Merge branch 'unstable' of https://github.com/sigp/lighthouse into merge-unstable-deneb-june-6th

This commit is contained in:
realbigsean 2023-07-12 13:05:30 -04:00
commit a6f48f5ecb
No known key found for this signature in database
GPG Key ID: BE1B3DB104F6C788
38 changed files with 2038 additions and 1907 deletions

28
Cargo.lock generated
View File

@ -717,6 +717,30 @@ dependencies = [
"unused_port", "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]] [[package]]
name = "bincode" name = "bincode"
version = "1.3.3" version = "1.3.3"
@ -1218,6 +1242,7 @@ name = "client"
version = "0.2.0" version = "0.2.0"
dependencies = [ dependencies = [
"beacon_chain", "beacon_chain",
"beacon_processor",
"directory", "directory",
"dirs", "dirs",
"environment", "environment",
@ -1235,6 +1260,7 @@ dependencies = [
"logging", "logging",
"monitoring_api", "monitoring_api",
"network", "network",
"num_cpus",
"operation_pool", "operation_pool",
"parking_lot 0.12.1", "parking_lot 0.12.1",
"sensitive_url", "sensitive_url",
@ -5539,6 +5565,7 @@ name = "network"
version = "0.2.0" version = "0.2.0"
dependencies = [ dependencies = [
"beacon_chain", "beacon_chain",
"beacon_processor",
"delay_map", "delay_map",
"derivative", "derivative",
"environment", "environment",
@ -5562,6 +5589,7 @@ dependencies = [
"matches", "matches",
"num_cpus", "num_cpus",
"operation_pool", "operation_pool",
"parking_lot 0.12.1",
"rand 0.8.5", "rand 0.8.5",
"rlp", "rlp",
"slog", "slog",

View File

@ -4,6 +4,7 @@ members = [
"beacon_node", "beacon_node",
"beacon_node/beacon_chain", "beacon_node/beacon_chain",
"beacon_node/beacon_processor",
"beacon_node/builder_client", "beacon_node/builder_client",
"beacon_node/client", "beacon_node/client",
"beacon_node/eth1", "beacon_node/eth1",

View 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"

View 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"]
);
}

View File

@ -10,17 +10,12 @@
//! //!
//! Aggregated and unaggregated attestations that failed verification due to referencing an unknown //! 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. //! 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::metrics;
use crate::sync::manager::BlockProcessType; use crate::{AsyncFn, BlockingFn, Work, WorkEvent};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::task::Poll; use futures::task::Poll;
use futures::{Stream, StreamExt}; use futures::{Stream, StreamExt};
use itertools::Itertools; use itertools::Itertools;
use lighthouse_network::{MessageId, PeerId};
use logging::TimeLatch; use logging::TimeLatch;
use slog::{crit, debug, error, trace, warn, Logger}; use slog::{crit, debug, error, trace, warn, Logger};
use slot_clock::SlotClock; use slot_clock::SlotClock;
@ -34,9 +29,7 @@ use task_executor::TaskExecutor;
use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::sync::mpsc::{self, Receiver, Sender};
use tokio::time::error::Error as TimeError; use tokio::time::error::Error as TimeError;
use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey};
use types::{ use types::{EthSpec, Hash256, Slot};
Attestation, EthSpec, Hash256, LightClientOptimisticUpdate, SignedAggregateAndProof, SubnetId,
};
const TASK_NAME: &str = "beacon_processor_reprocess_queue"; const TASK_NAME: &str = "beacon_processor_reprocess_queue";
const GOSSIP_BLOCKS: &str = "gossip_blocks"; const GOSSIP_BLOCKS: &str = "gossip_blocks";
@ -46,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. /// 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. /// 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. /// For how long to queue aggregated and unaggregated attestations for re-processing.
pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12); pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12);
@ -83,12 +76,12 @@ pub const BACKFILL_SCHEDULE_IN_SLOT: [(u32, u32); 3] = [
/// Messages that the scheduler can receive. /// Messages that the scheduler can receive.
#[derive(AsRefStr)] #[derive(AsRefStr)]
pub enum ReprocessQueueMessage<T: BeaconChainTypes> { pub enum ReprocessQueueMessage {
/// A block that has been received early and we should queue for later processing. /// 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 /// 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. /// 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 /// A block that was successfully processed. We use this to handle attestations and light client updates
/// for unknown blocks. /// for unknown blocks.
BlockImported { BlockImported {
@ -96,139 +89,127 @@ pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
parent_root: Hash256, parent_root: Hash256,
}, },
/// An unaggregated attestation that references an unknown block. /// An unaggregated attestation that references an unknown block.
UnknownBlockUnaggregate(QueuedUnaggregate<T::EthSpec>), UnknownBlockUnaggregate(QueuedUnaggregate),
/// An aggregated attestation that references an unknown block. /// 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. /// 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. /// 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. /// Events sent by the scheduler once they are ready for re-processing.
pub enum ReadyWork<T: BeaconChainTypes> { pub enum ReadyWork {
GossipBlock(QueuedGossipBlock<T>), Block(QueuedGossipBlock),
RpcBlock(QueuedRpcBlock<T::EthSpec>), RpcBlock(QueuedRpcBlock),
Unaggregate(QueuedUnaggregate<T::EthSpec>), IgnoredRpcBlock(IgnoredRpcBlock),
Aggregate(QueuedAggregate<T::EthSpec>), Unaggregate(QueuedUnaggregate),
LightClientUpdate(QueuedLightClientUpdate<T::EthSpec>), Aggregate(QueuedAggregate),
BackfillSync(QueuedBackfillBatch<T::EthSpec>), LightClientUpdate(QueuedLightClientUpdate),
BackfillSync(QueuedBackfillBatch),
} }
/// An Attestation for which the corresponding block was not seen while processing, queued for /// An Attestation for which the corresponding block was not seen while processing, queued for
/// later. /// later.
pub struct QueuedUnaggregate<T: EthSpec> { pub struct QueuedUnaggregate {
pub peer_id: PeerId, pub beacon_block_root: Hash256,
pub message_id: MessageId, pub process_fn: BlockingFn,
pub attestation: Box<Attestation<T>>,
pub subnet_id: SubnetId,
pub should_import: bool,
pub seen_timestamp: Duration,
} }
/// An aggregated attestation for which the corresponding block was not seen while processing, queued for /// An aggregated attestation for which the corresponding block was not seen while processing, queued for
/// later. /// later.
pub struct QueuedAggregate<T: EthSpec> { pub struct QueuedAggregate {
pub peer_id: PeerId, pub beacon_block_root: Hash256,
pub message_id: MessageId, pub process_fn: BlockingFn,
pub attestation: Box<SignedAggregateAndProof<T>>,
pub seen_timestamp: Duration,
} }
/// A light client update for which the corresponding parent block was not seen while processing, /// A light client update for which the corresponding parent block was not seen while processing,
/// queued for later. /// queued for later.
pub struct QueuedLightClientUpdate<T: EthSpec> { pub struct QueuedLightClientUpdate {
pub peer_id: PeerId,
pub message_id: MessageId,
pub light_client_optimistic_update: Box<LightClientOptimisticUpdate<T>>,
pub parent_root: Hash256, pub parent_root: Hash256,
pub seen_timestamp: Duration, pub process_fn: BlockingFn,
} }
/// A block that arrived early and has been queued for later import. /// A block that arrived early and has been queued for later import.
pub struct QueuedGossipBlock<T: BeaconChainTypes> { pub struct QueuedGossipBlock {
pub peer_id: PeerId, pub beacon_block_slot: Slot,
pub block: Box<GossipVerifiedBlock<T>>, pub beacon_block_root: Hash256,
pub seen_timestamp: Duration, pub process_fn: AsyncFn,
} }
/// A block that arrived for processing when the same block was being imported over gossip. /// A block that arrived for processing when the same block was being imported over gossip.
/// It is queued for later import. /// It is queued for later import.
pub struct QueuedRpcBlock<T: EthSpec> { pub struct QueuedRpcBlock {
pub block_root: Hash256, pub beacon_block_root: Hash256,
pub block: BlockWrapper<T>, /// Processes/imports the block.
pub process_type: BlockProcessType, pub process_fn: AsyncFn,
pub seen_timestamp: Duration, /// Ignores the block.
/// Indicates if the beacon chain should process this block or not. pub ignore_fn: BlockingFn,
/// We use this to ignore block processing when rpc block queues are full. }
pub should_process: bool,
/// 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. /// A backfill batch work that has been queued for processing later.
#[derive(Clone)] pub struct QueuedBackfillBatch(pub AsyncFn);
pub struct QueuedBackfillBatch<E: EthSpec> {
pub process_id: ChainSegmentProcessId,
pub blocks: Vec<BlockWrapper<E>>,
}
impl<T: BeaconChainTypes> TryFrom<WorkEvent<T>> for QueuedBackfillBatch<T::EthSpec> { impl<T: EthSpec> TryFrom<WorkEvent<T>> for QueuedBackfillBatch {
type Error = WorkEvent<T>; type Error = WorkEvent<T>;
fn try_from(event: WorkEvent<T>) -> Result<Self, WorkEvent<T>> { fn try_from(event: WorkEvent<T>) -> Result<Self, WorkEvent<T>> {
match event { match event {
WorkEvent { WorkEvent {
work: work: Work::ChainSegmentBackfill(process_fn),
Work::ChainSegment {
process_id: process_id @ ChainSegmentProcessId::BackSyncBatchId(_),
blocks,
},
.. ..
} => Ok(QueuedBackfillBatch { process_id, blocks }), } => Ok(QueuedBackfillBatch(process_fn)),
_ => Err(event), _ => Err(event),
} }
} }
} }
impl<T: BeaconChainTypes> From<QueuedBackfillBatch<T::EthSpec>> for WorkEvent<T> { impl<T: EthSpec> From<QueuedBackfillBatch> for WorkEvent<T> {
fn from(queued_backfill_batch: QueuedBackfillBatch<T::EthSpec>) -> WorkEvent<T> { fn from(queued_backfill_batch: QueuedBackfillBatch) -> WorkEvent<T> {
WorkEvent::chain_segment( WorkEvent {
queued_backfill_batch.process_id, drop_during_sync: false,
queued_backfill_batch.blocks, work: Work::ChainSegmentBackfill(queued_backfill_batch.0),
) }
} }
} }
/// Unifies the different messages processed by the block delay queue. /// 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. /// 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 /// A rpc block that was queued because the same gossip block was being imported
/// will now be retried for import. /// will now be retried for import.
ReadyRpcBlock(QueuedRpcBlock<T::EthSpec>), ReadyRpcBlock(QueuedRpcBlock),
/// An aggregated or unaggregated attestation is ready for re-processing. /// An aggregated or unaggregated attestation is ready for re-processing.
ReadyAttestation(QueuedAttestationId), ReadyAttestation(QueuedAttestationId),
/// A light client update that is ready for re-processing. /// A light client update that is ready for re-processing.
ReadyLightClientUpdate(QueuedLightClientUpdateId), ReadyLightClientUpdate(QueuedLightClientUpdateId),
/// A backfill batch that was queued is ready for processing. /// A backfill batch that was queued is ready for processing.
ReadyBackfillSync(QueuedBackfillBatch<T::EthSpec>), ReadyBackfillSync(QueuedBackfillBatch),
/// A `DelayQueue` returned an error. /// A `DelayQueue` returned an error.
DelayQueueError(TimeError, &'static str), DelayQueueError(TimeError, &'static str),
/// A message sent to the `ReprocessQueue` /// A message sent to the `ReprocessQueue`
Msg(ReprocessQueueMessage<T>), Msg(ReprocessQueueMessage),
} }
/// Manages scheduling works that need to be later re-processed. /// 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. /// 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 /// Sender of works once they become ready
ready_work_tx: Sender<ReadyWork<T>>, ready_work_tx: Sender<ReadyWork>,
/* Queues */ /* Queues */
/// Queue to manage scheduled early blocks. /// 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. /// 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. /// Queue to manage scheduled attestations.
attestations_delay_queue: DelayQueue<QueuedAttestationId>, attestations_delay_queue: DelayQueue<QueuedAttestationId>,
/// Queue to manage scheduled light client updates. /// Queue to manage scheduled light client updates.
@ -238,17 +219,17 @@ struct ReprocessQueue<T: BeaconChainTypes> {
/// Queued blocks. /// Queued blocks.
queued_gossip_block_roots: HashSet<Hash256>, queued_gossip_block_roots: HashSet<Hash256>,
/// Queued aggregated attestations. /// Queued aggregated attestations.
queued_aggregates: FnvHashMap<usize, (QueuedAggregate<T::EthSpec>, DelayKey)>, queued_aggregates: FnvHashMap<usize, (QueuedAggregate, DelayKey)>,
/// Queued attestations. /// Queued attestations.
queued_unaggregates: FnvHashMap<usize, (QueuedUnaggregate<T::EthSpec>, DelayKey)>, queued_unaggregates: FnvHashMap<usize, (QueuedUnaggregate, DelayKey)>,
/// Attestations (aggregated and unaggregated) per root. /// Attestations (aggregated and unaggregated) per root.
awaiting_attestations_per_root: HashMap<Hash256, Vec<QueuedAttestationId>>, awaiting_attestations_per_root: HashMap<Hash256, Vec<QueuedAttestationId>>,
/// Queued Light Client Updates. /// 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. /// Light Client Updates per parent_root.
awaiting_lc_updates_per_parent_root: HashMap<Hash256, Vec<QueuedLightClientUpdateId>>, awaiting_lc_updates_per_parent_root: HashMap<Hash256, Vec<QueuedLightClientUpdateId>>,
/// Queued backfill batches /// Queued backfill batches
queued_backfill_batches: Vec<QueuedBackfillBatch<T::EthSpec>>, queued_backfill_batches: Vec<QueuedBackfillBatch>,
/* Aux */ /* Aux */
/// Next attestation id, used for both aggregated and unaggregated attestations /// Next attestation id, used for both aggregated and unaggregated attestations
@ -259,7 +240,7 @@ struct ReprocessQueue<T: BeaconChainTypes> {
attestation_delay_debounce: TimeLatch, attestation_delay_debounce: TimeLatch,
lc_update_delay_debounce: TimeLatch, lc_update_delay_debounce: TimeLatch,
next_backfill_batch_event: Option<Pin<Box<tokio::time::Sleep>>>, 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; pub type QueuedLightClientUpdateId = usize;
@ -270,20 +251,20 @@ enum QueuedAttestationId {
Unaggregate(usize), Unaggregate(usize),
} }
impl<T: EthSpec> QueuedAggregate<T> { impl QueuedAggregate {
pub fn beacon_block_root(&self) -> &Hash256 { 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 { pub fn beacon_block_root(&self) -> &Hash256 {
&self.attestation.data.beacon_block_root &self.beacon_block_root
} }
} }
impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> { impl<S: SlotClock> Stream for ReprocessQueue<S> {
type Item = InboundEvent<T>; type Item = InboundEvent;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { 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 // NOTE: implementing `Stream` is not necessary but allows to maintain the future selection
@ -374,16 +355,13 @@ impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
/// Starts the job that manages scheduling works that need re-processing. The returned `Sender` /// 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 /// gives the communicating channel to receive those works. Once a work is ready, it is sent back
/// via `ready_work_tx`. /// via `ready_work_tx`.
pub fn spawn_reprocess_scheduler<T: BeaconChainTypes>( pub fn spawn_reprocess_scheduler<S: SlotClock + 'static>(
ready_work_tx: Sender<ReadyWork<T>>, ready_work_tx: Sender<ReadyWork>,
work_reprocessing_rx: Receiver<ReprocessQueueMessage>,
executor: &TaskExecutor, executor: &TaskExecutor,
slot_clock: T::SlotClock, slot_clock: S,
log: Logger, 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 { let mut queue = ReprocessQueue {
work_reprocessing_rx, work_reprocessing_rx,
ready_work_tx, ready_work_tx,
@ -422,19 +400,17 @@ pub fn spawn_reprocess_scheduler<T: BeaconChainTypes>(
}, },
TASK_NAME, TASK_NAME,
); );
work_reprocessing_tx
} }
impl<T: BeaconChainTypes> ReprocessQueue<T> { impl<S: SlotClock> ReprocessQueue<S> {
fn handle_message(&mut self, msg: InboundEvent<T>, slot_clock: &T::SlotClock, log: &Logger) { fn handle_message(&mut self, msg: InboundEvent, slot_clock: &S, log: &Logger) {
use ReprocessQueueMessage::*; use ReprocessQueueMessage::*;
match msg { match msg {
// Some block has been indicated as "early" and should be processed when the // Some block has been indicated as "early" and should be processed when the
// appropriate slot arrives. // appropriate slot arrives.
InboundEvent::Msg(EarlyBlock(early_block)) => { InboundEvent::Msg(EarlyBlock(early_block)) => {
let block_slot = early_block.block.block.slot(); let block_slot = early_block.beacon_block_slot;
let block_root = early_block.block.block_root; let block_root = early_block.beacon_block_root;
// Don't add the same block to the queue twice. This prevents DoS attacks. // Don't add the same block to the queue twice. This prevents DoS attacks.
if self.queued_gossip_block_roots.contains(&block_root) { if self.queued_gossip_block_roots.contains(&block_root) {
@ -493,7 +469,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
// for the same block hash is being imported. We wait for `QUEUED_RPC_BLOCK_DELAY` // 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 // and then send the rpc block back for processing assuming the gossip import
// has completed by then. // 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. // Check to ensure this won't over-fill the queue.
if self.rpc_block_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS { if self.rpc_block_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS {
if self.rpc_block_debounce.elapsed() { if self.rpc_block_debounce.elapsed() {
@ -506,10 +482,11 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
} }
// Return the block to the beacon processor signalling to // Return the block to the beacon processor signalling to
// ignore processing for this block // ignore processing for this block
rpc_block.should_process = false;
if self if self
.ready_work_tx .ready_work_tx
.try_send(ReadyWork::RpcBlock(rpc_block)) .try_send(ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock {
process_fn: rpc_block.ignore_fn,
}))
.is_err() .is_err()
{ {
error!( error!(
@ -528,7 +505,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
debug!( debug!(
log, log,
"Sending rpc block for reprocessing"; "Sending rpc block for reprocessing";
"block_root" => %queued_rpc_block.block_root "block_root" => %queued_rpc_block.beacon_block_root
); );
if self if self
.ready_work_tx .ready_work_tx
@ -766,7 +743,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
} }
// A block that was queued for later processing is now ready to be processed. // A block that was queued for later processing is now ready to be processed.
InboundEvent::ReadyGossipBlock(ready_block) => { 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) { if !self.queued_gossip_block_roots.remove(&block_root) {
// Log an error to alert that we've made a bad assumption about how this // Log an error to alert that we've made a bad assumption about how this
@ -884,18 +861,28 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
"millis_from_slot_start" => millis_from_slot_start "millis_from_slot_start" => millis_from_slot_start
); );
if self match self
.ready_work_tx .ready_work_tx
.try_send(ReadyWork::BackfillSync(queued_backfill_batch.clone())) .try_send(ReadyWork::BackfillSync(queued_backfill_batch))
.is_err()
{ {
// 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!( error!(
log, log,
"Failed to send scheduled backfill work"; "Failed to send scheduled backfill work";
"info" => "sending work back to queue" "info" => "sending work back to queue"
); );
self.queued_backfill_batches self.queued_backfill_batches.insert(0, batch)
.insert(0, queued_backfill_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";
),
} }
} }
} }
@ -926,7 +913,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
// only recompute the `next_backfill_batch_event` if there are backfill batches in the queue // only recompute the `next_backfill_batch_event` if there are backfill batches in the queue
if !self.queued_backfill_batches.is_empty() { if !self.queued_backfill_batches.is_empty() {
self.next_backfill_batch_event = Some(Box::pin(tokio::time::sleep( 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 { } else {
self.next_backfill_batch_event = None self.next_backfill_batch_event = None
@ -935,7 +922,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
/// Returns duration until the next scheduled processing time. The schedule ensure that backfill /// Returns duration until the next scheduled processing time. The schedule ensure that backfill
/// processing is done in windows of time that aren't critical /// 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(); let slot_duration = slot_clock.slot_duration();
slot_clock slot_clock
.millis_from_current_slot_start() .millis_from_current_slot_start()
@ -965,16 +952,9 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;
use beacon_chain::builder::Witness;
use beacon_chain::eth1_chain::CachingEth1Backend;
use slot_clock::TestingSlotClock; use slot_clock::TestingSlotClock;
use store::MemoryStore;
use types::MainnetEthSpec as E;
use types::Slot; use types::Slot;
type TestBeaconChainType =
Witness<TestingSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
#[test] #[test]
fn backfill_processing_schedule_calculation() { fn backfill_processing_schedule_calculation() {
let slot_duration = Duration::from_secs(12); let slot_duration = Duration::from_secs(12);
@ -987,7 +967,7 @@ mod tests {
for &event_duration_from_slot_start in event_times.iter() { for &event_duration_from_slot_start in event_times.iter() {
let duration_to_next_event = let duration_to_next_event =
ReprocessQueue::<TestBeaconChainType>::duration_until_next_backfill_batch_event( ReprocessQueue::<TestingSlotClock>::duration_until_next_backfill_batch_event(
&slot_clock, &slot_clock,
); );
@ -1004,7 +984,7 @@ mod tests {
// check for next event beyond the current slot // check for next event beyond the current slot
let duration_to_next_slot = slot_clock.duration_to_next_slot().unwrap(); let duration_to_next_slot = slot_clock.duration_to_next_slot().unwrap();
let duration_to_next_event = let duration_to_next_event =
ReprocessQueue::<TestBeaconChainType>::duration_until_next_backfill_batch_event( ReprocessQueue::<TestingSlotClock>::duration_until_next_backfill_batch_event(
&slot_clock, &slot_clock,
); );
assert_eq!( assert_eq!(

View File

@ -43,3 +43,5 @@ slasher = { path = "../../slasher" }
slasher_service = { path = "../../slasher/service" } slasher_service = { path = "../../slasher/service" }
monitoring_api = {path = "../../common/monitoring_api"} monitoring_api = {path = "../../common/monitoring_api"}
execution_layer = { path = "../execution_layer" } execution_layer = { path = "../execution_layer" }
beacon_processor = { path = "../beacon_processor" }
num_cpus = "1.13.0"

View File

@ -14,6 +14,10 @@ use beacon_chain::{
store::{HotColdDB, ItemStore, LevelDB, StoreConfig}, store::{HotColdDB, ItemStore, LevelDB, StoreConfig},
BeaconChain, BeaconChainTypes, Eth1ChainBackend, ServerSentEventHandler, 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 environment::RuntimeContext;
use eth1::{Config as Eth1Config, Service as Eth1Service}; use eth1::{Config as Eth1Config, Service as Eth1Service};
use eth2::{ use eth2::{
@ -29,12 +33,13 @@ use slasher::Slasher;
use slasher_service::SlasherService; use slasher_service::SlasherService;
use slog::{debug, info, warn, Logger}; use slog::{debug, info, warn, Logger};
use state_processing::per_slot_processing; use state_processing::per_slot_processing;
use std::cmp;
use std::net::TcpListener; use std::net::TcpListener;
use std::path::{Path, PathBuf}; use std::path::{Path, PathBuf};
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use timer::spawn_timer; use timer::spawn_timer;
use tokio::sync::oneshot; use tokio::sync::{mpsc, oneshot};
use types::{ use types::{
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec, test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec,
ExecutionBlockHash, Hash256, SignedBeaconBlock, ExecutionBlockHash, Hash256, SignedBeaconBlock,
@ -75,6 +80,10 @@ pub struct ClientBuilder<T: BeaconChainTypes> {
http_metrics_config: http_metrics::Config, http_metrics_config: http_metrics::Config,
slasher: Option<Arc<Slasher<T::EthSpec>>>, slasher: Option<Arc<Slasher<T::EthSpec>>>,
eth_spec_instance: 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> impl<TSlotClock, TEth1Backend, TEthSpec, THotStore, TColdStore>
@ -90,6 +99,10 @@ where
/// ///
/// The `eth_spec_instance` parameter is used to concretize `TEthSpec`. /// The `eth_spec_instance` parameter is used to concretize `TEthSpec`.
pub fn new(eth_spec_instance: TEthSpec) -> Self { 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 { Self {
slot_clock: None, slot_clock: None,
store: None, store: None,
@ -108,6 +121,10 @@ where
http_metrics_config: <_>::default(), http_metrics_config: <_>::default(),
slasher: None, slasher: None,
eth_spec_instance, eth_spec_instance,
beacon_processor_send: BeaconProcessorSend(beacon_processor_send),
beacon_processor_receive,
work_reprocessing_tx,
work_reprocessing_rx,
} }
} }
@ -551,6 +568,8 @@ where
gossipsub_registry gossipsub_registry
.as_mut() .as_mut()
.map(|registry| registry.sub_registry_with_prefix("gossipsub")), .map(|registry| registry.sub_registry_with_prefix("gossipsub")),
self.beacon_processor_send.clone(),
self.work_reprocessing_tx.clone(),
) )
.await .await
.map_err(|e| format!("Failed to start network: {:?}", e))?; .map_err(|e| format!("Failed to start network: {:?}", e))?;
@ -738,6 +757,27 @@ where
} }
if let Some(beacon_chain) = self.beacon_chain.as_ref() { 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_context = runtime_context.service_context("state_advance".into());
let state_advance_log = state_advance_context.log().clone(); let state_advance_log = state_advance_context.log().clone();
spawn_state_advance_timer( spawn_state_advance_timer(

View File

@ -40,12 +40,11 @@ use tokio::{
}; };
use tokio_stream::wrappers::WatchStream; use tokio_stream::wrappers::WatchStream;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use types::beacon_block_body::KzgCommitments; use types::{AbstractExecPayload, BeaconStateError, ExecPayload};
use types::blob_sidecar::Blobs;
use types::{AbstractExecPayload, BeaconStateError, ExecPayload, VersionedHash};
use types::{ use types::{
BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ExecutionPayload, BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionPayloadCapella, ExecutionPayloadMerge,
ExecutionPayloadCapella, ExecutionPayloadDeneb, ExecutionPayloadMerge, ForkName, ForkVersionedResponse, ProposerPreparationData, PublicKeyBytes, Signature, SignedBeaconBlock,
Slot,
}; };
use types::{KzgProofs, Withdrawals}; use types::{KzgProofs, Withdrawals};
use types::{ProposerPreparationData, PublicKeyBytes, Signature, Slot, Transaction, Uint256}; use types::{ProposerPreparationData, PublicKeyBytes, Signature, Slot, Transaction, Uint256};

View File

@ -327,11 +327,8 @@ async fn sync_committee_indices_across_fork() {
/// Assert that an HTTP API error has the given status code and indexed errors for the given indices. /// Assert that an HTTP API error has the given status code and indexed errors for the given indices.
fn assert_server_indexed_error(error: eth2::Error, status_code: u16, indices: Vec<usize>) { fn assert_server_indexed_error(error: eth2::Error, status_code: u16, indices: Vec<usize>) {
let eth2::Error::ServerIndexedMessage(IndexedErrorMessage { let eth2::Error::ServerIndexedMessage(IndexedErrorMessage { code, failures, .. }) = error
code, else {
failures,
..
}) = error else {
panic!("wrong error, expected ServerIndexedMessage, got: {error:?}") panic!("wrong error, expected ServerIndexedMessage, got: {error:?}")
}; };
assert_eq!(code, status_code); assert_eq!(code, status_code);

View File

@ -11,7 +11,6 @@ matches = "0.1.8"
exit-future = "0.2.0" exit-future = "0.2.0"
slog-term = "2.6.0" slog-term = "2.6.0"
slog-async = "2.5.0" slog-async = "2.5.0"
environment = { path = "../../lighthouse/environment" }
[dependencies] [dependencies]
beacon_chain = { path = "../beacon_chain" } beacon_chain = { path = "../beacon_chain" }
@ -47,6 +46,9 @@ delay_map = "0.3.0"
ethereum-types = { version = "0.14.1", optional = true } ethereum-types = { version = "0.14.1", optional = true }
operation_pool = { path = "../operation_pool" } operation_pool = { path = "../operation_pool" }
execution_layer = { path = "../execution_layer" } execution_layer = { path = "../execution_layer" }
beacon_processor = { path = "../beacon_processor" }
parking_lot = "0.12.0"
environment = { path = "../../lighthouse/environment" }
[features] [features]
spec-minimal = ["beacon_chain/spec-minimal"] spec-minimal = ["beacon_chain/spec-minimal"]

View File

@ -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>>,
}

View File

@ -6,10 +6,10 @@ pub mod error;
#[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy #[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy
pub mod service; pub mod service;
mod beacon_processor;
#[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy #[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy
mod metrics; mod metrics;
mod nat; mod nat;
mod network_beacon_processor;
mod persisted_dht; mod persisted_dht;
mod router; mod router;
mod status; mod status;

View File

@ -49,47 +49,8 @@ lazy_static! {
/* /*
* Gossip processor * 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. // 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( pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_gossip_block_verified_total", "beacon_processor_gossip_block_verified_total",
"Total number of gossip blocks verified for propagation." "Total number of gossip blocks verified for propagation."
@ -120,10 +81,6 @@ lazy_static! {
"Total number of gossip blobs imported to fork choice, etc." "Total number of gossip blobs imported to fork choice, etc."
); );
// Gossip Exits. // 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( pub static ref BEACON_PROCESSOR_EXIT_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_exit_verified_total", "beacon_processor_exit_verified_total",
"Total number of voluntary exits verified for propagation." "Total number of voluntary exits verified for propagation."
@ -133,10 +90,6 @@ lazy_static! {
"Total number of voluntary exits imported to the op pool." "Total number of voluntary exits imported to the op pool."
); );
// Gossip proposer slashings. // 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( pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_proposer_slashing_verified_total", "beacon_processor_proposer_slashing_verified_total",
"Total number of proposer slashings verified for propagation." "Total number of proposer slashings verified for propagation."
@ -146,10 +99,6 @@ lazy_static! {
"Total number of proposer slashings imported to the op pool." "Total number of proposer slashings imported to the op pool."
); );
// Gossip attester slashings. // 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( pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_attester_slashing_verified_total", "beacon_processor_attester_slashing_verified_total",
"Total number of attester slashings verified for propagation." "Total number of attester slashings verified for propagation."
@ -159,10 +108,6 @@ lazy_static! {
"Total number of attester slashings imported to the op pool." "Total number of attester slashings imported to the op pool."
); );
// Gossip BLS to execution changes. // 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( 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", "beacon_processor_bls_to_execution_change_verified_total",
"Total number of address changes verified for propagation." "Total number of address changes verified for propagation."
@ -176,10 +121,6 @@ lazy_static! {
// Need to split up this `lazy_static!` due to recursion limits. // Need to split up this `lazy_static!` due to recursion limits.
lazy_static! { lazy_static! {
// Rpc blocks. // 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( pub static ref BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_rpc_block_imported_total", "beacon_processor_rpc_block_imported_total",
"Total number of gossip blocks imported to fork choice, etc." "Total number of gossip blocks imported to fork choice, etc."
@ -194,14 +135,6 @@ lazy_static! {
"Total number of gossip blobs imported." "Total number of gossip blobs imported."
); );
// Chain segments. // 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( pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_SUCCESS_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_chain_segment_success_total", "beacon_processor_chain_segment_success_total",
"Total number of chain segments successfully processed." "Total number of chain segments successfully processed."
@ -219,10 +152,6 @@ lazy_static! {
"Total number of backfill chain segments that failed processing." "Total number of backfill chain segments that failed processing."
); );
// Unaggregated attestations. // 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( pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_unaggregated_attestation_verified_total", "beacon_processor_unaggregated_attestation_verified_total",
"Total number of unaggregated attestations verified for gossip." "Total number of unaggregated attestations verified for gossip."
@ -236,10 +165,6 @@ lazy_static! {
"Total number of unaggregated attestations that referenced an unknown block and were re-queued." "Total number of unaggregated attestations that referenced an unknown block and were re-queued."
); );
// Aggregated attestations. // 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( pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_aggregated_attestation_verified_total", "beacon_processor_aggregated_attestation_verified_total",
"Total number of aggregated attestations verified for gossip." "Total number of aggregated attestations verified for gossip."
@ -253,10 +178,6 @@ lazy_static! {
"Total number of aggregated attestations that referenced an unknown block and were re-queued." "Total number of aggregated attestations that referenced an unknown block and were re-queued."
); );
// Sync committee messages. // 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( pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_sync_message_verified_total", "beacon_processor_sync_message_verified_total",
"Total number of sync committee messages verified for gossip." "Total number of sync committee messages verified for gossip."
@ -266,10 +187,6 @@ lazy_static! {
"Total number of sync committee messages imported to fork choice, etc." "Total number of sync committee messages imported to fork choice, etc."
); );
// Sync contribution. // 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( pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_sync_contribution_verified_total", "beacon_processor_sync_contribution_verified_total",
"Total number of sync committee contributions verified for gossip." "Total number of sync committee contributions verified for gossip."
@ -305,12 +222,6 @@ lazy_static! {
"Gossipsub light_client_optimistic_update errors per error type", "Gossipsub light_client_optimistic_update errors per error type",
&["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"]
);
/* /*
@ -426,35 +337,9 @@ lazy_static! {
"Count of times when a gossip blob arrived from the network later than the attestation deadline.", "Count of times when a gossip blob 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. * 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( pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_SENT_OPTIMISTIC_UPDATES: Result<IntCounter> = try_create_int_counter(
"beacon_processor_reprocessing_queue_sent_optimistic_updates", "beacon_processor_reprocessing_queue_sent_optimistic_updates",
"Number of queued light client optimistic updates where as matching block has been imported." "Number of queued light client optimistic updates where as matching block has been imported."

View File

@ -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::blob_verification::{AsBlock, BlobError, GossipVerifiedBlob}; use beacon_chain::blob_verification::{AsBlock, BlobError, GossipVerifiedBlob};
use beacon_chain::store::Error; use beacon_chain::store::Error;
@ -31,14 +36,13 @@ use types::{
Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId,
}; };
use super::{ use beacon_processor::{
super::work_reprocessing_queue::{ work_reprocessing_queue::{
QueuedAggregate, QueuedGossipBlock, QueuedLightClientUpdate, QueuedUnaggregate, QueuedAggregate, QueuedGossipBlock, QueuedLightClientUpdate, QueuedUnaggregate,
ReprocessQueueMessage, 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 /// Set to `true` to introduce stricter penalties for peers who send some types of late consensus
/// messages. /// messages.
@ -145,65 +149,7 @@ impl<T: EthSpec> FailedAtt<T> {
} }
} }
/// Items required to verify a batch of unaggregated gossip attestations. impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
#[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> {
/* Auxiliary functions */ /* Auxiliary functions */
/// Penalizes a peer for misbehaviour. /// Penalizes a peer for misbehaviour.
@ -246,13 +192,13 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Raises a log if there are errors. /// Raises a log if there are errors.
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
pub fn process_gossip_attestation( pub fn process_gossip_attestation(
self, self: Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
attestation: Box<Attestation<T::EthSpec>>, attestation: Box<Attestation<T::EthSpec>>,
subnet_id: SubnetId, subnet_id: SubnetId,
should_import: bool, should_import: bool,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
seen_timestamp: Duration, seen_timestamp: Duration,
) { ) {
let result = match self let result = match self
@ -278,9 +224,9 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_attestation_batch( pub fn process_gossip_attestation_batch(
self, self: Arc<Self>,
packages: Vec<GossipAttestationPackage<T::EthSpec>>, packages: Vec<GossipAttestationPackage<T::EthSpec>>,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
) { ) {
let attestations_and_subnets = packages let attestations_and_subnets = packages
.iter() .iter()
@ -349,12 +295,12 @@ impl<T: BeaconChainTypes> Worker<T> {
// cant' be mixed-up) and creating a struct would result in more complexity. // cant' be mixed-up) and creating a struct would result in more complexity.
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
fn process_gossip_attestation_result( fn process_gossip_attestation_result(
&self, self: &Arc<Self>,
result: Result<VerifiedUnaggregate<T>, RejectedUnaggregate<T::EthSpec>>, result: Result<VerifiedUnaggregate<T>, RejectedUnaggregate<T::EthSpec>>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
subnet_id: SubnetId, subnet_id: SubnetId,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
should_import: bool, should_import: bool,
seen_timestamp: Duration, seen_timestamp: Duration,
) { ) {
@ -457,11 +403,11 @@ impl<T: BeaconChainTypes> Worker<T> {
/// ///
/// Raises a log if there are errors. /// Raises a log if there are errors.
pub fn process_gossip_aggregate( pub fn process_gossip_aggregate(
self, self: Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
aggregate: Box<SignedAggregateAndProof<T::EthSpec>>, aggregate: Box<SignedAggregateAndProof<T::EthSpec>>,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
seen_timestamp: Duration, seen_timestamp: Duration,
) { ) {
let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root; let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root;
@ -491,9 +437,9 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_aggregate_batch( pub fn process_gossip_aggregate_batch(
self, self: Arc<Self>,
packages: Vec<GossipAggregatePackage<T::EthSpec>>, 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()); let aggregates = packages.iter().map(|package| package.aggregate.as_ref());
@ -556,12 +502,12 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
fn process_gossip_aggregate_result( fn process_gossip_aggregate_result(
&self, self: &Arc<Self>,
result: Result<VerifiedAggregate<T>, RejectedAggregate<T::EthSpec>>, result: Result<VerifiedAggregate<T>, RejectedAggregate<T::EthSpec>>,
beacon_block_root: Hash256, beacon_block_root: Hash256,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
seen_timestamp: Duration, seen_timestamp: Duration,
) { ) {
match result { match result {
@ -803,12 +749,12 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Raises a log if there are errors. /// Raises a log if there are errors.
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
pub async fn process_gossip_block( pub async fn process_gossip_block(
self, self: Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
peer_client: Client, peer_client: Client,
block: Arc<SignedBeaconBlock<T::EthSpec>>, block: Arc<SignedBeaconBlock<T::EthSpec>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>, reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
duplicate_cache: DuplicateCache, duplicate_cache: DuplicateCache,
invalid_block_storage: InvalidBlockStorage, invalid_block_storage: InvalidBlockStorage,
seen_duration: Duration, seen_duration: Duration,
@ -852,12 +798,12 @@ impl<T: BeaconChainTypes> Worker<T> {
/// ///
/// Returns the `GossipVerifiedBlock` if verification passes and raises a log if there are errors. /// Returns the `GossipVerifiedBlock` if verification passes and raises a log if there are errors.
pub async fn process_gossip_unverified_block( pub async fn process_gossip_unverified_block(
&self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
peer_client: Client, peer_client: Client,
block: Arc<SignedBeaconBlock<T::EthSpec>>, block: Arc<SignedBeaconBlock<T::EthSpec>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>, reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
seen_duration: Duration, seen_duration: Duration,
) -> Option<GossipVerifiedBlock<T>> { ) -> Option<GossipVerifiedBlock<T>> {
let block_delay = let block_delay =
@ -1066,11 +1012,25 @@ impl<T: BeaconChainTypes> Worker<T> {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL); 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 if reprocess_tx
.try_send(ReprocessQueueMessage::EarlyBlock(QueuedGossipBlock { .try_send(ReprocessQueueMessage::EarlyBlock(QueuedGossipBlock {
peer_id, beacon_block_slot: block_slot,
block: Box::new(verified_block), beacon_block_root: block_root,
seen_timestamp: seen_duration, process_fn,
})) }))
.is_err() .is_err()
{ {
@ -1103,10 +1063,10 @@ impl<T: BeaconChainTypes> Worker<T> {
/// ///
/// Raises a log if there are errors. /// Raises a log if there are errors.
pub async fn process_gossip_verified_block( pub async fn process_gossip_verified_block(
self, self: Arc<Self>,
peer_id: PeerId, peer_id: PeerId,
verified_block: GossipVerifiedBlock<T>, verified_block: GossipVerifiedBlock<T>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>, reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
invalid_block_storage: InvalidBlockStorage, invalid_block_storage: InvalidBlockStorage,
// This value is not used presently, but it might come in handy for debugging. // This value is not used presently, but it might come in handy for debugging.
_seen_duration: Duration, _seen_duration: Duration,
@ -1214,7 +1174,7 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_voluntary_exit( pub fn process_gossip_voluntary_exit(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
voluntary_exit: SignedVoluntaryExit, voluntary_exit: SignedVoluntaryExit,
@ -1272,7 +1232,7 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_proposer_slashing( pub fn process_gossip_proposer_slashing(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
proposer_slashing: ProposerSlashing, proposer_slashing: ProposerSlashing,
@ -1334,7 +1294,7 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_attester_slashing( pub fn process_gossip_attester_slashing(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
attester_slashing: AttesterSlashing<T::EthSpec>, attester_slashing: AttesterSlashing<T::EthSpec>,
@ -1388,7 +1348,7 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_bls_to_execution_change( pub fn process_gossip_bls_to_execution_change(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
bls_to_execution_change: SignedBlsToExecutionChange, bls_to_execution_change: SignedBlsToExecutionChange,
@ -1471,7 +1431,7 @@ impl<T: BeaconChainTypes> Worker<T> {
/// ///
/// Raises a log if there are errors. /// Raises a log if there are errors.
pub fn process_gossip_sync_committee_signature( pub fn process_gossip_sync_committee_signature(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
sync_signature: SyncCommitteeMessage, sync_signature: SyncCommitteeMessage,
@ -1534,7 +1494,7 @@ impl<T: BeaconChainTypes> Worker<T> {
/// ///
/// Raises a log if there are errors. /// Raises a log if there are errors.
pub fn process_sync_committee_contribution( pub fn process_sync_committee_contribution(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
sync_contribution: SignedContributionAndProof<T::EthSpec>, sync_contribution: SignedContributionAndProof<T::EthSpec>,
@ -1589,7 +1549,7 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_finality_update( pub fn process_gossip_finality_update(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
light_client_finality_update: LightClientFinalityUpdate<T::EthSpec>, light_client_finality_update: LightClientFinalityUpdate<T::EthSpec>,
@ -1655,11 +1615,11 @@ impl<T: BeaconChainTypes> Worker<T> {
} }
pub fn process_gossip_optimistic_update( pub fn process_gossip_optimistic_update(
self, self: &Arc<Self>,
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
light_client_optimistic_update: LightClientOptimisticUpdate<T::EthSpec>, light_client_optimistic_update: LightClientOptimisticUpdate<T::EthSpec>,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
seen_timestamp: Duration, seen_timestamp: Duration,
) { ) {
match self.chain.verify_optimistic_update_for_gossip( match self.chain.verify_optimistic_update_for_gossip(
@ -1690,15 +1650,19 @@ impl<T: BeaconChainTypes> Worker<T> {
); );
if let Some(sender) = reprocess_tx { if let Some(sender) = reprocess_tx {
let processor = self.clone();
let msg = ReprocessQueueMessage::UnknownLightClientOptimisticUpdate( let msg = ReprocessQueueMessage::UnknownLightClientOptimisticUpdate(
QueuedLightClientUpdate { QueuedLightClientUpdate {
peer_id,
message_id,
light_client_optimistic_update: Box::new(
light_client_optimistic_update,
),
parent_root, 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, seen_timestamp,
)
}),
}, },
); );
@ -1787,11 +1751,11 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Handle an error whilst verifying an `Attestation` or `SignedAggregateAndProof` from the /// Handle an error whilst verifying an `Attestation` or `SignedAggregateAndProof` from the
/// network. /// network.
fn handle_attestation_verification_failure( fn handle_attestation_verification_failure(
&self, self: &Arc<Self>,
peer_id: PeerId, peer_id: PeerId,
message_id: MessageId, message_id: MessageId,
failed_att: FailedAtt<T::EthSpec>, failed_att: FailedAtt<T::EthSpec>,
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>, reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
error: AttnError, error: AttnError,
seen_timestamp: Duration, seen_timestamp: Duration,
) { ) {
@ -2026,11 +1990,18 @@ impl<T: BeaconChainTypes> Worker<T> {
metrics::inc_counter( metrics::inc_counter(
&metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL, &metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL,
); );
let processor = self.clone();
ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate {
peer_id, beacon_block_root: *beacon_block_root,
process_fn: Box::new(move || {
processor.process_gossip_aggregate(
message_id, message_id,
peer_id,
attestation, attestation,
None, // Do not allow this attestation to be re-processed beyond this point.
seen_timestamp, seen_timestamp,
)
}),
}) })
} }
FailedAtt::Unaggregate { FailedAtt::Unaggregate {
@ -2042,13 +2013,20 @@ impl<T: BeaconChainTypes> Worker<T> {
metrics::inc_counter( metrics::inc_counter(
&metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL, &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL,
); );
let processor = self.clone();
ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate {
peer_id, beacon_block_root: *beacon_block_root,
process_fn: Box::new(move || {
processor.process_gossip_attestation(
message_id, message_id,
peer_id,
attestation, attestation,
subnet_id, subnet_id,
should_import, should_import,
None, // Do not allow this attestation to be re-processed beyond this point.
seen_timestamp, seen_timestamp,
)
}),
}) })
} }
}; };

View 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
);
}
}

View File

@ -1,8 +1,9 @@
use crate::beacon_processor::{worker::FUTURE_SLOT_TOLERANCE, SendOnDrop}; use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE};
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use crate::status::ToStatusMessage; use crate::status::ToStatusMessage;
use crate::sync::SyncMessage; use crate::sync::SyncMessage;
use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped}; use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped};
use beacon_processor::SendOnDrop;
use itertools::process_results; use itertools::process_results;
use lighthouse_network::rpc::methods::{ use lighthouse_network::rpc::methods::{
BlobsByRangeRequest, BlobsByRootRequest, MAX_REQUEST_BLOB_SIDECARS, MAX_REQUEST_BLOCKS_DENEB, BlobsByRangeRequest, BlobsByRootRequest, MAX_REQUEST_BLOB_SIDECARS, MAX_REQUEST_BLOCKS_DENEB,
@ -12,15 +13,14 @@ use lighthouse_network::rpc::*;
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo}; use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
use slog::{debug, error, trace, warn}; use slog::{debug, error, trace, warn};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use std::sync::Arc;
use std::collections::{hash_map::Entry, HashMap}; use std::collections::{hash_map::Entry, HashMap};
use task_executor::TaskExecutor; use task_executor::TaskExecutor;
use tokio_stream::StreamExt; use tokio_stream::StreamExt;
use types::blob_sidecar::BlobIdentifier; use types::blob_sidecar::BlobIdentifier;
use types::{light_client_bootstrap::LightClientBootstrap, Epoch, EthSpec, Hash256, Slot}; use types::{light_client_bootstrap::LightClientBootstrap, Epoch, EthSpec, Hash256, Slot};
use super::Worker; impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
impl<T: BeaconChainTypes> Worker<T> {
/* Auxiliary functions */ /* Auxiliary functions */
/// Disconnects and ban's a peer, sending a Goodbye request with the associated reason. /// Disconnects and ban's a peer, sending a Goodbye request with the associated reason.
@ -132,7 +132,7 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Handle a `BlocksByRoot` request from the peer. /// Handle a `BlocksByRoot` request from the peer.
pub fn handle_blocks_by_root_request( pub fn handle_blocks_by_root_request(
self, self: Arc<Self>,
executor: TaskExecutor, executor: TaskExecutor,
send_on_drop: SendOnDrop, send_on_drop: SendOnDrop,
peer_id: PeerId, peer_id: PeerId,
@ -297,7 +297,7 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Handle a `BlocksByRoot` request from the peer. /// Handle a `BlocksByRoot` request from the peer.
pub fn handle_light_client_bootstrap( pub fn handle_light_client_bootstrap(
self, self: &Arc<Self>,
peer_id: PeerId, peer_id: PeerId,
request_id: PeerRequestId, request_id: PeerRequestId,
request: LightClientBootstrapRequest, request: LightClientBootstrapRequest,
@ -370,7 +370,7 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Handle a `BlocksByRange` request from the peer. /// Handle a `BlocksByRange` request from the peer.
pub fn handle_blocks_by_range_request( pub fn handle_blocks_by_range_request(
self, self: Arc<Self>,
executor: TaskExecutor, executor: TaskExecutor,
send_on_drop: SendOnDrop, send_on_drop: SendOnDrop,
peer_id: PeerId, peer_id: PeerId,

View File

@ -1,21 +1,21 @@
use std::time::Duration; 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::metrics;
use crate::sync::manager::{BlockProcessType, ResponseType, SyncMessage}; use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE};
use crate::sync::{BatchProcessResult, ChainId}; use crate::sync::BatchProcessResult;
use beacon_chain::blob_verification::BlockWrapper; use crate::sync::{
use beacon_chain::blob_verification::{AsBlock, MaybeAvailableBlock}; manager::{BlockProcessType, SyncMessage},
use beacon_chain::data_availability_checker::AvailabilityCheckError; ChainId,
use beacon_chain::AvailabilityProcessingStatus; };
use beacon_chain::{ use beacon_chain::{
observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms, observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms,
BeaconChainError, BeaconChainTypes, BlockError, ChainSegmentResult, HistoricalBlockError, BeaconChainError, BeaconChainTypes, BlockError, ChainSegmentResult, HistoricalBlockError,
NotifyExecutionLayer, NotifyExecutionLayer,
}; };
use beacon_processor::{
work_reprocessing_queue::{QueuedRpcBlock, ReprocessQueueMessage},
AsyncFn, BlockingFn, DuplicateCache,
};
use lighthouse_network::PeerAction; use lighthouse_network::PeerAction;
use slog::{debug, error, info, warn}; use slog::{debug, error, info, warn};
use slot_clock::SlotClock; use slot_clock::SlotClock;
@ -43,28 +43,72 @@ struct ChainSegmentFailed {
peer_action: Option<PeerAction>, peer_action: Option<PeerAction>,
} }
impl<T: BeaconChainTypes> Worker<T> { impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
/// Attempt to process a block received from a direct RPC request. /// Returns an async closure which processes a beacon block recieved via RPC.
#[allow(clippy::too_many_arguments)] ///
pub async fn process_rpc_block( /// This separate function was required to prevent a cycle during compiler
self, /// 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,
) -> 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_root: Hash256,
block: BlockWrapper<T::EthSpec>, block: BlockWrapper<T::EthSpec>,
seen_timestamp: Duration, seen_timestamp: Duration,
process_type: BlockProcessType, process_type: BlockProcessType,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>, ) -> (AsyncFn, BlockingFn) {
duplicate_cache: DuplicateCache, // An async closure which will import the block.
should_process: bool, let process_fn = self.clone().generate_rpc_beacon_block_process_fn(
) { block_root,
if !should_process { block,
seen_timestamp,
process_type.clone(),
);
// A closure which will ignore the block.
let ignore_fn = move || {
// Sync handles these results // Sync handles these results
self.send_sync_message(SyncMessage::BlockComponentProcessed { self.send_sync_message(SyncMessage::BlockComponentProcessed {
process_type, process_type,
result: crate::sync::manager::BlockProcessingResult::Ignored, result: crate::sync::manager::BlockProcessingResult::Ignored,
response_type: crate::sync::manager::ResponseType::Block, response_type: crate::sync::manager::ResponseType::Block,
}); });
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 // Check if the block is already being imported through another source
let handle = match duplicate_cache.check_and_insert(block_root) { let handle = match duplicate_cache.check_and_insert(block_root) {
Some(handle) => handle, Some(handle) => handle,
@ -75,13 +119,18 @@ impl<T: BeaconChainTypes> Worker<T> {
"action" => "sending rpc block to reprocessing queue", "action" => "sending rpc block to reprocessing queue",
"block_root" => %block_root, "block_root" => %block_root,
); );
// Send message to work reprocess queue to retry the block // 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_root,
block: block.clone(), block,
process_type,
seen_timestamp, 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() { if reprocess_tx.try_send(reprocess_msg).is_err() {
@ -135,12 +184,16 @@ impl<T: BeaconChainTypes> Worker<T> {
); );
// Send message to work reprocess queue to retry the block // 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_root,
block: block.clone(), block,
process_type,
seen_timestamp, 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() { if reprocess_tx.try_send(reprocess_msg).is_err() {

View File

@ -1,21 +1,22 @@
#![cfg(test)] #![cfg(test)]
use crate::beacon_processor::work_reprocessing_queue::{ use crate::{
QUEUED_ATTESTATION_DELAY, QUEUED_RPC_BLOCK_DELAY, 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::{ use beacon_chain::test_utils::{
test_spec, AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, test_spec, AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
}; };
use beacon_chain::{BeaconChain, ChainConfig, WhenSlotSkipped, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use lighthouse_network::discovery::ConnectionId; use beacon_processor::{work_reprocessing_queue::*, *};
use lighthouse_network::rpc::SubstreamId;
use lighthouse_network::{ use lighthouse_network::{
discv5::enr::{CombinedKey, EnrBuilder}, discv5::enr::{CombinedKey, EnrBuilder},
rpc::methods::{MetaData, MetaDataV2}, rpc::methods::{MetaData, MetaDataV2},
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield}, types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield},
MessageId, NetworkGlobals, PeerId, Response, Client, MessageId, NetworkGlobals, PeerId,
}; };
use slot_clock::SlotClock; use slot_clock::SlotClock;
use std::cmp; use std::cmp;
@ -24,8 +25,8 @@ use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{ use types::{
Attestation, AttesterSlashing, Epoch, MainnetEthSpec, ProposerSlashing, SignedBeaconBlock, Attestation, AttesterSlashing, Epoch, EthSpec, Hash256, MainnetEthSpec, ProposerSlashing,
SignedBlobSidecarList, SignedVoluntaryExit, Slot, SubnetId, SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
}; };
type E = MainnetEthSpec; type E = MainnetEthSpec;
@ -53,11 +54,12 @@ struct TestRig {
attester_slashing: AttesterSlashing<E>, attester_slashing: AttesterSlashing<E>,
proposer_slashing: ProposerSlashing, proposer_slashing: ProposerSlashing,
voluntary_exit: SignedVoluntaryExit, voluntary_exit: SignedVoluntaryExit,
beacon_processor_tx: mpsc::Sender<WorkEvent<T>>, beacon_processor_tx: BeaconProcessorSend<E>,
work_journal_rx: mpsc::Receiver<&'static str>, work_journal_rx: mpsc::Receiver<&'static str>,
_network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>, _network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
_sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>, _sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>,
duplicate_cache: DuplicateCache, duplicate_cache: DuplicateCache,
network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
_harness: BeaconChainHarness<T>, _harness: BeaconChainHarness<T>,
} }
@ -66,7 +68,7 @@ struct TestRig {
impl Drop for TestRig { impl Drop for TestRig {
fn drop(&mut self) { fn drop(&mut self) {
// Causes the beacon processor to shutdown. // 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);
} }
} }
@ -172,6 +174,7 @@ impl TestRig {
let log = harness.logger().clone(); let log = harness.logger().clone();
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); 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(); let (sync_tx, _sync_rx) = mpsc::unbounded_channel();
// Default metadata // Default metadata
@ -194,22 +197,40 @@ impl TestRig {
let executor = harness.runtime.task_executor.clone(); 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 (work_journal_tx, work_journal_rx) = mpsc::channel(16_364);
let duplicate_cache = DuplicateCache::default(); let duplicate_cache = DuplicateCache::default();
BeaconProcessor { let network_beacon_processor = NetworkBeaconProcessor {
beacon_chain: Arc::downgrade(&chain), beacon_processor_send: beacon_processor_tx.clone(),
duplicate_cache: duplicate_cache.clone(),
chain: harness.chain.clone(),
network_tx, network_tx,
sync_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, network_globals,
executor, executor,
max_workers: cmp::max(1, num_cpus::get()), max_workers: cmp::max(1, num_cpus::get()),
current_workers: 0, current_workers: 0,
importing_blocks: duplicate_cache.clone(), enable_backfill_rate_limiting: harness.chain.config.enable_backfill_rate_limiting,
invalid_block_storage: InvalidBlockStorage::Disabled,
log: log.clone(), 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 { Self {
chain, chain,
@ -226,6 +247,7 @@ impl TestRig {
_network_rx, _network_rx,
_sync_rx, _sync_rx,
duplicate_cache, duplicate_cache,
network_beacon_processor,
_harness: harness, _harness: harness,
} }
} }
@ -239,14 +261,14 @@ impl TestRig {
} }
pub fn enqueue_gossip_block(&self) { pub fn enqueue_gossip_block(&self) {
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::gossip_beacon_block( .send_gossip_beacon_block(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
Client::default(), Client::default(),
self.next_block.clone(), self.next_block.clone(),
Duration::from_secs(0), Duration::from_secs(0),
)) )
.unwrap(); .unwrap();
} }
@ -267,27 +289,28 @@ impl TestRig {
} }
pub fn enqueue_rpc_block(&self) { 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.canonical_root(),
self.next_block.clone().into(), self.next_block.clone().into(),
std::time::Duration::default(), std::time::Duration::default(),
BlockProcessType::ParentLookup { BlockProcessType::ParentLookup {
chain_hash: Hash256::random(), chain_hash: Hash256::random(),
}, },
); )
self.beacon_processor_tx.try_send(event).unwrap(); .unwrap();
} }
pub fn enqueue_single_lookup_rpc_block(&self) { 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.canonical_root(),
self.next_block.clone().into(), self.next_block.clone().into(),
std::time::Duration::default(), std::time::Duration::default(),
BlockProcessType::SingleBlock { id: 1 }, BlockProcessType::SingleBlock { id: 1 },
); )
self.beacon_processor_tx.try_send(event).unwrap(); .unwrap();
} }
pub fn enqueue_single_lookup_rpc_blobs(&self) { pub fn enqueue_single_lookup_rpc_blobs(&self) {
if let Some(blobs) = self.next_blobs.clone() { if let Some(blobs) = self.next_blobs.clone() {
let blobs = FixedBlobSidecarList::from( let blobs = FixedBlobSidecarList::from(
@ -319,68 +342,69 @@ impl TestRig {
} }
pub fn enqueue_backfill_batch(&self) { pub fn enqueue_backfill_batch(&self) {
let event = WorkEvent::chain_segment( self.network_beacon_processor
.send_chain_segment(
ChainSegmentProcessId::BackSyncBatchId(Epoch::default()), ChainSegmentProcessId::BackSyncBatchId(Epoch::default()),
Vec::default(), Vec::default(),
); )
self.beacon_processor_tx.try_send(event).unwrap(); .unwrap();
} }
pub fn enqueue_unaggregated_attestation(&self) { pub fn enqueue_unaggregated_attestation(&self) {
let (attestation, subnet_id) = self.attestations.first().unwrap().clone(); let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::unaggregated_attestation( .send_unaggregated_attestation(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
attestation, attestation,
subnet_id, subnet_id,
true, true,
Duration::from_secs(0), Duration::from_secs(0),
)) )
.unwrap(); .unwrap();
} }
pub fn enqueue_gossip_attester_slashing(&self) { pub fn enqueue_gossip_attester_slashing(&self) {
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::gossip_attester_slashing( .send_gossip_attester_slashing(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
Box::new(self.attester_slashing.clone()), Box::new(self.attester_slashing.clone()),
)) )
.unwrap(); .unwrap();
} }
pub fn enqueue_gossip_proposer_slashing(&self) { pub fn enqueue_gossip_proposer_slashing(&self) {
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::gossip_proposer_slashing( .send_gossip_proposer_slashing(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
Box::new(self.proposer_slashing.clone()), Box::new(self.proposer_slashing.clone()),
)) )
.unwrap(); .unwrap();
} }
pub fn enqueue_gossip_voluntary_exit(&self) { pub fn enqueue_gossip_voluntary_exit(&self) {
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::gossip_voluntary_exit( .send_gossip_voluntary_exit(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
Box::new(self.voluntary_exit.clone()), Box::new(self.voluntary_exit.clone()),
)) )
.unwrap(); .unwrap();
} }
pub fn enqueue_next_block_unaggregated_attestation(&self) { pub fn enqueue_next_block_unaggregated_attestation(&self) {
let (attestation, subnet_id) = self.next_block_attestations.first().unwrap().clone(); let (attestation, subnet_id) = self.next_block_attestations.first().unwrap().clone();
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::unaggregated_attestation( .send_unaggregated_attestation(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
attestation, attestation,
subnet_id, subnet_id,
true, true,
Duration::from_secs(0), Duration::from_secs(0),
)) )
.unwrap(); .unwrap();
} }
@ -390,13 +414,13 @@ impl TestRig {
.first() .first()
.unwrap() .unwrap()
.clone(); .clone();
self.beacon_processor_tx self.network_beacon_processor
.try_send(WorkEvent::aggregated_attestation( .send_aggregated_attestation(
junk_message_id(), junk_message_id(),
junk_peer_id(), junk_peer_id(),
aggregate, aggregate,
Duration::from_secs(0), Duration::from_secs(0),
)) )
.unwrap(); .unwrap();
} }

View File

@ -5,16 +5,16 @@
//! syncing-related responses to the Sync manager. //! syncing-related responses to the Sync manager.
#![allow(clippy::unit_arg)] #![allow(clippy::unit_arg)]
use crate::beacon_processor::{
BeaconProcessor, BeaconProcessorSend, InvalidBlockStorage, WorkEvent as BeaconWorkEvent,
MAX_WORK_EVENT_QUEUE_LEN,
};
use crate::error; use crate::error;
use crate::network_beacon_processor::{InvalidBlockStorage, NetworkBeaconProcessor};
use crate::service::{NetworkMessage, RequestId}; use crate::service::{NetworkMessage, RequestId};
use crate::status::status_message; use crate::status::status_message;
use crate::sync::manager::RequestId as SyncId; use crate::sync::manager::RequestId as SyncId;
use crate::sync::SyncMessage; use crate::sync::SyncMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use beacon_processor::{
work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache,
};
use futures::prelude::*; use futures::prelude::*;
use lighthouse_network::rpc::*; use lighthouse_network::rpc::*;
use lighthouse_network::{ use lighthouse_network::{
@ -23,7 +23,6 @@ use lighthouse_network::{
use logging::TimeLatch; use logging::TimeLatch;
use slog::{debug, o, trace}; use slog::{debug, o, trace};
use slog::{error, warn}; use slog::{error, warn};
use std::cmp;
use std::sync::Arc; use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH}; use std::time::{Duration, SystemTime, UNIX_EPOCH};
use tokio::sync::mpsc; use tokio::sync::mpsc;
@ -41,7 +40,7 @@ pub struct Router<T: BeaconChainTypes> {
/// A network context to return and handle RPC requests. /// A network context to return and handle RPC requests.
network: HandlerNetworkContext<T::EthSpec>, network: HandlerNetworkContext<T::EthSpec>,
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain. /// 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. /// The `Router` logger.
log: slog::Logger, log: slog::Logger,
/// Provides de-bounce functionality for logging. /// Provides de-bounce functionality for logging.
@ -81,12 +80,15 @@ pub enum RouterMessage<T: EthSpec> {
impl<T: BeaconChainTypes> Router<T> { impl<T: BeaconChainTypes> Router<T> {
/// Initializes and runs the Router. /// Initializes and runs the Router.
#[allow(clippy::too_many_arguments)]
pub fn spawn( pub fn spawn(
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>, network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>, network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
executor: task_executor::TaskExecutor, executor: task_executor::TaskExecutor,
invalid_block_storage: InvalidBlockStorage, invalid_block_storage: InvalidBlockStorage,
beacon_processor_send: BeaconProcessorSend<T::EthSpec>,
beacon_processor_reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
log: slog::Logger, log: slog::Logger,
) -> error::Result<mpsc::UnboundedSender<RouterMessage<T::EthSpec>>> { ) -> error::Result<mpsc::UnboundedSender<RouterMessage<T::EthSpec>>> {
let message_handler_log = log.new(o!("service"=> "router")); let message_handler_log = log.new(o!("service"=> "router"));
@ -94,34 +96,33 @@ impl<T: BeaconChainTypes> Router<T> {
let (handler_send, handler_recv) = mpsc::unbounded_channel(); 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")); 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 network_beacon_processor = NetworkBeaconProcessor {
let sync_send = crate::sync::manager::spawn( beacon_processor_send,
executor.clone(), duplicate_cache: DuplicateCache::default(),
beacon_chain.clone(), chain: beacon_chain.clone(),
network_globals.clone(),
network_send.clone(),
BeaconProcessorSend(beacon_processor_send.clone()),
sync_logger,
);
BeaconProcessor {
beacon_chain: Arc::downgrade(&beacon_chain),
network_tx: network_send.clone(), network_tx: network_send.clone(),
sync_tx: sync_send.clone(), sync_tx: sync_send.clone(),
reprocess_tx: beacon_processor_reprocess_tx,
network_globals: network_globals.clone(), network_globals: network_globals.clone(),
executor: executor.clone(),
max_workers: cmp::max(1, num_cpus::get()),
current_workers: 0,
importing_blocks: Default::default(),
invalid_block_storage, invalid_block_storage,
executor: executor.clone(),
log: log.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 // generate the Message handler
let mut handler = Router { let mut handler = Router {
@ -129,7 +130,7 @@ impl<T: BeaconChainTypes> Router<T> {
chain: beacon_chain, chain: beacon_chain,
sync_send, sync_send,
network: HandlerNetworkContext::new(network_send, log.clone()), network: HandlerNetworkContext::new(network_send, log.clone()),
beacon_processor_send: BeaconProcessorSend(beacon_processor_send), network_beacon_processor,
log: message_handler_log, log: message_handler_log,
logger_debounce: TimeLatch::default(), logger_debounce: TimeLatch::default(),
}; };
@ -199,11 +200,13 @@ impl<T: BeaconChainTypes> Router<T> {
Request::Status(status_message) => { Request::Status(status_message) => {
self.on_status_request(peer_id, request_id, status_message) self.on_status_request(peer_id, request_id, status_message)
} }
Request::BlocksByRange(request) => self.send_beacon_processor_work( Request::BlocksByRange(request) => self.handle_beacon_processor_send_result(
BeaconWorkEvent::blocks_by_range_request(peer_id, request_id, request), self.network_beacon_processor
.send_blocks_by_range_request(peer_id, request_id, request),
), ),
Request::BlocksByRoot(request) => self.send_beacon_processor_work( Request::BlocksByRoot(request) => self.handle_beacon_processor_send_result(
BeaconWorkEvent::blocks_by_roots_request(peer_id, request_id, request), self.network_beacon_processor
.send_blocks_by_roots_request(peer_id, request_id, request),
), ),
Request::BlobsByRange(request) => self.send_beacon_processor_work( Request::BlobsByRange(request) => self.send_beacon_processor_work(
BeaconWorkEvent::blobs_by_range_request(peer_id, request_id, request), BeaconWorkEvent::blobs_by_range_request(peer_id, request_id, request),
@ -211,8 +214,9 @@ impl<T: BeaconChainTypes> Router<T> {
Request::BlobsByRoot(request) => self.send_beacon_processor_work( Request::BlobsByRoot(request) => self.send_beacon_processor_work(
BeaconWorkEvent::blobs_by_root_request(peer_id, request_id, request), BeaconWorkEvent::blobs_by_root_request(peer_id, request_id, request),
), ),
Request::LightClientBootstrap(request) => self.send_beacon_processor_work( Request::LightClientBootstrap(request) => self.handle_beacon_processor_send_result(
BeaconWorkEvent::lightclient_bootstrap_request(peer_id, request_id, request), self.network_beacon_processor
.send_lightclient_bootstrap_request(peer_id, request_id, request),
), ),
} }
} }
@ -227,10 +231,10 @@ impl<T: BeaconChainTypes> Router<T> {
match response { match response {
Response::Status(status_message) => { Response::Status(status_message) => {
debug!(self.log, "Received Status Response"; "peer_id" => %peer_id, &status_message); debug!(self.log, "Received Status Response"; "peer_id" => %peer_id, &status_message);
self.send_beacon_processor_work(BeaconWorkEvent::status_message( self.handle_beacon_processor_send_result(
peer_id, self.network_beacon_processor
status_message, .send_status_message(peer_id, status_message),
)) )
} }
Response::BlocksByRange(beacon_block) => { Response::BlocksByRange(beacon_block) => {
self.on_blocks_by_range_response(peer_id, request_id, beacon_block); self.on_blocks_by_range_response(peer_id, request_id, beacon_block);
@ -261,24 +265,27 @@ impl<T: BeaconChainTypes> Router<T> {
) { ) {
match gossip_message { match gossip_message {
PubsubMessage::AggregateAndProofAttestation(aggregate_and_proof) => self 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, message_id,
peer_id, peer_id,
*aggregate_and_proof, *aggregate_and_proof,
timestamp_now(), 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, message_id,
peer_id, peer_id,
subnet_attestation.1, subnet_attestation.1,
subnet_attestation.0, subnet_attestation.0,
should_process, should_process,
timestamp_now(), timestamp_now(),
)) ),
} ),
PubsubMessage::BeaconBlock(block) => { PubsubMessage::BeaconBlock(block) => self.handle_beacon_processor_send_result(
self.send_beacon_processor_work(BeaconWorkEvent::gossip_beacon_block( self.network_beacon_processor.send_gossip_beacon_block(
message_id, message_id,
peer_id, peer_id,
self.network_globals.client(&peer_id), self.network_globals.client(&peer_id),
@ -300,9 +307,10 @@ impl<T: BeaconChainTypes> Router<T> {
} }
PubsubMessage::VoluntaryExit(exit) => { PubsubMessage::VoluntaryExit(exit) => {
debug!(self.log, "Received a voluntary exit"; "peer_id" => %peer_id); debug!(self.log, "Received a voluntary exit"; "peer_id" => %peer_id);
self.send_beacon_processor_work(BeaconWorkEvent::gossip_voluntary_exit( self.handle_beacon_processor_send_result(
message_id, peer_id, exit, self.network_beacon_processor
)) .send_gossip_voluntary_exit(message_id, peer_id, exit),
)
} }
PubsubMessage::ProposerSlashing(proposer_slashing) => { PubsubMessage::ProposerSlashing(proposer_slashing) => {
debug!( debug!(
@ -310,11 +318,13 @@ impl<T: BeaconChainTypes> Router<T> {
"Received a proposer slashing"; "Received a proposer slashing";
"peer_id" => %peer_id "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, message_id,
peer_id, peer_id,
proposer_slashing, proposer_slashing,
)) ),
)
} }
PubsubMessage::AttesterSlashing(attester_slashing) => { PubsubMessage::AttesterSlashing(attester_slashing) => {
debug!( debug!(
@ -322,11 +332,13 @@ impl<T: BeaconChainTypes> Router<T> {
"Received a attester slashing"; "Received a attester slashing";
"peer_id" => %peer_id "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, message_id,
peer_id, peer_id,
attester_slashing, attester_slashing,
)) ),
)
} }
PubsubMessage::SignedContributionAndProof(contribution_and_proof) => { PubsubMessage::SignedContributionAndProof(contribution_and_proof) => {
trace!( trace!(
@ -334,12 +346,14 @@ impl<T: BeaconChainTypes> Router<T> {
"Received sync committee aggregate"; "Received sync committee aggregate";
"peer_id" => %peer_id "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, message_id,
peer_id, peer_id,
*contribution_and_proof, *contribution_and_proof,
timestamp_now(), timestamp_now(),
)) ),
)
} }
PubsubMessage::SyncCommitteeMessage(sync_committtee_msg) => { PubsubMessage::SyncCommitteeMessage(sync_committtee_msg) => {
trace!( trace!(
@ -347,13 +361,15 @@ impl<T: BeaconChainTypes> Router<T> {
"Received sync committee signature"; "Received sync committee signature";
"peer_id" => %peer_id "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, message_id,
peer_id, peer_id,
sync_committtee_msg.1, sync_committtee_msg.1,
sync_committtee_msg.0, sync_committtee_msg.0,
timestamp_now(), timestamp_now(),
)) ),
)
} }
PubsubMessage::LightClientFinalityUpdate(light_client_finality_update) => { PubsubMessage::LightClientFinalityUpdate(light_client_finality_update) => {
trace!( trace!(
@ -361,11 +377,12 @@ impl<T: BeaconChainTypes> Router<T> {
"Received light client finality update"; "Received light client finality update";
"peer_id" => %peer_id "peer_id" => %peer_id
); );
self.send_beacon_processor_work( self.handle_beacon_processor_send_result(
BeaconWorkEvent::gossip_light_client_finality_update( self.network_beacon_processor
.send_gossip_light_client_finality_update(
message_id, message_id,
peer_id, peer_id,
light_client_finality_update, *light_client_finality_update,
timestamp_now(), timestamp_now(),
), ),
) )
@ -376,21 +393,25 @@ impl<T: BeaconChainTypes> Router<T> {
"Received light client optimistic update"; "Received light client optimistic update";
"peer_id" => %peer_id "peer_id" => %peer_id
); );
self.send_beacon_processor_work( self.handle_beacon_processor_send_result(
BeaconWorkEvent::gossip_light_client_optimistic_update( self.network_beacon_processor
.send_gossip_light_client_optimistic_update(
message_id, message_id,
peer_id, peer_id,
light_client_optimistic_update, *light_client_optimistic_update,
timestamp_now(), timestamp_now(),
), ),
) )
} }
PubsubMessage::BlsToExecutionChange(bls_to_execution_change) => self 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, message_id,
peer_id, peer_id,
bls_to_execution_change, bls_to_execution_change,
)), ),
),
} }
} }
@ -442,7 +463,10 @@ impl<T: BeaconChainTypes> Router<T> {
request_id, 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. /// Handle a `BlocksByRange` response from the peer.
@ -573,20 +597,22 @@ impl<T: BeaconChainTypes> Router<T> {
}); });
} }
fn send_beacon_processor_work(&mut self, work: BeaconWorkEvent<T>) { fn handle_beacon_processor_send_result(
self.beacon_processor_send &mut self,
.try_send(work) result: Result<(), crate::network_beacon_processor::Error<T::EthSpec>>,
.unwrap_or_else(|e| { ) {
let work_type = match &*e { if let Err(e) = result {
mpsc::error::TrySendError::Closed(work) let work_type = match &e {
| mpsc::error::TrySendError::Full(work) => work.work_type(), mpsc::error::TrySendError::Closed(work) | mpsc::error::TrySendError::Full(work) => {
work.work_type()
}
}; };
if self.logger_debounce.elapsed() { if self.logger_debounce.elapsed() {
error!(&self.log, "Unable to send message to the beacon processor"; error!(&self.log, "Unable to send message to the beacon processor";
"error" => %e, "type" => work_type) "error" => %e, "type" => work_type)
} }
}) }
} }
} }

View File

@ -1,5 +1,5 @@
use super::sync::manager::RequestId as SyncId; 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::persisted_dht::{clear_dht, load_dht, persist_dht};
use crate::router::{Router, RouterMessage}; use crate::router::{Router, RouterMessage};
use crate::subnet_service::SyncCommitteeService; use crate::subnet_service::SyncCommitteeService;
@ -9,6 +9,7 @@ use crate::{
NetworkConfig, NetworkConfig,
}; };
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use beacon_processor::{work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend};
use futures::channel::mpsc::Sender; use futures::channel::mpsc::Sender;
use futures::future::OptionFuture; use futures::future::OptionFuture;
use futures::prelude::*; use futures::prelude::*;
@ -224,6 +225,8 @@ impl<T: BeaconChainTypes> NetworkService<T> {
config: &NetworkConfig, config: &NetworkConfig,
executor: task_executor::TaskExecutor, executor: task_executor::TaskExecutor,
gossipsub_registry: Option<&'_ mut Registry>, 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>)> { ) -> error::Result<(Arc<NetworkGlobals<T::EthSpec>>, NetworkSenders<T::EthSpec>)> {
let network_log = executor.log().clone(); let network_log = executor.log().clone();
// build the channels for external comms // build the channels for external comms
@ -311,6 +314,8 @@ impl<T: BeaconChainTypes> NetworkService<T> {
network_senders.network_send(), network_senders.network_send(),
executor.clone(), executor.clone(),
invalid_block_storage, invalid_block_storage,
beacon_processor_send,
beacon_processor_reprocess_tx,
network_log.clone(), network_log.clone(),
)?; )?;

View File

@ -3,12 +3,17 @@
mod tests { mod tests {
use crate::persisted_dht::load_dht; use crate::persisted_dht::load_dht;
use crate::{NetworkConfig, NetworkService}; use crate::{NetworkConfig, NetworkService};
use beacon_chain::test_utils::EphemeralHarnessType; 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 lighthouse_network::Enr;
use slog::{o, Drain, Level, Logger}; use slog::{o, Drain, Level, Logger};
use sloggers::{null::NullLoggerBuilder, Build}; use sloggers::{null::NullLoggerBuilder, Build};
use std::str::FromStr; use std::str::FromStr;
use std::sync::Arc; use std::sync::Arc;
use tokio::{runtime::Runtime, sync::mpsc};
use types::MinimalEthSpec;
use tokio::runtime::Runtime; use tokio::runtime::Runtime;
use types::MinimalEthSpec as E; use types::MinimalEthSpec as E;
@ -69,8 +74,18 @@ mod tests {
// Create a new network service which implicitly gets dropped at the // Create a new network service which implicitly gets dropped at the
// end of the block. // end of the block.
let _network_service = let (beacon_processor_send, _beacon_processor_receive) =
NetworkService::start(beacon_chain.clone(), &config, executor, None) 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 .await
.unwrap(); .unwrap();
drop(signal); drop(signal);

View File

@ -8,7 +8,7 @@
//! If a batch fails, the backfill sync cannot progress. In this scenario, we mark the backfill //! 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. //! 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::manager::{BatchProcessResult, Id};
use crate::sync::network_context::SyncNetworkContext; use crate::sync::network_context::SyncNetworkContext;
use crate::sync::range_sync::{ use crate::sync::range_sync::{
@ -537,8 +537,10 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
let process_id = ChainSegmentProcessId::BackSyncBatchId(batch_id); let process_id = ChainSegmentProcessId::BackSyncBatchId(batch_id);
self.current_processing_batch = Some(batch_id); self.current_processing_batch = Some(batch_id);
let work_event = BeaconWorkEvent::chain_segment(process_id, blocks); if let Err(e) = network
if let Err(e) = network.processor_channel().try_send(work_event) { .beacon_processor()
.send_chain_segment(process_id, blocks)
{
crit!(self.log, "Failed to send backfill segment to processor."; "msg" => "process_batch", crit!(self.log, "Failed to send backfill segment to processor."; "msg" => "process_batch",
"error" => %e, "batch" => self.processing_target); "error" => %e, "batch" => self.processing_target);
// This is unlikely to happen but it would stall syncing since the batch now has no // This is unlikely to happen but it would stall syncing since the batch now has no
@ -1097,12 +1099,7 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
match self.batches.entry(batch_id) { match self.batches.entry(batch_id) {
Entry::Occupied(_) => { Entry::Occupied(_) => {
// this batch doesn't need downloading, let this same function decide the next batch // this batch doesn't need downloading, let this same function decide the next batch
if batch_id if self.would_complete(batch_id) {
== self
.beacon_chain
.genesis_backfill_slot
.epoch(T::EthSpec::slots_per_epoch())
{
self.last_batch_downloaded = true; self.last_batch_downloaded = true;
} }
@ -1118,12 +1115,7 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
BACKFILL_EPOCHS_PER_BATCH, BACKFILL_EPOCHS_PER_BATCH,
batch_type, batch_type,
)); ));
if batch_id if self.would_complete(batch_id) {
== self
.beacon_chain
.genesis_backfill_slot
.epoch(T::EthSpec::slots_per_epoch())
{
self.last_batch_downloaded = true; self.last_batch_downloaded = true;
} }
self.to_be_downloaded = self self.to_be_downloaded = self
@ -1155,14 +1147,8 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
/// Checks with the beacon chain if backfill sync has completed. /// Checks with the beacon chain if backfill sync has completed.
fn check_completed(&mut self) -> bool { fn check_completed(&mut self) -> bool {
if self.current_start if self.would_complete(self.current_start) {
== self
.beacon_chain
.genesis_backfill_slot
.epoch(T::EthSpec::slots_per_epoch())
{
// Check that the beacon chain agrees // Check that the beacon chain agrees
if let Some(anchor_info) = self.beacon_chain.store.get_anchor_info() { if let Some(anchor_info) = self.beacon_chain.store.get_anchor_info() {
// Conditions that we have completed a backfill sync // Conditions that we have completed a backfill sync
if anchor_info.block_backfill_complete(self.beacon_chain.genesis_backfill_slot) { if anchor_info.block_backfill_complete(self.beacon_chain.genesis_backfill_slot) {
@ -1175,6 +1161,15 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
false false
} }
/// Checks if backfill would complete by syncing to `start_epoch`.
fn would_complete(&self, start_epoch: Epoch) -> bool {
start_epoch
<= self
.beacon_chain
.genesis_backfill_slot
.epoch(T::EthSpec::slots_per_epoch())
}
/// Updates the global network state indicating the current state of a backfill sync. /// Updates the global network state indicating the current state of a backfill sync.
fn set_state(&self, state: BackFillState) { fn set_state(&self, state: BackFillState) {
*self.network_globals.backfill_state.write() = state; *self.network_globals.backfill_state.write() = state;

View File

@ -1,7 +1,10 @@
use beacon_chain::blob_verification::{AsBlock, BlockWrapper}; use std::collections::hash_map::Entry;
use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use std::collections::HashMap;
use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; use std::time::Duration;
use lighthouse_network::rpc::RPCError;
use crate::network_beacon_processor::ChainSegmentProcessId;
use beacon_chain::{BeaconChainTypes, BlockError};
use fnv::FnvHashMap;
use lighthouse_network::{PeerAction, PeerId}; use lighthouse_network::{PeerAction, PeerId};
use lru_cache::LRUTimeCache; use lru_cache::LRUTimeCache;
use slog::{debug, error, trace, warn, Logger}; use slog::{debug, error, trace, warn, Logger};
@ -9,11 +12,9 @@ use smallvec::SmallVec;
use std::collections::HashMap; use std::collections::HashMap;
use std::fmt::Debug; use std::fmt::Debug;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use store::{Hash256, SignedBeaconBlock};
use store::Hash256;
use strum::Display; use crate::metrics;
use types::blob_sidecar::FixedBlobSidecarList;
use types::{BlobSidecar, SignedBeaconBlock, Slot};
use self::parent_lookup::PARENT_FAIL_TOLERANCE; use self::parent_lookup::PARENT_FAIL_TOLERANCE;
use self::parent_lookup::{ParentLookup, ParentVerifyError}; use self::parent_lookup::{ParentLookup, ParentVerifyError};
@ -1054,8 +1055,8 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(_)) BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(_))
| BlockProcessingResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => { | BlockProcessingResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => {
// Check if the beacon processor is available // Check if the beacon processor is available
let beacon_processor_send = match cx.processor_channel_if_enabled() { let beacon_processor = match cx.beacon_processor_if_enabled() {
Some(channel) => channel, Some(beacon_processor) => beacon_processor,
None => { None => {
return trace!( return trace!(
self.log, self.log,
@ -1078,7 +1079,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
let process_id = ChainSegmentProcessId::ParentLookup(chain_hash); let process_id = ChainSegmentProcessId::ParentLookup(chain_hash);
let work = WorkEvent::chain_segment(process_id, blocks); let work = WorkEvent::chain_segment(process_id, blocks);
match beacon_processor_send.try_send(work) { match beacon_processor.send_chain_segment(process_id, blocks) {
Ok(_) => { Ok(_) => {
self.processing_parent_lookups self.processing_parent_lookups
.insert(chain_hash, (hashes, block_request)); .insert(chain_hash, (hashes, block_request));
@ -1235,11 +1236,15 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
process_type: BlockProcessType, process_type: BlockProcessType,
cx: &mut SyncNetworkContext<T>, cx: &mut SyncNetworkContext<T>,
) -> Result<(), ()> { ) -> Result<(), ()> {
match cx.processor_channel_if_enabled() { match cx.beacon_processor_if_enabled() {
Some(beacon_processor_send) => { Some(beacon_processor) => {
trace!(self.log, "Sending block for processing"; "block" => ?block_root, "process" => ?process_type); 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_rpc_beacon_block(
if let Err(e) = beacon_processor_send.try_send(event) { block_root,
block,
duration,
process_type,
) {
error!( error!(
self.log, self.log,
"Failed to send sync block to processor"; "Failed to send sync block to processor";

View File

@ -1,24 +1,19 @@
#![cfg(feature = "spec-minimal")] #![cfg(feature = "spec-minimal")]
use std::sync::Arc; use std::sync::Arc;
use crate::beacon_processor::BeaconProcessorSend; use crate::network_beacon_processor::NetworkBeaconProcessor;
use crate::service::RequestId; use crate::service::RequestId;
use crate::sync::manager::RequestId as SyncId; use crate::sync::manager::RequestId as SyncId;
use crate::NetworkMessage; use crate::NetworkMessage;
use super::*; use super::*;
use beacon_chain::{ use beacon_chain::builder::Witness;
builder::Witness, use beacon_chain::eth1_chain::CachingEth1Backend;
eth1_chain::CachingEth1Backend, use beacon_processor::WorkEvent;
test_utils::{build_log, BeaconChainHarness, EphemeralHarnessType},
};
use execution_layer::BlobsBundleV1;
pub use genesis::{interop_genesis_state, DEFAULT_ETH1_BLOCK_HASH};
use lighthouse_network::rpc::RPCResponseErrorCode;
use lighthouse_network::{NetworkGlobals, Request}; use lighthouse_network::{NetworkGlobals, Request};
use slot_clock::{SlotClock, TestingSlotClock}; use slog::{Drain, Level};
use std::time::Duration; use slot_clock::ManualSlotClock;
use store::MemoryStore; use store::MemoryStore;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{ use types::{
@ -27,10 +22,10 @@ use types::{
BeaconBlock, EthSpec, ForkName, FullPayloadDeneb, MinimalEthSpec as E, SignedBeaconBlock, BeaconBlock, EthSpec, ForkName, FullPayloadDeneb, MinimalEthSpec as E, SignedBeaconBlock,
}; };
type T = Witness<TestingSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>; type T = Witness<ManualSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
struct TestRig { struct TestRig {
beacon_processor_rx: mpsc::Receiver<WorkEvent<T>>, beacon_processor_rx: mpsc::Receiver<WorkEvent<E>>,
network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>, network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
rng: XorShiftRng, rng: XorShiftRng,
harness: BeaconChainHarness<T>, harness: BeaconChainHarness<T>,
@ -62,8 +57,10 @@ impl TestRig {
let chain = harness.chain.clone(); let chain = harness.chain.clone();
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(100);
let (network_tx, network_rx) = mpsc::unbounded_channel(); 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 rng = XorShiftRng::from_seed([42; 16]);
let rig = TestRig { let rig = TestRig {
beacon_processor_rx, beacon_processor_rx,
@ -77,11 +74,9 @@ impl TestRig {
log.new(slog::o!("component" => "block_lookups")), log.new(slog::o!("component" => "block_lookups")),
); );
let cx = { let cx = {
let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log));
SyncNetworkContext::new( SyncNetworkContext::new(
network_tx, network_tx,
globals, Arc::new(network_beacon_processor),
BeaconProcessorSend(beacon_processor_tx),
chain, chain,
log.new(slog::o!("component" => "network_context")), log.new(slog::o!("component" => "network_context")),
) )
@ -211,13 +206,13 @@ impl TestRig {
match response_type { match response_type {
ResponseType::Block => match self.beacon_processor_rx.try_recv() { ResponseType::Block => match self.beacon_processor_rx.try_recv() {
Ok(work) => { 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), other => panic!("Expected block process, found {:?}", other),
}, },
ResponseType::Blob => match self.beacon_processor_rx.try_recv() { ResponseType::Blob => match self.beacon_processor_rx.try_recv() {
Ok(work) => { Ok(work) => {
assert_eq!(work.work_type(), crate::beacon_processor::RPC_BLOB); assert_eq!(work.work_type(), beacon_processor::RPC_BLOB);
} }
other => panic!("Expected blob process, found {:?}", other), other => panic!("Expected blob process, found {:?}", other),
}, },
@ -228,7 +223,7 @@ impl TestRig {
fn expect_parent_chain_process(&mut self) { fn expect_parent_chain_process(&mut self) {
match self.beacon_processor_rx.try_recv() { match self.beacon_processor_rx.try_recv() {
Ok(work) => { 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), other => panic!("Expected chain segment process, found {:?}", other),
} }

View File

@ -38,7 +38,7 @@ use super::block_lookups::{BlockLookups, PeerShouldHave};
use super::network_context::{BlockOrBlob, SyncNetworkContext}; use super::network_context::{BlockOrBlob, SyncNetworkContext};
use super::peer_sync_info::{remote_sync_type, PeerSyncType}; use super::peer_sync_info::{remote_sync_type, PeerSyncType};
use super::range_sync::{RangeSync, RangeSyncType, EPOCHS_PER_BATCH}; 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::service::NetworkMessage;
use crate::status::ToStatusMessage; use crate::status::ToStatusMessage;
use crate::sync::block_lookups::delayed_lookup; use crate::sync::block_lookups::delayed_lookup;
@ -207,9 +207,6 @@ pub struct SyncManager<T: BeaconChainTypes> {
/// A reference to the underlying beacon chain. /// A reference to the underlying beacon chain.
chain: Arc<BeaconChain<T>>, 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. /// A receiving channel sent by the message processor thread.
input_channel: mpsc::UnboundedReceiver<SyncMessage<T::EthSpec>>, input_channel: mpsc::UnboundedReceiver<SyncMessage<T::EthSpec>>,
@ -236,32 +233,22 @@ pub struct SyncManager<T: BeaconChainTypes> {
pub fn spawn<T: BeaconChainTypes>( pub fn spawn<T: BeaconChainTypes>(
executor: task_executor::TaskExecutor, executor: task_executor::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<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, log: slog::Logger,
) -> mpsc::UnboundedSender<SyncMessage<T::EthSpec>> { ) {
assert!( assert!(
MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, 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" "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>>();
let (delayed_lookups_send, delayed_lookups_recv) =
mpsc::channel::<DelayedLookupMessage>(DELAY_QUEUE_CHANNEL_SIZE);
// create an instance of the SyncManager // create an instance of the SyncManager
let network_globals = beacon_processor.network_globals.clone();
let mut sync_manager = SyncManager { let mut sync_manager = SyncManager {
chain: beacon_chain.clone(), chain: beacon_chain.clone(),
network_globals: network_globals.clone(),
input_channel: sync_recv, input_channel: sync_recv,
network: SyncNetworkContext::new( network: SyncNetworkContext::new(network_send, beacon_processor, log.clone()),
network_send,
network_globals.clone(),
beacon_processor_send,
beacon_chain.clone(),
log.clone(),
),
range_sync: RangeSync::new(beacon_chain.clone(), log.clone()), range_sync: RangeSync::new(beacon_chain.clone(), log.clone()),
backfill_sync: BackFillSync::new(beacon_chain.clone(), network_globals, log.clone()), backfill_sync: BackFillSync::new(beacon_chain.clone(), network_globals, log.clone()),
block_lookups: BlockLookups::new( block_lookups: BlockLookups::new(
@ -285,10 +272,13 @@ pub fn spawn<T: BeaconChainTypes>(
// spawn the sync manager thread // spawn the sync manager thread
debug!(log_clone, "Sync Manager started"); debug!(log_clone, "Sync Manager started");
executor.spawn(async move { Box::pin(sync_manager.main()).await }, "sync"); executor.spawn(async move { Box::pin(sync_manager.main()).await }, "sync");
sync_send_clone
} }
impl<T: BeaconChainTypes> SyncManager<T> { impl<T: BeaconChainTypes> SyncManager<T> {
fn network_globals(&self) -> &NetworkGlobals<T::EthSpec> {
self.network.network_globals()
}
/* Input Handling Functions */ /* Input Handling Functions */
/// A peer has connected which has blocks that are unknown to us. /// A peer has connected which has blocks that are unknown to us.
@ -429,12 +419,12 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let rpr = new_state.as_str(); let rpr = new_state.as_str();
// Drop the write lock // Drop the write lock
let update_sync_status = self let update_sync_status = self
.network_globals .network_globals()
.peers .peers
.write() .write()
.update_sync_status(peer_id, new_state.clone()); .update_sync_status(peer_id, new_state.clone());
if let Some(was_updated) = update_sync_status { 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 { if was_updated {
debug!( debug!(
self.log, self.log,
@ -490,7 +480,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let head = self.chain.best_slot(); let head = self.chain.best_slot();
let current_slot = self.chain.slot().unwrap_or_else(|_| Slot::new(0)); 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 if current_slot >= head
&& current_slot.sub(head) <= (SLOT_IMPORT_TOLERANCE as u64) && current_slot.sub(head) <= (SLOT_IMPORT_TOLERANCE as u64)
&& head > 0 && head > 0
@ -552,8 +542,8 @@ impl<T: BeaconChainTypes> SyncManager<T> {
}, },
}; };
let old_state = self.network_globals.set_sync_state(new_state); let old_state = self.network_globals().set_sync_state(new_state);
let new_state = self.network_globals.sync_state.read(); let new_state = self.network_globals().sync_state.read().clone();
if !new_state.eq(&old_state) { if !new_state.eq(&old_state) {
info!(self.log, "Sync state updated"; "old_state" => %old_state, "new_state" => %new_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 // If we have become synced - Subscribe to all the core subnet topics

View File

@ -5,6 +5,7 @@ use super::block_sidecar_coupling::BlocksAndBlobsRequestInfo;
use super::manager::{Id, RequestId as SyncRequestId}; use super::manager::{Id, RequestId as SyncRequestId};
use super::range_sync::{BatchId, ByRangeRequestType, ChainId}; use super::range_sync::{BatchId, ByRangeRequestType, ChainId};
use crate::beacon_processor::BeaconProcessorSend; use crate::beacon_processor::BeaconProcessorSend;
duse crate::network_beacon_processor::NetworkBeaconProcessor;
use crate::service::{NetworkMessage, RequestId}; use crate::service::{NetworkMessage, RequestId};
use crate::status::ToStatusMessage; use crate::status::ToStatusMessage;
use crate::sync::block_lookups::{BlobRequestId, BlockRequestId}; use crate::sync::block_lookups::{BlobRequestId, BlockRequestId};
@ -36,9 +37,6 @@ pub struct SyncNetworkContext<T: BeaconChainTypes> {
/// The network channel to relay messages to the Network service. /// The network channel to relay messages to the Network service.
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>, 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. /// A sequential ID for all RPC requests.
request_id: Id, request_id: Id,
@ -59,8 +57,8 @@ pub struct SyncNetworkContext<T: BeaconChainTypes> {
/// `beacon_processor_send`. /// `beacon_processor_send`.
execution_engine_state: EngineState, execution_engine_state: EngineState,
/// Channel to send work to the beacon processor. /// Sends work to the beacon processor via a channel.
beacon_processor_send: BeaconProcessorSend<T>, network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
pub chain: Arc<BeaconChain<T>>, pub chain: Arc<BeaconChain<T>>,
@ -89,29 +87,29 @@ impl<T: EthSpec> From<Option<Arc<BlobSidecar<T>>>> for BlockOrBlob<T> {
impl<T: BeaconChainTypes> SyncNetworkContext<T> { impl<T: BeaconChainTypes> SyncNetworkContext<T> {
pub fn new( pub fn new(
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>, network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>, network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
beacon_processor_send: BeaconProcessorSend<T>,
chain: Arc<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
log: slog::Logger, log: slog::Logger,
) -> Self { ) -> Self {
SyncNetworkContext { SyncNetworkContext {
network_send, network_send,
network_globals,
request_id: 1,
range_requests: Default::default(),
backfill_requests: Default::default(),
range_blocks_and_blobs_requests: Default::default(),
backfill_blocks_and_blobs_requests: Default::default(),
execution_engine_state: EngineState::Online, // always assume `Online` at the start execution_engine_state: EngineState::Online, // always assume `Online` at the start
beacon_processor_send, request_id: 1,
range_requests: FnvHashMap::default(),
backfill_requests: FnvHashMap::default(),
network_beacon_processor,
chain, chain,
log, log,
} }
} }
pub fn network_globals(&self) -> &NetworkGlobals<T::EthSpec> {
&self.network_beacon_processor.network_globals
}
/// Returns the Client type of the peer if known /// Returns the Client type of the peer if known
pub fn client_type(&self, peer_id: &PeerId) -> Client { pub fn client_type(&self, peer_id: &PeerId) -> Client {
self.network_globals self.network_globals()
.peers .peers
.read() .read()
.peer_info(peer_id) .peer_info(peer_id)
@ -564,13 +562,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() self.is_execution_engine_online()
.then_some(&self.beacon_processor_send) .then_some(&self.network_beacon_processor)
} }
pub fn processor_channel(&self) -> &BeaconProcessorSend<T> { pub fn beacon_processor(&self) -> &Arc<NetworkBeaconProcessor<T>> {
&self.beacon_processor_send &self.network_beacon_processor
} }
fn next_id(&mut self) -> Id { fn next_id(&mut self) -> Id {

View File

@ -1,5 +1,5 @@
use super::batch::{BatchInfo, BatchProcessingResult, BatchState}; use super::batch::{BatchInfo, BatchProcessingResult, BatchState};
use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent as BeaconWorkEvent}; use crate::network_beacon_processor::ChainSegmentProcessId;
use crate::sync::{ use crate::sync::{
manager::Id, network_context::SyncNetworkContext, BatchOperationOutcome, BatchProcessResult, manager::Id, network_context::SyncNetworkContext, BatchOperationOutcome, BatchProcessResult,
}; };
@ -294,8 +294,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
return Ok(KeepChain); return Ok(KeepChain);
} }
let beacon_processor_send = match network.processor_channel_if_enabled() { let beacon_processor = match network.beacon_processor_if_enabled() {
Some(channel) => channel, Some(beacon_processor) => beacon_processor,
None => return Ok(KeepChain), None => return Ok(KeepChain),
}; };
@ -317,9 +317,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
let process_id = ChainSegmentProcessId::RangeBatchId(self.id, batch_id); let process_id = ChainSegmentProcessId::RangeBatchId(self.id, batch_id);
self.current_processing_batch = Some(batch_id); self.current_processing_batch = Some(batch_id);
let work_event = BeaconWorkEvent::chain_segment(process_id, blocks); if let Err(e) = beacon_processor.send_chain_segment(process_id, blocks) {
if let Err(e) = beacon_processor_send.try_send(work_event) {
crit!(self.log, "Failed to send chain segment to processor."; "msg" => "process_batch", crit!(self.log, "Failed to send chain segment to processor."; "msg" => "process_batch",
"error" => %e, "batch" => self.processing_target); "error" => %e, "batch" => self.processing_target);
// This is unlikely to happen but it would stall syncing since the batch now has no // This is unlikely to happen but it would stall syncing since the batch now has no

View File

@ -379,25 +379,25 @@ where
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use crate::network_beacon_processor::NetworkBeaconProcessor;
use crate::beacon_processor::{BeaconProcessorSend, WorkEvent as BeaconWorkEvent};
use crate::service::RequestId; use crate::service::RequestId;
use crate::NetworkMessage; use crate::NetworkMessage;
use beacon_chain::{
builder::Witness, use super::*;
eth1_chain::CachingEth1Backend, use beacon_chain::builder::Witness;
parking_lot::RwLock, use beacon_chain::eth1_chain::CachingEth1Backend;
test_utils::{build_log, BeaconChainHarness, EphemeralHarnessType}, use beacon_chain::parking_lot::RwLock;
EngineState, use beacon_chain::EngineState;
}; use beacon_processor::WorkEvent as BeaconWorkEvent;
use lighthouse_network::{ use lighthouse_network::rpc::BlocksByRangeRequest;
rpc::{BlocksByRangeRequest, StatusMessage}, use lighthouse_network::Request;
NetworkGlobals, Request, use lighthouse_network::{rpc::StatusMessage, NetworkGlobals};
}; use slog::{o, Drain};
use slog::o; use tokio::sync::mpsc;
use slot_clock::TestingSlotClock;
use std::{collections::HashSet, sync::Arc}; use slot_clock::ManualSlotClock;
use std::collections::HashSet;
use std::sync::Arc;
use store::MemoryStore; use store::MemoryStore;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{Hash256, MinimalEthSpec as E}; use types::{Hash256, MinimalEthSpec as E};
@ -449,11 +449,23 @@ mod tests {
type TestBeaconChainType = type TestBeaconChainType =
Witness<TestingSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>; Witness<TestingSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
fn build_log(level: slog::Level, enabled: bool) -> slog::Logger {
let decorator = slog_term::TermDecorator::new().build();
let drain = slog_term::FullFormat::new(decorator).build().fuse();
let drain = slog_async::Async::new(drain).build().fuse();
if enabled {
slog::Logger::root(drain.filter_level(level).fuse(), o!())
} else {
slog::Logger::root(drain.filter(|_| false).fuse(), o!())
}
}
#[allow(unused)] #[allow(unused)]
struct TestRig { struct TestRig {
log: slog::Logger, log: slog::Logger,
/// To check what does sync send to the beacon processor. /// 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. /// To set up different scenarios where sync is told about known/unkown blocks.
chain: Arc<FakeStorage>, chain: Arc<FakeStorage>,
/// Needed by range to handle communication with the network. /// Needed by range to handle communication with the network.
@ -581,7 +593,7 @@ mod tests {
fn expect_chain_segment(&mut self) { fn expect_chain_segment(&mut self) {
match self.beacon_processor_rx.try_recv() { match self.beacon_processor_rx.try_recv() {
Ok(work) => { 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), other => panic!("Expected chain segment process, found {:?}", other),
} }
@ -607,10 +619,11 @@ mod tests {
); );
let (network_tx, network_rx) = mpsc::unbounded_channel(); let (network_tx, network_rx) = mpsc::unbounded_channel();
let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log)); 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( let cx = SyncNetworkContext::new(
network_tx, network_tx,
globals.clone(), Arc::new(network_beacon_processor),
BeaconProcessorSend(beacon_processor_tx),
chain, chain,
log.new(o!("component" => "network_context")), log.new(o!("component" => "network_context")),
); );

View File

@ -890,7 +890,7 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.help("Set the timeout for checkpoint sync calls to remote beacon node HTTP endpoint.") .help("Set the timeout for checkpoint sync calls to remote beacon node HTTP endpoint.")
.value_name("SECONDS") .value_name("SECONDS")
.takes_value(true) .takes_value(true)
.default_value("60") .default_value("180")
) )
.arg( .arg(
Arg::with_name("reconstruct-historic-states") Arg::with_name("reconstruct-historic-states")

View File

@ -48,6 +48,17 @@ The Ethereum community provides various [public endpoints](https://eth-clients.g
lighthouse bn --checkpoint-sync-url https://example.com/ ... lighthouse bn --checkpoint-sync-url https://example.com/ ...
``` ```
### Adjusting the timeout
If the beacon node fails to start due to a timeout from the checkpoint sync server, you can try
running it again with a longer timeout by adding the flag `--checkpoint-sync-url-timeout`.
```
lighthouse bn --checkpoint-sync-url-timeout 300 --checkpoint-sync-url https://example.com/ ...
```
The flag takes a value in seconds. For more information see `lighthouse bn --help`.
## Backfilling Blocks ## Backfilling Blocks
Once forwards sync completes, Lighthouse will commence a "backfill sync" to download the blocks Once forwards sync completes, Lighthouse will commence a "backfill sync" to download the blocks

View File

@ -31,14 +31,19 @@ use clap::ArgMatches;
use clap_utils::{parse_optional, parse_required}; use clap_utils::{parse_optional, parse_required};
use environment::Environment; use environment::Environment;
use eth2::{types::BlockId, BeaconNodeHttpClient, SensitiveUrl, Timeouts}; use eth2::{types::BlockId, BeaconNodeHttpClient, SensitiveUrl, Timeouts};
use eth2_network_config::Eth2NetworkConfig;
use std::path::PathBuf; use std::path::PathBuf;
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
use types::{EthSpec, FullPayload, SignedBeaconBlock}; use types::{EthSpec, FullPayload, SignedBeaconBlock};
const HTTP_TIMEOUT: Duration = Duration::from_secs(5); const HTTP_TIMEOUT: Duration = Duration::from_secs(5);
pub fn run<T: EthSpec>(env: Environment<T>, matches: &ArgMatches) -> Result<(), String> { pub fn run<T: EthSpec>(
let spec = &T::default_spec(); env: Environment<T>,
network_config: Eth2NetworkConfig,
matches: &ArgMatches,
) -> Result<(), String> {
let spec = &network_config.chain_spec::<T>()?;
let executor = env.core_context().executor; let executor = env.core_context().executor;
/* /*

View File

@ -15,11 +15,13 @@ mod new_testnet;
mod parse_ssz; mod parse_ssz;
mod replace_state_pubkeys; mod replace_state_pubkeys;
mod skip_slots; mod skip_slots;
mod state_root;
mod transition_blocks; mod transition_blocks;
use clap::{App, Arg, ArgMatches, SubCommand}; use clap::{App, Arg, ArgMatches, SubCommand};
use clap_utils::parse_path_with_default_in_home_dir; use clap_utils::parse_optional;
use environment::{EnvironmentBuilder, LoggerConfig}; use environment::{EnvironmentBuilder, LoggerConfig};
use eth2_network_config::Eth2NetworkConfig;
use parse_ssz::run_parse_ssz; use parse_ssz::run_parse_ssz;
use std::path::PathBuf; use std::path::PathBuf;
use std::process; use std::process;
@ -50,7 +52,16 @@ fn main() {
.value_name("PATH") .value_name("PATH")
.takes_value(true) .takes_value(true)
.global(true) .global(true)
.help("The testnet dir. Defaults to ~/.lighthouse/testnet"), .help("The testnet dir."),
)
.arg(
Arg::with_name("network")
.long("network")
.value_name("NAME")
.takes_value(true)
.global(true)
.help("The network to use. Defaults to mainnet.")
.conflicts_with("testnet-dir")
) )
.subcommand( .subcommand(
SubCommand::with_name("skip-slots") SubCommand::with_name("skip-slots")
@ -126,7 +137,7 @@ fn main() {
.takes_value(true) .takes_value(true)
.conflicts_with("beacon-url") .conflicts_with("beacon-url")
.requires("block-path") .requires("block-path")
.help("Path to load a BeaconState from file as SSZ."), .help("Path to load a BeaconState from as SSZ."),
) )
.arg( .arg(
Arg::with_name("block-path") Arg::with_name("block-path")
@ -135,7 +146,7 @@ fn main() {
.takes_value(true) .takes_value(true)
.conflicts_with("beacon-url") .conflicts_with("beacon-url")
.requires("pre-state-path") .requires("pre-state-path")
.help("Path to load a SignedBeaconBlock from file as SSZ."), .help("Path to load a SignedBeaconBlock from as SSZ."),
) )
.arg( .arg(
Arg::with_name("post-state-output-path") Arg::with_name("post-state-output-path")
@ -817,14 +828,14 @@ fn main() {
) )
.subcommand( .subcommand(
SubCommand::with_name("block-root") SubCommand::with_name("block-root")
.about("Computes the block root of some block") .about("Computes the block root of some block.")
.arg( .arg(
Arg::with_name("block-path") Arg::with_name("block-path")
.long("block-path") .long("block-path")
.value_name("PATH") .value_name("PATH")
.takes_value(true) .takes_value(true)
.conflicts_with("beacon-url") .conflicts_with("beacon-url")
.help("Path to load a SignedBeaconBlock from file as SSZ."), .help("Path to load a SignedBeaconBlock from as SSZ."),
) )
.arg( .arg(
Arg::with_name("beacon-url") Arg::with_name("beacon-url")
@ -850,6 +861,41 @@ fn main() {
.help("Number of repeat runs, useful for benchmarking."), .help("Number of repeat runs, useful for benchmarking."),
) )
) )
.subcommand(
SubCommand::with_name("state-root")
.about("Computes the state root of some state.")
.arg(
Arg::with_name("state-path")
.long("state-path")
.value_name("PATH")
.takes_value(true)
.conflicts_with("beacon-url")
.help("Path to load a BeaconState from as SSZ."),
)
.arg(
Arg::with_name("beacon-url")
.long("beacon-url")
.value_name("URL")
.takes_value(true)
.help("URL to a beacon-API provider."),
)
.arg(
Arg::with_name("state-id")
.long("state-id")
.value_name("BLOCK_ID")
.takes_value(true)
.requires("beacon-url")
.help("Identifier for a state as per beacon-API standards (slot, root, etc.)"),
)
.arg(
Arg::with_name("runs")
.long("runs")
.value_name("INTEGER")
.takes_value(true)
.default_value("1")
.help("Number of repeat runs, useful for benchmarking."),
)
)
.get_matches(); .get_matches();
let result = matches let result = matches
@ -896,17 +942,44 @@ fn run<T: EthSpec>(
.build() .build()
.map_err(|e| format!("should build env: {:?}", e))?; .map_err(|e| format!("should build env: {:?}", e))?;
let testnet_dir = parse_path_with_default_in_home_dir( // Determine testnet-dir path or network name depending on CLI flags.
matches, let (testnet_dir, network_name) =
"testnet-dir", if let Some(testnet_dir) = parse_optional::<PathBuf>(matches, "testnet-dir")? {
PathBuf::from(directory::DEFAULT_ROOT_DIR).join("testnet"), (Some(testnet_dir), None)
)?; } else {
let network_name =
parse_optional(matches, "network")?.unwrap_or_else(|| "mainnet".to_string());
(None, Some(network_name))
};
// Lazily load either the testnet dir or the network config, as required.
// Some subcommands like new-testnet need the testnet dir but not the network config.
let get_testnet_dir = || testnet_dir.clone().ok_or("testnet-dir is required");
let get_network_config = || {
if let Some(testnet_dir) = &testnet_dir {
Eth2NetworkConfig::load(testnet_dir.clone()).map_err(|e| {
format!(
"Unable to open testnet dir at {}: {}",
testnet_dir.display(),
e
)
})
} else {
let network_name = network_name.ok_or("no network name or testnet-dir provided")?;
Eth2NetworkConfig::constant(&network_name)?.ok_or("invalid network name".into())
}
};
match matches.subcommand() { match matches.subcommand() {
("transition-blocks", Some(matches)) => transition_blocks::run::<T>(env, matches) ("transition-blocks", Some(matches)) => {
.map_err(|e| format!("Failed to transition blocks: {}", e)), let network_config = get_network_config()?;
transition_blocks::run::<T>(env, network_config, matches)
.map_err(|e| format!("Failed to transition blocks: {}", e))
}
("skip-slots", Some(matches)) => { ("skip-slots", Some(matches)) => {
skip_slots::run::<T>(env, matches).map_err(|e| format!("Failed to skip slots: {}", e)) let network_config = get_network_config()?;
skip_slots::run::<T>(env, network_config, matches)
.map_err(|e| format!("Failed to skip slots: {}", e))
} }
("pretty-ssz", Some(matches)) => { ("pretty-ssz", Some(matches)) => {
run_parse_ssz::<T>(matches).map_err(|e| format!("Failed to pretty print hex: {}", e)) run_parse_ssz::<T>(matches).map_err(|e| format!("Failed to pretty print hex: {}", e))
@ -915,22 +988,33 @@ fn run<T: EthSpec>(
deploy_deposit_contract::run::<T>(env, matches) deploy_deposit_contract::run::<T>(env, matches)
.map_err(|e| format!("Failed to run deploy-deposit-contract command: {}", e)) .map_err(|e| format!("Failed to run deploy-deposit-contract command: {}", e))
} }
("eth1-genesis", Some(matches)) => eth1_genesis::run::<T>(env, testnet_dir, matches) ("eth1-genesis", Some(matches)) => {
.map_err(|e| format!("Failed to run eth1-genesis command: {}", e)), let testnet_dir = get_testnet_dir()?;
("interop-genesis", Some(matches)) => interop_genesis::run::<T>(testnet_dir, matches) eth1_genesis::run::<T>(env, testnet_dir, matches)
.map_err(|e| format!("Failed to run interop-genesis command: {}", e)), .map_err(|e| format!("Failed to run eth1-genesis command: {}", e))
}
("interop-genesis", Some(matches)) => {
let testnet_dir = get_testnet_dir()?;
interop_genesis::run::<T>(testnet_dir, matches)
.map_err(|e| format!("Failed to run interop-genesis command: {}", e))
}
("change-genesis-time", Some(matches)) => { ("change-genesis-time", Some(matches)) => {
let testnet_dir = get_testnet_dir()?;
change_genesis_time::run::<T>(testnet_dir, matches) change_genesis_time::run::<T>(testnet_dir, matches)
.map_err(|e| format!("Failed to run change-genesis-time command: {}", e)) .map_err(|e| format!("Failed to run change-genesis-time command: {}", e))
} }
("create-payload-header", Some(matches)) => create_payload_header::run::<T>(matches) ("create-payload-header", Some(matches)) => create_payload_header::run::<T>(matches)
.map_err(|e| format!("Failed to run create-payload-header command: {}", e)), .map_err(|e| format!("Failed to run create-payload-header command: {}", e)),
("replace-state-pubkeys", Some(matches)) => { ("replace-state-pubkeys", Some(matches)) => {
let testnet_dir = get_testnet_dir()?;
replace_state_pubkeys::run::<T>(testnet_dir, matches) replace_state_pubkeys::run::<T>(testnet_dir, matches)
.map_err(|e| format!("Failed to run replace-state-pubkeys command: {}", e)) .map_err(|e| format!("Failed to run replace-state-pubkeys command: {}", e))
} }
("new-testnet", Some(matches)) => new_testnet::run::<T>(testnet_dir, matches) ("new-testnet", Some(matches)) => {
.map_err(|e| format!("Failed to run new_testnet command: {}", e)), let testnet_dir = get_testnet_dir()?;
new_testnet::run::<T>(testnet_dir, matches)
.map_err(|e| format!("Failed to run new_testnet command: {}", e))
}
("check-deposit-data", Some(matches)) => check_deposit_data::run(matches) ("check-deposit-data", Some(matches)) => check_deposit_data::run(matches)
.map_err(|e| format!("Failed to run check-deposit-data command: {}", e)), .map_err(|e| format!("Failed to run check-deposit-data command: {}", e)),
("generate-bootnode-enr", Some(matches)) => generate_bootnode_enr::run::<T>(matches) ("generate-bootnode-enr", Some(matches)) => generate_bootnode_enr::run::<T>(matches)
@ -941,8 +1025,16 @@ fn run<T: EthSpec>(
.map_err(|e| format!("Failed to run mnemonic-validators command: {}", e)), .map_err(|e| format!("Failed to run mnemonic-validators command: {}", e)),
("indexed-attestations", Some(matches)) => indexed_attestations::run::<T>(matches) ("indexed-attestations", Some(matches)) => indexed_attestations::run::<T>(matches)
.map_err(|e| format!("Failed to run indexed-attestations command: {}", e)), .map_err(|e| format!("Failed to run indexed-attestations command: {}", e)),
("block-root", Some(matches)) => block_root::run::<T>(env, matches) ("block-root", Some(matches)) => {
.map_err(|e| format!("Failed to run block-root command: {}", e)), let network_config = get_network_config()?;
block_root::run::<T>(env, network_config, matches)
.map_err(|e| format!("Failed to run block-root command: {}", e))
}
("state-root", Some(matches)) => {
let network_config = get_network_config()?;
state_root::run::<T>(env, network_config, matches)
.map_err(|e| format!("Failed to run state-root command: {}", e))
}
(other, _) => Err(format!("Unknown subcommand {}. See --help.", other)), (other, _) => Err(format!("Unknown subcommand {}. See --help.", other)),
} }
} }

View File

@ -49,6 +49,7 @@ use clap::ArgMatches;
use clap_utils::{parse_optional, parse_required}; use clap_utils::{parse_optional, parse_required};
use environment::Environment; use environment::Environment;
use eth2::{types::StateId, BeaconNodeHttpClient, SensitiveUrl, Timeouts}; use eth2::{types::StateId, BeaconNodeHttpClient, SensitiveUrl, Timeouts};
use eth2_network_config::Eth2NetworkConfig;
use ssz::Encode; use ssz::Encode;
use state_processing::state_advance::{complete_state_advance, partial_state_advance}; use state_processing::state_advance::{complete_state_advance, partial_state_advance};
use std::fs::File; use std::fs::File;
@ -59,8 +60,12 @@ use types::{BeaconState, CloneConfig, EthSpec, Hash256};
const HTTP_TIMEOUT: Duration = Duration::from_secs(10); const HTTP_TIMEOUT: Duration = Duration::from_secs(10);
pub fn run<T: EthSpec>(env: Environment<T>, matches: &ArgMatches) -> Result<(), String> { pub fn run<T: EthSpec>(
let spec = &T::default_spec(); env: Environment<T>,
network_config: Eth2NetworkConfig,
matches: &ArgMatches,
) -> Result<(), String> {
let spec = &network_config.chain_spec::<T>()?;
let executor = env.core_context().executor; let executor = env.core_context().executor;
let output_path: Option<PathBuf> = parse_optional(matches, "output-path")?; let output_path: Option<PathBuf> = parse_optional(matches, "output-path")?;

76
lcli/src/state_root.rs Normal file
View File

@ -0,0 +1,76 @@
use crate::transition_blocks::load_from_ssz_with;
use clap::ArgMatches;
use clap_utils::{parse_optional, parse_required};
use environment::Environment;
use eth2::{types::StateId, BeaconNodeHttpClient, SensitiveUrl, Timeouts};
use eth2_network_config::Eth2NetworkConfig;
use std::path::PathBuf;
use std::time::{Duration, Instant};
use types::{BeaconState, EthSpec};
const HTTP_TIMEOUT: Duration = Duration::from_secs(10);
pub fn run<T: EthSpec>(
env: Environment<T>,
network_config: Eth2NetworkConfig,
matches: &ArgMatches,
) -> Result<(), String> {
let executor = env.core_context().executor;
let spec = &network_config.chain_spec::<T>()?;
let state_path: Option<PathBuf> = parse_optional(matches, "state-path")?;
let beacon_url: Option<SensitiveUrl> = parse_optional(matches, "beacon-url")?;
let runs: usize = parse_required(matches, "runs")?;
info!(
"Using {} network ({} spec)",
spec.config_name.as_deref().unwrap_or("unknown"),
T::spec_name()
);
info!("Doing {} runs", runs);
let state = match (state_path, beacon_url) {
(Some(state_path), None) => {
info!("State path: {:?}", state_path);
load_from_ssz_with(&state_path, spec, BeaconState::from_ssz_bytes)?
}
(None, Some(beacon_url)) => {
let state_id: StateId = parse_required(matches, "state-id")?;
let client = BeaconNodeHttpClient::new(beacon_url, Timeouts::set_all(HTTP_TIMEOUT));
executor
.handle()
.ok_or("shutdown in progress")?
.block_on(async move {
client
.get_debug_beacon_states::<T>(state_id)
.await
.map_err(|e| format!("Failed to download state: {:?}", e))
})
.map_err(|e| format!("Failed to complete task: {:?}", e))?
.ok_or_else(|| format!("Unable to locate state at {:?}", state_id))?
.data
}
_ => return Err("must supply either --state-path or --beacon-url".into()),
};
/*
* Perform the core "runs".
*/
let mut state_root = None;
for i in 0..runs {
let mut state = state.clone();
let timer = Instant::now();
state_root = Some(
state
.update_tree_hash_cache()
.map_err(|e| format!("error computing state root: {e:?}"))?,
);
info!("Run {}: {:?}", i, timer.elapsed());
}
if let Some(state_root) = state_root {
info!("State root is {:?}", state_root);
}
Ok(())
}

View File

@ -71,6 +71,7 @@ use eth2::{
types::{BlockId, StateId}, types::{BlockId, StateId},
BeaconNodeHttpClient, SensitiveUrl, Timeouts, BeaconNodeHttpClient, SensitiveUrl, Timeouts,
}; };
use eth2_network_config::Eth2NetworkConfig;
use ssz::Encode; use ssz::Encode;
use state_processing::{ use state_processing::{
block_signature_verifier::BlockSignatureVerifier, per_block_processing, per_slot_processing, block_signature_verifier::BlockSignatureVerifier, per_block_processing, per_slot_processing,
@ -94,8 +95,12 @@ struct Config {
exclude_post_block_thc: bool, exclude_post_block_thc: bool,
} }
pub fn run<T: EthSpec>(env: Environment<T>, matches: &ArgMatches) -> Result<(), String> { pub fn run<T: EthSpec>(
let spec = &T::default_spec(); env: Environment<T>,
network_config: Eth2NetworkConfig,
matches: &ArgMatches,
) -> Result<(), String> {
let spec = &network_config.chain_spec::<T>()?;
let executor = env.core_context().executor; let executor = env.core_context().executor;
/* /*

View File

@ -177,7 +177,7 @@ fn checkpoint_sync_url_timeout_default() {
CommandLineTest::new() CommandLineTest::new()
.run_with_zero_port() .run_with_zero_port()
.with_config(|config| { .with_config(|config| {
assert_eq!(config.chain.checkpoint_sync_url_timeout, 60); assert_eq!(config.chain.checkpoint_sync_url_timeout, 180);
}); });
} }