Skip to content

Commit

Permalink
Move the BeaconProcessor into a new crate (sigp#4435)
Browse files Browse the repository at this point in the history
*Replaces sigp#4434. It is identical, but this PR has a smaller diff due to a curated commit history.*

NA

This PR moves the scheduling logic for the `BeaconProcessor` into a new crate in `beacon_node/beacon_processor`. Previously it existed in the `beacon_node/network` crate.

This addresses a circular-dependency problem where it's not possible to use the `BeaconProcessor` from the `beacon_chain` crate. The `network` crate depends on the `beacon_chain` crate (`network -> beacon_chain`), but importing the `BeaconProcessor` into the `beacon_chain` crate would create a circular dependancy of `beacon_chain -> network`.

The `BeaconProcessor` was designed to provide queuing and prioritized scheduling for messages from the network. It has proven to be quite valuable and I believe we'd make Lighthouse more stable and effective by using it elsewhere. In particular, I think we should use the `BeaconProcessor` for:

1. HTTP API requests.
1. Scheduled tasks in the `BeaconChain` (e.g., state advance).

Using the `BeaconProcessor` for these tasks would help prevent the BN from becoming overwhelmed and would also help it to prioritize operations (e.g., choosing to process blocks from gossip before responding to low-priority HTTP API requests).

This PR is intended to have zero impact on runtime behaviour. It aims to simply separate the *scheduling* code (i.e., the `BeaconProcessor`) from the *business logic* in the `network` crate (i.e., the `Worker` impls). Future PRs (see sigp#4462) can build upon these works to actually use the `BeaconProcessor` for more operations.

I've gone to some effort to use `git mv` to make the diff look more like "file was moved and modified" rather than "file was deleted and a new one added". This should reduce review burden and help maintain commit attribution.
  • Loading branch information
paulhauner authored and Woodpile37 committed Jan 6, 2024
1 parent 8e132ad commit ebaac13
Show file tree
Hide file tree
Showing 19 changed files with 840 additions and 173 deletions.
28 changes: 28 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ members = [

"beacon_node",
"beacon_node/beacon_chain",
"beacon_node/beacon_processor",
"beacon_node/builder_client",
"beacon_node/client",
"beacon_node/eth1",
Expand Down
2 changes: 1 addition & 1 deletion beacon_node/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -43,4 +43,4 @@ monitoring_api = { path = "../common/monitoring_api" }
sensitive_url = { path = "../common/sensitive_url" }
http_api = { path = "http_api" }
unused_port = { path = "../common/unused_port" }
strum = "0.24.1"
strum = "0.24.1"
24 changes: 24 additions & 0 deletions beacon_node/beacon_processor/Cargo.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
[package]
name = "beacon_processor"
version = "0.1.0"
edition = "2021"

[dependencies]
slog = { version = "2.5.2", features = ["max_level_trace"] }
itertools = "0.10.0"
logging = { path = "../../common/logging" }
tokio = { version = "1.14.0", features = ["full"] }
tokio-util = { version = "0.6.3", features = ["time"] }
futures = "0.3.7"
fnv = "1.0.7"
strum = "0.24.0"
task_executor = { path = "../../common/task_executor" }
slot_clock = { path = "../../common/slot_clock" }
lighthouse_network = { path = "../lighthouse_network" }
hex = "0.4.2"
derivative = "2.2.0"
types = { path = "../../consensus/types" }
ethereum_ssz = "0.5.0"
lazy_static = "1.4.0"
lighthouse_metrics = { path = "../../common/lighthouse_metrics" }
parking_lot = "0.12.0"
File renamed without changes.
141 changes: 141 additions & 0 deletions beacon_node/beacon_processor/src/metrics.rs
Original file line number Diff line number Diff line change
@@ -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"]
);
}
2 changes: 2 additions & 0 deletions beacon_node/client/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -43,3 +43,5 @@ slasher = { path = "../../slasher" }
slasher_service = { path = "../../slasher/service" }
monitoring_api = {path = "../../common/monitoring_api"}
execution_layer = { path = "../execution_layer" }
beacon_processor = { path = "../beacon_processor" }
num_cpus = "1.13.0"
42 changes: 41 additions & 1 deletion beacon_node/client/src/builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,10 @@ use beacon_chain::{
store::{HotColdDB, ItemStore, LevelDB, StoreConfig},
BeaconChain, BeaconChainTypes, Eth1ChainBackend, ServerSentEventHandler,
};
use beacon_processor::{
work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessor, BeaconProcessorSend,
WorkEvent, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN,
};
use environment::RuntimeContext;
use eth1::{Config as Eth1Config, Service as Eth1Service};
use eth2::{
Expand All @@ -27,12 +31,13 @@ use network::{NetworkConfig, NetworkSenders, NetworkService};
use slasher::Slasher;
use slasher_service::SlasherService;
use slog::{debug, info, warn, Logger};
use std::cmp;
use std::net::TcpListener;
use std::path::{Path, PathBuf};
use std::sync::Arc;
use std::time::Duration;
use timer::spawn_timer;
use tokio::sync::oneshot;
use tokio::sync::{mpsc, oneshot};
use types::{
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec,
ExecutionBlockHash, Hash256, SignedBeaconBlock,
Expand Down Expand Up @@ -73,6 +78,10 @@ pub struct ClientBuilder<T: BeaconChainTypes> {
http_metrics_config: http_metrics::Config,
slasher: Option<Arc<Slasher<T::EthSpec>>>,
eth_spec_instance: T::EthSpec,
beacon_processor_send: BeaconProcessorSend<T::EthSpec>,
beacon_processor_receive: mpsc::Receiver<WorkEvent<T::EthSpec>>,
work_reprocessing_tx: mpsc::Sender<ReprocessQueueMessage>,
work_reprocessing_rx: mpsc::Receiver<ReprocessQueueMessage>,
}

impl<TSlotClock, TEth1Backend, TEthSpec, THotStore, TColdStore>
Expand All @@ -88,6 +97,10 @@ where
///
/// The `eth_spec_instance` parameter is used to concretize `TEthSpec`.
pub fn new(eth_spec_instance: TEthSpec) -> Self {
let (beacon_processor_send, beacon_processor_receive) =
mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
let (work_reprocessing_tx, work_reprocessing_rx) =
mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
Self {
slot_clock: None,
store: None,
Expand All @@ -106,6 +119,10 @@ where
http_metrics_config: <_>::default(),
slasher: None,
eth_spec_instance,
beacon_processor_send: BeaconProcessorSend(beacon_processor_send),
beacon_processor_receive,
work_reprocessing_tx,
work_reprocessing_rx,
}
}

Expand Down Expand Up @@ -576,6 +593,8 @@ where
gossipsub_registry
.as_mut()
.map(|registry| registry.sub_registry_with_prefix("gossipsub")),
self.beacon_processor_send.clone(),
self.work_reprocessing_tx.clone(),
)
.await
.map_err(|e| format!("Failed to start network: {:?}", e))?;
Expand Down Expand Up @@ -763,6 +782,27 @@ where
}

if let Some(beacon_chain) = self.beacon_chain.as_ref() {
if let Some(network_globals) = &self.network_globals {
let beacon_processor_context = runtime_context.service_context("bproc".into());
BeaconProcessor {
network_globals: network_globals.clone(),
executor: beacon_processor_context.executor.clone(),
max_workers: cmp::max(1, num_cpus::get()),
current_workers: 0,
enable_backfill_rate_limiting: beacon_chain
.config
.enable_backfill_rate_limiting,
log: beacon_processor_context.log().clone(),
}
.spawn_manager(
self.beacon_processor_receive,
self.work_reprocessing_tx,
self.work_reprocessing_rx,
None,
beacon_chain.slot_clock.clone(),
);
}

let state_advance_context = runtime_context.service_context("state_advance".into());
let state_advance_log = state_advance_context.log().clone();
spawn_state_advance_timer(
Expand Down
6 changes: 4 additions & 2 deletions beacon_node/network/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ matches = "0.1.8"
exit-future = "0.2.0"
slog-term = "2.6.0"
slog-async = "2.5.0"
environment = { path = "../../lighthouse/environment" }

[dependencies]
beacon_chain = { path = "../beacon_chain" }
Expand Down Expand Up @@ -46,4 +45,7 @@ derivative = "2.2.0"
delay_map = "0.3.0"
ethereum-types = { version = "0.14.1", optional = true }
operation_pool = { path = "../operation_pool" }
execution_layer = { path = "../execution_layer" }
execution_layer = { path = "../execution_layer" }
beacon_processor = { path = "../beacon_processor" }
parking_lot = "0.12.0"
environment = { path = "../../lighthouse/environment" }
Loading

0 comments on commit ebaac13

Please sign in to comment.