From 71ee4cf2d83911e0dc99fe63b4e3b3574f38e08a Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 13 Jun 2023 16:43:38 +1000 Subject: [PATCH 01/50] Rework Validator Client fallback mechanism --- beacon_node/http_api/tests/tests.rs | 25 +- common/eth2/src/lib.rs | 14 +- lighthouse/tests/validator_client.rs | 14 +- testing/simulator/src/checks.rs | 65 +++ testing/simulator/src/cli.rs | 40 +- testing/simulator/src/eth1_sim.rs | 2 +- testing/simulator/src/fallback_sim.rs | 333 +++++++++++++ testing/simulator/src/local_network.rs | 42 ++ testing/simulator/src/main.rs | 8 + validator_client/src/beacon_node_fallback.rs | 479 +++++++++++++------ validator_client/src/beacon_node_health.rs | 363 ++++++++++++++ validator_client/src/block_service.rs | 12 +- validator_client/src/check_synced.rs | 30 +- validator_client/src/cli.rs | 8 + validator_client/src/config.rs | 25 +- validator_client/src/lib.rs | 16 +- validator_client/src/notifier.rs | 2 +- 17 files changed, 1303 insertions(+), 175 deletions(-) create mode 100644 testing/simulator/src/fallback_sim.rs create mode 100644 validator_client/src/beacon_node_health.rs diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index 3ae495378e3..26718ad294a 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -1231,7 +1231,11 @@ impl ApiTester { pub async fn test_post_beacon_blocks_valid(mut self) -> Self { let next_block = &self.next_block; - self.client.post_beacon_blocks(next_block).await.unwrap(); + self.client + .clone() + .post_beacon_blocks(next_block) + .await + .unwrap(); assert!( self.network_rx.network_recv.recv().await.is_some(), @@ -1270,7 +1274,13 @@ impl ApiTester { .await .0; - assert!(self.client.post_beacon_blocks(&block).await.is_err()); + + assert!(self + .client + .clone() + .post_beacon_blocks(&next_block) + .await + .is_err()); assert!( self.network_rx.network_recv.recv().await.is_some(), @@ -2419,7 +2429,11 @@ impl ApiTester { let signed_block = block.sign(&sk, &fork, genesis_validators_root, &self.chain.spec); - self.client.post_beacon_blocks(&signed_block).await.unwrap(); + self.client + .clone() + .post_beacon_blocks(&signed_block) + .await + .unwrap(); assert_eq!(self.chain.head_beacon_block().as_ref(), &signed_block); @@ -4281,6 +4295,7 @@ impl ApiTester { }); self.client + .clone() .post_beacon_blocks(&self.next_block) .await .unwrap(); @@ -4312,6 +4327,7 @@ impl ApiTester { self.harness.advance_slot(); self.client + .clone() .post_beacon_blocks(&self.reorg_block) .await .unwrap(); @@ -4389,6 +4405,7 @@ impl ApiTester { }); self.client + .clone() .post_beacon_blocks(&self.next_block) .await .unwrap(); @@ -5107,4 +5124,4 @@ async fn optimistic_responses() { .await .test_check_optimistic_responses() .await; -} +} \ No newline at end of file diff --git a/common/eth2/src/lib.rs b/common/eth2/src/lib.rs index 146a832e388..1fc65f9301c 100644 --- a/common/eth2/src/lib.rs +++ b/common/eth2/src/lib.rs @@ -108,7 +108,7 @@ impl fmt::Display for Error { /// A struct to define a variety of different timeouts for different validator tasks to ensure /// proper fallback behaviour. -#[derive(Clone)] +#[derive(Clone, Debug, PartialEq, Eq)] pub struct Timeouts { pub attestation: Duration, pub attester_duties: Duration, @@ -141,13 +141,21 @@ impl Timeouts { /// A wrapper around `reqwest::Client` which provides convenience methods for interfacing with a /// Lighthouse Beacon Node HTTP server (`http_api`). -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct BeaconNodeHttpClient { client: reqwest::Client, server: SensitiveUrl, timeouts: Timeouts, } +impl PartialEq for BeaconNodeHttpClient { + fn eq(&self, other: &Self) -> bool { + self.server == other.server && self.timeouts == other.timeouts + } +} + +impl Eq for BeaconNodeHttpClient {} + impl fmt::Display for BeaconNodeHttpClient { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.server.fmt(f) @@ -686,7 +694,7 @@ impl BeaconNodeHttpClient { /// /// Returns `Ok(None)` on a 404 error. pub async fn post_beacon_blocks>( - &self, + self, block: &SignedBeaconBlock, ) -> Result<(), Error> { let mut path = self.eth_path(V1)?; diff --git a/lighthouse/tests/validator_client.rs b/lighthouse/tests/validator_client.rs index 9bcfe2a1d50..400ca777118 100644 --- a/lighthouse/tests/validator_client.rs +++ b/lighthouse/tests/validator_client.rs @@ -487,7 +487,7 @@ fn monitoring_endpoint() { #[test] fn disable_run_on_all_default() { CommandLineTest::new().run().with_config(|config| { - assert!(!config.disable_run_on_all); + assert!(!config.beacon_node_fallback.disable_run_on_all); }); } @@ -497,7 +497,17 @@ fn disable_run_on_all() { .flag("disable-run-on-all", None) .run() .with_config(|config| { - assert!(config.disable_run_on_all); + assert!(config.beacon_node_fallback.disable_run_on_all); + }); +} + +#[test] +fn sync_tolerance_flag() { + CommandLineTest::new() + .flag("beacon-node-sync-tolerance", Some("8")) + .run() + .with_config(|config| { + assert_eq!(config.beacon_node_fallback.sync_tolerance, Some(8)); }); } diff --git a/testing/simulator/src/checks.rs b/testing/simulator/src/checks.rs index d34cdbc9ff1..01e50eb7fba 100644 --- a/testing/simulator/src/checks.rs +++ b/testing/simulator/src/checks.rs @@ -1,5 +1,6 @@ use crate::local_network::LocalNetwork; use node_test_rig::eth2::types::{BlockId, StateId}; + use std::time::Duration; use types::{Epoch, EthSpec, ExecPayload, ExecutionBlockHash, Hash256, Slot, Unsigned}; @@ -243,3 +244,67 @@ pub async fn verify_transition_block_finalized( )) } } + +pub async fn disconnect_from_execution_layer( + network: LocalNetwork, + transition_epoch: Epoch, + slot_duration: Duration, +) -> Result<(), String> { + epoch_delay(transition_epoch + 1, slot_duration, E::slots_per_epoch()).await; + + eprintln!("Disabling Execution Layer"); + + // Take the execution node at position 0 and force it to return the `syncing` status. + network.execution_nodes.read()[0] + .server + .all_payloads_syncing(false); + + // Run for 2 epochs with the 0th execution node stalled. + epoch_delay( + transition_epoch + 1 + 2, + slot_duration, + E::slots_per_epoch(), + ) + .await; + + // Restore the functionality of the 0th execution node. + network.execution_nodes.read()[0] + .server + .all_payloads_valid(); + + eprintln!("Re-enabling Execution Layer"); + Ok(()) +} + +/// Ensure all validators have attested correctly. +pub async fn check_attestation_correctness( + network: LocalNetwork, + upto_epoch: Epoch, + slots_per_epoch: u64, + slot_duration: Duration, +) -> Result<(), String> { + let upto_slot = upto_epoch.start_slot(slots_per_epoch); + slot_delay(upto_slot, slot_duration).await; + + let remote_node = &network.remote_nodes()?[1]; + + let results = remote_node + .get_lighthouse_analysis_attestation_performance( + Epoch::new(2), + upto_epoch - 2, + "global".to_string(), + ) + .await + .map_err(|e| format!("Unable to get attestation performance: {e}"))?; + + for result in results { + for epochs in result.epochs.values() { + assert!(epochs.active); + assert!(epochs.head); + assert!(epochs.target); + assert!(epochs.source); + } + } + + Ok(()) +} diff --git a/testing/simulator/src/cli.rs b/testing/simulator/src/cli.rs index ff80201051f..0b888314cf1 100644 --- a/testing/simulator/src/cli.rs +++ b/testing/simulator/src/cli.rs @@ -119,7 +119,45 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { .takes_value(true) .default_value("all") .possible_values(&["one-node", "two-nodes", "mixed", "all"]) - .help("Sync verification strategy to run."), + .help("Sync verification strategy to run.")) + ) + .subcommand( + SubCommand::with_name("fallback-sim") + .about("Run the fallback simulation") + .arg(Arg::with_name("vc_count") + .short("c") + .long("vc-count") + .takes_value(true) + .default_value("4") + .help("Number of validator clients")) + .arg(Arg::with_name("bns_per_vc") + .short("b") + .long("bns_per_vc") + .takes_value(true) + .default_value("2") + .help("Number of beacon nodes to connect to each validator client")) + .arg(Arg::with_name("validators_per_vc") + .short("v") + .long("validators_per_vc") + .takes_value(true) + .default_value("20") + .help("Number of validators per client")) + .arg(Arg::with_name("speed_up_factor") + .short("s") + .long("speed_up_factor") + .takes_value(true) + .default_value("3") + .help("Speed up factor. Please use a divisor of 12.")) + .arg(Arg::with_name("post-merge") + .short("m") + .long("post-merge") + .takes_value(false) + .help("Simulate the merge transition")) + .arg(Arg::with_name("continue_after_checks") + .short("c") + .long("continue_after_checks") + .takes_value(false) + .help("Continue after checks (default false)") ), ) } diff --git a/testing/simulator/src/eth1_sim.rs b/testing/simulator/src/eth1_sim.rs index 57c944cf1a7..03a0205574d 100644 --- a/testing/simulator/src/eth1_sim.rs +++ b/testing/simulator/src/eth1_sim.rs @@ -64,7 +64,7 @@ pub fn run_eth1_sim(matches: &ArgMatches) -> Result<(), String> { let mut env = EnvironmentBuilder::minimal() .initialize_logger(LoggerConfig { path: None, - debug_level: String::from("debug"), + debug_level: String::from("info"), logfile_debug_level: String::from("debug"), log_format: None, logfile_format: None, diff --git a/testing/simulator/src/fallback_sim.rs b/testing/simulator/src/fallback_sim.rs new file mode 100644 index 00000000000..63b50d5c3ca --- /dev/null +++ b/testing/simulator/src/fallback_sim.rs @@ -0,0 +1,333 @@ +use futures::prelude::*; + +use std::cmp::max; +use std::net::Ipv4Addr; +use std::time::Duration; + +use crate::local_network::{EXECUTION_PORT, TERMINAL_BLOCK, TERMINAL_DIFFICULTY}; +use crate::{checks, LocalNetwork, E}; +use clap::ArgMatches; +use eth1::{Eth1Endpoint, DEFAULT_CHAIN_ID}; +use eth1_test_rig::AnvilEth1Instance; + +use execution_layer::http::deposit_methods::Eth1Id; + +use node_test_rig::{ + environment::{EnvironmentBuilder, LoggerConfig}, + testing_client_config, testing_validator_config, ClientGenesis, ValidatorFiles, +}; +use rayon::prelude::*; +use sensitive_url::SensitiveUrl; +use tokio::time::sleep; +use types::{Epoch, EthSpec, MinimalEthSpec}; + +const END_EPOCH: u64 = 20; +const ALTAIR_FORK_EPOCH: u64 = 1; +const BELLATRIX_FORK_EPOCH: u64 = 2; + +const SUGGESTED_FEE_RECIPIENT: [u8; 20] = + [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; + +pub fn run_fallback_sim(matches: &ArgMatches) -> Result<(), String> { + let speed_up_factor = + value_t!(matches, "speed_up_factor", u64).expect("missing speed_up_factor default"); + let vc_count = value_t!(matches, "vc_count", usize).expect("missing vc_count default"); + let validators_per_vc = + value_t!(matches, "validators_per_vc", usize).expect("missing validators_per_vc default"); + let bns_per_vc = value_t!(matches, "bns_per_vc", usize).expect("missing bns_per_vc default"); + let continue_after_checks = matches.is_present("continue_after_checks"); + //let post_merge_sim = matches.is_present("post-merge"); + let post_merge_sim = true; + + println!("Fallback Simulator:"); + println!(" Validator Clients: {}", vc_count); + println!(" Validators per Client: {}", validators_per_vc); + println!(" Beacon Nodes per Validator Client: {}", bns_per_vc); + println!(" speed up factor:{}", speed_up_factor); + + let log_level = "debug"; + + fallback_sim( + speed_up_factor, + vc_count, + validators_per_vc, + bns_per_vc, + post_merge_sim, + continue_after_checks, + log_level, + ) +} + +fn fallback_sim( + speed_up_factor: u64, + vc_count: usize, + validators_per_vc: usize, + bns_per_vc: usize, + post_merge_sim: bool, + continue_after_checks: bool, + log_level: &str, +) -> Result<(), String> { + // Generate the directories and keystores required for the validator clients. + let validator_files = (0..vc_count) + .into_par_iter() + .map(|i| { + println!( + "Generating keystores for validator {} of {}", + i + 1, + vc_count + ); + + let indices = (i * validators_per_vc..(i + 1) * validators_per_vc).collect::>(); + ValidatorFiles::with_keystores(&indices).unwrap() + }) + .collect::>(); + + let mut env = EnvironmentBuilder::minimal() + .initialize_logger(LoggerConfig { + path: None, + debug_level: String::from(log_level), + logfile_debug_level: String::from("debug"), + log_format: None, + logfile_format: None, + log_color: false, + disable_log_timestamp: false, + max_log_size: 0, + max_log_number: 0, + compression: false, + is_restricted: true, + sse_logging: false, + })? + .multi_threaded_tokio_runtime()? + .build()?; + + let eth1_block_time = Duration::from_millis(15_000 / speed_up_factor); + + let spec = &mut env.eth2_config.spec; + + let total_validator_count = validators_per_vc * vc_count; + let node_count = vc_count * bns_per_vc; + //let altair_fork_version = spec.altair_fork_version; + //let bellatrix_fork_version = spec.bellatrix_fork_version; + + spec.seconds_per_slot /= speed_up_factor; + spec.seconds_per_slot = max(1, spec.seconds_per_slot); + spec.eth1_follow_distance = 16; + spec.genesis_delay = eth1_block_time.as_secs() * spec.eth1_follow_distance * 2; + spec.min_genesis_time = 0; + spec.min_genesis_active_validator_count = total_validator_count as u64; + spec.seconds_per_eth1_block = eth1_block_time.as_secs(); + spec.altair_fork_epoch = Some(Epoch::new(ALTAIR_FORK_EPOCH)); + // Set these parameters only if we are doing a merge simulation + if post_merge_sim { + spec.terminal_total_difficulty = TERMINAL_DIFFICULTY.into(); + spec.bellatrix_fork_epoch = Some(Epoch::new(BELLATRIX_FORK_EPOCH)); + } + + let seconds_per_slot = spec.seconds_per_slot; + let slot_duration = Duration::from_secs(spec.seconds_per_slot); + let _initial_validator_count = spec.min_genesis_active_validator_count as usize; + let deposit_amount = env.eth2_config.spec.max_effective_balance; + + let context = env.core_context(); + + let main_future = async { + /* + * Deploy the deposit contract, spawn tasks to keep creating new blocks and deposit + * validators. + */ + let anvil_eth1_instance = AnvilEth1Instance::new(DEFAULT_CHAIN_ID.into()).await?; + let deposit_contract = anvil_eth1_instance.deposit_contract; + let chain_id = anvil_eth1_instance.anvil.chain_id(); + let anvil = anvil_eth1_instance.anvil; + let eth1_endpoint = SensitiveUrl::parse(anvil.endpoint().as_str()) + .expect("Unable to parse anvil endpoint."); + let deposit_contract_address = deposit_contract.address(); + + // Start a timer that produces eth1 blocks on an interval. + tokio::spawn(async move { + let mut interval = tokio::time::interval(eth1_block_time); + loop { + interval.tick().await; + let _ = anvil.evm_mine().await; + } + }); + + // Submit deposits to the deposit contract. + tokio::spawn(async move { + for i in 0..total_validator_count { + println!("Submitting deposit for validator {}...", i); + let _ = deposit_contract + .deposit_deterministic_async::(i, deposit_amount) + .await; + } + }); + + let mut beacon_config = testing_client_config(); + + beacon_config.genesis = ClientGenesis::DepositContract; + beacon_config.eth1.endpoint = Eth1Endpoint::NoAuth(eth1_endpoint); + beacon_config.eth1.deposit_contract_address = deposit_contract_address; + beacon_config.eth1.deposit_contract_deploy_block = 0; + beacon_config.eth1.lowest_cached_block_number = 0; + beacon_config.eth1.follow_distance = 1; + beacon_config.eth1.node_far_behind_seconds = 20; + beacon_config.dummy_eth1_backend = false; + beacon_config.sync_eth1_chain = true; + beacon_config.eth1.auto_update_interval_millis = eth1_block_time.as_millis() as u64; + beacon_config.eth1.chain_id = Eth1Id::from(chain_id); + beacon_config.network.target_peers = node_count - 1; + + beacon_config.network.enr_address = (Some(Ipv4Addr::LOCALHOST), None); + + if post_merge_sim { + let el_config = execution_layer::Config { + execution_endpoints: vec![SensitiveUrl::parse(&format!( + "http://localhost:{}", + EXECUTION_PORT + )) + .unwrap()], + ..Default::default() + }; + + beacon_config.execution_layer = Some(el_config); + } + + /* + * Create a new `LocalNetwork` with one beacon node. + */ + let network = LocalNetwork::new(context.clone(), beacon_config.clone()).await?; + + /* + * One by one, add beacon nodes to the network. + */ + for _ in 0..node_count - 1 { + network + .add_beacon_node(beacon_config.clone(), false) + .await?; + } + + /* + * One by one, add validators to the network. + */ + let executor = context.executor.clone(); + for (i, files) in validator_files.into_iter().enumerate() { + let network_1 = network.clone(); + let beacon_nodes = if i == vc_count { + vec![i, 0] + } else { + vec![i, i + 1] + }; + executor.spawn( + async move { + let mut validator_config = testing_validator_config(); + if post_merge_sim { + validator_config.fee_recipient = Some(SUGGESTED_FEE_RECIPIENT.into()); + } + println!("Adding validator client {}", i); + network_1 + .add_validator_client_with_fallbacks( + validator_config, + i, + beacon_nodes, + files, + ) + .await + .expect("should add validator"); + }, + "vc", + ); + } + + let duration_to_genesis = network.duration_to_genesis().await; + println!("Duration to genesis: {}", duration_to_genesis.as_secs()); + sleep(duration_to_genesis).await; + + if post_merge_sim { + let executor = executor.clone(); + let network_2 = network.clone(); + executor.spawn( + async move { + println!("Mining pow blocks"); + let mut interval = tokio::time::interval(Duration::from_secs(seconds_per_slot)); + for i in 1..=TERMINAL_BLOCK + 1 { + interval.tick().await; + let _ = network_2.mine_pow_blocks(i); + } + }, + "pow_mining", + ); + } + /* + * Start the checks that ensure the network performs as expected. + * + * We start these checks immediately after the validators have started. This means we're + * relying on the validator futures to all return immediately after genesis so that these + * tests start at the right time. Whilst this is works well for now, it's subject to + * breakage by changes to the VC. + */ + + let ( + //finalization, + //block_prod, + //validator_count, + //onboarding, + fallback, + check_attestations, + //fork, + //sync_aggregate, + //transition, + ) = futures::join!( + //checks::verify_first_finalization(network.clone(), slot_duration), + checks::disconnect_from_execution_layer( + network.clone(), + Epoch::new(BELLATRIX_FORK_EPOCH), + slot_duration + ), + checks::check_attestation_correctness( + network.clone(), + Epoch::new(END_EPOCH), + MinimalEthSpec::slots_per_epoch(), + slot_duration + ), + //checks::stall_node(network.clone(), 0, 30, seconds_per_slot), + ); + + //block_prod?; + //finalization?; + //validator_count?; + //onboarding?; + fallback?; + check_attestations?; + //fork?; + //sync_aggregate?; + //transition?; + + // The `final_future` either completes immediately or never completes, depending on the value + // of `continue_after_checks`. + + if continue_after_checks { + future::pending::<()>().await; + } + /* + * End the simulation by dropping the network. This will kill all running beacon nodes and + * validator clients. + */ + println!( + "Simulation complete. Finished with {} beacon nodes and {} validator clients", + network.beacon_node_count(), + network.validator_client_count() + ); + + // Be explicit about dropping the network, as this kills all the nodes. This ensures + // all the checks have adequate time to pass. + drop(network); + Ok::<(), String>(()) + }; + + env.runtime().block_on(main_future).unwrap(); + + env.fire_signal(); + env.shutdown_on_idle(); + + Ok(()) +} diff --git a/testing/simulator/src/local_network.rs b/testing/simulator/src/local_network.rs index e35870d126c..b3d7afd7b9b 100644 --- a/testing/simulator/src/local_network.rs +++ b/testing/simulator/src/local_network.rs @@ -265,6 +265,48 @@ impl LocalNetwork { Ok(()) } + pub async fn add_validator_client_with_fallbacks( + &self, + mut validator_config: ValidatorConfig, + validator_index: usize, + beacon_nodes: Vec, + validator_files: ValidatorFiles, + ) -> Result<(), String> { + let context = self + .context + .service_context(format!("validator_{}", validator_index)); + let self_1 = self.clone(); + let mut beacon_node_urls = vec![]; + for beacon_node in beacon_nodes { + let socket_addr = { + let read_lock = self.beacon_nodes.read(); + let beacon_node = read_lock + .get(beacon_node) + .ok_or_else(|| format!("No beacon node for index {}", beacon_node))?; + beacon_node + .client + .http_api_listen_addr() + .expect("Must have http started") + }; + let beacon_node = SensitiveUrl::parse( + format!("http://{}:{}", socket_addr.ip(), socket_addr.port()).as_str(), + ) + .unwrap(); + beacon_node_urls.push(beacon_node); + } + + validator_config.beacon_nodes = beacon_node_urls; + + let validator_client = LocalValidatorClient::production_with_insecure_keypairs( + context, + validator_config, + validator_files, + ) + .await?; + self_1.validator_clients.write().push(validator_client); + Ok(()) + } + /// For all beacon nodes in `Self`, return a HTTP client to access each nodes HTTP API. pub fn remote_nodes(&self) -> Result, String> { let beacon_nodes = self.beacon_nodes.read(); diff --git a/testing/simulator/src/main.rs b/testing/simulator/src/main.rs index e8af9c18067..83ca1135efc 100644 --- a/testing/simulator/src/main.rs +++ b/testing/simulator/src/main.rs @@ -19,6 +19,7 @@ extern crate clap; mod checks; mod cli; mod eth1_sim; +mod fallback_sim; mod local_network; mod no_eth1_sim; mod retry; @@ -58,6 +59,13 @@ fn main() { std::process::exit(1) } }, + ("fallback-sim", Some(matches)) => match fallback_sim::run_fallback_sim(matches) { + Ok(()) => println!("Simulation exited successfully"), + Err(e) => { + eprintln!("Simulation exited with an error: {}", e); + std::process::exit(1) + } + }, _ => { eprintln!("Invalid subcommand. Use --help to see available options"); std::process::exit(1) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 531cec08ac5..cb432271e4c 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -2,13 +2,19 @@ //! "fallback" behaviour; it will try a request on all of the nodes until one or none of them //! succeed. -use crate::check_synced::check_synced; +use crate::beacon_node_health::{ + BeaconNodeHealth, BeaconNodeSyncDistanceTiers, ExecutionEngineHealth, SyncDistanceTier, +}; +use crate::check_synced::{check_node_health, check_synced}; use crate::http_metrics::metrics::{inc_counter_vec, ENDPOINT_ERRORS, ENDPOINT_REQUESTS}; use environment::RuntimeContext; use eth2::BeaconNodeHttpClient; use futures::future; +use parking_lot::RwLock as PLRwLock; +use serde_derive::{Deserialize, Serialize}; use slog::{debug, error, info, warn, Logger}; use slot_clock::SlotClock; +use std::cmp::Ordering; use std::fmt; use std::fmt::Debug; use std::future::Future; @@ -16,7 +22,7 @@ use std::marker::PhantomData; use std::sync::Arc; use std::time::{Duration, Instant}; use tokio::{sync::RwLock, time::sleep}; -use types::{ChainSpec, Config, EthSpec}; +use types::{ChainSpec, Config as ConfigSpec, EthSpec}; /// Message emitted when the VC detects the BN is using a different spec. const UPDATE_REQUIRED_LOG_HINT: &str = "this VC or the remote BN may need updating"; @@ -30,6 +36,16 @@ const UPDATE_REQUIRED_LOG_HINT: &str = "this VC or the remote BN may need updati /// having the correct nodes up and running prior to the start of the slot. const SLOT_LOOKAHEAD: Duration = Duration::from_secs(2); +// Configuration for the Beacon Node fallback. +#[derive(Copy, Clone, Debug, Default, Serialize, Deserialize)] +pub struct Config { + /// Disables publishing http api requests to all beacon nodes for select api calls. + pub disable_run_on_all: bool, + /// Sets the number of slots behind the head a beacon node is allowed to be to still be + /// considered `synced`. + pub sync_tolerance: Option, +} + /// Indicates a measurement of latency between the VC and a BN. pub struct LatencyMeasurement { /// An identifier for the beacon node (e.g. the URL). @@ -139,21 +155,52 @@ pub enum CandidateError { Offline, Incompatible, NotSynced, + TimeDiscrepancy, } /// Represents a `BeaconNodeHttpClient` inside a `BeaconNodeFallback` that may or may not be used /// for a query. +#[derive(Debug)] pub struct CandidateBeaconNode { + id: usize, beacon_node: BeaconNodeHttpClient, + health: PLRwLock>, status: RwLock>, _phantom: PhantomData, } +impl PartialEq for CandidateBeaconNode { + fn eq(&self, other: &Self) -> bool { + self.id == other.id && self.beacon_node == other.beacon_node + } +} + +impl Eq for CandidateBeaconNode {} + +impl Ord for CandidateBeaconNode { + fn cmp(&self, other: &Self) -> Ordering { + match (&(*self.health.read()), &(*other.health.read())) { + (None, None) => Ordering::Equal, + (None, _) => Ordering::Greater, + (_, None) => Ordering::Less, + (Some(health_1), Some(health_2)) => health_1.cmp(health_2), + } + } +} + +impl PartialOrd for CandidateBeaconNode { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + impl CandidateBeaconNode { /// Instantiate a new node. - pub fn new(beacon_node: BeaconNodeHttpClient) -> Self { + pub fn new(beacon_node: BeaconNodeHttpClient, id: usize) -> Self { Self { + id, beacon_node, + health: PLRwLock::new(None), status: RwLock::new(Err(CandidateError::Uninitialized)), _phantom: PhantomData, } @@ -204,6 +251,64 @@ impl CandidateBeaconNode { new_status } + pub async fn refresh_health( + &self, + distance_tiers: &BeaconNodeSyncDistanceTiers, + slot_clock: Option<&T>, + spec: &ChainSpec, + log: &Logger, + ) -> Result<(), CandidateError> { + if let Err(e) = self.is_compatible(spec, log).await { + *self.status.write().await = Err(e); + return Ok(()); + } + + if let Some(slot_clock) = slot_clock { + match check_node_health(&self.beacon_node, log).await { + Ok((head, is_optimistic, el_offline)) => { + // Currently ExecutionEngineHealth is solely determined by online status. + let execution_status = if el_offline { + ExecutionEngineHealth::Unhealthy + } else { + ExecutionEngineHealth::Healthy + }; + + let new_health = BeaconNodeHealth::from_status( + self.id, + head, + is_optimistic, + execution_status, + distance_tiers, + slot_clock, + ); + + warn!( + log, + "Health of Beacon Node: {}, updated. Health tier: {}", + new_health.get_id(), + new_health.get_health_tier() + ); + + *self.health.write() = Some(new_health); + *self.status.write().await = Ok(()); + + Ok(()) + } + Err(status) => { + // Set the health as None which is sorted last in the list. + *self.health.write() = None; + *self.status.write().await = Err(status); + Ok(()) + } + } + } else { + // Slot clock will only be None at startup. + // Assume compatible nodes are available. + *self.status.write().await = Ok(()); + Ok(()) + } + } + /// Checks if the node is reachable. async fn is_online(&self, was_offline: bool, log: &Logger) -> Result<(), CandidateError> { let result = self @@ -240,7 +345,7 @@ impl CandidateBeaconNode { async fn is_compatible(&self, spec: &ChainSpec, log: &Logger) -> Result<(), CandidateError> { let config = self .beacon_node - .get_config_spec::() + .get_config_spec::() .await .map_err(|e| { error!( @@ -319,10 +424,12 @@ impl CandidateBeaconNode { /// A collection of `CandidateBeaconNode` that can be used to perform requests with "fallback" /// behaviour, where the failure of one candidate results in the next candidate receiving an /// identical query. +#[derive(Clone, Debug)] pub struct BeaconNodeFallback { - candidates: Vec>, - slot_clock: Option, + candidates: Arc>>>, disable_run_on_all: bool, + distance_tiers: BeaconNodeSyncDistanceTiers, + slot_clock: Option, spec: ChainSpec, log: Logger, } @@ -330,14 +437,16 @@ pub struct BeaconNodeFallback { impl BeaconNodeFallback { pub fn new( candidates: Vec>, - disable_run_on_all: bool, + config: Config, spec: ChainSpec, log: Logger, ) -> Self { + let distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); Self { - candidates, + candidates: Arc::new(RwLock::new(candidates)), + disable_run_on_all: config.disable_run_on_all, + distance_tiers, slot_clock: None, - disable_run_on_all, spec, log, } @@ -353,16 +462,22 @@ impl BeaconNodeFallback { } /// The count of candidates, regardless of their state. - pub fn num_total(&self) -> usize { - self.candidates.len() + pub async fn num_total(&self) -> usize { + self.candidates.read().await.len() } /// The count of synced and ready candidates. pub async fn num_synced(&self) -> usize { let mut n = 0; - for candidate in &self.candidates { - if candidate.status(RequireSynced::Yes).await.is_ok() { - n += 1 + for candidate in self.candidates.read().await.iter() { + if let Some(cand) = candidate.health.read().as_ref() { + if self + .distance_tiers + .distance_tier(cand.health_tier.sync_distance) + == SyncDistanceTier::Synced + { + n += 1 + } } } n @@ -371,9 +486,15 @@ impl BeaconNodeFallback { /// The count of synced and ready fallbacks excluding the primary beacon node candidate. pub async fn num_synced_fallback(&self) -> usize { let mut n = 0; - for candidate in self.candidates.iter().skip(1) { - if candidate.status(RequireSynced::Yes).await.is_ok() { - n += 1 + for candidate in self.candidates.read().await.iter().skip(1) { + if let Some(cand) = candidate.health.read().as_ref() { + if self + .distance_tiers + .distance_tier(cand.health_tier.sync_distance) + == SyncDistanceTier::Synced + { + n += 1 + } } } n @@ -382,7 +503,7 @@ impl BeaconNodeFallback { /// The count of candidates that are online and compatible, but not necessarily synced. pub async fn num_available(&self) -> usize { let mut n = 0; - for candidate in &self.candidates { + for candidate in self.candidates.read().await.iter() { if candidate.status(RequireSynced::No).await.is_ok() { n += 1 } @@ -396,24 +517,36 @@ impl BeaconNodeFallback { /// low quality responses. To route around this it's best to poll all connected beacon nodes. /// A previous implementation of this function polled only the unavailable BNs. pub async fn update_all_candidates(&self) { - let futures = self - .candidates + let candidates = self.candidates.read().await; + + let futures = candidates .iter() .map(|candidate| { - candidate.refresh_status(self.slot_clock.as_ref(), &self.spec, &self.log) + candidate.refresh_health( + &self.distance_tiers, + self.slot_clock.as_ref(), + &self.spec, + &self.log, + ) }) .collect::>(); - // run all updates concurrently and ignore errors + // Run all updates concurrently and ignore errors. let _ = future::join_all(futures).await; + + drop(candidates); + + // Sort the list to put the healthiest candidate first. + let mut write = self.candidates.write().await; + write.sort(); } /// Concurrently send a request to all candidates (regardless of /// offline/online) status and attempt to collect a rough reading on the /// latency between the VC and candidate. pub async fn measure_latency(&self) -> Vec { - let futures: Vec<_> = self - .candidates + let candidates = self.candidates.read().await; + let futures: Vec<_> = candidates .iter() .map(|candidate| async { let beacon_node_id = candidate.beacon_node.to_string(); @@ -455,20 +588,18 @@ impl BeaconNodeFallback { /// First this function will try all nodes with a suitable status. If no candidates are suitable /// or all the requests fail, it will try updating the status of all unsuitable nodes and /// re-running `func` again. - pub async fn first_success<'a, F, O, Err, R>( - &'a self, - require_synced: RequireSynced, - offline_on_failure: OfflineOnFailure, + pub async fn first_success( + &self, + _require_synced: RequireSynced, + _offline_on_failure: OfflineOnFailure, func: F, ) -> Result> where - F: Fn(&'a BeaconNodeHttpClient) -> R, + F: Fn(BeaconNodeHttpClient) -> R, R: Future>, Err: Debug, { let mut errors = vec![]; - let mut to_retry = vec![]; - let mut retry_unsynced = vec![]; let log = &self.log.clone(); // Run `func` using a `candidate`, returning the value or capturing errors. @@ -481,7 +612,7 @@ impl BeaconNodeFallback { // There exists a race condition where `func` may be called when the candidate is // actually not ready. We deem this an acceptable inefficiency. - match func(&$candidate.beacon_node).await { + match func($candidate.beacon_node.clone()).await { Ok(val) => return Ok(val), Err(e) => { debug!( @@ -495,9 +626,9 @@ impl BeaconNodeFallback { // There exists a race condition where the candidate may have been marked // as ready between the `func` call and now. We deem this an acceptable // inefficiency. - if matches!(offline_on_failure, OfflineOnFailure::Yes) { - $candidate.set_offline().await; - } + //if matches!(offline_on_failure, OfflineOnFailure::Yes) { + // $candidate.set_offline().await; + //} errors.push(($candidate.beacon_node.to_string(), Error::RequestFailed(e))); inc_counter_vec(&ENDPOINT_ERRORS, &[$candidate.beacon_node.as_ref()]); } @@ -508,53 +639,9 @@ impl BeaconNodeFallback { // First pass: try `func` on all synced and ready candidates. // // This ensures that we always choose a synced node if it is available. - for candidate in &self.candidates { - match candidate.status(RequireSynced::Yes).await { - Err(e @ CandidateError::NotSynced) if require_synced == false => { - // This client is unsynced we will try it after trying all synced clients - retry_unsynced.push(candidate); - errors.push((candidate.beacon_node.to_string(), Error::Unavailable(e))); - } - Err(e) => { - // This client was not ready on the first pass, we might try it again later. - to_retry.push(candidate); - errors.push((candidate.beacon_node.to_string(), Error::Unavailable(e))); - } - _ => try_func!(candidate), - } - } - - // Second pass: try `func` on ready unsynced candidates. This only runs if we permit - // unsynced candidates. - // - // Due to async race-conditions, it is possible that we will send a request to a candidate - // that has been set to an offline/unready status. This is acceptable. - if require_synced == false { - for candidate in retry_unsynced { - try_func!(candidate); - } - } - - // Third pass: try again, attempting to make non-ready clients become ready. - for candidate in to_retry { - // If the candidate hasn't luckily transferred into the correct state in the meantime, - // force an update of the state. - let new_status = match candidate.status(require_synced).await { - Ok(()) => Ok(()), - Err(_) => { - candidate - .refresh_status(self.slot_clock.as_ref(), &self.spec, &self.log) - .await - } - }; - - match new_status { - Ok(()) => try_func!(candidate), - Err(CandidateError::NotSynced) if require_synced == false => try_func!(candidate), - Err(e) => { - errors.push((candidate.beacon_node.to_string(), Error::Unavailable(e))); - } - } + let candidates = self.candidates.read().await; + for candidate in candidates.iter() { + try_func!(candidate); } // There were no candidates already ready and we were unable to make any of them ready. @@ -571,19 +658,17 @@ impl BeaconNodeFallback { /// It returns a list of errors along with the beacon node id that failed for `func`. /// Since this ignores the actual result of `func`, this function should only be used for beacon /// node calls whose results we do not care about, only that they completed successfully. - pub async fn run_on_all<'a, F, O, Err, R>( - &'a self, - require_synced: RequireSynced, - offline_on_failure: OfflineOnFailure, + pub async fn run_on_all( + &self, + _require_synced: RequireSynced, + _offline_on_failure: OfflineOnFailure, func: F, ) -> Result<(), Errors> where - F: Fn(&'a BeaconNodeHttpClient) -> R, + F: Fn(BeaconNodeHttpClient) -> R, R: Future>, { let mut results = vec![]; - let mut to_retry = vec![]; - let mut retry_unsynced = vec![]; // Run `func` using a `candidate`, returning the value or capturing errors. // @@ -595,7 +680,7 @@ impl BeaconNodeFallback { // There exists a race condition where `func` may be called when the candidate is // actually not ready. We deem this an acceptable inefficiency. - match func(&$candidate.beacon_node).await { + match func($candidate.beacon_node.clone()).await { Ok(val) => results.push(Ok(val)), Err(e) => { // If we have an error on this function, make the client as not-ready. @@ -603,9 +688,9 @@ impl BeaconNodeFallback { // There exists a race condition where the candidate may have been marked // as ready between the `func` call and now. We deem this an acceptable // inefficiency. - if matches!(offline_on_failure, OfflineOnFailure::Yes) { - $candidate.set_offline().await; - } + //if matches!(offline_on_failure, OfflineOnFailure::Yes) { + // $candidate.set_offline().await; + //} results.push(Err(( $candidate.beacon_node.to_string(), Error::RequestFailed(e), @@ -619,54 +704,9 @@ impl BeaconNodeFallback { // First pass: try `func` on all synced and ready candidates. // // This ensures that we always choose a synced node if it is available. - for candidate in &self.candidates { - match candidate.status(RequireSynced::Yes).await { - Err(CandidateError::NotSynced) if require_synced == false => { - // This client is unsynced we will try it after trying all synced clients - retry_unsynced.push(candidate); - } - Err(_) => { - // This client was not ready on the first pass, we might try it again later. - to_retry.push(candidate); - } - Ok(_) => try_func!(candidate), - } - } - - // Second pass: try `func` on ready unsynced candidates. This only runs if we permit - // unsynced candidates. - // - // Due to async race-conditions, it is possible that we will send a request to a candidate - // that has been set to an offline/unready status. This is acceptable. - if require_synced == false { - for candidate in retry_unsynced { - try_func!(candidate); - } - } - - // Third pass: try again, attempting to make non-ready clients become ready. - for candidate in to_retry { - // If the candidate hasn't luckily transferred into the correct state in the meantime, - // force an update of the state. - let new_status = match candidate.status(require_synced).await { - Ok(()) => Ok(()), - Err(_) => { - candidate - .refresh_status(self.slot_clock.as_ref(), &self.spec, &self.log) - .await - } - }; - - match new_status { - Ok(()) => try_func!(candidate), - Err(CandidateError::NotSynced) if require_synced == false => try_func!(candidate), - Err(e) => { - results.push(Err(( - candidate.beacon_node.to_string(), - Error::Unavailable(e), - ))); - } - } + let candidates = self.candidates.read().await; + for candidate in candidates.iter() { + try_func!(candidate); } let errors: Vec<_> = results.into_iter().filter_map(|res| res.err()).collect(); @@ -680,14 +720,14 @@ impl BeaconNodeFallback { /// Call `func` on first beacon node that returns success or on all beacon nodes /// depending on the value of `disable_run_on_all`. - pub async fn run<'a, F, Err, R>( - &'a self, + pub async fn run( + &self, require_synced: RequireSynced, offline_on_failure: OfflineOnFailure, func: F, ) -> Result<(), Errors> where - F: Fn(&'a BeaconNodeHttpClient) -> R, + F: Fn(BeaconNodeHttpClient) -> R, R: Future>, Err: Debug, { @@ -701,3 +741,154 @@ impl BeaconNodeFallback { } } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::beacon_node_health::BeaconNodeHealthTier; + use crate::SensitiveUrl; + use eth2::Timeouts; + use types::{MainnetEthSpec, Slot}; + + type E = MainnetEthSpec; + + #[test] + fn check_candidate_order() { + let candidate_1: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_1.com").unwrap(), + Timeouts::set_all(Duration::from_secs(1)), + ), + 1, + ); + let expected_candidate_1: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_1.com").unwrap(), + Timeouts::set_all(Duration::from_secs(1)), + ), + 1, + ); + let candidate_2: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_2.com").unwrap(), + Timeouts::set_all(Duration::from_secs(2)), + ), + 2, + ); + let expected_candidate_2: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_2.com").unwrap(), + Timeouts::set_all(Duration::from_secs(2)), + ), + 2, + ); + let candidate_3: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_3.com").unwrap(), + Timeouts::set_all(Duration::from_secs(3)), + ), + 3, + ); + let expected_candidate_3: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_3.com").unwrap(), + Timeouts::set_all(Duration::from_secs(3)), + ), + 3, + ); + let candidate_4: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_4.com").unwrap(), + Timeouts::set_all(Duration::from_secs(4)), + ), + 3, + ); + let expected_candidate_4: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_4.com").unwrap(), + Timeouts::set_all(Duration::from_secs(4)), + ), + 3, + ); + let candidate_5: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_5.com").unwrap(), + Timeouts::set_all(Duration::from_secs(5)), + ), + 3, + ); + let expected_candidate_5: CandidateBeaconNode = CandidateBeaconNode::new( + BeaconNodeHttpClient::new( + SensitiveUrl::parse("http://example_5.com").unwrap(), + Timeouts::set_all(Duration::from_secs(5)), + ), + 3, + ); + + // All health parameters other than `health_tier` are irrelevant for ordering. + let health_1 = BeaconNodeHealth { + id: 1, + head: Slot::new(99), + optimistic_status: false, + execution_status: ExecutionEngineHealth::Healthy, + health_tier: BeaconNodeHealthTier::new(1, Slot::new(1)), + }; + + let health_2 = BeaconNodeHealth { + id: 2, + head: Slot::new(99), + optimistic_status: false, + execution_status: ExecutionEngineHealth::Healthy, + health_tier: BeaconNodeHealthTier::new(2, Slot::new(1)), + }; + + let health_3 = BeaconNodeHealth { + id: 3, + head: Slot::new(99), + optimistic_status: false, + execution_status: ExecutionEngineHealth::Healthy, + health_tier: BeaconNodeHealthTier::new(3, Slot::new(1)), + }; + + let health_4 = BeaconNodeHealth { + id: 4, + head: Slot::new(99), + optimistic_status: false, + execution_status: ExecutionEngineHealth::Healthy, + health_tier: BeaconNodeHealthTier::new(4, Slot::new(1)), + }; + + let health_5 = BeaconNodeHealth { + id: 5, + head: Slot::new(99), + optimistic_status: false, + execution_status: ExecutionEngineHealth::Unhealthy, + health_tier: BeaconNodeHealthTier::new(4, Slot::new(5)), + }; + + *candidate_1.health.write() = Some(health_1); + *candidate_2.health.write() = Some(health_2); + *candidate_3.health.write() = Some(health_3); + *candidate_4.health.write() = Some(health_4); + *candidate_5.health.write() = Some(health_5); + + let mut candidates = vec![ + candidate_3, + candidate_5, + candidate_1, + candidate_4, + candidate_2, + ]; + let expected_candidates = vec![ + expected_candidate_1, + expected_candidate_2, + expected_candidate_3, + expected_candidate_4, + expected_candidate_5, + ]; + + candidates.sort(); + + assert_eq!(candidates, expected_candidates); + } +} diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs new file mode 100644 index 00000000000..3ab6b473e0e --- /dev/null +++ b/validator_client/src/beacon_node_health.rs @@ -0,0 +1,363 @@ +use crate::beacon_node_fallback::Config; +use slot_clock::SlotClock; +use std::cmp::Ordering; +use std::fmt::{Debug, Display, Formatter}; +use types::Slot; + +// Sync distances between 0 and DEFAULT_SYNC_TOLERANCE are considered `synced`. +// Sync distance tiers are determined by the different modifiers. +const DEFAULT_SYNC_TOLERANCE: Slot = Slot::new(4); +const SYNC_DISTANCE_SMALL_MODIFIER: Slot = Slot::new(7); +const SYNC_DISTANCE_MEDIUM_MODIFIER: Slot = Slot::new(31); + +type HealthTier = u8; +type SyncDistance = Slot; +type OptimisticStatus = bool; + +/// Helpful enum which is used when pattern matching to determine health tier. +#[derive(PartialEq, Debug)] +pub enum SyncDistanceTier { + Synced, + Small, + Medium, + Large, +} + +/// Contains the different sync distance tiers which are determined at runtime by the +/// `sync_tolerance` CLI flag. +#[derive(Clone, Debug)] +pub struct BeaconNodeSyncDistanceTiers { + synced: SyncDistance, + small: SyncDistance, + medium: SyncDistance, +} + +impl BeaconNodeSyncDistanceTiers { + pub fn from_config(config: &Config) -> Self { + if let Some(sync_tolerance) = config.sync_tolerance { + Self { + synced: Slot::new(sync_tolerance), + small: Slot::new(sync_tolerance) + SYNC_DISTANCE_SMALL_MODIFIER, + medium: Slot::new(sync_tolerance) + SYNC_DISTANCE_MEDIUM_MODIFIER, + } + } else { + Self::default() + } + } + + /// Takes a given sync distance and determines its tier based on the `sync_tolerance` defined by + /// the CLI. + pub fn distance_tier(&self, distance: SyncDistance) -> SyncDistanceTier { + let distance = distance.as_u64(); + // Add 1 since we are using exclusive ranges. + let synced = self.synced.as_u64() + 1; + let small = self.small.as_u64() + 1; + let medium = self.medium.as_u64() + 1; + + if (0..synced).contains(&distance) { + SyncDistanceTier::Synced + } else if (synced..small).contains(&distance) { + SyncDistanceTier::Small + } else if (small..medium).contains(&distance) { + SyncDistanceTier::Medium + } else { + SyncDistanceTier::Large + } + } +} + +impl Default for BeaconNodeSyncDistanceTiers { + fn default() -> Self { + Self { + synced: DEFAULT_SYNC_TOLERANCE, + small: DEFAULT_SYNC_TOLERANCE + SYNC_DISTANCE_SMALL_MODIFIER, + medium: DEFAULT_SYNC_TOLERANCE + SYNC_DISTANCE_MEDIUM_MODIFIER, + } + } +} + +/// Execution Node health metrics. +#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] +#[allow(dead_code)] +pub enum ExecutionEngineHealth { + Healthy, + Unhealthy, +} + +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub struct BeaconNodeHealthTier { + pub tier: HealthTier, + pub sync_distance: SyncDistance, +} + +impl Display for BeaconNodeHealthTier { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Tier{}({})", self.tier, self.sync_distance) + } +} + +impl Ord for BeaconNodeHealthTier { + fn cmp(&self, other: &Self) -> Ordering { + self.tier.cmp(&other.tier) + } +} + +impl PartialOrd for BeaconNodeHealthTier { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl BeaconNodeHealthTier { + pub fn new(tier: HealthTier, sync_distance: SyncDistance) -> Self { + Self { + tier, + sync_distance, + } + } +} + +/// Beacon Node Health metrics. +#[derive(Debug, PartialEq, Eq)] +pub struct BeaconNodeHealth { + // The ID of the Beacon Node. This should correspond with its position in the `--beacon-nodes` + // list. Note that the ID field is used to tie-break nodes with the same health so that nodes + // with a lower ID are preferred. + pub id: usize, + // The slot number of the head. + pub head: Slot, + // Whether the node is optimistically synced. + pub optimistic_status: OptimisticStatus, + // The status of the nodes connected Execution Engine. + pub execution_status: ExecutionEngineHealth, + // The overall health tier of the Beacon Node. Used to rank the nodes for the purposes of + // fallbacks. + pub health_tier: BeaconNodeHealthTier, +} + +impl Ord for BeaconNodeHealth { + fn cmp(&self, other: &Self) -> Ordering { + let ordering = self.health_tier.cmp(&other.health_tier); + if ordering == Ordering::Equal { + // Tie-break node health by ID. + self.id.cmp(&other.id) + } else { + ordering + } + } +} + +impl PartialOrd for BeaconNodeHealth { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl BeaconNodeHealth { + pub fn from_status( + id: usize, + head: Slot, + optimistic_status: OptimisticStatus, + execution_status: ExecutionEngineHealth, + distance_tiers: &BeaconNodeSyncDistanceTiers, + slot_clock: &T, + ) -> Self { + let sync_distance = BeaconNodeHealth::compute_sync_distance(head, slot_clock); + let health_tier = BeaconNodeHealth::compute_health_tier( + sync_distance, + optimistic_status, + execution_status, + distance_tiers, + ); + + Self { + id, + head, + optimistic_status, + execution_status, + health_tier, + } + } + + pub fn get_id(&self) -> usize { + self.id + } + + pub fn get_health_tier(&self) -> BeaconNodeHealthTier { + self.health_tier + } + + fn compute_sync_distance(head: Slot, slot_clock: &T) -> SyncDistance { + // TODO(mac) May be worth distinguishing between nodes that are ahead of the `slot_clock`. + slot_clock + .now() + .map(|head_slot| head_slot.saturating_sub(head)) + .unwrap_or(Slot::max_value()) + } + + fn compute_health_tier( + sync_distance: SyncDistance, + optimistic_status: OptimisticStatus, + execution_status: ExecutionEngineHealth, + sync_distance_tiers: &BeaconNodeSyncDistanceTiers, + ) -> BeaconNodeHealthTier { + let sync_distance_tier = sync_distance_tiers.distance_tier(sync_distance); + let health = (sync_distance_tier, optimistic_status, execution_status); + + match health { + (SyncDistanceTier::Synced, false, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(1, sync_distance) + } + (SyncDistanceTier::Small, false, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(2, sync_distance) + } + (SyncDistanceTier::Synced, false, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(3, sync_distance) + } + (SyncDistanceTier::Medium, false, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(4, sync_distance) + } + (SyncDistanceTier::Synced, true, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(5, sync_distance) + } + (SyncDistanceTier::Synced, true, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(6, sync_distance) + } + (SyncDistanceTier::Small, false, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(7, sync_distance) + } + (SyncDistanceTier::Small, true, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(8, sync_distance) + } + (SyncDistanceTier::Small, true, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(9, sync_distance) + } + (SyncDistanceTier::Large, false, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(10, sync_distance) + } + (SyncDistanceTier::Medium, false, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(11, sync_distance) + } + (SyncDistanceTier::Medium, true, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(12, sync_distance) + } + (SyncDistanceTier::Medium, true, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(13, sync_distance) + } + (SyncDistanceTier::Large, false, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(14, sync_distance) + } + (SyncDistanceTier::Large, true, ExecutionEngineHealth::Healthy) => { + BeaconNodeHealthTier::new(15, sync_distance) + } + (SyncDistanceTier::Large, true, ExecutionEngineHealth::Unhealthy) => { + BeaconNodeHealthTier::new(16, sync_distance) + } + } + } +} + +#[cfg(test)] +mod tests { + + use super::ExecutionEngineHealth::{Healthy, Unhealthy}; + use super::{BeaconNodeHealth, BeaconNodeSyncDistanceTiers, SyncDistanceTier}; + use crate::beacon_node_fallback::Config; + use slot_clock::{SlotClock, TestingSlotClock}; + use std::time::Duration; + use types::Slot; + + #[test] + fn all_possible_health_tiers() { + let current_head = Slot::new(64); + + let config = Config::default(); + let beacon_node_sync_distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); + + let slot_clock = + TestingSlotClock::new(current_head, Duration::from_secs(0), Duration::from_secs(1)); + + let mut health_vec = vec![]; + + for head_slot in (0..=64).rev() { + for optimistic_status in &[false, true] { + for ee_health in &[Healthy, Unhealthy] { + let health = BeaconNodeHealth::from_status( + 0, + Slot::new(head_slot), + *optimistic_status, + *ee_health, + &beacon_node_sync_distance_tiers, + &slot_clock, + ); + health_vec.push(health); + } + } + } + + for health in health_vec { + let health_tier = health.get_health_tier(); + let tier = health_tier.tier; + let distance = health_tier.sync_distance; + + let distance_tier = beacon_node_sync_distance_tiers.distance_tier(distance); + + // Check sync distance. + if [1, 3, 5, 6].contains(&tier) { + assert!(distance_tier == SyncDistanceTier::Synced) + } else if [2, 7, 8, 9].contains(&tier) { + assert!(distance_tier == SyncDistanceTier::Small); + } else if [4, 11, 12, 13].contains(&tier) { + assert!(distance_tier == SyncDistanceTier::Medium); + } else { + assert!(distance_tier == SyncDistanceTier::Large); + } + + // Check optimistic status. + if [1, 2, 3, 4, 7, 10, 11, 14].contains(&tier) { + assert!(!health.optimistic_status); + } else { + assert!(health.optimistic_status); + } + + // Check execution health. + if [3, 6, 7, 9, 11, 13, 14, 16].contains(&tier) { + assert_eq!(health.execution_status, Unhealthy); + } else { + assert_eq!(health.execution_status, Healthy); + } + } + } + + #[test] + fn sync_tolerance() { + let config = Config { + disable_run_on_all: false, + sync_tolerance: Some(8), + }; + let distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); + + let synced_low = + BeaconNodeHealth::compute_health_tier(Slot::new(0), false, Healthy, &distance_tiers); + let synced_high = + BeaconNodeHealth::compute_health_tier(Slot::new(8), false, Healthy, &distance_tiers); + let small_low = + BeaconNodeHealth::compute_health_tier(Slot::new(9), false, Healthy, &distance_tiers); + let small_high = + BeaconNodeHealth::compute_health_tier(Slot::new(15), false, Healthy, &distance_tiers); + let medium_low = + BeaconNodeHealth::compute_health_tier(Slot::new(16), false, Healthy, &distance_tiers); + let medium_high = + BeaconNodeHealth::compute_health_tier(Slot::new(39), false, Healthy, &distance_tiers); + let large = + BeaconNodeHealth::compute_health_tier(Slot::new(40), false, Healthy, &distance_tiers); + + assert!(synced_low.tier == 1); + assert!(synced_high.tier == 1); + assert!(small_low.tier == 2); + assert!(small_high.tier == 2); + assert!(medium_low.tier == 4); + assert!(medium_high.tier == 4); + assert!(large.tier == 10); + } +} diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index 2a09455b6ff..8b0b5dcacbc 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -145,14 +145,14 @@ pub struct ProposerFallback { impl ProposerFallback { // Try `func` on `self.proposer_nodes` first. If that doesn't work, try `self.beacon_nodes`. - pub async fn first_success_try_proposers_first<'a, F, O, Err, R>( - &'a self, + pub async fn first_success_try_proposers_first( + &self, require_synced: RequireSynced, offline_on_failure: OfflineOnFailure, func: F, ) -> Result> where - F: Fn(&'a BeaconNodeHttpClient) -> R + Clone, + F: Fn(BeaconNodeHttpClient) -> R + Clone, R: Future>, Err: Debug, { @@ -173,14 +173,14 @@ impl ProposerFallback { } // Try `func` on `self.beacon_nodes` first. If that doesn't work, try `self.proposer_nodes`. - pub async fn first_success_try_proposers_last<'a, F, O, Err, R>( - &'a self, + pub async fn first_success_try_proposers_last( + &self, require_synced: RequireSynced, offline_on_failure: OfflineOnFailure, func: F, ) -> Result> where - F: Fn(&'a BeaconNodeHttpClient) -> R + Clone, + F: Fn(BeaconNodeHttpClient) -> R + Clone, R: Future>, Err: Debug, { diff --git a/validator_client/src/check_synced.rs b/validator_client/src/check_synced.rs index fb88d33dae3..be6d591c9ef 100644 --- a/validator_client/src/check_synced.rs +++ b/validator_client/src/check_synced.rs @@ -1,5 +1,5 @@ use crate::beacon_node_fallback::CandidateError; -use eth2::BeaconNodeHttpClient; +use eth2::{types::Slot, BeaconNodeHttpClient}; use slog::{debug, error, warn, Logger}; use slot_clock::SlotClock; @@ -70,6 +70,8 @@ pub async fn check_synced( "local_slot" => local_slot, "endpoint" => %beacon_node, ); + + return Err(CandidateError::TimeDiscrepancy); } } } @@ -80,3 +82,29 @@ pub async fn check_synced( Err(CandidateError::NotSynced) } } + +pub async fn check_node_health( + beacon_node: &BeaconNodeHttpClient, + log: &Logger, +) -> Result<(Slot, bool, bool), CandidateError> { + let resp = match beacon_node.get_node_syncing().await { + Ok(resp) => resp, + Err(e) => { + warn!( + log, + "Unable connect to beacon node"; + "error" => %e + ); + + return Err(CandidateError::Offline); + } + }; + + Ok(( + resp.data.head_slot, + // Note that optimistic and EL status will both default to their healthy variants which may + // be undesirable. + resp.data.is_optimistic.unwrap_or(false), + resp.data.el_offline.unwrap_or(false), + )) +} diff --git a/validator_client/src/cli.rs b/validator_client/src/cli.rs index 0789ac78a00..012c31ff736 100644 --- a/validator_client/src/cli.rs +++ b/validator_client/src/cli.rs @@ -363,6 +363,14 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { .default_value("500") .takes_value(true), ) + .arg( + Arg::with_name("beacon-node-sync-tolerance") + .long("beacon-node-sync-tolerance") + .help("Sets the number of slots behind the head that each connected Beacon Node can be \ + to still be considered synced. Effectively this gives more priority to the first \ + connected Beacon Node.") + .takes_value(true), + ) /* * Experimental/development options. */ diff --git a/validator_client/src/config.rs b/validator_client/src/config.rs index 7c662db9371..81e1ac1411f 100644 --- a/validator_client/src/config.rs +++ b/validator_client/src/config.rs @@ -1,5 +1,5 @@ use crate::graffiti_file::GraffitiFile; -use crate::{http_api, http_metrics}; +use crate::{beacon_node_fallback, http_api, http_metrics}; use clap::ArgMatches; use clap_utils::{flags::DISABLE_MALLOC_TUNING_FLAG, parse_optional, parse_required}; use directory::{ @@ -19,7 +19,7 @@ use types::{Address, GRAFFITI_BYTES_LEN}; pub const DEFAULT_BEACON_NODE: &str = "http://localhost:5052/"; /// Stores the core configuration for this validator instance. -#[derive(Clone, Serialize, Deserialize)] +#[derive(Clone, Debug, Serialize, Deserialize)] pub struct Config { /// The data directory, which stores all validator databases pub validator_dir: PathBuf, @@ -50,6 +50,8 @@ pub struct Config { pub http_api: http_api::Config, /// Configuration for the HTTP REST API. pub http_metrics: http_metrics::Config, + /// Configuration for the Beacon Node fallback. + pub beacon_node_fallback: beacon_node_fallback::Config, /// Configuration for sending metrics to a remote explorer endpoint. pub monitoring_api: Option, /// If true, enable functionality that monitors the network for attestations or proposals from @@ -73,8 +75,6 @@ pub struct Config { /// /// This is *not* recommended in prod and should only be used for testing. pub block_delay: Option, - /// Disables publishing http api requests to all beacon nodes for select api calls. - pub disable_run_on_all: bool, /// Enables a service which attempts to measure latency between the VC and BNs. pub enable_latency_measurement_service: bool, /// Defines the number of validators per `validator/register_validator` request sent to the BN. @@ -109,6 +109,7 @@ impl Default for Config { fee_recipient: None, http_api: <_>::default(), http_metrics: <_>::default(), + beacon_node_fallback: <_>::default(), monitoring_api: None, enable_doppelganger_protection: false, enable_high_validator_count_metrics: false, @@ -117,7 +118,6 @@ impl Default for Config { builder_proposals: false, builder_registration_timestamp_override: None, gas_limit: None, - disable_run_on_all: false, enable_latency_measurement_service: true, validator_registration_batch_size: 500, } @@ -215,7 +215,6 @@ impl Config { "msg" => "it no longer has any effect", ); } - config.disable_run_on_all = cli_args.is_present("disable-run-on-all"); config.disable_auto_discover = cli_args.is_present("disable-auto-discover"); config.init_slashing_protection = cli_args.is_present("init-slashing-protection"); config.use_long_timeouts = cli_args.is_present("use-long-timeouts"); @@ -258,6 +257,20 @@ impl Config { config.beacon_nodes_tls_certs = Some(tls_certs.split(',').map(PathBuf::from).collect()); } + /* + * Beacon node fallback + */ + + config.beacon_node_fallback.disable_run_on_all = cli_args.is_present("disable-run-on-all"); + + if let Some(sync_tolerance) = cli_args.value_of("beacon-node-sync-tolerance") { + config.beacon_node_fallback.sync_tolerance = Some( + sync_tolerance + .parse::() + .map_err(|_| "beacon-node-sync-tolerance is not a valid u64.")?, + ); + } + /* * Http API server */ diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index 6f071055a4a..de6dc55f5bf 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -1,5 +1,6 @@ mod attestation_service; mod beacon_node_fallback; +mod beacon_node_health; mod block_service; mod check_synced; mod cli; @@ -334,15 +335,18 @@ impl ProductionValidatorClient { .collect::, String>>()?; let num_nodes = beacon_nodes.len(); + let candidates = beacon_nodes .into_iter() - .map(CandidateBeaconNode::new) + .zip(0..num_nodes) + .map(|(node, id)| CandidateBeaconNode::new(node, id)) .collect(); let proposer_nodes_num = proposer_nodes.len(); let proposer_candidates = proposer_nodes .into_iter() - .map(CandidateBeaconNode::new) + .zip(0..num_nodes) + .map(|(node, id)| CandidateBeaconNode::new(node, id)) .collect(); // Set the count for beacon node fallbacks excluding the primary beacon node. @@ -364,14 +368,14 @@ impl ProductionValidatorClient { let mut beacon_nodes: BeaconNodeFallback<_, T> = BeaconNodeFallback::new( candidates, - config.disable_run_on_all, + config.beacon_node_fallback, context.eth2_config.spec.clone(), log.clone(), ); let mut proposer_nodes: BeaconNodeFallback<_, T> = BeaconNodeFallback::new( proposer_candidates, - config.disable_run_on_all, + config.beacon_node_fallback, context.eth2_config.spec.clone(), log.clone(), ); @@ -625,10 +629,10 @@ async fn init_from_beacon_node( proposer_nodes.update_all_candidates().await; let num_available = beacon_nodes.num_available().await; - let num_total = beacon_nodes.num_total(); + let num_total = beacon_nodes.num_total().await; let proposer_available = proposer_nodes.num_available().await; - let proposer_total = proposer_nodes.num_total(); + let proposer_total = proposer_nodes.num_total().await; if proposer_total > 0 && proposer_available == 0 { warn!( diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index 909e64a78a6..e8b5f4c5ba2 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -49,7 +49,7 @@ async fn notify( &http_metrics::metrics::SYNCED_BEACON_NODES_COUNT, num_synced as i64, ); - let num_total = duties_service.beacon_nodes.num_total(); + let num_total = duties_service.beacon_nodes.num_total().await; set_gauge( &http_metrics::metrics::TOTAL_BEACON_NODES_COUNT, num_total as i64, From 52f2be5cf24044bafa24cea2c5f25f1d6947f5fc Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 13 Jun 2023 17:34:46 +1000 Subject: [PATCH 02/50] Add CI workflow for fallback simulator --- .github/workflows/test-suite.yml | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/.github/workflows/test-suite.yml b/.github/workflows/test-suite.yml index 91a0b734537..2abff41835c 100644 --- a/.github/workflows/test-suite.yml +++ b/.github/workflows/test-suite.yml @@ -213,7 +213,22 @@ jobs: run: rustup update stable - name: Run the beacon chain sim without an eth1 connection run: cargo run --release --bin simulator no-eth1-sim - syncing-simulator-ubuntu: + fallback-simulator-ubuntu: + name: fallback-simulator-ubuntu + runs-on: ubuntu-latest + needs: cargo-fmt + steps: + - uses: actions/checkout@v3 + - name: Get latest version of stable Rust + run: rustup update stable + - name: Install Protoc + uses: arduino/setup-protoc@e52d9eb8f7b63115df1ac544a1376fdbf5a39612 + with: + repo-token: ${{ secrets.GITHUB_TOKEN }} + - name: Install anvil + run: cargo install --git https://github.com/foundry-rs/foundry --locked anvil + - name: Run the fallback simulator + run: cargo run --release --bin simulator fallback-sim fallback-simulator-ubuntu: name: syncing-simulator-ubuntu runs-on: ubuntu-latest needs: cargo-fmt @@ -355,4 +370,4 @@ jobs: - name: Use Rust beta run: rustup override set beta - name: Run make - run: make + run: make \ No newline at end of file From ce4df50af55522bd7748d0b50174c99a71510f79 Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 13 Jun 2023 17:35:18 +1000 Subject: [PATCH 03/50] Tie-break with sync distance for non-synced nodes --- validator_client/src/beacon_node_health.rs | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 3ab6b473e0e..33940102c1f 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -98,7 +98,19 @@ impl Display for BeaconNodeHealthTier { impl Ord for BeaconNodeHealthTier { fn cmp(&self, other: &Self) -> Ordering { - self.tier.cmp(&other.tier) + let ordering = self.tier.cmp(&other.tier); + if ordering == Ordering::Equal { + // These tiers represent `synced`. + if [1, 3, 5, 6].contains(&self.tier) { + // Don't tie-break on sync distance in these cases. + // This ensures validator clients don't artificially prefer one node. + ordering + } else { + self.sync_distance.cmp(&other.sync_distance) + } + } else { + ordering + } } } From 6262013ae09340f9baa6ab8a4f61bd3bdecad59d Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 13 Jun 2023 17:38:34 +1000 Subject: [PATCH 04/50] Fix simulator --- .github/workflows/test-suite.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/test-suite.yml b/.github/workflows/test-suite.yml index 2abff41835c..c1e6a1913c8 100644 --- a/.github/workflows/test-suite.yml +++ b/.github/workflows/test-suite.yml @@ -228,7 +228,8 @@ jobs: - name: Install anvil run: cargo install --git https://github.com/foundry-rs/foundry --locked anvil - name: Run the fallback simulator - run: cargo run --release --bin simulator fallback-sim fallback-simulator-ubuntu: + run: cargo run --release --bin simulator fallback-sim + syncing-simulator-ubuntu: name: syncing-simulator-ubuntu runs-on: ubuntu-latest needs: cargo-fmt From f5154be2071ec6a3a732182a2097b5ce8019a08c Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 14 Jun 2023 16:51:16 +1000 Subject: [PATCH 05/50] Cleanup unused code --- beacon_node/http_api/tests/tests.rs | 3 +- validator_client/src/attestation_service.rs | 109 ++++----- validator_client/src/beacon_node_fallback.rs | 153 +----------- validator_client/src/block_service.rs | 217 ++++++++---------- validator_client/src/check_synced.rs | 83 +------ validator_client/src/doppelganger_service.rs | 39 ++-- validator_client/src/duties_service.rs | 96 ++++---- validator_client/src/duties_service/sync.rs | 23 +- validator_client/src/lib.rs | 15 +- validator_client/src/preparation_service.rs | 27 +-- .../src/sync_committee_service.rs | 71 +++--- 11 files changed, 254 insertions(+), 582 deletions(-) diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index 26718ad294a..1ab519dd906 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -1274,7 +1274,6 @@ impl ApiTester { .await .0; - assert!(self .client .clone() @@ -5124,4 +5123,4 @@ async fn optimistic_responses() { .await .test_check_optimistic_responses() .await; -} \ No newline at end of file +} diff --git a/validator_client/src/attestation_service.rs b/validator_client/src/attestation_service.rs index f0a9258c747..9869fb8c3f9 100644 --- a/validator_client/src/attestation_service.rs +++ b/validator_client/src/attestation_service.rs @@ -1,9 +1,8 @@ -use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced}; +use crate::beacon_node_fallback::BeaconNodeFallback; use crate::{ duties_service::{DutiesService, DutyAndProof}, http_metrics::metrics, validator_store::{Error as ValidatorStoreError, ValidatorStore}, - OfflineOnFailure, }; use environment::RuntimeContext; use futures::future::join_all; @@ -338,21 +337,17 @@ impl AttestationService { let attestation_data = self .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::ATTESTATION_SERVICE_TIMES, - &[metrics::ATTESTATIONS_HTTP_GET], - ); - beacon_node - .get_validator_attestation_data(slot, committee_index) - .await - .map_err(|e| format!("Failed to produce attestation data: {:?}", e)) - .map(|result| result.data) - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::ATTESTATION_SERVICE_TIMES, + &[metrics::ATTESTATIONS_HTTP_GET], + ); + beacon_node + .get_validator_attestation_data(slot, committee_index) + .await + .map_err(|e| format!("Failed to produce attestation data: {:?}", e)) + .map(|result| result.data) + }) .await .map_err(|e| e.to_string())?; @@ -433,19 +428,15 @@ impl AttestationService { // Post the attestations to the BN. match self .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::ATTESTATION_SERVICE_TIMES, - &[metrics::ATTESTATIONS_HTTP_POST], - ); - beacon_node - .post_beacon_pool_attestations(attestations) - .await - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::ATTESTATION_SERVICE_TIMES, + &[metrics::ATTESTATIONS_HTTP_POST], + ); + beacon_node + .post_beacon_pool_attestations(attestations) + .await + }) .await { Ok(()) => info!( @@ -493,27 +484,21 @@ impl AttestationService { let aggregated_attestation = &self .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::ATTESTATION_SERVICE_TIMES, - &[metrics::AGGREGATES_HTTP_GET], - ); - beacon_node - .get_validator_aggregate_attestation( - attestation_data.slot, - attestation_data.tree_hash_root(), - ) - .await - .map_err(|e| { - format!("Failed to produce an aggregate attestation: {:?}", e) - })? - .ok_or_else(|| format!("No aggregate available for {:?}", attestation_data)) - .map(|result| result.data) - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::ATTESTATION_SERVICE_TIMES, + &[metrics::AGGREGATES_HTTP_GET], + ); + beacon_node + .get_validator_aggregate_attestation( + attestation_data.slot, + attestation_data.tree_hash_root(), + ) + .await + .map_err(|e| format!("Failed to produce an aggregate attestation: {:?}", e))? + .ok_or_else(|| format!("No aggregate available for {:?}", attestation_data)) + .map(|result| result.data) + }) .await .map_err(|e| e.to_string())?; @@ -574,19 +559,15 @@ impl AttestationService { let signed_aggregate_and_proofs_slice = signed_aggregate_and_proofs.as_slice(); match self .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::ATTESTATION_SERVICE_TIMES, - &[metrics::AGGREGATES_HTTP_POST], - ); - beacon_node - .post_validator_aggregate_and_proof(signed_aggregate_and_proofs_slice) - .await - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::ATTESTATION_SERVICE_TIMES, + &[metrics::AGGREGATES_HTTP_POST], + ); + beacon_node + .post_validator_aggregate_and_proof(signed_aggregate_and_proofs_slice) + .await + }) .await { Ok(()) => { diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index cb432271e4c..e3d5da9585b 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -5,14 +5,14 @@ use crate::beacon_node_health::{ BeaconNodeHealth, BeaconNodeSyncDistanceTiers, ExecutionEngineHealth, SyncDistanceTier, }; -use crate::check_synced::{check_node_health, check_synced}; +use crate::check_synced::check_node_health; use crate::http_metrics::metrics::{inc_counter_vec, ENDPOINT_ERRORS, ENDPOINT_REQUESTS}; use environment::RuntimeContext; use eth2::BeaconNodeHttpClient; use futures::future; use parking_lot::RwLock as PLRwLock; use serde_derive::{Deserialize, Serialize}; -use slog::{debug, error, info, warn, Logger}; +use slog::{debug, error, warn, Logger}; use slot_clock::SlotClock; use std::cmp::Ordering; use std::fmt; @@ -90,30 +90,6 @@ pub fn start_fallback_updater_service( Ok(()) } -/// Indicates if a beacon node must be synced before some action is performed on it. -#[derive(PartialEq, Clone, Copy)] -pub enum RequireSynced { - Yes, - No, -} - -/// Indicates if a beacon node should be set to `Offline` if a request fails. -#[derive(PartialEq, Clone, Copy)] -pub enum OfflineOnFailure { - Yes, - No, -} - -impl PartialEq for RequireSynced { - fn eq(&self, other: &bool) -> bool { - if *other { - *self == RequireSynced::Yes - } else { - *self == RequireSynced::No - } - } -} - #[derive(Debug)] pub enum Error { /// The node was unavailable and we didn't attempt to contact it. @@ -207,48 +183,8 @@ impl CandidateBeaconNode { } /// Returns the status of `self`. - /// - /// If `RequiredSynced::No`, any `NotSynced` node will be ignored and mapped to `Ok(())`. - pub async fn status(&self, synced: RequireSynced) -> Result<(), CandidateError> { - match *self.status.read().await { - Err(CandidateError::NotSynced) if synced == false => Ok(()), - other => other, - } - } - - /// Indicate that `self` is offline. - pub async fn set_offline(&self) { - *self.status.write().await = Err(CandidateError::Offline) - } - - /// Perform some queries against the node to determine if it is a good candidate, updating - /// `self.status` and returning that result. - pub async fn refresh_status( - &self, - slot_clock: Option<&T>, - spec: &ChainSpec, - log: &Logger, - ) -> Result<(), CandidateError> { - let previous_status = self.status(RequireSynced::Yes).await; - let was_offline = matches!(previous_status, Err(CandidateError::Offline)); - - let new_status = if let Err(e) = self.is_online(was_offline, log).await { - Err(e) - } else if let Err(e) = self.is_compatible(spec, log).await { - Err(e) - } else if let Err(e) = self.is_synced(slot_clock, log).await { - Err(e) - } else { - Ok(()) - }; - - // In case of concurrent use, the latest value will always be used. It's possible that a - // long time out might over-ride a recent successful response, leading to a falsely-offline - // status. I deem this edge-case acceptable in return for the concurrency benefits of not - // holding a write-lock whilst we check the online status of the node. - *self.status.write().await = new_status; - - new_status + pub async fn status(&self) -> Result<(), CandidateError> { + *self.status.read().await } pub async fn refresh_health( @@ -309,38 +245,6 @@ impl CandidateBeaconNode { } } - /// Checks if the node is reachable. - async fn is_online(&self, was_offline: bool, log: &Logger) -> Result<(), CandidateError> { - let result = self - .beacon_node - .get_node_version() - .await - .map(|body| body.data.version); - - match result { - Ok(version) => { - if was_offline { - info!( - log, - "Connected to beacon node"; - "version" => version, - "endpoint" => %self.beacon_node, - ); - } - Ok(()) - } - Err(e) => { - warn!( - log, - "Offline beacon node"; - "error" => %e, - "endpoint" => %self.beacon_node, - ); - Err(CandidateError::Offline) - } - } - } - /// Checks if the node has the correct specification. async fn is_compatible(&self, spec: &ChainSpec, log: &Logger) -> Result<(), CandidateError> { let config = self @@ -405,20 +309,6 @@ impl CandidateBeaconNode { Ok(()) } - - /// Checks if the beacon node is synced. - async fn is_synced( - &self, - slot_clock: Option<&T>, - log: &Logger, - ) -> Result<(), CandidateError> { - if let Some(slot_clock) = slot_clock { - check_synced(&self.beacon_node, slot_clock, Some(log)).await - } else { - // Skip this check if we don't supply a slot clock. - Ok(()) - } - } } /// A collection of `CandidateBeaconNode` that can be used to perform requests with "fallback" @@ -504,7 +394,7 @@ impl BeaconNodeFallback { pub async fn num_available(&self) -> usize { let mut n = 0; for candidate in self.candidates.read().await.iter() { - if candidate.status(RequireSynced::No).await.is_ok() { + if candidate.status().await.is_ok() { n += 1 } } @@ -588,12 +478,7 @@ impl BeaconNodeFallback { /// First this function will try all nodes with a suitable status. If no candidates are suitable /// or all the requests fail, it will try updating the status of all unsuitable nodes and /// re-running `func` again. - pub async fn first_success( - &self, - _require_synced: RequireSynced, - _offline_on_failure: OfflineOnFailure, - func: F, - ) -> Result> + pub async fn first_success(&self, func: F) -> Result> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, @@ -626,9 +511,6 @@ impl BeaconNodeFallback { // There exists a race condition where the candidate may have been marked // as ready between the `func` call and now. We deem this an acceptable // inefficiency. - //if matches!(offline_on_failure, OfflineOnFailure::Yes) { - // $candidate.set_offline().await; - //} errors.push(($candidate.beacon_node.to_string(), Error::RequestFailed(e))); inc_counter_vec(&ENDPOINT_ERRORS, &[$candidate.beacon_node.as_ref()]); } @@ -658,12 +540,7 @@ impl BeaconNodeFallback { /// It returns a list of errors along with the beacon node id that failed for `func`. /// Since this ignores the actual result of `func`, this function should only be used for beacon /// node calls whose results we do not care about, only that they completed successfully. - pub async fn run_on_all( - &self, - _require_synced: RequireSynced, - _offline_on_failure: OfflineOnFailure, - func: F, - ) -> Result<(), Errors> + pub async fn run_on_all(&self, func: F) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, @@ -688,9 +565,6 @@ impl BeaconNodeFallback { // There exists a race condition where the candidate may have been marked // as ready between the `func` call and now. We deem this an acceptable // inefficiency. - //if matches!(offline_on_failure, OfflineOnFailure::Yes) { - // $candidate.set_offline().await; - //} results.push(Err(( $candidate.beacon_node.to_string(), Error::RequestFailed(e), @@ -720,24 +594,17 @@ impl BeaconNodeFallback { /// Call `func` on first beacon node that returns success or on all beacon nodes /// depending on the value of `disable_run_on_all`. - pub async fn run( - &self, - require_synced: RequireSynced, - offline_on_failure: OfflineOnFailure, - func: F, - ) -> Result<(), Errors> + pub async fn run(&self, func: F) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, Err: Debug, { if self.disable_run_on_all { - self.first_success(require_synced, offline_on_failure, func) - .await?; + self.first_success(func).await?; Ok(()) } else { - self.run_on_all(require_synced, offline_on_failure, func) - .await + self.run_on_all(func).await } } } diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index 8b0b5dcacbc..255b5328621 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -1,9 +1,6 @@ use crate::beacon_node_fallback::{Error as FallbackError, Errors}; use crate::{ - beacon_node_fallback::{BeaconNodeFallback, RequireSynced}, - determine_graffiti, - graffiti_file::GraffitiFile, - OfflineOnFailure, + beacon_node_fallback::BeaconNodeFallback, determine_graffiti, graffiti_file::GraffitiFile, }; use crate::{ http_metrics::metrics, @@ -147,8 +144,6 @@ impl ProposerFallback { // Try `func` on `self.proposer_nodes` first. If that doesn't work, try `self.beacon_nodes`. pub async fn first_success_try_proposers_first( &self, - require_synced: RequireSynced, - offline_on_failure: OfflineOnFailure, func: F, ) -> Result> where @@ -158,25 +153,18 @@ impl ProposerFallback { { // If there are proposer nodes, try calling `func` on them and return early if they are successful. if let Some(proposer_nodes) = &self.proposer_nodes { - if let Ok(result) = proposer_nodes - .first_success(require_synced, offline_on_failure, func.clone()) - .await - { + if let Ok(result) = proposer_nodes.first_success(func.clone()).await { return Ok(result); } } // If the proposer nodes failed, try on the non-proposer nodes. - self.beacon_nodes - .first_success(require_synced, offline_on_failure, func) - .await + self.beacon_nodes.first_success(func).await } // Try `func` on `self.beacon_nodes` first. If that doesn't work, try `self.proposer_nodes`. pub async fn first_success_try_proposers_last( &self, - require_synced: RequireSynced, - offline_on_failure: OfflineOnFailure, func: F, ) -> Result> where @@ -185,10 +173,7 @@ impl ProposerFallback { Err: Debug, { // Try running `func` on the non-proposer beacon nodes. - let beacon_nodes_result = self - .beacon_nodes - .first_success(require_synced, offline_on_failure, func.clone()) - .await; + let beacon_nodes_result = self.beacon_nodes.first_success(func.clone()).await; match (beacon_nodes_result, &self.proposer_nodes) { // The non-proposer node call succeed, return the result. @@ -196,11 +181,7 @@ impl ProposerFallback { // The non-proposer node call failed, but we don't have any proposer nodes. Return an error. (Err(e), None) => Err(e), // The non-proposer node call failed, try the same call on the proposer nodes. - (Err(_), Some(proposer_nodes)) => { - proposer_nodes - .first_success(require_synced, offline_on_failure, func) - .await - } + (Err(_), Some(proposer_nodes)) => proposer_nodes.first_success(func).await, } } } @@ -474,68 +455,64 @@ impl BlockService { // Try the proposer nodes last, since it's likely that they don't have a // great view of attestations on the network. let block = proposer_fallback - .first_success_try_proposers_last( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let block = match Payload::block_type() { - BlockType::Full => { - let _get_timer = metrics::start_timer_vec( - &metrics::BLOCK_SERVICE_TIMES, - &[metrics::BEACON_BLOCK_HTTP_GET], - ); - beacon_node - .get_validator_blocks::( - slot, - randao_reveal_ref, - graffiti.as_ref(), - ) - .await - .map_err(|e| { - BlockError::Recoverable(format!( - "Error from beacon node when producing block: {:?}", - e - )) - })? - .data - } - BlockType::Blinded => { - let _get_timer = metrics::start_timer_vec( - &metrics::BLOCK_SERVICE_TIMES, - &[metrics::BLINDED_BEACON_BLOCK_HTTP_GET], - ); - beacon_node - .get_validator_blinded_blocks::( - slot, - randao_reveal_ref, - graffiti.as_ref(), - ) - .await - .map_err(|e| { - BlockError::Recoverable(format!( - "Error from beacon node when producing block: {:?}", - e - )) - })? - .data - } - }; - - info!( - log, - "Received unsigned block"; - "slot" => slot.as_u64(), - ); - if proposer_index != Some(block.proposer_index()) { - return Err(BlockError::Recoverable( - "Proposer index does not match block proposer. Beacon chain re-orged" - .to_string(), - )); + .first_success_try_proposers_last(|beacon_node| async move { + let block = match Payload::block_type() { + BlockType::Full => { + let _get_timer = metrics::start_timer_vec( + &metrics::BLOCK_SERVICE_TIMES, + &[metrics::BEACON_BLOCK_HTTP_GET], + ); + beacon_node + .get_validator_blocks::( + slot, + randao_reveal_ref, + graffiti.as_ref(), + ) + .await + .map_err(|e| { + BlockError::Recoverable(format!( + "Error from beacon node when producing block: {:?}", + e + )) + })? + .data } + BlockType::Blinded => { + let _get_timer = metrics::start_timer_vec( + &metrics::BLOCK_SERVICE_TIMES, + &[metrics::BLINDED_BEACON_BLOCK_HTTP_GET], + ); + beacon_node + .get_validator_blinded_blocks::( + slot, + randao_reveal_ref, + graffiti.as_ref(), + ) + .await + .map_err(|e| { + BlockError::Recoverable(format!( + "Error from beacon node when producing block: {:?}", + e + )) + })? + .data + } + }; - Ok::<_, BlockError>(block) - }, - ) + info!( + log, + "Received unsigned block"; + "slot" => slot.as_u64(), + ); + if proposer_index != Some(block.proposer_index()) { + return Err(BlockError::Recoverable( + "Proposer index does not match block proposer. Beacon chain re-orged" + .to_string(), + )); + } + + Ok::<_, BlockError>(block) + }) .await?; let signing_timer = metrics::start_timer(&metrics::BLOCK_SIGNING_TIMES); @@ -580,45 +557,41 @@ impl BlockService { // protect them from DoS attacks and they're most likely to successfully // publish a block. proposer_fallback - .first_success_try_proposers_first( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async { - match Payload::block_type() { - BlockType::Full => { - let _post_timer = metrics::start_timer_vec( - &metrics::BLOCK_SERVICE_TIMES, - &[metrics::BEACON_BLOCK_HTTP_POST], - ); - beacon_node - .post_beacon_blocks(&signed_block) - .await - .map_err(|e| { - BlockError::Irrecoverable(format!( - "Error from beacon node when publishing block: {:?}", - e - )) - })? - } - BlockType::Blinded => { - let _post_timer = metrics::start_timer_vec( - &metrics::BLOCK_SERVICE_TIMES, - &[metrics::BLINDED_BEACON_BLOCK_HTTP_POST], - ); - beacon_node - .post_beacon_blinded_blocks(&signed_block) - .await - .map_err(|e| { - BlockError::Irrecoverable(format!( - "Error from beacon node when publishing block: {:?}", - e - )) - })? - } + .first_success_try_proposers_first(|beacon_node| async { + match Payload::block_type() { + BlockType::Full => { + let _post_timer = metrics::start_timer_vec( + &metrics::BLOCK_SERVICE_TIMES, + &[metrics::BEACON_BLOCK_HTTP_POST], + ); + beacon_node + .post_beacon_blocks(&signed_block) + .await + .map_err(|e| { + BlockError::Irrecoverable(format!( + "Error from beacon node when publishing block: {:?}", + e + )) + })? } - Ok::<_, BlockError>(()) - }, - ) + BlockType::Blinded => { + let _post_timer = metrics::start_timer_vec( + &metrics::BLOCK_SERVICE_TIMES, + &[metrics::BLINDED_BEACON_BLOCK_HTTP_POST], + ); + beacon_node + .post_beacon_blinded_blocks(&signed_block) + .await + .map_err(|e| { + BlockError::Irrecoverable(format!( + "Error from beacon node when publishing block: {:?}", + e + )) + })? + } + } + Ok::<_, BlockError>(()) + }) .await?; info!( diff --git a/validator_client/src/check_synced.rs b/validator_client/src/check_synced.rs index be6d591c9ef..a7b81df9d8c 100644 --- a/validator_client/src/check_synced.rs +++ b/validator_client/src/check_synced.rs @@ -1,87 +1,6 @@ use crate::beacon_node_fallback::CandidateError; use eth2::{types::Slot, BeaconNodeHttpClient}; -use slog::{debug, error, warn, Logger}; -use slot_clock::SlotClock; - -/// A distance in slots. -const SYNC_TOLERANCE: u64 = 4; - -/// Returns -/// -/// `Ok(())` if the beacon node is synced and ready for action, -/// `Err(CandidateError::Offline)` if the beacon node is unreachable, -/// `Err(CandidateError::NotSynced)` if the beacon node indicates that it is syncing **AND** -/// it is more than `SYNC_TOLERANCE` behind the highest -/// known slot. -/// -/// The second condition means the even if the beacon node thinks that it's syncing, we'll still -/// try to use it if it's close enough to the head. -pub async fn check_synced( - beacon_node: &BeaconNodeHttpClient, - slot_clock: &T, - log_opt: Option<&Logger>, -) -> Result<(), CandidateError> { - let resp = match beacon_node.get_node_syncing().await { - Ok(resp) => resp, - Err(e) => { - if let Some(log) = log_opt { - warn!( - log, - "Unable connect to beacon node"; - "error" => %e - ) - } - - return Err(CandidateError::Offline); - } - }; - - // Default EL status to "online" for backwards-compatibility with BNs that don't include it. - let el_offline = resp.data.el_offline.unwrap_or(false); - let bn_is_synced = !resp.data.is_syncing || (resp.data.sync_distance.as_u64() < SYNC_TOLERANCE); - let is_synced = bn_is_synced && !el_offline; - - if let Some(log) = log_opt { - if !is_synced { - debug!( - log, - "Beacon node sync status"; - "status" => format!("{:?}", resp), - ); - - warn!( - log, - "Beacon node is not synced"; - "sync_distance" => resp.data.sync_distance.as_u64(), - "head_slot" => resp.data.head_slot.as_u64(), - "endpoint" => %beacon_node, - "el_offline" => el_offline, - ); - } - - if let Some(local_slot) = slot_clock.now() { - let remote_slot = resp.data.head_slot + resp.data.sync_distance; - if remote_slot + 1 < local_slot || local_slot + 1 < remote_slot { - error!( - log, - "Time discrepancy with beacon node"; - "msg" => "check the system time on this host and the beacon node", - "beacon_node_slot" => remote_slot, - "local_slot" => local_slot, - "endpoint" => %beacon_node, - ); - - return Err(CandidateError::TimeDiscrepancy); - } - } - } - - if is_synced { - Ok(()) - } else { - Err(CandidateError::NotSynced) - } -} +use slog::{warn, Logger}; pub async fn check_node_health( beacon_node: &BeaconNodeHttpClient, diff --git a/validator_client/src/doppelganger_service.rs b/validator_client/src/doppelganger_service.rs index 558b9e199f0..efb14babdb1 100644 --- a/validator_client/src/doppelganger_service.rs +++ b/validator_client/src/doppelganger_service.rs @@ -29,9 +29,8 @@ //! //! Doppelganger protection is a best-effort, last-line-of-defence mitigation. Do not rely upon it. -use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced}; +use crate::beacon_node_fallback::BeaconNodeFallback; use crate::validator_store::ValidatorStore; -use crate::OfflineOnFailure; use environment::RuntimeContext; use eth2::types::LivenessResponseData; use parking_lot::RwLock; @@ -177,17 +176,13 @@ async fn beacon_node_liveness<'a, T: 'static + SlotClock, E: EthSpec>( } else { // Request the previous epoch liveness state from the beacon node. beacon_nodes - .first_success( - RequireSynced::Yes, - OfflineOnFailure::Yes, - |beacon_node| async move { - beacon_node - .post_lighthouse_liveness(validator_indices, previous_epoch) - .await - .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) - .map(|result| result.data) - }, - ) + .first_success(|beacon_node| async move { + beacon_node + .post_lighthouse_liveness(validator_indices, previous_epoch) + .await + .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) + .map(|result| result.data) + }) .await .unwrap_or_else(|e| { crit!( @@ -204,17 +199,13 @@ async fn beacon_node_liveness<'a, T: 'static + SlotClock, E: EthSpec>( // Request the current epoch liveness state from the beacon node. let current_epoch_responses = beacon_nodes - .first_success( - RequireSynced::Yes, - OfflineOnFailure::Yes, - |beacon_node| async move { - beacon_node - .post_lighthouse_liveness(validator_indices, current_epoch) - .await - .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) - .map(|result| result.data) - }, - ) + .first_success(|beacon_node| async move { + beacon_node + .post_lighthouse_liveness(validator_indices, current_epoch) + .await + .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) + .map(|result| result.data) + }) .await .unwrap_or_else(|e| { crit!( diff --git a/validator_client/src/duties_service.rs b/validator_client/src/duties_service.rs index a3b3cabcccd..da697025f1e 100644 --- a/validator_client/src/duties_service.rs +++ b/validator_client/src/duties_service.rs @@ -8,7 +8,7 @@ mod sync; -use crate::beacon_node_fallback::{BeaconNodeFallback, OfflineOnFailure, RequireSynced}; +use crate::beacon_node_fallback::BeaconNodeFallback; use crate::http_metrics::metrics::{get_int_gauge, set_int_gauge, ATTESTATION_DUTY}; use crate::{ block_service::BlockServiceNotification, @@ -415,22 +415,18 @@ async fn poll_validator_indices( // Query the remote BN to resolve a pubkey to a validator index. let download_result = duties_service .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::DUTIES_SERVICE_TIMES, - &[metrics::VALIDATOR_ID_HTTP_GET], - ); - beacon_node - .get_beacon_states_validator_id( - StateId::Head, - &ValidatorId::PublicKey(pubkey), - ) - .await - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::DUTIES_SERVICE_TIMES, + &[metrics::VALIDATOR_ID_HTTP_GET], + ); + beacon_node + .get_beacon_states_validator_id( + StateId::Head, + &ValidatorId::PublicKey(pubkey), + ) + .await + }) .await; let fee_recipient = duties_service @@ -612,19 +608,15 @@ async fn poll_beacon_attesters( let subscriptions_ref = &subscriptions; if let Err(e) = duties_service .beacon_nodes - .run( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::DUTIES_SERVICE_TIMES, - &[metrics::SUBSCRIPTIONS_HTTP_POST], - ); - beacon_node - .post_validator_beacon_committee_subscriptions(subscriptions_ref) - .await - }, - ) + .run(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::DUTIES_SERVICE_TIMES, + &[metrics::SUBSCRIPTIONS_HTTP_POST], + ); + beacon_node + .post_validator_beacon_committee_subscriptions(subscriptions_ref) + .await + }) .await { error!( @@ -850,19 +842,15 @@ async fn post_validator_duties_attester( ) -> Result>, Error> { duties_service .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::DUTIES_SERVICE_TIMES, - &[metrics::ATTESTER_DUTIES_HTTP_POST], - ); - beacon_node - .post_validator_duties_attester(epoch, validator_indices) - .await - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::DUTIES_SERVICE_TIMES, + &[metrics::ATTESTER_DUTIES_HTTP_POST], + ); + beacon_node + .post_validator_duties_attester(epoch, validator_indices) + .await + }) .await .map_err(|e| Error::FailedToDownloadAttesters(e.to_string())) } @@ -1071,19 +1059,15 @@ async fn poll_beacon_proposers( if !local_pubkeys.is_empty() { let download_result = duties_service .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::DUTIES_SERVICE_TIMES, - &[metrics::PROPOSER_DUTIES_HTTP_GET], - ); - beacon_node - .get_validator_duties_proposer(current_epoch) - .await - }, - ) + .first_success(|beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::DUTIES_SERVICE_TIMES, + &[metrics::PROPOSER_DUTIES_HTTP_GET], + ); + beacon_node + .get_validator_duties_proposer(current_epoch) + .await + }) .await; match download_result { diff --git a/validator_client/src/duties_service/sync.rs b/validator_client/src/duties_service/sync.rs index cf63d8ac625..bf21097385d 100644 --- a/validator_client/src/duties_service/sync.rs +++ b/validator_client/src/duties_service/sync.rs @@ -1,4 +1,3 @@ -use crate::beacon_node_fallback::{OfflineOnFailure, RequireSynced}; use crate::{ doppelganger_service::DoppelgangerStatus, duties_service::{DutiesService, Error}, @@ -424,19 +423,15 @@ pub async fn poll_sync_committee_duties_for_period( let genesis = loop { match beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |node| async move { node.get_beacon_genesis().await }, - ) + .first_success(|node| async move { node.get_beacon_genesis().await }) .await { Ok(genesis) => break genesis.data, @@ -769,11 +764,7 @@ async fn poll_whilst_waiting_for_genesis( ) -> Result<(), String> { loop { match beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { beacon_node.get_lighthouse_staking().await }, - ) + .first_success(|beacon_node| async move { beacon_node.get_lighthouse_staking().await }) .await { Ok(is_staking) => { diff --git a/validator_client/src/preparation_service.rs b/validator_client/src/preparation_service.rs index 2d2221680f9..c250a124d27 100644 --- a/validator_client/src/preparation_service.rs +++ b/validator_client/src/preparation_service.rs @@ -1,6 +1,5 @@ -use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced}; +use crate::beacon_node_fallback::BeaconNodeFallback; use crate::validator_store::{DoppelgangerStatus, Error as ValidatorStoreError, ValidatorStore}; -use crate::OfflineOnFailure; use bls::PublicKeyBytes; use environment::RuntimeContext; use parking_lot::RwLock; @@ -342,15 +341,11 @@ impl PreparationService { let preparation_entries = preparation_data.as_slice(); match self .beacon_nodes - .run( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - beacon_node - .post_validator_prepare_beacon_proposer(preparation_entries) - .await - }, - ) + .run(|beacon_node| async move { + beacon_node + .post_validator_prepare_beacon_proposer(preparation_entries) + .await + }) .await { Ok(()) => debug!( @@ -476,13 +471,9 @@ impl PreparationService { for batch in signed.chunks(self.validator_registration_batch_size) { match self .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::No, - |beacon_node| async move { - beacon_node.post_validator_register_validator(batch).await - }, - ) + .first_success(|beacon_node| async move { + beacon_node.post_validator_register_validator(batch).await + }) .await { Ok(()) => info!( diff --git a/validator_client/src/sync_committee_service.rs b/validator_client/src/sync_committee_service.rs index e01bf09cf2f..0318a1d5bf0 100644 --- a/validator_client/src/sync_committee_service.rs +++ b/validator_client/src/sync_committee_service.rs @@ -1,8 +1,7 @@ -use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced}; +use crate::beacon_node_fallback::BeaconNodeFallback; use crate::{ duties_service::DutiesService, validator_store::{Error as ValidatorStoreError, ValidatorStore}, - OfflineOnFailure, }; use environment::RuntimeContext; use eth2::types::BlockId; @@ -182,8 +181,6 @@ impl SyncCommitteeService { let response = self .beacon_nodes .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, |beacon_node| async move { match beacon_node.get_beacon_blocks_root(BlockId::Head).await { Ok(Some(block)) if block.execution_optimistic == Some(false) => { @@ -301,15 +298,11 @@ impl SyncCommitteeService { .collect::>(); self.beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - beacon_node - .post_beacon_pool_sync_committee_signatures(committee_signatures) - .await - }, - ) + .first_success(|beacon_node| async move { + beacon_node + .post_beacon_pool_sync_committee_signatures(committee_signatures) + .await + }) .await .map_err(|e| { error!( @@ -372,21 +365,17 @@ impl SyncCommitteeService { let contribution = &self .beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let sync_contribution_data = SyncContributionData { - slot, - beacon_block_root, - subcommittee_index: subnet_id.into(), - }; + .first_success(|beacon_node| async move { + let sync_contribution_data = SyncContributionData { + slot, + beacon_block_root, + subcommittee_index: subnet_id.into(), + }; - beacon_node - .get_validator_sync_committee_contribution::(&sync_contribution_data) - .await - }, - ) + beacon_node + .get_validator_sync_committee_contribution::(&sync_contribution_data) + .await + }) .await .map_err(|e| { crit!( @@ -454,15 +443,11 @@ impl SyncCommitteeService { // Publish to the beacon node. self.beacon_nodes - .first_success( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - beacon_node - .post_validator_contribution_and_proofs(signed_contributions) - .await - }, - ) + .first_success(|beacon_node| async move { + beacon_node + .post_validator_contribution_and_proofs(signed_contributions) + .await + }) .await .map_err(|e| { error!( @@ -596,15 +581,11 @@ impl SyncCommitteeService { if let Err(e) = self .beacon_nodes - .run( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - beacon_node - .post_validator_sync_committee_subscriptions(subscriptions_slice) - .await - }, - ) + .run(|beacon_node| async move { + beacon_node + .post_validator_sync_committee_subscriptions(subscriptions_slice) + .await + }) .await { error!( From d65967162a7cb783622e667a8cbc8beb76284d20 Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 14 Jun 2023 20:47:32 +1000 Subject: [PATCH 06/50] More improvements --- validator_client/src/beacon_node_fallback.rs | 159 ++++++++----------- validator_client/src/beacon_node_health.rs | 45 +++--- 2 files changed, 90 insertions(+), 114 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index e3d5da9585b..436c2fe0414 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -621,116 +621,84 @@ mod tests { #[test] fn check_candidate_order() { - let candidate_1: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_1.com").unwrap(), - Timeouts::set_all(Duration::from_secs(1)), - ), - 1, - ); - let expected_candidate_1: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_1.com").unwrap(), - Timeouts::set_all(Duration::from_secs(1)), - ), - 1, - ); - let candidate_2: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_2.com").unwrap(), - Timeouts::set_all(Duration::from_secs(2)), - ), - 2, - ); - let expected_candidate_2: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_2.com").unwrap(), - Timeouts::set_all(Duration::from_secs(2)), - ), - 2, - ); - let candidate_3: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_3.com").unwrap(), - Timeouts::set_all(Duration::from_secs(3)), - ), - 3, - ); - let expected_candidate_3: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_3.com").unwrap(), - Timeouts::set_all(Duration::from_secs(3)), - ), - 3, - ); - let candidate_4: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_4.com").unwrap(), - Timeouts::set_all(Duration::from_secs(4)), - ), - 3, - ); - let expected_candidate_4: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_4.com").unwrap(), - Timeouts::set_all(Duration::from_secs(4)), - ), - 3, - ); - let candidate_5: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_5.com").unwrap(), - Timeouts::set_all(Duration::from_secs(5)), - ), - 3, - ); - let expected_candidate_5: CandidateBeaconNode = CandidateBeaconNode::new( - BeaconNodeHttpClient::new( - SensitiveUrl::parse("http://example_5.com").unwrap(), - Timeouts::set_all(Duration::from_secs(5)), - ), - 3, - ); - - // All health parameters other than `health_tier` are irrelevant for ordering. + fn new_candidate(id: usize) -> CandidateBeaconNode { + let beacon_node = BeaconNodeHttpClient::new( + SensitiveUrl::parse(&format!("http://example_{id}.com")).unwrap(), + Timeouts::set_all(Duration::from_secs(id as u64)), + ); + CandidateBeaconNode::new(beacon_node, id) + } + + // These fields is irrelvant for sorting. They are set to arbitrary values. + let head = Slot::new(99); + let optimistic_status = false; + let execution_status = ExecutionEngineHealth::Healthy; + + let candidate_1 = new_candidate(1); + let expected_candidate_1 = new_candidate(1); + let candidate_2 = new_candidate(2); + let expected_candidate_2 = new_candidate(2); + let candidate_3 = new_candidate(3); + let expected_candidate_3 = new_candidate(3); + let candidate_4 = new_candidate(4); + let expected_candidate_4 = new_candidate(4); + let candidate_5 = new_candidate(5); + let expected_candidate_5 = new_candidate(5); + let candidate_6 = new_candidate(6); + let expected_candidate_6 = new_candidate(6); + + let synced = SyncDistanceTier::Synced; + let small = SyncDistanceTier::Small; + + // Despite `health_1` having a larger sync distance, it is inside the `synced` range which + // does not tie-break on sync distance and so will tie-break on `id` instead. let health_1 = BeaconNodeHealth { id: 1, - head: Slot::new(99), - optimistic_status: false, - execution_status: ExecutionEngineHealth::Healthy, - health_tier: BeaconNodeHealthTier::new(1, Slot::new(1)), + head, + optimistic_status, + execution_status, + health_tier: BeaconNodeHealthTier::new(1, Slot::new(2), synced), }; - let health_2 = BeaconNodeHealth { id: 2, - head: Slot::new(99), - optimistic_status: false, - execution_status: ExecutionEngineHealth::Healthy, - health_tier: BeaconNodeHealthTier::new(2, Slot::new(1)), + head, + optimistic_status, + execution_status, + health_tier: BeaconNodeHealthTier::new(2, Slot::new(1), synced), }; + // `health_3` and `health_4` have the same health tier and sync distance so should + // tie-break on `id`. let health_3 = BeaconNodeHealth { id: 3, - head: Slot::new(99), - optimistic_status: false, - execution_status: ExecutionEngineHealth::Healthy, - health_tier: BeaconNodeHealthTier::new(3, Slot::new(1)), + head, + optimistic_status, + execution_status, + health_tier: BeaconNodeHealthTier::new(3, Slot::new(8), small), }; - let health_4 = BeaconNodeHealth { id: 4, - head: Slot::new(99), - optimistic_status: false, - execution_status: ExecutionEngineHealth::Healthy, - health_tier: BeaconNodeHealthTier::new(4, Slot::new(1)), + head, + optimistic_status, + execution_status, + health_tier: BeaconNodeHealthTier::new(3, Slot::new(8), small), }; + // `health_5` has a smaller sync distance and is outside the `synced` range so should be + // sorted first. let health_5 = BeaconNodeHealth { id: 5, - head: Slot::new(99), - optimistic_status: false, - execution_status: ExecutionEngineHealth::Unhealthy, - health_tier: BeaconNodeHealthTier::new(4, Slot::new(5)), + head, + optimistic_status, + execution_status, + health_tier: BeaconNodeHealthTier::new(4, Slot::new(8), small), + }; + let health_6 = BeaconNodeHealth { + id: 6, + head, + optimistic_status, + execution_status, + health_tier: BeaconNodeHealthTier::new(4, Slot::new(9), small), }; *candidate_1.health.write() = Some(health_1); @@ -738,9 +706,11 @@ mod tests { *candidate_3.health.write() = Some(health_3); *candidate_4.health.write() = Some(health_4); *candidate_5.health.write() = Some(health_5); + *candidate_6.health.write() = Some(health_6); let mut candidates = vec![ candidate_3, + candidate_6, candidate_5, candidate_1, candidate_4, @@ -752,6 +722,7 @@ mod tests { expected_candidate_3, expected_candidate_4, expected_candidate_5, + expected_candidate_6, ]; candidates.sort(); diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 33940102c1f..decd0f80445 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -15,7 +15,7 @@ type SyncDistance = Slot; type OptimisticStatus = bool; /// Helpful enum which is used when pattern matching to determine health tier. -#[derive(PartialEq, Debug)] +#[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum SyncDistanceTier { Synced, Small, @@ -88,6 +88,7 @@ pub enum ExecutionEngineHealth { pub struct BeaconNodeHealthTier { pub tier: HealthTier, pub sync_distance: SyncDistance, + pub distance_tier: SyncDistanceTier, } impl Display for BeaconNodeHealthTier { @@ -100,8 +101,7 @@ impl Ord for BeaconNodeHealthTier { fn cmp(&self, other: &Self) -> Ordering { let ordering = self.tier.cmp(&other.tier); if ordering == Ordering::Equal { - // These tiers represent `synced`. - if [1, 3, 5, 6].contains(&self.tier) { + if self.distance_tier == SyncDistanceTier::Synced { // Don't tie-break on sync distance in these cases. // This ensures validator clients don't artificially prefer one node. ordering @@ -121,10 +121,15 @@ impl PartialOrd for BeaconNodeHealthTier { } impl BeaconNodeHealthTier { - pub fn new(tier: HealthTier, sync_distance: SyncDistance) -> Self { + pub fn new( + tier: HealthTier, + sync_distance: SyncDistance, + distance_tier: SyncDistanceTier, + ) -> Self { Self { tier, sync_distance, + distance_tier, } } } @@ -218,52 +223,52 @@ impl BeaconNodeHealth { match health { (SyncDistanceTier::Synced, false, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(1, sync_distance) + BeaconNodeHealthTier::new(1, sync_distance, sync_distance_tier) } (SyncDistanceTier::Small, false, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(2, sync_distance) + BeaconNodeHealthTier::new(2, sync_distance, sync_distance_tier) } (SyncDistanceTier::Synced, false, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(3, sync_distance) + BeaconNodeHealthTier::new(3, sync_distance, sync_distance_tier) } (SyncDistanceTier::Medium, false, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(4, sync_distance) + BeaconNodeHealthTier::new(4, sync_distance, sync_distance_tier) } (SyncDistanceTier::Synced, true, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(5, sync_distance) + BeaconNodeHealthTier::new(5, sync_distance, sync_distance_tier) } (SyncDistanceTier::Synced, true, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(6, sync_distance) + BeaconNodeHealthTier::new(6, sync_distance, sync_distance_tier) } (SyncDistanceTier::Small, false, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(7, sync_distance) + BeaconNodeHealthTier::new(7, sync_distance, sync_distance_tier) } (SyncDistanceTier::Small, true, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(8, sync_distance) + BeaconNodeHealthTier::new(8, sync_distance, sync_distance_tier) } (SyncDistanceTier::Small, true, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(9, sync_distance) + BeaconNodeHealthTier::new(9, sync_distance, sync_distance_tier) } (SyncDistanceTier::Large, false, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(10, sync_distance) + BeaconNodeHealthTier::new(10, sync_distance, sync_distance_tier) } (SyncDistanceTier::Medium, false, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(11, sync_distance) + BeaconNodeHealthTier::new(11, sync_distance, sync_distance_tier) } (SyncDistanceTier::Medium, true, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(12, sync_distance) + BeaconNodeHealthTier::new(12, sync_distance, sync_distance_tier) } (SyncDistanceTier::Medium, true, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(13, sync_distance) + BeaconNodeHealthTier::new(13, sync_distance, sync_distance_tier) } (SyncDistanceTier::Large, false, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(14, sync_distance) + BeaconNodeHealthTier::new(14, sync_distance, sync_distance_tier) } (SyncDistanceTier::Large, true, ExecutionEngineHealth::Healthy) => { - BeaconNodeHealthTier::new(15, sync_distance) + BeaconNodeHealthTier::new(15, sync_distance, sync_distance_tier) } (SyncDistanceTier::Large, true, ExecutionEngineHealth::Unhealthy) => { - BeaconNodeHealthTier::new(16, sync_distance) + BeaconNodeHealthTier::new(16, sync_distance, sync_distance_tier) } } } From dbce701aeea68a30b447577d2a6596d76adbf43e Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 15 Jun 2023 10:54:16 +1000 Subject: [PATCH 07/50] Add IsOptimistic enum for readability --- validator_client/src/beacon_node_fallback.rs | 13 ++- validator_client/src/beacon_node_health.rs | 114 ++++++++++++------- 2 files changed, 84 insertions(+), 43 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 436c2fe0414..a380440e9e7 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -3,7 +3,8 @@ //! succeed. use crate::beacon_node_health::{ - BeaconNodeHealth, BeaconNodeSyncDistanceTiers, ExecutionEngineHealth, SyncDistanceTier, + BeaconNodeHealth, BeaconNodeSyncDistanceTiers, ExecutionEngineHealth, IsOptimistic, + SyncDistanceTier, }; use crate::check_synced::check_node_health; use crate::http_metrics::metrics::{inc_counter_vec, ENDPOINT_ERRORS, ENDPOINT_REQUESTS}; @@ -209,10 +210,16 @@ impl CandidateBeaconNode { ExecutionEngineHealth::Healthy }; + let optimistic_status = if is_optimistic { + IsOptimistic::Yes + } else { + IsOptimistic::No + }; + let new_health = BeaconNodeHealth::from_status( self.id, head, - is_optimistic, + optimistic_status, execution_status, distance_tiers, slot_clock, @@ -631,7 +638,7 @@ mod tests { // These fields is irrelvant for sorting. They are set to arbitrary values. let head = Slot::new(99); - let optimistic_status = false; + let optimistic_status = IsOptimistic::No; let execution_status = ExecutionEngineHealth::Healthy; let candidate_1 = new_candidate(1); diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index decd0f80445..1831c8fa0d7 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -12,7 +12,6 @@ const SYNC_DISTANCE_MEDIUM_MODIFIER: Slot = Slot::new(31); type HealthTier = u8; type SyncDistance = Slot; -type OptimisticStatus = bool; /// Helpful enum which is used when pattern matching to determine health tier. #[derive(Copy, Clone, Debug, PartialEq, Eq)] @@ -77,13 +76,20 @@ impl Default for BeaconNodeSyncDistanceTiers { } /// Execution Node health metrics. -#[derive(Copy, Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] -#[allow(dead_code)] +/// +/// Currently only considers `el_offline`. +#[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum ExecutionEngineHealth { Healthy, Unhealthy, } +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum IsOptimistic { + Yes, + No, +} + #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub struct BeaconNodeHealthTier { pub tier: HealthTier, @@ -144,7 +150,7 @@ pub struct BeaconNodeHealth { // The slot number of the head. pub head: Slot, // Whether the node is optimistically synced. - pub optimistic_status: OptimisticStatus, + pub optimistic_status: IsOptimistic, // The status of the nodes connected Execution Engine. pub execution_status: ExecutionEngineHealth, // The overall health tier of the Beacon Node. Used to rank the nodes for the purposes of @@ -174,7 +180,7 @@ impl BeaconNodeHealth { pub fn from_status( id: usize, head: Slot, - optimistic_status: OptimisticStatus, + optimistic_status: IsOptimistic, execution_status: ExecutionEngineHealth, distance_tiers: &BeaconNodeSyncDistanceTiers, slot_clock: &T, @@ -214,7 +220,7 @@ impl BeaconNodeHealth { fn compute_health_tier( sync_distance: SyncDistance, - optimistic_status: OptimisticStatus, + optimistic_status: IsOptimistic, execution_status: ExecutionEngineHealth, sync_distance_tiers: &BeaconNodeSyncDistanceTiers, ) -> BeaconNodeHealthTier { @@ -222,52 +228,52 @@ impl BeaconNodeHealth { let health = (sync_distance_tier, optimistic_status, execution_status); match health { - (SyncDistanceTier::Synced, false, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Synced, IsOptimistic::No, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(1, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Small, false, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Small, IsOptimistic::No, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(2, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Synced, false, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Synced, IsOptimistic::No, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(3, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Medium, false, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Medium, IsOptimistic::No, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(4, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Synced, true, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Synced, IsOptimistic::Yes, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(5, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Synced, true, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Synced, IsOptimistic::Yes, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(6, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Small, false, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Small, IsOptimistic::No, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(7, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Small, true, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Small, IsOptimistic::Yes, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(8, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Small, true, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Small, IsOptimistic::Yes, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(9, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Large, false, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Large, IsOptimistic::No, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(10, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Medium, false, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Medium, IsOptimistic::No, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(11, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Medium, true, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Medium, IsOptimistic::Yes, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(12, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Medium, true, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Medium, IsOptimistic::Yes, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(13, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Large, false, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Large, IsOptimistic::No, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(14, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Large, true, ExecutionEngineHealth::Healthy) => { + (SyncDistanceTier::Large, IsOptimistic::Yes, ExecutionEngineHealth::Healthy) => { BeaconNodeHealthTier::new(15, sync_distance, sync_distance_tier) } - (SyncDistanceTier::Large, true, ExecutionEngineHealth::Unhealthy) => { + (SyncDistanceTier::Large, IsOptimistic::Yes, ExecutionEngineHealth::Unhealthy) => { BeaconNodeHealthTier::new(16, sync_distance, sync_distance_tier) } } @@ -278,7 +284,7 @@ impl BeaconNodeHealth { mod tests { use super::ExecutionEngineHealth::{Healthy, Unhealthy}; - use super::{BeaconNodeHealth, BeaconNodeSyncDistanceTiers, SyncDistanceTier}; + use super::{BeaconNodeHealth, BeaconNodeSyncDistanceTiers, IsOptimistic, SyncDistanceTier}; use crate::beacon_node_fallback::Config; use slot_clock::{SlotClock, TestingSlotClock}; use std::time::Duration; @@ -297,7 +303,7 @@ mod tests { let mut health_vec = vec![]; for head_slot in (0..=64).rev() { - for optimistic_status in &[false, true] { + for optimistic_status in &[IsOptimistic::No, IsOptimistic::Yes] { for ee_health in &[Healthy, Unhealthy] { let health = BeaconNodeHealth::from_status( 0, @@ -332,9 +338,9 @@ mod tests { // Check optimistic status. if [1, 2, 3, 4, 7, 10, 11, 14].contains(&tier) { - assert!(!health.optimistic_status); + assert_eq!(health.optimistic_status, IsOptimistic::No); } else { - assert!(health.optimistic_status); + assert_eq!(health.optimistic_status, IsOptimistic::Yes); } // Check execution health. @@ -354,20 +360,48 @@ mod tests { }; let distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); - let synced_low = - BeaconNodeHealth::compute_health_tier(Slot::new(0), false, Healthy, &distance_tiers); - let synced_high = - BeaconNodeHealth::compute_health_tier(Slot::new(8), false, Healthy, &distance_tiers); - let small_low = - BeaconNodeHealth::compute_health_tier(Slot::new(9), false, Healthy, &distance_tiers); - let small_high = - BeaconNodeHealth::compute_health_tier(Slot::new(15), false, Healthy, &distance_tiers); - let medium_low = - BeaconNodeHealth::compute_health_tier(Slot::new(16), false, Healthy, &distance_tiers); - let medium_high = - BeaconNodeHealth::compute_health_tier(Slot::new(39), false, Healthy, &distance_tiers); - let large = - BeaconNodeHealth::compute_health_tier(Slot::new(40), false, Healthy, &distance_tiers); + let synced_low = BeaconNodeHealth::compute_health_tier( + Slot::new(0), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); + let synced_high = BeaconNodeHealth::compute_health_tier( + Slot::new(8), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); + let small_low = BeaconNodeHealth::compute_health_tier( + Slot::new(9), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); + let small_high = BeaconNodeHealth::compute_health_tier( + Slot::new(15), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); + let medium_low = BeaconNodeHealth::compute_health_tier( + Slot::new(16), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); + let medium_high = BeaconNodeHealth::compute_health_tier( + Slot::new(39), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); + let large = BeaconNodeHealth::compute_health_tier( + Slot::new(40), + IsOptimistic::No, + Healthy, + &distance_tiers, + ); assert!(synced_low.tier == 1); assert!(synced_high.tier == 1); From 774eb913781ab0d510ad6aff286892a8078d8e1f Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 16 Jun 2023 14:29:45 +1000 Subject: [PATCH 08/50] Use configurable sync distance tiers --- lighthouse/tests/validator_client.rs | 31 +++- validator_client/src/beacon_node_fallback.rs | 30 ++-- validator_client/src/beacon_node_health.rs | 152 +++++++++++-------- validator_client/src/cli.rs | 18 +++ validator_client/src/config.rs | 18 +++ 5 files changed, 173 insertions(+), 76 deletions(-) diff --git a/lighthouse/tests/validator_client.rs b/lighthouse/tests/validator_client.rs index 400ca777118..369783b7f8a 100644 --- a/lighthouse/tests/validator_client.rs +++ b/lighthouse/tests/validator_client.rs @@ -501,13 +501,38 @@ fn disable_run_on_all() { }); } +/// Tests for validator fallback parameter flags. #[test] -fn sync_tolerance_flag() { +fn beacon_node_sync_tolerance_flag() { CommandLineTest::new() - .flag("beacon-node-sync-tolerance", Some("8")) + .flag("beacon-node-sync-tolerance", Some("4")) .run() .with_config(|config| { - assert_eq!(config.beacon_node_fallback.sync_tolerance, Some(8)); + assert_eq!(config.beacon_node_fallback.sync_tolerance, Some(4)); + }); +} +#[test] +fn beacon_node_small_sync_distance_modifier_flag() { + CommandLineTest::new() + .flag("beacon-node-small-sync-distance-modifer", Some("16")) + .run() + .with_config(|config| { + assert_eq!( + config.beacon_node_fallback.small_sync_distance_modifier, + Some(16) + ); + }); +} +#[test] +fn beacon_node_medium_sync_distance_modifier_flag() { + CommandLineTest::new() + .flag("beacon-node-medium-sync-distance-modifer", Some("32")) + .run() + .with_config(|config| { + assert_eq!( + config.beacon_node_fallback.medium_sync_distance_modifier, + Some(32) + ); }); } diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index a380440e9e7..5d841791ef2 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -45,6 +45,12 @@ pub struct Config { /// Sets the number of slots behind the head a beacon node is allowed to be to still be /// considered `synced`. pub sync_tolerance: Option, + /// Sets the size of the range of the `small` sync distance tier. This range starts immediately + /// after `sync_tolerance`. + pub small_sync_distance_modifier: Option, + /// Sets the size of the range of the `medium` sync distance tier. This range starts immediately + /// after the `small` range. + pub medium_sync_distance_modifier: Option, } /// Indicates a measurement of latency between the VC and a BN. @@ -628,6 +634,11 @@ mod tests { #[test] fn check_candidate_order() { + // These fields is irrelvant for sorting. They are set to arbitrary values. + let head = Slot::new(99); + let optimistic_status = IsOptimistic::No; + let execution_status = ExecutionEngineHealth::Healthy; + fn new_candidate(id: usize) -> CandidateBeaconNode { let beacon_node = BeaconNodeHttpClient::new( SensitiveUrl::parse(&format!("http://example_{id}.com")).unwrap(), @@ -636,11 +647,6 @@ mod tests { CandidateBeaconNode::new(beacon_node, id) } - // These fields is irrelvant for sorting. They are set to arbitrary values. - let head = Slot::new(99); - let optimistic_status = IsOptimistic::No; - let execution_status = ExecutionEngineHealth::Healthy; - let candidate_1 = new_candidate(1); let expected_candidate_1 = new_candidate(1); let candidate_2 = new_candidate(2); @@ -681,31 +687,31 @@ mod tests { head, optimistic_status, execution_status, - health_tier: BeaconNodeHealthTier::new(3, Slot::new(8), small), + health_tier: BeaconNodeHealthTier::new(3, Slot::new(9), small), }; let health_4 = BeaconNodeHealth { id: 4, head, optimistic_status, execution_status, - health_tier: BeaconNodeHealthTier::new(3, Slot::new(8), small), + health_tier: BeaconNodeHealthTier::new(3, Slot::new(9), small), }; // `health_5` has a smaller sync distance and is outside the `synced` range so should be - // sorted first. + // sorted first. Note the values of `id`. let health_5 = BeaconNodeHealth { - id: 5, + id: 6, head, optimistic_status, execution_status, - health_tier: BeaconNodeHealthTier::new(4, Slot::new(8), small), + health_tier: BeaconNodeHealthTier::new(4, Slot::new(9), small), }; let health_6 = BeaconNodeHealth { - id: 6, + id: 5, head, optimistic_status, execution_status, - health_tier: BeaconNodeHealthTier::new(4, Slot::new(9), small), + health_tier: BeaconNodeHealthTier::new(4, Slot::new(10), small), }; *candidate_1.health.write() = Some(health_1); diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 1831c8fa0d7..4f8402d2e5d 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -6,9 +6,15 @@ use types::Slot; // Sync distances between 0 and DEFAULT_SYNC_TOLERANCE are considered `synced`. // Sync distance tiers are determined by the different modifiers. -const DEFAULT_SYNC_TOLERANCE: Slot = Slot::new(4); -const SYNC_DISTANCE_SMALL_MODIFIER: Slot = Slot::new(7); -const SYNC_DISTANCE_MEDIUM_MODIFIER: Slot = Slot::new(31); +// +// The default range is the following: +// Synced: 0..=8 +// Small: 9..=16 +// Medium: 17..=64 +// Large: 65.. +const DEFAULT_SYNC_TOLERANCE: Slot = Slot::new(8); +const DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER: Slot = Slot::new(8); +const DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER: Slot = Slot::new(48); type HealthTier = u8; type SyncDistance = Slot; @@ -23,7 +29,7 @@ pub enum SyncDistanceTier { } /// Contains the different sync distance tiers which are determined at runtime by the -/// `sync_tolerance` CLI flag. +/// `sync_tolerance` CLI flag and the `sync_distance_modifier` flags. #[derive(Clone, Debug)] pub struct BeaconNodeSyncDistanceTiers { synced: SyncDistance, @@ -33,14 +39,23 @@ pub struct BeaconNodeSyncDistanceTiers { impl BeaconNodeSyncDistanceTiers { pub fn from_config(config: &Config) -> Self { - if let Some(sync_tolerance) = config.sync_tolerance { - Self { - synced: Slot::new(sync_tolerance), - small: Slot::new(sync_tolerance) + SYNC_DISTANCE_SMALL_MODIFIER, - medium: Slot::new(sync_tolerance) + SYNC_DISTANCE_MEDIUM_MODIFIER, - } - } else { - Self::default() + let synced = config + .sync_tolerance + .map(Slot::new) + .unwrap_or(DEFAULT_SYNC_TOLERANCE); + let small_mod = config + .small_sync_distance_modifier + .map(Slot::new) + .unwrap_or(DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER); + let medium_mod = config + .medium_sync_distance_modifier + .map(Slot::new) + .unwrap_or(DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER); + + Self { + synced, + small: synced + small_mod, + medium: synced + small_mod + medium_mod, } } @@ -69,8 +84,10 @@ impl Default for BeaconNodeSyncDistanceTiers { fn default() -> Self { Self { synced: DEFAULT_SYNC_TOLERANCE, - small: DEFAULT_SYNC_TOLERANCE + SYNC_DISTANCE_SMALL_MODIFIER, - medium: DEFAULT_SYNC_TOLERANCE + SYNC_DISTANCE_MEDIUM_MODIFIER, + small: DEFAULT_SYNC_TOLERANCE + DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER, + medium: DEFAULT_SYNC_TOLERANCE + + DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER + + DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER, } } } @@ -284,7 +301,10 @@ impl BeaconNodeHealth { mod tests { use super::ExecutionEngineHealth::{Healthy, Unhealthy}; - use super::{BeaconNodeHealth, BeaconNodeSyncDistanceTiers, IsOptimistic, SyncDistanceTier}; + use super::{ + BeaconNodeHealth, BeaconNodeHealthTier, BeaconNodeSyncDistanceTiers, IsOptimistic, + SyncDistanceTier, + }; use crate::beacon_node_fallback::Config; use slot_clock::{SlotClock, TestingSlotClock}; use std::time::Duration; @@ -302,7 +322,7 @@ mod tests { let mut health_vec = vec![]; - for head_slot in (0..=64).rev() { + for head_slot in 0..=64 { for optimistic_status in &[IsOptimistic::No, IsOptimistic::Yes] { for ee_health in &[Healthy, Unhealthy] { let health = BeaconNodeHealth::from_status( @@ -352,63 +372,73 @@ mod tests { } } + fn new_distance_tier( + distance: u64, + distance_tiers: &BeaconNodeSyncDistanceTiers, + ) -> BeaconNodeHealthTier { + BeaconNodeHealth::compute_health_tier( + Slot::new(distance), + IsOptimistic::No, + Healthy, + distance_tiers, + ) + } + + #[test] + fn sync_tolerance_default() { + let distance_tiers = BeaconNodeSyncDistanceTiers::default(); + + let synced_low = new_distance_tier(0, &distance_tiers); + let synced_high = new_distance_tier(8, &distance_tiers); + + let small_low = new_distance_tier(9, &distance_tiers); + let small_high = new_distance_tier(16, &distance_tiers); + + let medium_low = new_distance_tier(17, &distance_tiers); + let medium_high = new_distance_tier(64, &distance_tiers); + let large = new_distance_tier(65, &distance_tiers); + + assert!(synced_low.tier == 1); + assert!(synced_high.tier == 1); + assert!(small_low.tier == 2); + assert!(small_high.tier == 2); + assert!(medium_low.tier == 4); + assert_eq!(medium_high.tier, 4); + assert!(large.tier == 10); + } + #[test] - fn sync_tolerance() { + fn sync_tolerance_from_config() { + // Config should set the tiers as: + // synced: 0..=4 + // small: 5..=8 + // medium 9..=12 + // large: 13.. let config = Config { disable_run_on_all: false, - sync_tolerance: Some(8), + sync_tolerance: Some(4), + small_sync_distance_modifier: Some(4), + medium_sync_distance_modifier: Some(4), }; let distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); - let synced_low = BeaconNodeHealth::compute_health_tier( - Slot::new(0), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); - let synced_high = BeaconNodeHealth::compute_health_tier( - Slot::new(8), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); - let small_low = BeaconNodeHealth::compute_health_tier( - Slot::new(9), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); - let small_high = BeaconNodeHealth::compute_health_tier( - Slot::new(15), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); - let medium_low = BeaconNodeHealth::compute_health_tier( - Slot::new(16), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); - let medium_high = BeaconNodeHealth::compute_health_tier( - Slot::new(39), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); - let large = BeaconNodeHealth::compute_health_tier( - Slot::new(40), - IsOptimistic::No, - Healthy, - &distance_tiers, - ); + let synced_low = new_distance_tier(0, &distance_tiers); + let synced_high = new_distance_tier(4, &distance_tiers); + + let small_low = new_distance_tier(5, &distance_tiers); + let small_high = new_distance_tier(8, &distance_tiers); + + let medium_low = new_distance_tier(9, &distance_tiers); + let medium_high = new_distance_tier(12, &distance_tiers); + + let large = new_distance_tier(13, &distance_tiers); assert!(synced_low.tier == 1); assert!(synced_high.tier == 1); assert!(small_low.tier == 2); assert!(small_high.tier == 2); assert!(medium_low.tier == 4); - assert!(medium_high.tier == 4); + assert_eq!(medium_high.tier, 4); assert!(large.tier == 10); } } diff --git a/validator_client/src/cli.rs b/validator_client/src/cli.rs index 012c31ff736..fe7ca3b1d1c 100644 --- a/validator_client/src/cli.rs +++ b/validator_client/src/cli.rs @@ -371,6 +371,24 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { connected Beacon Node.") .takes_value(true), ) + .arg( + Arg::with_name("beacon-node-small-sync-distance-modifier") + .long("beacon-node-small-sync-distance-modifier") + .help("Only use this if you know what you are doing. Incorrectly setting this value \ + can result in suboptimal fallback behaviour. Sets the size (in slots) of the \ + `small` sync distance range when calculating the health tiers of connected \ + Beacon Nodes. The range falls immediately after the end of the `synced` range.") + .takes_value(true) + ) + .arg( + Arg::with_name("beacon-node-medium-sync-distance-modifier") + .long("beacon-node-medium-sync-distance-modifier") + .help("Only use this if you know what you are doing. Incorrectly setting this value \ + can result in suboptimal fallback behaviour. Sets the size (in slots) of the \ + `medium` sync distance range when calculating the health tiers of connected \ + Beacon Nodes. The range falls immediately after the end of the `small` range.") + .takes_value(true) + ) /* * Experimental/development options. */ diff --git a/validator_client/src/config.rs b/validator_client/src/config.rs index 81e1ac1411f..3d53dfa5624 100644 --- a/validator_client/src/config.rs +++ b/validator_client/src/config.rs @@ -271,6 +271,24 @@ impl Config { ); } + if let Some(small_modifier) = cli_args.value_of("beacon-node-small-sync-distance-modifier") + { + config.beacon_node_fallback.small_sync_distance_modifier = Some( + small_modifier + .parse::() + .map_err(|_| "beacon-node-small-sync-distance-modifier is not a valid u64.")?, + ); + } + + if let Some(medium_modifier) = + cli_args.value_of("beacon-node-medium-sync-distance-modifier") + { + config.beacon_node_fallback.medium_sync_distance_modifier = + Some(medium_modifier.parse::().map_err(|_| { + "beacon-node-medium-sync-distance-modifier is not a valid u64." + })?); + } + /* * Http API server */ From d839620be1901d2d73b029d5f3df37276d5313a7 Mon Sep 17 00:00:00 2001 From: Mac L Date: Mon, 19 Jun 2023 10:08:13 +1000 Subject: [PATCH 09/50] Fix tests --- lighthouse/tests/validator_client.rs | 4 ++-- validator_client/src/beacon_node_health.rs | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/lighthouse/tests/validator_client.rs b/lighthouse/tests/validator_client.rs index 369783b7f8a..f20ff4e0f1f 100644 --- a/lighthouse/tests/validator_client.rs +++ b/lighthouse/tests/validator_client.rs @@ -514,7 +514,7 @@ fn beacon_node_sync_tolerance_flag() { #[test] fn beacon_node_small_sync_distance_modifier_flag() { CommandLineTest::new() - .flag("beacon-node-small-sync-distance-modifer", Some("16")) + .flag("beacon-node-small-sync-distance-modifier", Some("16")) .run() .with_config(|config| { assert_eq!( @@ -526,7 +526,7 @@ fn beacon_node_small_sync_distance_modifier_flag() { #[test] fn beacon_node_medium_sync_distance_modifier_flag() { CommandLineTest::new() - .flag("beacon-node-medium-sync-distance-modifer", Some("32")) + .flag("beacon-node-medium-sync-distance-modifier", Some("32")) .run() .with_config(|config| { assert_eq!( diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 4f8402d2e5d..1166123f564 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -299,7 +299,6 @@ impl BeaconNodeHealth { #[cfg(test)] mod tests { - use super::ExecutionEngineHealth::{Healthy, Unhealthy}; use super::{ BeaconNodeHealth, BeaconNodeHealthTier, BeaconNodeSyncDistanceTiers, IsOptimistic, From 5153aff196bb480a166b317a854b484e051c5ec7 Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 21 Jun 2023 15:51:37 +1000 Subject: [PATCH 10/50] Combine status and health and improve logging --- testing/simulator/src/checks.rs | 1 - testing/simulator/src/eth1_sim.rs | 2 +- validator_client/src/beacon_node_fallback.rs | 133 ++++++++++--------- validator_client/src/beacon_node_health.rs | 2 +- validator_client/src/notifier.rs | 33 ++++- 5 files changed, 106 insertions(+), 65 deletions(-) diff --git a/testing/simulator/src/checks.rs b/testing/simulator/src/checks.rs index 01e50eb7fba..ac672cbc214 100644 --- a/testing/simulator/src/checks.rs +++ b/testing/simulator/src/checks.rs @@ -1,6 +1,5 @@ use crate::local_network::LocalNetwork; use node_test_rig::eth2::types::{BlockId, StateId}; - use std::time::Duration; use types::{Epoch, EthSpec, ExecPayload, ExecutionBlockHash, Hash256, Slot, Unsigned}; diff --git a/testing/simulator/src/eth1_sim.rs b/testing/simulator/src/eth1_sim.rs index 03a0205574d..57c944cf1a7 100644 --- a/testing/simulator/src/eth1_sim.rs +++ b/testing/simulator/src/eth1_sim.rs @@ -64,7 +64,7 @@ pub fn run_eth1_sim(matches: &ArgMatches) -> Result<(), String> { let mut env = EnvironmentBuilder::minimal() .initialize_logger(LoggerConfig { path: None, - debug_level: String::from("info"), + debug_level: String::from("debug"), logfile_debug_level: String::from("debug"), log_format: None, logfile_format: None, diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 5d841791ef2..73c7b7d0b04 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -137,18 +137,23 @@ pub enum CandidateError { Uninitialized, Offline, Incompatible, - NotSynced, TimeDiscrepancy, } +#[derive(Debug, Clone)] +pub struct CandidateInfo { + pub id: usize, + pub node: String, + pub health: Option, +} + /// Represents a `BeaconNodeHttpClient` inside a `BeaconNodeFallback` that may or may not be used /// for a query. #[derive(Debug)] pub struct CandidateBeaconNode { id: usize, beacon_node: BeaconNodeHttpClient, - health: PLRwLock>, - status: RwLock>, + health: PLRwLock>, _phantom: PhantomData, } @@ -162,11 +167,11 @@ impl Eq for CandidateBeaconNode {} impl Ord for CandidateBeaconNode { fn cmp(&self, other: &Self) -> Ordering { - match (&(*self.health.read()), &(*other.health.read())) { - (None, None) => Ordering::Equal, - (None, _) => Ordering::Greater, - (_, None) => Ordering::Less, - (Some(health_1), Some(health_2)) => health_1.cmp(health_2), + match (&(self.health()), &(other.health())) { + (Err(_), Err(_)) => Ordering::Equal, + (Err(_), _) => Ordering::Greater, + (_, Err(_)) => Ordering::Less, + (Ok(health_1), Ok(health_2)) => health_1.cmp(health_2), } } } @@ -183,15 +188,14 @@ impl CandidateBeaconNode { Self { id, beacon_node, - health: PLRwLock::new(None), - status: RwLock::new(Err(CandidateError::Uninitialized)), + health: PLRwLock::new(Err(CandidateError::Uninitialized)), _phantom: PhantomData, } } - /// Returns the status of `self`. - pub async fn status(&self) -> Result<(), CandidateError> { - *self.status.read().await + /// Returns the health of `self`. + pub fn health(&self) -> Result { + *self.health.read() } pub async fn refresh_health( @@ -202,7 +206,7 @@ impl CandidateBeaconNode { log: &Logger, ) -> Result<(), CandidateError> { if let Err(e) = self.is_compatible(spec, log).await { - *self.status.write().await = Err(e); + *self.health.write() = Err(e); return Ok(()); } @@ -231,29 +235,20 @@ impl CandidateBeaconNode { slot_clock, ); - warn!( - log, - "Health of Beacon Node: {}, updated. Health tier: {}", - new_health.get_id(), - new_health.get_health_tier() - ); - - *self.health.write() = Some(new_health); - *self.status.write().await = Ok(()); + // TODO(mac): Set metric here. + *self.health.write() = Ok(new_health); Ok(()) } - Err(status) => { - // Set the health as None which is sorted last in the list. - *self.health.write() = None; - *self.status.write().await = Err(status); + Err(e) => { + // Set the health as `Err` which is sorted last in the list. + *self.health.write() = Err(e); Ok(()) } } } else { - // Slot clock will only be None at startup. + // Slot clock will only be `None` at startup. // Assume compatible nodes are available. - *self.status.write().await = Ok(()); Ok(()) } } @@ -373,7 +368,7 @@ impl BeaconNodeFallback { pub async fn num_synced(&self) -> usize { let mut n = 0; for candidate in self.candidates.read().await.iter() { - if let Some(cand) = candidate.health.read().as_ref() { + if let Ok(cand) = candidate.health().as_ref() { if self .distance_tiers .distance_tier(cand.health_tier.sync_distance) @@ -390,7 +385,7 @@ impl BeaconNodeFallback { pub async fn num_synced_fallback(&self) -> usize { let mut n = 0; for candidate in self.candidates.read().await.iter().skip(1) { - if let Some(cand) = candidate.health.read().as_ref() { + if let Ok(cand) = candidate.health().as_ref() { if self .distance_tiers .distance_tier(cand.health_tier.sync_distance) @@ -407,13 +402,26 @@ impl BeaconNodeFallback { pub async fn num_available(&self) -> usize { let mut n = 0; for candidate in self.candidates.read().await.iter() { - if candidate.status().await.is_ok() { + if candidate.health().is_ok() { n += 1 } } n } + pub async fn get_all_candidate_info(&self) -> Vec { + let candidates = self.candidates.read().await; + let mut results = Vec::with_capacity(candidates.len()); + for candidate in candidates.iter() { + let id = candidate.id; + let node = candidate.beacon_node.to_string(); + let health = candidate.health().ok(); + let info = CandidateInfo { id, node, health }; + results.push(info); + } + results + } + /// Loop through ALL candidates in `self.candidates` and update their sync status. /// /// It is possible for a node to return an unsynced status while continuing to serve @@ -421,21 +429,33 @@ impl BeaconNodeFallback { /// A previous implementation of this function polled only the unavailable BNs. pub async fn update_all_candidates(&self) { let candidates = self.candidates.read().await; + let mut futures = Vec::with_capacity(candidates.len()); + let mut nodes = Vec::with_capacity(candidates.len()); - let futures = candidates - .iter() - .map(|candidate| { - candidate.refresh_health( - &self.distance_tiers, - self.slot_clock.as_ref(), - &self.spec, - &self.log, - ) - }) - .collect::>(); + for candidate in candidates.iter() { + futures.push(candidate.refresh_health( + &self.distance_tiers, + self.slot_clock.as_ref(), + &self.spec, + &self.log, + )); + nodes.push(candidate.beacon_node.to_string()); + } - // Run all updates concurrently and ignore errors. - let _ = future::join_all(futures).await; + // Run all updates concurrently. + let future_results = future::join_all(futures).await; + let results = future_results.iter().zip(nodes); + + for (result, node) in results { + if let Err(e) = result { + warn!( + self.log, + "A connected beacon node errored during routine health check."; + "error" => ?e, + "endpoint" => node, + ); + } + } drop(candidates); @@ -519,11 +539,7 @@ impl BeaconNodeFallback { "node" => $candidate.beacon_node.to_string(), "error" => ?e, ); - // If we have an error on this function, make the client as not-ready. - // - // There exists a race condition where the candidate may have been marked - // as ready between the `func` call and now. We deem this an acceptable - // inefficiency. + errors.push(($candidate.beacon_node.to_string(), Error::RequestFailed(e))); inc_counter_vec(&ENDPOINT_ERRORS, &[$candidate.beacon_node.as_ref()]); } @@ -573,11 +589,6 @@ impl BeaconNodeFallback { match func($candidate.beacon_node.clone()).await { Ok(val) => results.push(Ok(val)), Err(e) => { - // If we have an error on this function, make the client as not-ready. - // - // There exists a race condition where the candidate may have been marked - // as ready between the `func` call and now. We deem this an acceptable - // inefficiency. results.push(Err(( $candidate.beacon_node.to_string(), Error::RequestFailed(e), @@ -714,12 +725,12 @@ mod tests { health_tier: BeaconNodeHealthTier::new(4, Slot::new(10), small), }; - *candidate_1.health.write() = Some(health_1); - *candidate_2.health.write() = Some(health_2); - *candidate_3.health.write() = Some(health_3); - *candidate_4.health.write() = Some(health_4); - *candidate_5.health.write() = Some(health_5); - *candidate_6.health.write() = Some(health_6); + *candidate_1.health.write() = Ok(health_1); + *candidate_2.health.write() = Ok(health_2); + *candidate_3.health.write() = Ok(health_3); + *candidate_4.health.write() = Ok(health_4); + *candidate_5.health.write() = Ok(health_5); + *candidate_6.health.write() = Ok(health_6); let mut candidates = vec![ candidate_3, diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 1166123f564..f1a6c7bbec0 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -158,7 +158,7 @@ impl BeaconNodeHealthTier { } /// Beacon Node Health metrics. -#[derive(Debug, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, PartialEq, Eq)] pub struct BeaconNodeHealth { // The ID of the Beacon Node. This should correspond with its position in the `--beacon-nodes` // list. Note that the ID field is used to tie-break nodes with the same health so that nodes diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index e8b5f4c5ba2..984ffadf892 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -1,7 +1,7 @@ use crate::http_metrics; use crate::{DutiesService, ProductionValidatorClient}; use lighthouse_metrics::set_gauge; -use slog::{error, info, Logger}; +use slog::{debug, error, info, Logger}; use slot_clock::SlotClock; use tokio::time::{sleep, Duration}; use types::EthSpec; @@ -39,6 +39,7 @@ async fn notify( duties_service: &DutiesService, log: &Logger, ) { + let candidate_info = duties_service.beacon_nodes.get_all_candidate_info().await; let num_available = duties_service.beacon_nodes.num_available().await; set_gauge( &http_metrics::metrics::AVAILABLE_BEACON_NODES_COUNT, @@ -55,9 +56,14 @@ async fn notify( num_total as i64, ); if num_synced > 0 { + let primary = candidate_info + .get(0) + .map(|candidate| candidate.node.as_str()) + .unwrap_or("None"); info!( log, "Connected to beacon node(s)"; + "primary" => primary, "total" => num_total, "available" => num_available, "synced" => num_synced, @@ -78,6 +84,31 @@ async fn notify( set_gauge(&http_metrics::metrics::ETH2_FALLBACK_CONNECTED, 0); } + // TODO(mac) Store all connected node info into metrics. + for info in candidate_info { + if let Some(health) = info.health { + debug!( + log, + "Beacon node info"; + "status" => "Connected", + "id" => info.id, + "endpoint" => info.node, + "head_slot" => %health.head, + "is_optimistic" => ?health.optimistic_status, + "execution_engine_status" => ?health.execution_status, + "health_tier" => ?health.health_tier, + ); + } else { + debug!( + log, + "Beacon node info"; + "status" => "Disconnected", + "id" => info.id, + "endpoint" => info.node, + ); + } + } + if let Some(slot) = duties_service.slot_clock.now() { let epoch = slot.epoch(E::slots_per_epoch()); From 61bc700fdfa93870f223f810a477cb7ecb703297 Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 22 Jun 2023 11:51:30 +1000 Subject: [PATCH 11/50] Fix nodes not being marked as available --- validator_client/src/beacon_node_fallback.rs | 9 +++++---- validator_client/src/notifier.rs | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 73c7b7d0b04..759148060ab 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -207,7 +207,7 @@ impl CandidateBeaconNode { ) -> Result<(), CandidateError> { if let Err(e) = self.is_compatible(spec, log).await { *self.health.write() = Err(e); - return Ok(()); + return Err(e); } if let Some(slot_clock) = slot_clock { @@ -243,7 +243,7 @@ impl CandidateBeaconNode { Err(e) => { // Set the health as `Err` which is sorted last in the list. *self.health.write() = Err(e); - Ok(()) + Err(e) } } } else { @@ -402,8 +402,9 @@ impl BeaconNodeFallback { pub async fn num_available(&self) -> usize { let mut n = 0; for candidate in self.candidates.read().await.iter() { - if candidate.health().is_ok() { - n += 1 + match candidate.health() { + Ok(_) | Err(CandidateError::Uninitialized) => n += 1, + Err(_) => continue, } } n diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index 984ffadf892..1e7940de579 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -96,7 +96,7 @@ async fn notify( "head_slot" => %health.head, "is_optimistic" => ?health.optimistic_status, "execution_engine_status" => ?health.execution_status, - "health_tier" => ?health.health_tier, + "health_tier" => %health.health_tier, ); } else { debug!( From 129568565e46eeec20c15c02a3155faacf3d4c68 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 21 Jul 2023 17:49:52 +1000 Subject: [PATCH 12/50] Fix simulator --- testing/simulator/src/checks.rs | 81 +++++++++++++++++++++++---- testing/simulator/src/cli.rs | 5 -- testing/simulator/src/fallback_sim.rs | 46 ++++++--------- testing/simulator/src/main.rs | 8 +++ 4 files changed, 94 insertions(+), 46 deletions(-) diff --git a/testing/simulator/src/checks.rs b/testing/simulator/src/checks.rs index ac672cbc214..c0be219d306 100644 --- a/testing/simulator/src/checks.rs +++ b/testing/simulator/src/checks.rs @@ -1,4 +1,5 @@ use crate::local_network::LocalNetwork; +use crate::ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE; use node_test_rig::eth2::types::{BlockId, StateId}; use std::time::Duration; use types::{Epoch, EthSpec, ExecPayload, ExecutionBlockHash, Hash256, Slot, Unsigned}; @@ -244,30 +245,42 @@ pub async fn verify_transition_block_finalized( } } +// Causes the execution node at `node_index` to disconnect from the execution layer 1 epoch after +// the merge transition. pub async fn disconnect_from_execution_layer( network: LocalNetwork, transition_epoch: Epoch, slot_duration: Duration, + node_index: usize, ) -> Result<(), String> { epoch_delay(transition_epoch + 1, slot_duration, E::slots_per_epoch()).await; eprintln!("Disabling Execution Layer"); - // Take the execution node at position 0 and force it to return the `syncing` status. - network.execution_nodes.read()[0] + // Force the execution node to return the `syncing` status. + network.execution_nodes.read()[node_index] .server .all_payloads_syncing(false); + Ok(()) +} - // Run for 2 epochs with the 0th execution node stalled. +pub async fn reconnect_to_execution_layer( + network: LocalNetwork, + transition_epoch: Epoch, + slot_duration: Duration, + node_index: usize, + epochs_offline: u64, +) -> Result<(), String> { + // Ensure this is configurable by only reconnecting after `epoch_offline`. epoch_delay( - transition_epoch + 1 + 2, + transition_epoch + epochs_offline, slot_duration, E::slots_per_epoch(), ) .await; - // Restore the functionality of the 0th execution node. - network.execution_nodes.read()[0] + // Restore the functionality of the execution node. + network.execution_nodes.read()[node_index] .server .all_payloads_valid(); @@ -278,32 +291,76 @@ pub async fn disconnect_from_execution_layer( /// Ensure all validators have attested correctly. pub async fn check_attestation_correctness( network: LocalNetwork, + start_epoch: Epoch, + // Must be 2 epochs less than the end of the simulation. upto_epoch: Epoch, slots_per_epoch: u64, slot_duration: Duration, + // Select which node to query. Will use this node to determine the global network performance. + node_index: usize, ) -> Result<(), String> { let upto_slot = upto_epoch.start_slot(slots_per_epoch); slot_delay(upto_slot, slot_duration).await; - let remote_node = &network.remote_nodes()?[1]; + let remote_node = &network.remote_nodes()?[node_index]; let results = remote_node .get_lighthouse_analysis_attestation_performance( - Epoch::new(2), + start_epoch, upto_epoch - 2, "global".to_string(), ) .await .map_err(|e| format!("Unable to get attestation performance: {e}"))?; + let mut active_successes: f64 = 0.0; + let mut head_successes: f64 = 0.0; + let mut target_successes: f64 = 0.0; + let mut source_successes: f64 = 0.0; + + let mut total: f64 = 0.0; + for result in results { for epochs in result.epochs.values() { - assert!(epochs.active); - assert!(epochs.head); - assert!(epochs.target); - assert!(epochs.source); + total += 1.0; + + if epochs.active { + active_successes += 1.0; + } + if epochs.head { + head_successes += 1.0; + } + if epochs.target { + target_successes += 1.0; + } + if epochs.source { + source_successes += 1.0; + } } } + let active_percent = active_successes / total * 100.0; + let head_percent = head_successes / total * 100.0; + let target_percent = target_successes / total * 100.0; + let source_percent = source_successes / total * 100.0; + + eprintln!("Total Attestations: {}", total); + eprintln!("Active: {}: {}%", active_successes, active_percent); + eprintln!("Head: {}: {}%", head_successes, head_percent); + eprintln!("Target: {}: {}%", target_successes, target_percent); + eprintln!("Source: {}: {}%", source_successes, source_percent); + + if active_percent < ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE { + return Err("Active percent was below required level".to_string()); + } + if head_percent < ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE { + return Err("Head percent was below required level".to_string()); + } + if target_percent < ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE { + return Err("Target percent was below required level".to_string()); + } + if source_percent < ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE { + return Err("Source percent was below required level".to_string()); + } Ok(()) } diff --git a/testing/simulator/src/cli.rs b/testing/simulator/src/cli.rs index 0b888314cf1..2eff6c52430 100644 --- a/testing/simulator/src/cli.rs +++ b/testing/simulator/src/cli.rs @@ -148,11 +148,6 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { .takes_value(true) .default_value("3") .help("Speed up factor. Please use a divisor of 12.")) - .arg(Arg::with_name("post-merge") - .short("m") - .long("post-merge") - .takes_value(false) - .help("Simulate the merge transition")) .arg(Arg::with_name("continue_after_checks") .short("c") .long("continue_after_checks") diff --git a/testing/simulator/src/fallback_sim.rs b/testing/simulator/src/fallback_sim.rs index 63b50d5c3ca..c122df7b331 100644 --- a/testing/simulator/src/fallback_sim.rs +++ b/testing/simulator/src/fallback_sim.rs @@ -21,7 +21,7 @@ use sensitive_url::SensitiveUrl; use tokio::time::sleep; use types::{Epoch, EthSpec, MinimalEthSpec}; -const END_EPOCH: u64 = 20; +const END_EPOCH: u64 = 16; const ALTAIR_FORK_EPOCH: u64 = 1; const BELLATRIX_FORK_EPOCH: u64 = 2; @@ -36,7 +36,6 @@ pub fn run_fallback_sim(matches: &ArgMatches) -> Result<(), String> { value_t!(matches, "validators_per_vc", usize).expect("missing validators_per_vc default"); let bns_per_vc = value_t!(matches, "bns_per_vc", usize).expect("missing bns_per_vc default"); let continue_after_checks = matches.is_present("continue_after_checks"); - //let post_merge_sim = matches.is_present("post-merge"); let post_merge_sim = true; println!("Fallback Simulator:"); @@ -106,8 +105,6 @@ fn fallback_sim( let total_validator_count = validators_per_vc * vc_count; let node_count = vc_count * bns_per_vc; - //let altair_fork_version = spec.altair_fork_version; - //let bellatrix_fork_version = spec.bellatrix_fork_version; spec.seconds_per_slot /= speed_up_factor; spec.seconds_per_slot = max(1, spec.seconds_per_slot); @@ -266,41 +263,32 @@ fn fallback_sim( * breakage by changes to the VC. */ - let ( - //finalization, - //block_prod, - //validator_count, - //onboarding, - fallback, - check_attestations, - //fork, - //sync_aggregate, - //transition, - ) = futures::join!( - //checks::verify_first_finalization(network.clone(), slot_duration), + let (disconnect, reconnect, check_attestations) = futures::join!( checks::disconnect_from_execution_layer( network.clone(), Epoch::new(BELLATRIX_FORK_EPOCH), - slot_duration + slot_duration, + 0 + ), + checks::reconnect_to_execution_layer( + network.clone(), + Epoch::new(BELLATRIX_FORK_EPOCH), + slot_duration, + 0, + 2, ), checks::check_attestation_correctness( network.clone(), - Epoch::new(END_EPOCH), + Epoch::new(0), + Epoch::new(END_EPOCH - 2), MinimalEthSpec::slots_per_epoch(), - slot_duration + slot_duration, + 1, ), - //checks::stall_node(network.clone(), 0, 30, seconds_per_slot), ); - - //block_prod?; - //finalization?; - //validator_count?; - //onboarding?; - fallback?; + disconnect?; + reconnect?; check_attestations?; - //fork?; - //sync_aggregate?; - //transition?; // The `final_future` either completes immediately or never completes, depending on the value // of `continue_after_checks`. diff --git a/testing/simulator/src/main.rs b/testing/simulator/src/main.rs index 83ca1135efc..4b27184f744 100644 --- a/testing/simulator/src/main.rs +++ b/testing/simulator/src/main.rs @@ -30,6 +30,14 @@ use env_logger::{Builder, Env}; use local_network::LocalNetwork; use types::MinimalEthSpec; +// Since simulator tests are non-deterministic and there is a non-zero chance of missed +// attestations, define an acceptable network-wide attestation performance. +// +// This has potential to block CI so it should be set conservatively enough that spurious failures +// don't become very common, but not so conservatively that regressions to the fallback mechanism +// cannot be detected. +pub(crate) const ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE: f64 = 99.0; + pub type E = MinimalEthSpec; fn main() { From 59c6b0904f756cda57b02215ad84a0d3d3355639 Mon Sep 17 00:00:00 2001 From: Mac L Date: Mon, 24 Jul 2023 12:38:38 +1000 Subject: [PATCH 13/50] Fix tests again --- beacon_node/http_api/tests/tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index 1ab519dd906..e8e78382e96 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -1277,7 +1277,7 @@ impl ApiTester { assert!(self .client .clone() - .post_beacon_blocks(&next_block) + .post_beacon_blocks(&block) .await .is_err()); From 6fc12e65ded635b89172512c3e30e29103cd2cbb Mon Sep 17 00:00:00 2001 From: Mac L Date: Mon, 24 Jul 2023 14:29:13 +1000 Subject: [PATCH 14/50] Increase fallback simulator tolerance --- testing/simulator/src/main.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing/simulator/src/main.rs b/testing/simulator/src/main.rs index 4b27184f744..7bcec31ed73 100644 --- a/testing/simulator/src/main.rs +++ b/testing/simulator/src/main.rs @@ -36,7 +36,7 @@ use types::MinimalEthSpec; // This has potential to block CI so it should be set conservatively enough that spurious failures // don't become very common, but not so conservatively that regressions to the fallback mechanism // cannot be detected. -pub(crate) const ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE: f64 = 99.0; +pub(crate) const ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE: f64 = 95.0; pub type E = MinimalEthSpec; From c9bb547a1e53317455e85df3f5086ffe0c674608 Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 1 Aug 2023 13:38:22 +1000 Subject: [PATCH 15/50] Add http api endpoint --- validator_client/src/beacon_node_fallback.rs | 10 ++--- validator_client/src/beacon_node_health.rs | 11 +++--- validator_client/src/block_service.rs | 8 ++-- validator_client/src/http_api/mod.rs | 40 +++++++++++++++++++- validator_client/src/http_api/test_utils.rs | 1 + validator_client/src/lib.rs | 1 + 6 files changed, 56 insertions(+), 15 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 759148060ab..4e18246dad9 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -132,7 +132,7 @@ impl fmt::Display for Errors { } /// Reasons why a candidate might not be ready. -#[derive(Debug, Clone, Copy)] +#[derive(Debug, Clone, Copy, Deserialize, Serialize)] pub enum CandidateError { Uninitialized, Offline, @@ -151,9 +151,9 @@ pub struct CandidateInfo { /// for a query. #[derive(Debug)] pub struct CandidateBeaconNode { - id: usize, - beacon_node: BeaconNodeHttpClient, - health: PLRwLock>, + pub id: usize, + pub beacon_node: BeaconNodeHttpClient, + pub health: PLRwLock>, _phantom: PhantomData, } @@ -324,7 +324,7 @@ impl CandidateBeaconNode { /// identical query. #[derive(Clone, Debug)] pub struct BeaconNodeFallback { - candidates: Arc>>>, + pub candidates: Arc>>>, disable_run_on_all: bool, distance_tiers: BeaconNodeSyncDistanceTiers, slot_clock: Option, diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index f1a6c7bbec0..ed91d3a2c58 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -1,4 +1,5 @@ use crate::beacon_node_fallback::Config; +use serde_derive::{Deserialize, Serialize}; use slot_clock::SlotClock; use std::cmp::Ordering; use std::fmt::{Debug, Display, Formatter}; @@ -20,7 +21,7 @@ type HealthTier = u8; type SyncDistance = Slot; /// Helpful enum which is used when pattern matching to determine health tier. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub enum SyncDistanceTier { Synced, Small, @@ -95,19 +96,19 @@ impl Default for BeaconNodeSyncDistanceTiers { /// Execution Node health metrics. /// /// Currently only considers `el_offline`. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub enum ExecutionEngineHealth { Healthy, Unhealthy, } -#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub enum IsOptimistic { Yes, No, } -#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub struct BeaconNodeHealthTier { pub tier: HealthTier, pub sync_distance: SyncDistance, @@ -158,7 +159,7 @@ impl BeaconNodeHealthTier { } /// Beacon Node Health metrics. -#[derive(Copy, Clone, Debug, PartialEq, Eq)] +#[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub struct BeaconNodeHealth { // The ID of the Beacon Node. This should correspond with its position in the `--beacon-nodes` // list. Note that the ID field is used to tie-break nodes with the same health so that nodes diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index 255b5328621..98b7f69456d 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -47,8 +47,8 @@ impl From> for BlockError { pub struct BlockServiceBuilder { validator_store: Option>>, slot_clock: Option>, - beacon_nodes: Option>>, - proposer_nodes: Option>>, + pub beacon_nodes: Option>>, + pub proposer_nodes: Option>>, context: Option>, graffiti: Option, graffiti_file: Option, @@ -190,8 +190,8 @@ impl ProposerFallback { pub struct Inner { validator_store: Arc>, slot_clock: Arc, - beacon_nodes: Arc>, - proposer_nodes: Option>>, + pub beacon_nodes: Arc>, + pub proposer_nodes: Option>>, context: RuntimeContext, graffiti: Option, graffiti_file: Option, diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index f654833cbb4..1528ce712be 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -7,8 +7,10 @@ mod tests; pub mod test_utils; +use crate::beacon_node_fallback::CandidateError; +use crate::beacon_node_health::BeaconNodeHealth; use crate::http_api::create_signed_voluntary_exit::create_signed_voluntary_exit; -use crate::{determine_graffiti, GraffitiFile, ValidatorStore}; +use crate::{determine_graffiti, BlockService, GraffitiFile, ValidatorStore}; use account_utils::{ mnemonic_from_phrase, validator_definitions::{SigningDefinition, ValidatorDefinition, Web3SignerDefinition}, @@ -73,6 +75,7 @@ impl From for Error { pub struct Context { pub task_executor: TaskExecutor, pub api_secret: ApiSecret, + pub block_service: Option>, pub validator_store: Option>>, pub validator_dir: Option, pub secrets_dir: Option, @@ -173,6 +176,17 @@ pub fn serve( let signer = ctx.api_secret.signer(); let signer = warp::any().map(move || signer.clone()); + let inner_block_service = ctx.block_service.clone(); + let block_service_filter = warp::any() + .map(move || inner_block_service.clone()) + .and_then(|block_service: Option<_>| async move { + block_service.ok_or_else(|| { + warp_utils::reject::custom_not_found( + "block service is not initialized.".to_string(), + ) + }) + }); + let inner_validator_store = ctx.validator_store.clone(); let validator_store_filter = warp::any() .map(move || inner_validator_store.clone()) @@ -410,6 +424,29 @@ pub fn serve( }, ); + // GET lighthouse/ui/fallback_health + let get_lighthouse_ui_fallback_health = warp::path("lighthouse") + .and(warp::path("ui")) + .and(warp::path("fallback_health")) + .and(warp::path::end()) + .and(signer.clone()) + .and(block_service_filter.clone()) + .and_then(|signer, block_filter: BlockService| async move { + let mut result: HashMap> = + HashMap::new(); + for node in &*block_filter.beacon_nodes.candidates.read().await { + result.insert(node.beacon_node.to_string(), *node.health.read()); + } + if let Some(proposer_nodes) = &block_filter.proposer_nodes { + for node in &*proposer_nodes.candidates.read().await { + result.insert(node.beacon_node.to_string(), *node.health.read()); + } + } + + blocking_signed_json_task(signer, move || Ok(api_types::GenericResponse::from(result))) + .await + }); + // POST lighthouse/validators/ let post_validators = warp::path("lighthouse") .and(warp::path("validators")) @@ -1173,6 +1210,7 @@ pub fn serve( .or(get_lighthouse_validators_pubkey) .or(get_lighthouse_ui_health) .or(get_lighthouse_ui_graffiti) + .or(get_lighthouse_ui_fallback_health) .or(get_fee_recipient) .or(get_gas_limit) .or(get_std_keystores) diff --git a/validator_client/src/http_api/test_utils.rs b/validator_client/src/http_api/test_utils.rs index c7558dd586d..7e7af5e711b 100644 --- a/validator_client/src/http_api/test_utils.rs +++ b/validator_client/src/http_api/test_utils.rs @@ -126,6 +126,7 @@ impl ApiTester { let context = Arc::new(Context { task_executor: test_runtime.task_executor.clone(), api_secret, + block_service: None, validator_dir: Some(validator_dir.path().into()), secrets_dir: Some(secrets_dir.path().into()), validator_store: Some(validator_store.clone()), diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index fbabb011bce..51da8b7ce4d 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -577,6 +577,7 @@ impl ProductionValidatorClient { let ctx = Arc::new(http_api::Context { task_executor: self.context.executor.clone(), api_secret, + block_service: Some(self.block_service.clone()), validator_store: Some(self.validator_store.clone()), validator_dir: Some(self.config.validator_dir.clone()), secrets_dir: Some(self.config.secrets_dir.clone()), From b25a1352c1eecd50eba47fb4b3cd390a828554b0 Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 1 Aug 2023 17:55:43 +1000 Subject: [PATCH 16/50] Fix todos and tests --- validator_client/src/beacon_node_fallback.rs | 17 +++++++-- validator_client/src/beacon_node_health.rs | 39 ++++++-------------- validator_client/src/http_api/tests.rs | 1 + validator_client/src/notifier.rs | 1 - 4 files changed, 25 insertions(+), 33 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 4e18246dad9..f012f55c605 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -23,7 +23,7 @@ use std::marker::PhantomData; use std::sync::Arc; use std::time::{Duration, Instant}; use tokio::{sync::RwLock, time::sleep}; -use types::{ChainSpec, Config as ConfigSpec, EthSpec}; +use types::{ChainSpec, Config as ConfigSpec, EthSpec, Slot}; /// Message emitted when the VC detects the BN is using a different spec. const UPDATE_REQUIRED_LOG_HINT: &str = "this VC or the remote BN may need updating"; @@ -37,6 +37,10 @@ const UPDATE_REQUIRED_LOG_HINT: &str = "this VC or the remote BN may need updati /// having the correct nodes up and running prior to the start of the slot. const SLOT_LOOKAHEAD: Duration = Duration::from_secs(2); +/// If the beacon node slot_clock is within 1 slot, this is deemed acceptable. Otherwise the node +/// will be marked as CandidateError::TimeDiscrepancy. +const FUTURE_SLOT_TOLERANCE: Slot = Slot::new(1); + // Configuration for the Beacon Node fallback. #[derive(Copy, Clone, Debug, Default, Serialize, Deserialize)] pub struct Config { @@ -213,6 +217,13 @@ impl CandidateBeaconNode { if let Some(slot_clock) = slot_clock { match check_node_health(&self.beacon_node, log).await { Ok((head, is_optimistic, el_offline)) => { + let slot_clock_head = slot_clock.now().ok_or(CandidateError::Uninitialized)?; + + if head > slot_clock_head + FUTURE_SLOT_TOLERANCE { + return Err(CandidateError::TimeDiscrepancy); + } + let sync_distance = slot_clock_head.saturating_sub(head); + // Currently ExecutionEngineHealth is solely determined by online status. let execution_status = if el_offline { ExecutionEngineHealth::Unhealthy @@ -228,15 +239,13 @@ impl CandidateBeaconNode { let new_health = BeaconNodeHealth::from_status( self.id, + sync_distance, head, optimistic_status, execution_status, distance_tiers, - slot_clock, ); - // TODO(mac): Set metric here. - *self.health.write() = Ok(new_health); Ok(()) } diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index ed91d3a2c58..81df0cd9925 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -1,18 +1,17 @@ use crate::beacon_node_fallback::Config; use serde_derive::{Deserialize, Serialize}; -use slot_clock::SlotClock; use std::cmp::Ordering; use std::fmt::{Debug, Display, Formatter}; use types::Slot; -// Sync distances between 0 and DEFAULT_SYNC_TOLERANCE are considered `synced`. -// Sync distance tiers are determined by the different modifiers. -// -// The default range is the following: -// Synced: 0..=8 -// Small: 9..=16 -// Medium: 17..=64 -// Large: 65.. +/// Sync distances between 0 and DEFAULT_SYNC_TOLERANCE are considered `synced`. +/// Sync distance tiers are determined by the different modifiers. +/// +/// The default range is the following: +/// Synced: 0..=8 +/// Small: 9..=16 +/// Medium: 17..=64 +/// Large: 65.. const DEFAULT_SYNC_TOLERANCE: Slot = Slot::new(8); const DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER: Slot = Slot::new(8); const DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER: Slot = Slot::new(48); @@ -195,15 +194,14 @@ impl PartialOrd for BeaconNodeHealth { } impl BeaconNodeHealth { - pub fn from_status( + pub fn from_status( id: usize, + sync_distance: Slot, head: Slot, optimistic_status: IsOptimistic, execution_status: ExecutionEngineHealth, distance_tiers: &BeaconNodeSyncDistanceTiers, - slot_clock: &T, ) -> Self { - let sync_distance = BeaconNodeHealth::compute_sync_distance(head, slot_clock); let health_tier = BeaconNodeHealth::compute_health_tier( sync_distance, optimistic_status, @@ -228,14 +226,6 @@ impl BeaconNodeHealth { self.health_tier } - fn compute_sync_distance(head: Slot, slot_clock: &T) -> SyncDistance { - // TODO(mac) May be worth distinguishing between nodes that are ahead of the `slot_clock`. - slot_clock - .now() - .map(|head_slot| head_slot.saturating_sub(head)) - .unwrap_or(Slot::max_value()) - } - fn compute_health_tier( sync_distance: SyncDistance, optimistic_status: IsOptimistic, @@ -306,20 +296,13 @@ mod tests { SyncDistanceTier, }; use crate::beacon_node_fallback::Config; - use slot_clock::{SlotClock, TestingSlotClock}; - use std::time::Duration; use types::Slot; #[test] fn all_possible_health_tiers() { - let current_head = Slot::new(64); - let config = Config::default(); let beacon_node_sync_distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); - let slot_clock = - TestingSlotClock::new(current_head, Duration::from_secs(0), Duration::from_secs(1)); - let mut health_vec = vec![]; for head_slot in 0..=64 { @@ -327,11 +310,11 @@ mod tests { for ee_health in &[Healthy, Unhealthy] { let health = BeaconNodeHealth::from_status( 0, + Slot::new(0), Slot::new(head_slot), *optimistic_status, *ee_health, &beacon_node_sync_distance_tiers, - &slot_clock, ); health_vec.push(health); } diff --git a/validator_client/src/http_api/tests.rs b/validator_client/src/http_api/tests.rs index 3bff444703b..35dcfffc557 100644 --- a/validator_client/src/http_api/tests.rs +++ b/validator_client/src/http_api/tests.rs @@ -110,6 +110,7 @@ impl ApiTester { let context = Arc::new(Context { task_executor: test_runtime.task_executor.clone(), api_secret, + block_service: None, validator_dir: Some(validator_dir.path().into()), secrets_dir: Some(secrets_dir.path().into()), validator_store: Some(validator_store.clone()), diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index 1e7940de579..521ff49a147 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -84,7 +84,6 @@ async fn notify( set_gauge(&http_metrics::metrics::ETH2_FALLBACK_CONNECTED, 0); } - // TODO(mac) Store all connected node info into metrics. for info in candidate_info { if let Some(health) = info.health { debug!( From 23f68ad61c3a5a8e61cef4a42a5307c1180b31e9 Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 2 Aug 2023 19:00:10 +1000 Subject: [PATCH 17/50] Update simulator --- .github/workflows/test-suite.yml | 4 +- testing/simulator/src/common.rs | 97 ++++++++++++++++++++ testing/simulator/src/eth1_sim.rs | 98 +------------------- testing/simulator/src/fallback_sim.rs | 125 ++++++-------------------- testing/simulator/src/main.rs | 1 + 5 files changed, 130 insertions(+), 195 deletions(-) create mode 100644 testing/simulator/src/common.rs diff --git a/.github/workflows/test-suite.yml b/.github/workflows/test-suite.yml index c1e6a1913c8..debc45bc75c 100644 --- a/.github/workflows/test-suite.yml +++ b/.github/workflows/test-suite.yml @@ -225,8 +225,8 @@ jobs: uses: arduino/setup-protoc@e52d9eb8f7b63115df1ac544a1376fdbf5a39612 with: repo-token: ${{ secrets.GITHUB_TOKEN }} - - name: Install anvil - run: cargo install --git https://github.com/foundry-rs/foundry --locked anvil + - name: Install Foundry (anvil) + uses: foundry-rs/foundry-toolchain@v1 - name: Run the fallback simulator run: cargo run --release --bin simulator fallback-sim syncing-simulator-ubuntu: diff --git a/testing/simulator/src/common.rs b/testing/simulator/src/common.rs new file mode 100644 index 00000000000..27548384ad8 --- /dev/null +++ b/testing/simulator/src/common.rs @@ -0,0 +1,97 @@ +use crate::local_network::EXECUTION_PORT; +use crate::LocalNetwork; +use eth1::{Eth1Endpoint, DEFAULT_CHAIN_ID}; +use eth1_test_rig::AnvilEth1Instance; + +use execution_layer::http::deposit_methods::Eth1Id; +use node_test_rig::environment::RuntimeContext; +use node_test_rig::{testing_client_config, ClientConfig, ClientGenesis}; +use sensitive_url::SensitiveUrl; +use std::net::Ipv4Addr; +use std::time::Duration; +use types::EthSpec; + +pub struct LocalNetworkParams { + pub eth1_block_time: Duration, + pub total_validator_count: usize, + pub deposit_amount: u64, + pub node_count: usize, + pub proposer_nodes: usize, + pub post_merge_sim: bool, +} + +pub async fn create_local_network( + LocalNetworkParams { + eth1_block_time, + total_validator_count, + deposit_amount, + node_count, + proposer_nodes, + post_merge_sim, + }: LocalNetworkParams, + context: RuntimeContext, +) -> Result<(LocalNetwork, ClientConfig), String> { + /* + * Deploy the deposit contract, spawn tasks to keep creating new blocks and deposit + * validators. + */ + let anvil_eth1_instance = AnvilEth1Instance::new(DEFAULT_CHAIN_ID.into()).await?; + let deposit_contract = anvil_eth1_instance.deposit_contract; + let chain_id = anvil_eth1_instance.anvil.chain_id(); + let anvil = anvil_eth1_instance.anvil; + let eth1_endpoint = + SensitiveUrl::parse(anvil.endpoint().as_str()).expect("Unable to parse anvil endpoint."); + let deposit_contract_address = deposit_contract.address(); + + // Start a timer that produces eth1 blocks on an interval. + tokio::spawn(async move { + let mut interval = tokio::time::interval(eth1_block_time); + loop { + interval.tick().await; + let _ = anvil.evm_mine().await; + } + }); + + // Submit deposits to the deposit contract. + tokio::spawn(async move { + for i in 0..total_validator_count { + println!("Submitting deposit for validator {}...", i); + let _ = deposit_contract + .deposit_deterministic_async::(i, deposit_amount) + .await; + } + }); + + let mut beacon_config = testing_client_config(); + + beacon_config.genesis = ClientGenesis::DepositContract; + beacon_config.eth1.endpoint = Eth1Endpoint::NoAuth(eth1_endpoint); + beacon_config.eth1.deposit_contract_address = deposit_contract_address; + beacon_config.eth1.deposit_contract_deploy_block = 0; + beacon_config.eth1.lowest_cached_block_number = 0; + beacon_config.eth1.follow_distance = 1; + beacon_config.eth1.node_far_behind_seconds = 20; + beacon_config.dummy_eth1_backend = false; + beacon_config.sync_eth1_chain = true; + beacon_config.eth1.auto_update_interval_millis = eth1_block_time.as_millis() as u64; + beacon_config.eth1.chain_id = Eth1Id::from(chain_id); + beacon_config.network.target_peers = node_count + proposer_nodes - 1; + + beacon_config.network.enr_address = (Some(Ipv4Addr::LOCALHOST), None); + + if post_merge_sim { + let el_config = execution_layer::Config { + execution_endpoints: vec![SensitiveUrl::parse(&format!( + "http://localhost:{}", + EXECUTION_PORT + )) + .unwrap()], + ..Default::default() + }; + + beacon_config.execution_layer = Some(el_config); + } + + let network = LocalNetwork::new(context, beacon_config.clone()).await?; + Ok((network, beacon_config)) +} diff --git a/testing/simulator/src/eth1_sim.rs b/testing/simulator/src/eth1_sim.rs index 57c944cf1a7..3657f548a32 100644 --- a/testing/simulator/src/eth1_sim.rs +++ b/testing/simulator/src/eth1_sim.rs @@ -1,21 +1,16 @@ -use crate::local_network::{EXECUTION_PORT, TERMINAL_BLOCK, TERMINAL_DIFFICULTY}; -use crate::{checks, LocalNetwork}; +use crate::checks; +use crate::local_network::{TERMINAL_BLOCK, TERMINAL_DIFFICULTY}; use clap::ArgMatches; -use eth1::{Eth1Endpoint, DEFAULT_CHAIN_ID}; -use eth1_test_rig::AnvilEth1Instance; +use crate::common::{create_local_network, LocalNetworkParams}; use crate::retry::with_retry; -use execution_layer::http::deposit_methods::Eth1Id; use futures::prelude::*; -use node_test_rig::environment::RuntimeContext; use node_test_rig::{ environment::{EnvironmentBuilder, LoggerConfig}, - testing_client_config, testing_validator_config, ClientConfig, ClientGenesis, ValidatorFiles, + testing_validator_config, ValidatorFiles, }; use rayon::prelude::*; -use sensitive_url::SensitiveUrl; use std::cmp::max; -use std::net::Ipv4Addr; use std::time::Duration; use tokio::time::sleep; use types::{Epoch, EthSpec, MinimalEthSpec}; @@ -296,88 +291,3 @@ pub fn run_eth1_sim(matches: &ArgMatches) -> Result<(), String> { Ok(()) } - -struct LocalNetworkParams { - eth1_block_time: Duration, - total_validator_count: usize, - deposit_amount: u64, - node_count: usize, - proposer_nodes: usize, - post_merge_sim: bool, -} - -async fn create_local_network( - LocalNetworkParams { - eth1_block_time, - total_validator_count, - deposit_amount, - node_count, - proposer_nodes, - post_merge_sim, - }: LocalNetworkParams, - context: RuntimeContext, -) -> Result<(LocalNetwork, ClientConfig), String> { - /* - * Deploy the deposit contract, spawn tasks to keep creating new blocks and deposit - * validators. - */ - let anvil_eth1_instance = AnvilEth1Instance::new(DEFAULT_CHAIN_ID.into()).await?; - let deposit_contract = anvil_eth1_instance.deposit_contract; - let chain_id = anvil_eth1_instance.anvil.chain_id(); - let anvil = anvil_eth1_instance.anvil; - let eth1_endpoint = - SensitiveUrl::parse(anvil.endpoint().as_str()).expect("Unable to parse anvil endpoint."); - let deposit_contract_address = deposit_contract.address(); - - // Start a timer that produces eth1 blocks on an interval. - tokio::spawn(async move { - let mut interval = tokio::time::interval(eth1_block_time); - loop { - interval.tick().await; - let _ = anvil.evm_mine().await; - } - }); - - // Submit deposits to the deposit contract. - tokio::spawn(async move { - for i in 0..total_validator_count { - println!("Submitting deposit for validator {}...", i); - let _ = deposit_contract - .deposit_deterministic_async::(i, deposit_amount) - .await; - } - }); - - let mut beacon_config = testing_client_config(); - - beacon_config.genesis = ClientGenesis::DepositContract; - beacon_config.eth1.endpoint = Eth1Endpoint::NoAuth(eth1_endpoint); - beacon_config.eth1.deposit_contract_address = deposit_contract_address; - beacon_config.eth1.deposit_contract_deploy_block = 0; - beacon_config.eth1.lowest_cached_block_number = 0; - beacon_config.eth1.follow_distance = 1; - beacon_config.eth1.node_far_behind_seconds = 20; - beacon_config.dummy_eth1_backend = false; - beacon_config.sync_eth1_chain = true; - beacon_config.eth1.auto_update_interval_millis = eth1_block_time.as_millis() as u64; - beacon_config.eth1.chain_id = Eth1Id::from(chain_id); - beacon_config.network.target_peers = node_count + proposer_nodes - 1; - - beacon_config.network.enr_address = (Some(Ipv4Addr::LOCALHOST), None); - - if post_merge_sim { - let el_config = execution_layer::Config { - execution_endpoints: vec![SensitiveUrl::parse(&format!( - "http://localhost:{}", - EXECUTION_PORT - )) - .unwrap()], - ..Default::default() - }; - - beacon_config.execution_layer = Some(el_config); - } - - let network = LocalNetwork::new(context, beacon_config.clone()).await?; - Ok((network, beacon_config)) -} diff --git a/testing/simulator/src/fallback_sim.rs b/testing/simulator/src/fallback_sim.rs index c122df7b331..9afd6756a3e 100644 --- a/testing/simulator/src/fallback_sim.rs +++ b/testing/simulator/src/fallback_sim.rs @@ -1,23 +1,17 @@ -use futures::prelude::*; - -use std::cmp::max; -use std::net::Ipv4Addr; -use std::time::Duration; - -use crate::local_network::{EXECUTION_PORT, TERMINAL_BLOCK, TERMINAL_DIFFICULTY}; -use crate::{checks, LocalNetwork, E}; +use crate::checks; +use crate::local_network::{TERMINAL_BLOCK, TERMINAL_DIFFICULTY}; use clap::ArgMatches; -use eth1::{Eth1Endpoint, DEFAULT_CHAIN_ID}; -use eth1_test_rig::AnvilEth1Instance; - -use execution_layer::http::deposit_methods::Eth1Id; +use crate::common::{create_local_network, LocalNetworkParams}; +use crate::retry::with_retry; +use futures::prelude::*; use node_test_rig::{ environment::{EnvironmentBuilder, LoggerConfig}, - testing_client_config, testing_validator_config, ClientGenesis, ValidatorFiles, + testing_validator_config, ValidatorFiles, }; use rayon::prelude::*; -use sensitive_url::SensitiveUrl; +use std::cmp::max; +use std::time::Duration; use tokio::time::sleep; use types::{Epoch, EthSpec, MinimalEthSpec}; @@ -29,12 +23,13 @@ const SUGGESTED_FEE_RECIPIENT: [u8; 20] = [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; pub fn run_fallback_sim(matches: &ArgMatches) -> Result<(), String> { - let speed_up_factor = - value_t!(matches, "speed_up_factor", u64).expect("missing speed_up_factor default"); let vc_count = value_t!(matches, "vc_count", usize).expect("missing vc_count default"); let validators_per_vc = value_t!(matches, "validators_per_vc", usize).expect("missing validators_per_vc default"); let bns_per_vc = value_t!(matches, "bns_per_vc", usize).expect("missing bns_per_vc default"); + assert!(bns_per_vc > 1); + let speed_up_factor = + value_t!(matches, "speed_up_factor", u64).expect("missing speed_up_factor default"); let continue_after_checks = matches.is_present("continue_after_checks"); let post_merge_sim = true; @@ -46,26 +41,6 @@ pub fn run_fallback_sim(matches: &ArgMatches) -> Result<(), String> { let log_level = "debug"; - fallback_sim( - speed_up_factor, - vc_count, - validators_per_vc, - bns_per_vc, - post_merge_sim, - continue_after_checks, - log_level, - ) -} - -fn fallback_sim( - speed_up_factor: u64, - vc_count: usize, - validators_per_vc: usize, - bns_per_vc: usize, - post_merge_sim: bool, - continue_after_checks: bool, - log_level: &str, -) -> Result<(), String> { // Generate the directories and keystores required for the validator clients. let validator_files = (0..vc_count) .into_par_iter() @@ -122,77 +97,29 @@ fn fallback_sim( let seconds_per_slot = spec.seconds_per_slot; let slot_duration = Duration::from_secs(spec.seconds_per_slot); - let _initial_validator_count = spec.min_genesis_active_validator_count as usize; let deposit_amount = env.eth2_config.spec.max_effective_balance; let context = env.core_context(); let main_future = async { - /* - * Deploy the deposit contract, spawn tasks to keep creating new blocks and deposit - * validators. - */ - let anvil_eth1_instance = AnvilEth1Instance::new(DEFAULT_CHAIN_ID.into()).await?; - let deposit_contract = anvil_eth1_instance.deposit_contract; - let chain_id = anvil_eth1_instance.anvil.chain_id(); - let anvil = anvil_eth1_instance.anvil; - let eth1_endpoint = SensitiveUrl::parse(anvil.endpoint().as_str()) - .expect("Unable to parse anvil endpoint."); - let deposit_contract_address = deposit_contract.address(); - - // Start a timer that produces eth1 blocks on an interval. - tokio::spawn(async move { - let mut interval = tokio::time::interval(eth1_block_time); - loop { - interval.tick().await; - let _ = anvil.evm_mine().await; - } - }); - - // Submit deposits to the deposit contract. - tokio::spawn(async move { - for i in 0..total_validator_count { - println!("Submitting deposit for validator {}...", i); - let _ = deposit_contract - .deposit_deterministic_async::(i, deposit_amount) - .await; - } - }); - - let mut beacon_config = testing_client_config(); - - beacon_config.genesis = ClientGenesis::DepositContract; - beacon_config.eth1.endpoint = Eth1Endpoint::NoAuth(eth1_endpoint); - beacon_config.eth1.deposit_contract_address = deposit_contract_address; - beacon_config.eth1.deposit_contract_deploy_block = 0; - beacon_config.eth1.lowest_cached_block_number = 0; - beacon_config.eth1.follow_distance = 1; - beacon_config.eth1.node_far_behind_seconds = 20; - beacon_config.dummy_eth1_backend = false; - beacon_config.sync_eth1_chain = true; - beacon_config.eth1.auto_update_interval_millis = eth1_block_time.as_millis() as u64; - beacon_config.eth1.chain_id = Eth1Id::from(chain_id); - beacon_config.network.target_peers = node_count - 1; - - beacon_config.network.enr_address = (Some(Ipv4Addr::LOCALHOST), None); - - if post_merge_sim { - let el_config = execution_layer::Config { - execution_endpoints: vec![SensitiveUrl::parse(&format!( - "http://localhost:{}", - EXECUTION_PORT - )) - .unwrap()], - ..Default::default() - }; - - beacon_config.execution_layer = Some(el_config); - } - /* * Create a new `LocalNetwork` with one beacon node. */ - let network = LocalNetwork::new(context.clone(), beacon_config.clone()).await?; + let max_retries = 3; + let (network, beacon_config) = with_retry(max_retries, || { + Box::pin(create_local_network( + LocalNetworkParams { + eth1_block_time, + total_validator_count, + deposit_amount, + node_count, + proposer_nodes: 0, + post_merge_sim, + }, + context.clone(), + )) + }) + .await?; /* * One by one, add beacon nodes to the network. diff --git a/testing/simulator/src/main.rs b/testing/simulator/src/main.rs index 7bcec31ed73..a0b9e0a57de 100644 --- a/testing/simulator/src/main.rs +++ b/testing/simulator/src/main.rs @@ -18,6 +18,7 @@ extern crate clap; mod checks; mod cli; +mod common; mod eth1_sim; mod fallback_sim; mod local_network; From 1d704411574c3ed7ec927b0876a0d24d3ffba78a Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 3 Nov 2023 13:22:32 +1100 Subject: [PATCH 18/50] Add suggestions --- Cargo.lock | 1 + common/eth2/Cargo.toml | 1 + common/eth2/src/lib.rs | 11 ++++------- validator_client/src/lib.rs | 8 ++++---- 4 files changed, 10 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index af9d0a0ad88..6b60c58c247 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2099,6 +2099,7 @@ version = "0.1.0" dependencies = [ "account_utils", "bytes", + "derivative", "eth2_keystore", "ethereum_serde_utils", "ethereum_ssz", diff --git a/common/eth2/Cargo.toml b/common/eth2/Cargo.toml index 02460551a9e..027136ac01d 100644 --- a/common/eth2/Cargo.toml +++ b/common/eth2/Cargo.toml @@ -31,6 +31,7 @@ slashing_protection = { workspace = true } mediatype = "0.19.13" mime = "0.3.16" pretty_reqwest_error = { workspace = true } +derivative = { workspace = true } [dev-dependencies] tokio = { workspace = true } diff --git a/common/eth2/src/lib.rs b/common/eth2/src/lib.rs index 9df1f2f600f..ca244fd89c7 100644 --- a/common/eth2/src/lib.rs +++ b/common/eth2/src/lib.rs @@ -16,6 +16,7 @@ pub mod types; use self::mixin::{RequestAccept, ResponseOptional}; use self::types::{Error as ResponseError, *}; +use derivative::Derivative; use futures::Stream; use futures_util::StreamExt; use lighthouse_network::PeerId; @@ -143,19 +144,15 @@ impl Timeouts { /// A wrapper around `reqwest::Client` which provides convenience methods for interfacing with a /// Lighthouse Beacon Node HTTP server (`http_api`). -#[derive(Clone, Debug)] +#[derive(Clone, Debug, Derivative)] +#[derivative(PartialEq)] pub struct BeaconNodeHttpClient { + #[derivative(PartialEq = "ignore")] client: reqwest::Client, server: SensitiveUrl, timeouts: Timeouts, } -impl PartialEq for BeaconNodeHttpClient { - fn eq(&self, other: &Self) -> bool { - self.server == other.server && self.timeouts == other.timeouts - } -} - impl Eq for BeaconNodeHttpClient {} impl fmt::Display for BeaconNodeHttpClient { diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index 8212841dbb8..b45e5ad1efd 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -342,15 +342,15 @@ impl ProductionValidatorClient { let candidates = beacon_nodes .into_iter() - .zip(0..num_nodes) - .map(|(node, id)| CandidateBeaconNode::new(node, id)) + .enumerate() + .map(|(id, node)| CandidateBeaconNode::new(node, id)) .collect(); let proposer_nodes_num = proposer_nodes.len(); let proposer_candidates = proposer_nodes .into_iter() - .zip(0..num_nodes) - .map(|(node, id)| CandidateBeaconNode::new(node, id)) + .enumerate() + .map(|(id, node)| CandidateBeaconNode::new(node, id)) .collect(); // Set the count for beacon node fallbacks excluding the primary beacon node. From 4ddfc7d534ee091c9bd5ae61fa3c7bc983a39243 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 3 Nov 2023 15:43:28 +1100 Subject: [PATCH 19/50] Add id to ui endpoint --- validator_client/src/http_api/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index 1cd503b7302..19b112a584c 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -432,14 +432,14 @@ pub fn serve( .and(signer.clone()) .and(block_service_filter.clone()) .and_then(|signer, block_filter: BlockService| async move { - let mut result: HashMap> = + let mut result: HashMap<(usize, String), Result> = HashMap::new(); for node in &*block_filter.beacon_nodes.candidates.read().await { - result.insert(node.beacon_node.to_string(), *node.health.read()); + result.insert((node.id, node.beacon_node.to_string()), *node.health.read()); } if let Some(proposer_nodes) = &block_filter.proposer_nodes { for node in &*proposer_nodes.candidates.read().await { - result.insert(node.beacon_node.to_string(), *node.health.read()); + result.insert((node.id, node.beacon_node.to_string()), *node.health.read()); } } From 21c34c280b48b6dda7838ca33deff36ce1d22c50 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 3 Nov 2023 16:34:18 +1100 Subject: [PATCH 20/50] Remove unnecessary clones --- beacon_node/http_api/tests/tests.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index 884c988c19e..774505cf7ad 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -1259,7 +1259,6 @@ impl ApiTester { let next_block = self.next_block.clone(); self.client - .clone() .post_beacon_blocks(next_block) .await .unwrap(); @@ -1303,7 +1302,6 @@ impl ApiTester { assert!(self .client - .clone() .post_beacon_blocks(&SignedBlockContents::from(block)) .await .is_err()); @@ -2529,7 +2527,6 @@ impl ApiTester { SignedBlockContents::try_from(signed_block.clone()).unwrap(); self.client - .clone() .post_beacon_blocks(&signed_block_contents) .await .unwrap(); @@ -4500,7 +4497,6 @@ impl ApiTester { }); self.client - .clone() .post_beacon_blocks(&self.next_block) .await .unwrap(); @@ -4536,7 +4532,6 @@ impl ApiTester { self.harness.advance_slot(); self.client - .clone() .post_beacon_blocks(&self.reorg_block) .await .unwrap(); @@ -4680,7 +4675,6 @@ impl ApiTester { }); self.client - .clone() .post_beacon_blocks(&self.next_block) .await .unwrap(); From ba92f5dff2fc6a7d6af8eb213bbe63ddbedcc0d8 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 3 Nov 2023 16:35:13 +1100 Subject: [PATCH 21/50] Formatting --- beacon_node/http_api/tests/tests.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index 774505cf7ad..90e9969ec94 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -1258,10 +1258,7 @@ impl ApiTester { pub async fn test_post_beacon_blocks_valid(mut self) -> Self { let next_block = self.next_block.clone(); - self.client - .post_beacon_blocks(next_block) - .await - .unwrap(); + self.client.post_beacon_blocks(next_block).await.unwrap(); assert!( self.network_rx.network_recv.recv().await.is_some(), From 6117ceffc257697799ca9929ecc0e0d234186d16 Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 13 Dec 2023 13:17:21 +1100 Subject: [PATCH 22/50] Fix flag tests --- lighthouse/tests/validator_client.rs | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/lighthouse/tests/validator_client.rs b/lighthouse/tests/validator_client.rs index 7795b0d0816..bba38aa3e04 100644 --- a/lighthouse/tests/validator_client.rs +++ b/lighthouse/tests/validator_client.rs @@ -476,11 +476,13 @@ fn monitoring_endpoint() { }); } #[test] -fn disable_run_on_all_default() { - CommandLineTest::new().run().with_config(|config| { - assert!(!config.beacon_node_fallback.disable_run_on_all); - assert_eq!(config.broadcast_topics, vec![]); - }); +fn disable_run_on_all_flag() { + CommandLineTest::new() + .flag("disable-run-on-all", None) + .run() + .with_config(|config| { + assert_eq!(config.broadcast_topics, vec![]); + }); // --broadcast flag takes precedence CommandLineTest::new() .flag("disable-run-on-all", None) From ba322391d404d66b8cd14125f7c8e7cffdf192d2 Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 16 Jan 2024 20:38:46 +1100 Subject: [PATCH 23/50] Fix conflicts --- validator_client/src/block_service.rs | 78 ++++++++++++--------------- 1 file changed, 35 insertions(+), 43 deletions(-) diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index 550c2659947..3b3ed6be802 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -553,34 +553,30 @@ impl BlockService { // Try the proposer nodes last, since it's likely that they don't have a // great view of attestations on the network. let unsigned_block = proposer_fallback - .request_proposers_last( - RequireSynced::No, - OfflineOnFailure::Yes, - |beacon_node| async move { - let _get_timer = metrics::start_timer_vec( - &metrics::BLOCK_SERVICE_TIMES, - &[metrics::BEACON_BLOCK_HTTP_GET], - ); - let block_response = Self::get_validator_block_v3( - beacon_node, - slot, - randao_reveal_ref, - graffiti, - proposer_index, - builder_boost_factor, - log, - ) - .await - .map_err(|e| { - BlockError::Recoverable(format!( - "Error from beacon node when producing block: {:?}", - e - )) - }); - - Ok::<_, BlockError>(block_response) - }, - ) + .request_proposers_last(|beacon_node| async move { + let _get_timer = metrics::start_timer_vec( + &metrics::BLOCK_SERVICE_TIMES, + &[metrics::BEACON_BLOCK_HTTP_GET], + ); + let block_response = Self::get_validator_block_v3( + &beacon_node, + slot, + randao_reveal_ref, + graffiti, + proposer_index, + builder_boost_factor, + log, + ) + .await + .map_err(|e| { + BlockError::Recoverable(format!( + "Error from beacon node when producing block: {:?}", + e + )) + }); + + Ok::<_, BlockError>(block_response) + }) .await??; self_ref @@ -660,21 +656,17 @@ impl BlockService { // Try the proposer nodes last, since it's likely that they don't have a // great view of attestations on the network. let unsigned_block = proposer_fallback - .request_proposers_last( - RequireSynced::No, - OfflineOnFailure::Yes, - move |beacon_node| { - Self::get_validator_block( - beacon_node, - slot, - randao_reveal_ref, - graffiti, - proposer_index, - builder_proposal, - log, - ) - }, - ) + .request_proposers_last(move |beacon_node| { + Self::get_validator_block( + beacon_node, + slot, + randao_reveal_ref, + graffiti, + proposer_index, + builder_proposal, + log, + ) + }) .await?; self_ref From 1802080cabe8ec03010f899bd53fececbee88491 Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 1 Feb 2024 17:10:25 +1100 Subject: [PATCH 24/50] Remove unnecessary pubs --- validator_client/src/block_service.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index 2150b2417ff..51f0d94989a 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -53,8 +53,8 @@ impl From> for BlockError { pub struct BlockServiceBuilder { validator_store: Option>>, slot_clock: Option>, - pub beacon_nodes: Option>>, - pub proposer_nodes: Option>>, + beacon_nodes: Option>>, + proposer_nodes: Option>>, context: Option>, graffiti: Option, graffiti_file: Option, @@ -186,8 +186,8 @@ impl ProposerFallback { pub struct Inner { validator_store: Arc>, slot_clock: Arc, - pub beacon_nodes: Arc>, - pub proposer_nodes: Option>>, + pub(crate) beacon_nodes: Arc>, + pub(crate) proposer_nodes: Option>>, context: RuntimeContext, graffiti: Option, graffiti_file: Option, From a5e934883a66bf382f23d2403820985448726592 Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 1 Feb 2024 17:35:22 +1100 Subject: [PATCH 25/50] Simplify `compute_distance_tier` and reduce notifier awaits --- validator_client/src/beacon_node_fallback.rs | 65 +++++++------------- validator_client/src/beacon_node_health.rs | 18 ++---- validator_client/src/notifier.rs | 9 ++- 3 files changed, 33 insertions(+), 59 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 0a99efc8681..34230aafbce 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -383,40 +383,6 @@ impl BeaconNodeFallback { self.candidates.read().await.len() } - /// The count of synced and ready candidates. - pub async fn num_synced(&self) -> usize { - let mut n = 0; - for candidate in self.candidates.read().await.iter() { - if let Ok(cand) = candidate.health().as_ref() { - if self - .distance_tiers - .distance_tier(cand.health_tier.sync_distance) - == SyncDistanceTier::Synced - { - n += 1 - } - } - } - n - } - - /// The count of synced and ready fallbacks excluding the primary beacon node candidate. - pub async fn num_synced_fallback(&self) -> usize { - let mut n = 0; - for candidate in self.candidates.read().await.iter().skip(1) { - if let Ok(cand) = candidate.health().as_ref() { - if self - .distance_tiers - .distance_tier(cand.health_tier.sync_distance) - == SyncDistanceTier::Synced - { - n += 1 - } - } - } - n - } - /// The count of candidates that are online and compatible, but not necessarily synced. pub async fn num_available(&self) -> usize { let mut n = 0; @@ -429,17 +395,32 @@ impl BeaconNodeFallback { n } - pub async fn get_all_candidate_info(&self) -> Vec { + // Returns all data required by the VC notifier. + pub async fn get_notifier_info(&self) -> (Vec, usize, usize) { let candidates = self.candidates.read().await; - let mut results = Vec::with_capacity(candidates.len()); + + let mut candidate_info = Vec::with_capacity(candidates.len()); + let mut num_available = 0; + let mut num_synced = 0; + for candidate in candidates.iter() { - let id = candidate.id; - let node = candidate.beacon_node.to_string(); - let health = candidate.health().ok(); - let info = CandidateInfo { id, node, health }; - results.push(info); + let health = candidate.health(); + + match candidate.health() { + Ok(health) => { + if self.distance_tiers.compute_distance_tier(health.health_tier.sync_distance) == SyncDistanceTier::Synced { + num_synced += 1; + } + num_available += 1; + } + Err(CandidateError::Uninitialized) => num_available += 1, + Err(_) => continue, + } + + candidate_info.push(CandidateInfo { id: candidate.id, node: candidate.beacon_node.to_string(), health: health.ok() }); } - results + + (candidate_info, num_available, num_synced) } /// Loop through ALL candidates in `self.candidates` and update their sync status. diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 08fd803822e..4761ad2cec2 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -61,18 +61,12 @@ impl BeaconNodeSyncDistanceTiers { /// Takes a given sync distance and determines its tier based on the `sync_tolerance` defined by /// the CLI. - pub fn distance_tier(&self, distance: SyncDistance) -> SyncDistanceTier { - let distance = distance.as_u64(); - // Add 1 since we are using exclusive ranges. - let synced = self.synced.as_u64() + 1; - let small = self.small.as_u64() + 1; - let medium = self.medium.as_u64() + 1; - - if (0..synced).contains(&distance) { + pub fn compute_distance_tier(&self, distance: SyncDistance) -> SyncDistanceTier { + if distance.as_u64() <= self.synced.as_u64() { SyncDistanceTier::Synced - } else if (synced..small).contains(&distance) { + } else if distance <= self.small.as_u64() { SyncDistanceTier::Small - } else if (small..medium).contains(&distance) { + } else if distance <= self.medium.as_u64() { SyncDistanceTier::Medium } else { SyncDistanceTier::Large @@ -232,7 +226,7 @@ impl BeaconNodeHealth { execution_status: ExecutionEngineHealth, sync_distance_tiers: &BeaconNodeSyncDistanceTiers, ) -> BeaconNodeHealthTier { - let sync_distance_tier = sync_distance_tiers.distance_tier(sync_distance); + let sync_distance_tier = sync_distance_tiers.compute_distance_tier(sync_distance); let health = (sync_distance_tier, optimistic_status, execution_status); match health { @@ -326,7 +320,7 @@ mod tests { let tier = health_tier.tier; let distance = health_tier.sync_distance; - let distance_tier = beacon_node_sync_distance_tiers.distance_tier(distance); + let distance_tier = beacon_node_sync_distance_tiers.compute_distance_tier(distance); // Check sync distance. if [1, 3, 5, 6].contains(&tier) { diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index 521ff49a147..a2c353a5e80 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -39,18 +39,18 @@ async fn notify( duties_service: &DutiesService, log: &Logger, ) { - let candidate_info = duties_service.beacon_nodes.get_all_candidate_info().await; - let num_available = duties_service.beacon_nodes.num_available().await; + let (candidate_info, num_available, num_synced) = duties_service.beacon_nodes.get_notifier_info().await; + let num_total = candidate_info.len(); + let num_synced_fallback = num_synced.saturating_sub(1); + set_gauge( &http_metrics::metrics::AVAILABLE_BEACON_NODES_COUNT, num_available as i64, ); - let num_synced = duties_service.beacon_nodes.num_synced().await; set_gauge( &http_metrics::metrics::SYNCED_BEACON_NODES_COUNT, num_synced as i64, ); - let num_total = duties_service.beacon_nodes.num_total().await; set_gauge( &http_metrics::metrics::TOTAL_BEACON_NODES_COUNT, num_total as i64, @@ -77,7 +77,6 @@ async fn notify( "synced" => num_synced, ) } - let num_synced_fallback = duties_service.beacon_nodes.num_synced_fallback().await; if num_synced_fallback > 0 { set_gauge(&http_metrics::metrics::ETH2_FALLBACK_CONNECTED, 1); } else { From e368397bf581da7f76b3d79a8bab855c54f466a9 Mon Sep 17 00:00:00 2001 From: Mac L Date: Mon, 5 Feb 2024 16:32:30 +1100 Subject: [PATCH 26/50] Use the more descriptive `user_index` instead of `id` --- validator_client/src/beacon_node_fallback.rs | 50 ++++++++++++-------- validator_client/src/beacon_node_health.rs | 20 ++++---- validator_client/src/http_api/mod.rs | 10 +++- validator_client/src/lib.rs | 9 ++-- validator_client/src/notifier.rs | 7 +-- 5 files changed, 57 insertions(+), 39 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 34230aafbce..f6ee1681bdb 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -147,7 +147,7 @@ pub enum CandidateError { #[derive(Debug, Clone)] pub struct CandidateInfo { - pub id: usize, + pub index: usize, pub node: String, pub health: Option, } @@ -156,7 +156,7 @@ pub struct CandidateInfo { /// for a query. #[derive(Debug)] pub struct CandidateBeaconNode { - pub id: usize, + pub index: usize, pub beacon_node: BeaconNodeHttpClient, pub health: PLRwLock>, _phantom: PhantomData, @@ -164,7 +164,7 @@ pub struct CandidateBeaconNode { impl PartialEq for CandidateBeaconNode { fn eq(&self, other: &Self) -> bool { - self.id == other.id && self.beacon_node == other.beacon_node + self.index == other.index && self.beacon_node == other.beacon_node } } @@ -189,9 +189,9 @@ impl PartialOrd for CandidateBeaconNode { impl CandidateBeaconNode { /// Instantiate a new node. - pub fn new(beacon_node: BeaconNodeHttpClient, id: usize) -> Self { + pub fn new(beacon_node: BeaconNodeHttpClient, index: usize) -> Self { Self { - id, + index, beacon_node, health: PLRwLock::new(Err(CandidateError::Uninitialized)), _phantom: PhantomData, @@ -239,7 +239,7 @@ impl CandidateBeaconNode { }; let new_health = BeaconNodeHealth::from_status( - self.id, + self.index, sync_distance, head, optimistic_status, @@ -408,7 +408,11 @@ impl BeaconNodeFallback { match candidate.health() { Ok(health) => { - if self.distance_tiers.compute_distance_tier(health.health_tier.sync_distance) == SyncDistanceTier::Synced { + if self + .distance_tiers + .compute_distance_tier(health.health_tier.sync_distance) + == SyncDistanceTier::Synced + { num_synced += 1; } num_available += 1; @@ -417,7 +421,11 @@ impl BeaconNodeFallback { Err(_) => continue, } - candidate_info.push(CandidateInfo { id: candidate.id, node: candidate.beacon_node.to_string(), health: health.ok() }); + candidate_info.push(CandidateInfo { + index: candidate.index, + node: candidate.beacon_node.to_string(), + health: health.ok(), + }); } (candidate_info, num_available, num_synced) @@ -680,12 +688,12 @@ mod tests { let optimistic_status = IsOptimistic::No; let execution_status = ExecutionEngineHealth::Healthy; - fn new_candidate(id: usize) -> CandidateBeaconNode { + fn new_candidate(index: usize) -> CandidateBeaconNode { let beacon_node = BeaconNodeHttpClient::new( - SensitiveUrl::parse(&format!("http://example_{id}.com")).unwrap(), - Timeouts::set_all(Duration::from_secs(id as u64)), + SensitiveUrl::parse(&format!("http://example_{index}.com")).unwrap(), + Timeouts::set_all(Duration::from_secs(index as u64)), ); - CandidateBeaconNode::new(beacon_node, id) + CandidateBeaconNode::new(beacon_node, index) } let candidate_1 = new_candidate(1); @@ -705,16 +713,16 @@ mod tests { let small = SyncDistanceTier::Small; // Despite `health_1` having a larger sync distance, it is inside the `synced` range which - // does not tie-break on sync distance and so will tie-break on `id` instead. + // does not tie-break on sync distance and so will tie-break on `user_index` instead. let health_1 = BeaconNodeHealth { - id: 1, + user_index: 1, head, optimistic_status, execution_status, health_tier: BeaconNodeHealthTier::new(1, Slot::new(2), synced), }; let health_2 = BeaconNodeHealth { - id: 2, + user_index: 2, head, optimistic_status, execution_status, @@ -722,16 +730,16 @@ mod tests { }; // `health_3` and `health_4` have the same health tier and sync distance so should - // tie-break on `id`. + // tie-break on `user_index`. let health_3 = BeaconNodeHealth { - id: 3, + user_index: 3, head, optimistic_status, execution_status, health_tier: BeaconNodeHealthTier::new(3, Slot::new(9), small), }; let health_4 = BeaconNodeHealth { - id: 4, + user_index: 4, head, optimistic_status, execution_status, @@ -739,16 +747,16 @@ mod tests { }; // `health_5` has a smaller sync distance and is outside the `synced` range so should be - // sorted first. Note the values of `id`. + // sorted first. Note the values of `user_index`. let health_5 = BeaconNodeHealth { - id: 6, + user_index: 6, head, optimistic_status, execution_status, health_tier: BeaconNodeHealthTier::new(4, Slot::new(9), small), }; let health_6 = BeaconNodeHealth { - id: 5, + user_index: 5, head, optimistic_status, execution_status, diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 4761ad2cec2..0c060f9e11e 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -154,10 +154,10 @@ impl BeaconNodeHealthTier { /// Beacon Node Health metrics. #[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub struct BeaconNodeHealth { - // The ID of the Beacon Node. This should correspond with its position in the `--beacon-nodes` - // list. Note that the ID field is used to tie-break nodes with the same health so that nodes - // with a lower ID are preferred. - pub id: usize, + // The index of the Beacon Node. This should correspond with its position in the + // `--beacon-nodes` list. Note that the `user_index` field is used to tie-break nodes with the + // same health so that nodes with a lower index are preferred. + pub user_index: usize, // The slot number of the head. pub head: Slot, // Whether the node is optimistically synced. @@ -173,8 +173,8 @@ impl Ord for BeaconNodeHealth { fn cmp(&self, other: &Self) -> Ordering { let ordering = self.health_tier.cmp(&other.health_tier); if ordering == Ordering::Equal { - // Tie-break node health by ID. - self.id.cmp(&other.id) + // Tie-break node health by `user_index`. + self.user_index.cmp(&other.user_index) } else { ordering } @@ -189,7 +189,7 @@ impl PartialOrd for BeaconNodeHealth { impl BeaconNodeHealth { pub fn from_status( - id: usize, + user_index: usize, sync_distance: Slot, head: Slot, optimistic_status: IsOptimistic, @@ -204,7 +204,7 @@ impl BeaconNodeHealth { ); Self { - id, + user_index, head, optimistic_status, execution_status, @@ -212,8 +212,8 @@ impl BeaconNodeHealth { } } - pub fn get_id(&self) -> usize { - self.id + pub fn get_index(&self) -> usize { + self.user_index } pub fn get_health_tier(&self) -> BeaconNodeHealthTier { diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index ddf7cce3ebc..b7e7e790ed2 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -441,11 +441,17 @@ pub fn serve( let mut result: HashMap<(usize, String), Result> = HashMap::new(); for node in &*block_filter.beacon_nodes.candidates.read().await { - result.insert((node.id, node.beacon_node.to_string()), *node.health.read()); + result.insert( + (node.index, node.beacon_node.to_string()), + *node.health.read(), + ); } if let Some(proposer_nodes) = &block_filter.proposer_nodes { for node in &*proposer_nodes.candidates.read().await { - result.insert((node.id, node.beacon_node.to_string()), *node.health.read()); + result.insert( + (node.index, node.beacon_node.to_string()), + *node.health.read(), + ); } } diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index 1a9972b44d3..fce470f22da 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -339,18 +339,21 @@ impl ProductionValidatorClient { .collect::, String>>()?; let num_nodes = beacon_nodes.len(); - + // User order of `beacon_nodes` is preserved, so `index` corresponds to the position of + // the node in `--beacon_nodes`. let candidates = beacon_nodes .into_iter() .enumerate() - .map(|(id, node)| CandidateBeaconNode::new(node, id)) + .map(|(index, node)| CandidateBeaconNode::new(node, index)) .collect(); let proposer_nodes_num = proposer_nodes.len(); + // User order of `proposer_nodes` is preserved, so `index` corresponds to the position of + // the node in `--proposer_nodes`. let proposer_candidates = proposer_nodes .into_iter() .enumerate() - .map(|(id, node)| CandidateBeaconNode::new(node, id)) + .map(|(index, node)| CandidateBeaconNode::new(node, index)) .collect(); // Set the count for beacon node fallbacks excluding the primary beacon node. diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index a2c353a5e80..e23989f68e8 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -39,7 +39,8 @@ async fn notify( duties_service: &DutiesService, log: &Logger, ) { - let (candidate_info, num_available, num_synced) = duties_service.beacon_nodes.get_notifier_info().await; + let (candidate_info, num_available, num_synced) = + duties_service.beacon_nodes.get_notifier_info().await; let num_total = candidate_info.len(); let num_synced_fallback = num_synced.saturating_sub(1); @@ -89,7 +90,7 @@ async fn notify( log, "Beacon node info"; "status" => "Connected", - "id" => info.id, + "index" => info.index, "endpoint" => info.node, "head_slot" => %health.head, "is_optimistic" => ?health.optimistic_status, @@ -101,7 +102,7 @@ async fn notify( log, "Beacon node info"; "status" => "Disconnected", - "id" => info.id, + "index" => info.index, "endpoint" => info.node, ); } From ee75a72b6b121375cfb77ffd1fbfd893035e020c Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 6 Feb 2024 19:13:19 +1100 Subject: [PATCH 27/50] Combine sync distance tolerance flags into one --- lighthouse/tests/validator_client.rs | 44 +++----- validator_client/src/beacon_node_fallback.rs | 21 ++-- validator_client/src/beacon_node_health.rs | 113 +++++++++---------- validator_client/src/cli.rs | 47 ++++---- validator_client/src/config.rs | 34 ++---- validator_client/src/lib.rs | 1 + 6 files changed, 118 insertions(+), 142 deletions(-) diff --git a/lighthouse/tests/validator_client.rs b/lighthouse/tests/validator_client.rs index 98c34b3a9b8..5087acf4507 100644 --- a/lighthouse/tests/validator_client.rs +++ b/lighthouse/tests/validator_client.rs @@ -1,4 +1,4 @@ -use validator_client::{ApiTopic, Config}; +use validator_client::{ApiTopic, BeaconNodeSyncDistanceTiers, Config}; use crate::exec::CommandLineTestExec; use bls::{Keypair, PublicKeyBytes}; @@ -10,7 +10,7 @@ use std::process::Command; use std::str::FromStr; use std::string::ToString; use tempfile::TempDir; -use types::Address; +use types::{Address, Slot}; /// Returns the `lighthouse validator_client` command. fn base_cmd() -> Command { @@ -576,37 +576,29 @@ fn broadcast_flag() { }); } -/// Tests for validator fallback parameter flags. +/// Tests for validator fallback flags. #[test] -fn beacon_node_sync_tolerance_flag() { - CommandLineTest::new() - .flag("beacon-node-sync-tolerance", Some("4")) - .run() - .with_config(|config| { - assert_eq!(config.beacon_node_fallback.sync_tolerance, Some(4)); - }); -} -#[test] -fn beacon_node_small_sync_distance_modifier_flag() { - CommandLineTest::new() - .flag("beacon-node-small-sync-distance-modifier", Some("16")) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_node_fallback.small_sync_distance_modifier, - Some(16) - ); - }); +fn beacon_nodes_sync_tolerances_flag_default() { + CommandLineTest::new().run().with_config(|config| { + assert_eq!( + config.beacon_node_fallback.sync_tolerances, + BeaconNodeSyncDistanceTiers::default() + ) + }); } #[test] -fn beacon_node_medium_sync_distance_modifier_flag() { +fn beacon_nodes_sync_tolerances_flag() { CommandLineTest::new() - .flag("beacon-node-medium-sync-distance-modifier", Some("32")) + .flag("beacon-nodes-sync-tolerances", Some("4,4,4")) .run() .with_config(|config| { assert_eq!( - config.beacon_node_fallback.medium_sync_distance_modifier, - Some(32) + config.beacon_node_fallback.sync_tolerances, + BeaconNodeSyncDistanceTiers { + synced: Slot::new(4), + small: Slot::new(8), + medium: Slot::new(12), + } ); }); } diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index f6ee1681bdb..8e22c80c981 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -47,15 +47,16 @@ const FUTURE_SLOT_TOLERANCE: Slot = Slot::new(1); pub struct Config { /// Disables publishing http api requests to all beacon nodes for select api calls. pub disable_run_on_all: bool, - /// Sets the number of slots behind the head a beacon node is allowed to be to still be - /// considered `synced`. - pub sync_tolerance: Option, - /// Sets the size of the range of the `small` sync distance tier. This range starts immediately - /// after `sync_tolerance`. - pub small_sync_distance_modifier: Option, - /// Sets the size of the range of the `medium` sync distance tier. This range starts immediately - /// after the `small` range. - pub medium_sync_distance_modifier: Option, + pub sync_tolerances: BeaconNodeSyncDistanceTiers, + // Sets the number of slots behind the head a beacon node is allowed to be to still be + // considered `synced`. + //pub sync_tolerance: Option, + // Sets the size of the range of the `small` sync distance tier. This range starts immediately + // after `sync_tolerance`. + //pub small_sync_distance_modifier: Option, + // Sets the size of the range of the `medium` sync distance tier. This range starts immediately + // after the `small` range. + //pub medium_sync_distance_modifier: Option, } /// Indicates a measurement of latency between the VC and a BN. @@ -358,7 +359,7 @@ impl BeaconNodeFallback { spec: ChainSpec, log: Logger, ) -> Self { - let distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); + let distance_tiers = config.sync_tolerances; Self { candidates: Arc::new(RwLock::new(candidates)), distance_tiers, diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 0c060f9e11e..d2e515de16e 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -1,7 +1,8 @@ -use crate::beacon_node_fallback::Config; +use itertools::Itertools; use serde::{Deserialize, Serialize}; use std::cmp::Ordering; use std::fmt::{Debug, Display, Formatter}; +use std::str::FromStr; use types::Slot; /// Sync distances between 0 and DEFAULT_SYNC_TOLERANCE are considered `synced`. @@ -30,35 +31,49 @@ pub enum SyncDistanceTier { /// Contains the different sync distance tiers which are determined at runtime by the /// `sync_tolerance` CLI flag and the `sync_distance_modifier` flags. -#[derive(Clone, Debug)] +#[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub struct BeaconNodeSyncDistanceTiers { - synced: SyncDistance, - small: SyncDistance, - medium: SyncDistance, + pub synced: SyncDistance, + pub small: SyncDistance, + pub medium: SyncDistance, } -impl BeaconNodeSyncDistanceTiers { - pub fn from_config(config: &Config) -> Self { - let synced = config - .sync_tolerance - .map(Slot::new) - .unwrap_or(DEFAULT_SYNC_TOLERANCE); - let small_mod = config - .small_sync_distance_modifier - .map(Slot::new) - .unwrap_or(DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER); - let medium_mod = config - .medium_sync_distance_modifier - .map(Slot::new) - .unwrap_or(DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER); - +impl Default for BeaconNodeSyncDistanceTiers { + fn default() -> Self { Self { - synced, - small: synced + small_mod, - medium: synced + small_mod + medium_mod, + synced: DEFAULT_SYNC_TOLERANCE, + small: DEFAULT_SYNC_TOLERANCE + DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER, + medium: DEFAULT_SYNC_TOLERANCE + + DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER + + DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER, } } +} + +impl FromStr for BeaconNodeSyncDistanceTiers { + type Err = String; + + fn from_str(s: &str) -> Result { + let values: (u64, u64, u64) = s + .split(',') + .map(|s| { + s.parse() + .map_err(|e| format!("Invalid sync distance modifier: {e:?}")) + }) + .collect::, _>>()? + .into_iter() + .collect_tuple() + .ok_or("Invalid number of sync distance modifiers".to_string())?; + + Ok(BeaconNodeSyncDistanceTiers { + synced: Slot::new(values.0), + small: Slot::new(values.0 + values.1), + medium: Slot::new(values.0 + values.1 + values.2), + }) + } +} +impl BeaconNodeSyncDistanceTiers { /// Takes a given sync distance and determines its tier based on the `sync_tolerance` defined by /// the CLI. pub fn compute_distance_tier(&self, distance: SyncDistance) -> SyncDistanceTier { @@ -74,18 +89,6 @@ impl BeaconNodeSyncDistanceTiers { } } -impl Default for BeaconNodeSyncDistanceTiers { - fn default() -> Self { - Self { - synced: DEFAULT_SYNC_TOLERANCE, - small: DEFAULT_SYNC_TOLERANCE + DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER, - medium: DEFAULT_SYNC_TOLERANCE - + DEFAULT_SMALL_SYNC_DISTANCE_MODIFIER - + DEFAULT_MEDIUM_SYNC_DISTANCE_MODIFIER, - } - } -} - /// Execution Node health metrics. /// /// Currently only considers `el_offline`. @@ -290,12 +293,13 @@ mod tests { SyncDistanceTier, }; use crate::beacon_node_fallback::Config; + use std::str::FromStr; use types::Slot; #[test] fn all_possible_health_tiers() { let config = Config::default(); - let beacon_node_sync_distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); + let beacon_node_sync_distance_tiers = config.sync_tolerances; let mut health_vec = vec![]; @@ -375,29 +379,24 @@ mod tests { let medium_high = new_distance_tier(64, &distance_tiers); let large = new_distance_tier(65, &distance_tiers); - assert!(synced_low.tier == 1); - assert!(synced_high.tier == 1); - assert!(small_low.tier == 2); - assert!(small_high.tier == 2); - assert!(medium_low.tier == 4); + assert_eq!(synced_low.tier, 1); + assert_eq!(synced_high.tier, 1); + assert_eq!(small_low.tier, 2); + assert_eq!(small_high.tier, 2); + assert_eq!(medium_low.tier, 4); assert_eq!(medium_high.tier, 4); - assert!(large.tier == 10); + assert_eq!(large.tier, 10); } #[test] - fn sync_tolerance_from_config() { - // Config should set the tiers as: + fn sync_tolerance_from_str() { + // String should set the tiers as: // synced: 0..=4 // small: 5..=8 // medium 9..=12 // large: 13.. - let config = Config { - disable_run_on_all: false, - sync_tolerance: Some(4), - small_sync_distance_modifier: Some(4), - medium_sync_distance_modifier: Some(4), - }; - let distance_tiers = BeaconNodeSyncDistanceTiers::from_config(&config); + + let distance_tiers = BeaconNodeSyncDistanceTiers::from_str("4,4,4").unwrap(); let synced_low = new_distance_tier(0, &distance_tiers); let synced_high = new_distance_tier(4, &distance_tiers); @@ -410,12 +409,12 @@ mod tests { let large = new_distance_tier(13, &distance_tiers); - assert!(synced_low.tier == 1); - assert!(synced_high.tier == 1); - assert!(small_low.tier == 2); - assert!(small_high.tier == 2); - assert!(medium_low.tier == 4); + assert_eq!(synced_low.tier, 1); + assert_eq!(synced_high.tier, 1); + assert_eq!(small_low.tier, 2); + assert_eq!(small_high.tier, 2); + assert_eq!(medium_low.tier, 4); assert_eq!(medium_high.tier, 4); - assert!(large.tier == 10); + assert_eq!(large.tier, 10); } } diff --git a/validator_client/src/cli.rs b/validator_client/src/cli.rs index 28dce1c6dfa..ee939de31f3 100644 --- a/validator_client/src/cli.rs +++ b/validator_client/src/cli.rs @@ -368,29 +368,28 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { .takes_value(false), ) .arg( - Arg::with_name("beacon-node-sync-tolerance") - .long("beacon-node-sync-tolerance") - .help("Sets the number of slots behind the head that each connected Beacon Node can be \ - to still be considered synced. Effectively this gives more priority to the first \ - connected Beacon Node.") - .takes_value(true), - ) - .arg( - Arg::with_name("beacon-node-small-sync-distance-modifier") - .long("beacon-node-small-sync-distance-modifier") - .help("Only use this if you know what you are doing. Incorrectly setting this value \ - can result in suboptimal fallback behaviour. Sets the size (in slots) of the \ - `small` sync distance range when calculating the health tiers of connected \ - Beacon Nodes. The range falls immediately after the end of the `synced` range.") - .takes_value(true), - ) - .arg( - Arg::with_name("beacon-node-medium-sync-distance-modifier") - .long("beacon-node-medium-sync-distance-modifier") - .help("Only use this if you know what you are doing. Incorrectly setting this value \ - can result in suboptimal fallback behaviour. Sets the size (in slots) of the \ - `medium` sync distance range when calculating the health tiers of connected \ - Beacon Nodes. The range falls immediately after the end of the `small` range.") - .takes_value(true), + Arg::with_name("beacon-nodes-sync-tolerances") + .long("beacon-nodes-sync-tolerances") + .value_name("SYNC_TOLERANCES") + .help("A comma-separated list of 3 values which sets the size of each sync distance range when \ + determining the health of each connected beacon node. \ + The first value determines the `Synced` range. \ + If a connected beacon node is synced to within this number of slots it is considered 'Synced'. \ + The second value determines the `Small` sync distance range. \ + This range starts immediately after the `Synced` range. \ + The third value determines the `Medium` sync distance range. \ + This range starts immediately after the `Small` range. \ + Any sync distance value beyond that is considered `Large`. \ + For example, a value of `8,8,48` would have ranges like the following: \ + `Synced`: 0..=8 \ + `Small`: 9..=16 \ + `Medium`: 17..=64 \ + `Large`: 65.. \ + These values are used to determine what ordering beacon node fallbacks are used in. \ + Generally, `Synced` nodes are preferred over `Small` and so on. \ + Nodes in the `Synced` range will tie-break based on their ordering in `--beacon-nodes`. \ + This ensures the primary beacon node is prioritised. \ + [default: 8,8,48]") + .takes_value(true) ) } diff --git a/validator_client/src/config.rs b/validator_client/src/config.rs index a89c035c446..b79837f9c9f 100644 --- a/validator_client/src/config.rs +++ b/validator_client/src/config.rs @@ -1,6 +1,8 @@ use crate::beacon_node_fallback::ApiTopic; use crate::graffiti_file::GraffitiFile; -use crate::{beacon_node_fallback, http_api, http_metrics}; +use crate::{ + beacon_node_fallback, beacon_node_health::BeaconNodeSyncDistanceTiers, http_api, http_metrics, +}; use clap::ArgMatches; use clap_utils::{flags::DISABLE_MALLOC_TUNING_FLAG, parse_optional, parse_required}; use directory::{ @@ -14,6 +16,7 @@ use slog::{info, warn, Logger}; use std::fs; use std::net::IpAddr; use std::path::PathBuf; +use std::str::FromStr; use types::{Address, GRAFFITI_BYTES_LEN}; pub const DEFAULT_BEACON_NODE: &str = "http://localhost:5052/"; @@ -254,30 +257,11 @@ impl Config { config.beacon_node_fallback.disable_run_on_all = cli_args.is_present("disable-run-on-all"); - if let Some(sync_tolerance) = cli_args.value_of("beacon-node-sync-tolerance") { - config.beacon_node_fallback.sync_tolerance = Some( - sync_tolerance - .parse::() - .map_err(|_| "beacon-node-sync-tolerance is not a valid u64.")?, - ); - } - - if let Some(small_modifier) = cli_args.value_of("beacon-node-small-sync-distance-modifier") - { - config.beacon_node_fallback.small_sync_distance_modifier = Some( - small_modifier - .parse::() - .map_err(|_| "beacon-node-small-sync-distance-modifier is not a valid u64.")?, - ); - } - - if let Some(medium_modifier) = - cli_args.value_of("beacon-node-medium-sync-distance-modifier") - { - config.beacon_node_fallback.medium_sync_distance_modifier = - Some(medium_modifier.parse::().map_err(|_| { - "beacon-node-medium-sync-distance-modifier is not a valid u64." - })?); + if let Some(sync_tolerance) = cli_args.value_of("beacon-nodes-sync-tolerances") { + config.beacon_node_fallback.sync_tolerances = + BeaconNodeSyncDistanceTiers::from_str(sync_tolerance)?; + } else { + config.beacon_node_fallback.sync_tolerances = BeaconNodeSyncDistanceTiers::default(); } /* diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index fce470f22da..31046c10c62 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -21,6 +21,7 @@ pub mod initialized_validators; pub mod validator_store; pub use beacon_node_fallback::ApiTopic; +pub use beacon_node_health::BeaconNodeSyncDistanceTiers; pub use cli::cli_app; pub use config::Config; use initialized_validators::InitializedValidators; From 4c5dd6716f7be9b3bcb2baf95e74a65f2586a900 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 26 Apr 2024 12:46:52 +1000 Subject: [PATCH 28/50] wip --- lighthouse/tests/validator_client.rs | 4 +++- testing/simulator/src/cli.rs | 28 ++++++++++++++++------------ 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/lighthouse/tests/validator_client.rs b/lighthouse/tests/validator_client.rs index c785dc24fce..b85af5d2a86 100644 --- a/lighthouse/tests/validator_client.rs +++ b/lighthouse/tests/validator_client.rs @@ -1,4 +1,6 @@ -use validator_client::{config::DEFAULT_WEB3SIGNER_KEEP_ALIVE, ApiTopic, BeaconNodeSyncDistanceTiers, Config}; +use validator_client::{ + config::DEFAULT_WEB3SIGNER_KEEP_ALIVE, ApiTopic, BeaconNodeSyncDistanceTiers, Config, +}; use crate::exec::CommandLineTestExec; use bls::{Keypair, PublicKeyBytes}; diff --git a/testing/simulator/src/cli.rs b/testing/simulator/src/cli.rs index 27298153698..0584f151377 100644 --- a/testing/simulator/src/cli.rs +++ b/testing/simulator/src/cli.rs @@ -120,18 +120,22 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { .short("c") .long("continue_after_checks") .takes_value(false) - .help("Number of validators per client")) - .arg(Arg::with_name("speed_up_factor") - .short("s") - .long("speed_up_factor") - .takes_value(true) - .default_value("3") - .help("Speed up factor. Please use a divisor of 12.")) - .arg(Arg::with_name("continue_after_checks") - .short("c") - .long("continue_after_checks") - .takes_value(false) - .help("Continue after checks (default false)") + .help("Number of validators per client"), + ) + .arg( + Arg::with_name("speed_up_factor") + .short("s") + .long("speed_up_factor") + .takes_value(true) + .default_value("3") + .help("Speed up factor. Please use a divisor of 12."), + ) + .arg( + Arg::with_name("continue_after_checks") + .short("c") + .long("continue_after_checks") + .takes_value(false) + .help("Continue after checks (default false)") .help("Continue after checks (default false)"), ), ) From 85443e8b1900a08376fb3670473b2918e7e92bba Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 26 Apr 2024 13:07:22 +1000 Subject: [PATCH 29/50] Use new simulator from unstable --- testing/simulator/src/basic_sim.rs | 1 - testing/simulator/src/checks.rs | 1 - testing/simulator/src/cli.rs | 16 ----- testing/simulator/src/common.rs | 100 -------------------------- testing/simulator/src/fallback_sim.rs | 2 +- testing/simulator/src/main.rs | 15 ---- 6 files changed, 1 insertion(+), 134 deletions(-) delete mode 100644 testing/simulator/src/common.rs diff --git a/testing/simulator/src/basic_sim.rs b/testing/simulator/src/basic_sim.rs index 8ac3270b271..755bb71b430 100644 --- a/testing/simulator/src/basic_sim.rs +++ b/testing/simulator/src/basic_sim.rs @@ -3,7 +3,6 @@ use crate::local_network::TERMINAL_BLOCK; use crate::{checks, LocalNetwork}; use clap::ArgMatches; -use crate::common::{create_local_network, LocalNetworkParams}; use crate::retry::with_retry; use futures::prelude::*; use node_test_rig::{ diff --git a/testing/simulator/src/checks.rs b/testing/simulator/src/checks.rs index fde09755958..03cc17fab3e 100644 --- a/testing/simulator/src/checks.rs +++ b/testing/simulator/src/checks.rs @@ -1,5 +1,4 @@ use crate::local_network::LocalNetwork; -use crate::ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE; use node_test_rig::eth2::types::{BlockId, FinalityCheckpointsData, StateId}; use std::time::Duration; use types::{Epoch, EthSpec, ExecPayload, ExecutionBlockHash, Slot, Unsigned}; diff --git a/testing/simulator/src/cli.rs b/testing/simulator/src/cli.rs index 0584f151377..00af7e560ce 100644 --- a/testing/simulator/src/cli.rs +++ b/testing/simulator/src/cli.rs @@ -120,22 +120,6 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> { .short("c") .long("continue_after_checks") .takes_value(false) - .help("Number of validators per client"), - ) - .arg( - Arg::with_name("speed_up_factor") - .short("s") - .long("speed_up_factor") - .takes_value(true) - .default_value("3") - .help("Speed up factor. Please use a divisor of 12."), - ) - .arg( - Arg::with_name("continue_after_checks") - .short("c") - .long("continue_after_checks") - .takes_value(false) - .help("Continue after checks (default false)") .help("Continue after checks (default false)"), ), ) diff --git a/testing/simulator/src/common.rs b/testing/simulator/src/common.rs deleted file mode 100644 index 318356f47e2..00000000000 --- a/testing/simulator/src/common.rs +++ /dev/null @@ -1,100 +0,0 @@ -use crate::local_network::EXECUTION_PORT; -use crate::LocalNetwork; -use eth1::{Eth1Endpoint, DEFAULT_CHAIN_ID}; -use eth1_test_rig::AnvilEth1Instance; - -use execution_layer::http::deposit_methods::Eth1Id; -use node_test_rig::environment::RuntimeContext; -use node_test_rig::{testing_client_config, ClientConfig, ClientGenesis}; -use sensitive_url::SensitiveUrl; -use std::net::Ipv4Addr; -use std::time::Duration; -use types::EthSpec; - -pub struct LocalNetworkParams { - pub eth1_block_time: Duration, - pub total_validator_count: usize, - pub deposit_amount: u64, - pub node_count: usize, - pub proposer_nodes: usize, - pub post_merge_sim: bool, -} - -pub async fn create_local_network( - LocalNetworkParams { - eth1_block_time, - total_validator_count, - deposit_amount, - node_count, - proposer_nodes, - post_merge_sim, - }: LocalNetworkParams, - context: RuntimeContext, -) -> Result<(LocalNetwork, ClientConfig), String> { - /* - * Deploy the deposit contract, spawn tasks to keep creating new blocks and deposit - * validators. - */ - let anvil_eth1_instance = AnvilEth1Instance::new(DEFAULT_CHAIN_ID.into()).await?; - let deposit_contract = anvil_eth1_instance.deposit_contract; - let chain_id = anvil_eth1_instance.anvil.chain_id(); - let anvil = anvil_eth1_instance.anvil; - let eth1_endpoint = - SensitiveUrl::parse(anvil.endpoint().as_str()).expect("Unable to parse anvil endpoint."); - let deposit_contract_address = deposit_contract.address(); - - // Start a timer that produces eth1 blocks on an interval. - tokio::spawn(async move { - let mut interval = tokio::time::interval(eth1_block_time); - loop { - interval.tick().await; - let _ = anvil.evm_mine().await; - } - }); - - // Submit deposits to the deposit contract. - tokio::spawn(async move { - for i in 0..total_validator_count { - println!("Submitting deposit for validator {}...", i); - let _ = deposit_contract - .deposit_deterministic_async::(i, deposit_amount) - .await; - } - }); - - let mut beacon_config = testing_client_config(); - - beacon_config.genesis = ClientGenesis::DepositContract; - beacon_config.eth1.endpoint = Eth1Endpoint::NoAuth(eth1_endpoint); - beacon_config.eth1.deposit_contract_address = deposit_contract_address; - beacon_config.eth1.deposit_contract_deploy_block = 0; - beacon_config.eth1.lowest_cached_block_number = 0; - beacon_config.eth1.follow_distance = 1; - beacon_config.eth1.node_far_behind_seconds = 20; - beacon_config.dummy_eth1_backend = false; - beacon_config.sync_eth1_chain = true; - beacon_config.eth1.auto_update_interval_millis = eth1_block_time.as_millis() as u64; - beacon_config.eth1.chain_id = Eth1Id::from(chain_id); - beacon_config.network.target_peers = node_count + proposer_nodes - 1; - - beacon_config.network.enr_address = (Some(Ipv4Addr::LOCALHOST), None); - beacon_config.network.enable_light_client_server = true; - beacon_config.chain.enable_light_client_server = true; - beacon_config.http_api.enable_light_client_server = true; - - if post_merge_sim { - let el_config = execution_layer::Config { - execution_endpoints: vec![SensitiveUrl::parse(&format!( - "http://localhost:{}", - EXECUTION_PORT - )) - .unwrap()], - ..Default::default() - }; - - beacon_config.execution_layer = Some(el_config); - } - - let network = LocalNetwork::new(context, beacon_config.clone()).await?; - Ok((network, beacon_config)) -} diff --git a/testing/simulator/src/fallback_sim.rs b/testing/simulator/src/fallback_sim.rs index c9deeba04d9..01f7c8418e4 100644 --- a/testing/simulator/src/fallback_sim.rs +++ b/testing/simulator/src/fallback_sim.rs @@ -28,7 +28,7 @@ const DENEB_FORK_EPOCH: u64 = 2; // This has potential to block CI so it should be set conservatively enough that spurious failures // don't become very common, but not so conservatively that regressions to the fallback mechanism // cannot be detected. -const ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE: f64 = 85.0; +const ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE: f64 = 95.0; const SUGGESTED_FEE_RECIPIENT: [u8; 20] = [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1]; diff --git a/testing/simulator/src/main.rs b/testing/simulator/src/main.rs index 53f3de916c5..d1a2d0dc672 100644 --- a/testing/simulator/src/main.rs +++ b/testing/simulator/src/main.rs @@ -26,14 +26,6 @@ use env_logger::{Builder, Env}; use local_network::LocalNetwork; use types::MinimalEthSpec; -// Since simulator tests are non-deterministic and there is a non-zero chance of missed -// attestations, define an acceptable network-wide attestation performance. -// -// This has potential to block CI so it should be set conservatively enough that spurious failures -// don't become very common, but not so conservatively that regressions to the fallback mechanism -// cannot be detected. -pub(crate) const ACCEPTABLE_FALLBACK_ATTESTATION_HIT_PERCENTAGE: f64 = 95.0; - pub type E = MinimalEthSpec; fn main() { @@ -56,13 +48,6 @@ fn main() { std::process::exit(1) } }, - ("fallback-sim", Some(matches)) => match fallback_sim::run_fallback_sim(matches) { - Ok(()) => println!("Simulation exited successfully"), - Err(e) => { - eprintln!("Simulation exited with an error: {}", e); - std::process::exit(1) - } - }, _ => { eprintln!("Invalid subcommand. Use --help to see available options"); std::process::exit(1) From 037448b0b0a257f9901cc19e2439ce0c7458391b Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 26 Apr 2024 14:29:52 +1000 Subject: [PATCH 30/50] Fix cli text --- book/src/help_vc.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/book/src/help_vc.md b/book/src/help_vc.md index 1b7e7f2b0af..ede8e5d9b23 100644 --- a/book/src/help_vc.md +++ b/book/src/help_vc.md @@ -83,6 +83,17 @@ OPTIONS: --beacon-nodes Comma-separated addresses to one or more beacon node HTTP APIs. Default is http://localhost:5052. + --beacon-nodes-sync-tolerances + A comma-separated list of 3 values which sets the size of each sync distance range when determining the + health of each connected beacon node. The first value determines the `Synced` range. If a connected beacon + node is synced to within this number of slots it is considered 'Synced'. The second value determines the + `Small` sync distance range. This range starts immediately after the `Synced` range. The third value + determines the `Medium` sync distance range. This range starts immediately after the `Small` range. Any sync + distance value beyond that is considered `Large`. For example, a value of `8,8,48` would have ranges like + the following: `Synced`: 0..=8 `Small`: 9..=16 `Medium`: 17..=64 `Large`: 65.. These values are used to + determine what ordering beacon node fallbacks are used in. Generally, `Synced` nodes are preferred over + `Small` and so on. Nodes in the `Synced` range will tie-break based on their ordering in `--beacon-nodes`. + This ensures the primary beacon node is prioritised. [default: 8,8,48] --beacon-nodes-tls-certs Comma-separated paths to custom TLS certificates to use when connecting to a beacon node (and/or proposer node). These certificates must be in PEM format and are used in addition to the OS trust store. Commas must From b72d2191a876c12b3fbca9c0a2bfd1167811fd34 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 26 Apr 2024 14:33:50 +1000 Subject: [PATCH 31/50] Remove leftover files --- .github/workflows/test-suite.yml | 53 -- .github/workflows/test-suite.yml.orig | 475 ------------ .../tests/validator_client_BACKUP_4094175.rs | 705 ------------------ .../tests/validator_client_BASE_4094175.rs | 655 ---------------- .../tests/validator_client_LOCAL_4094175.rs | 681 ----------------- .../tests/validator_client_REMOTE_4094175.rs | 679 ----------------- 6 files changed, 3248 deletions(-) delete mode 100644 .github/workflows/test-suite.yml.orig delete mode 100644 lighthouse/tests/validator_client_BACKUP_4094175.rs delete mode 100644 lighthouse/tests/validator_client_BASE_4094175.rs delete mode 100644 lighthouse/tests/validator_client_LOCAL_4094175.rs delete mode 100644 lighthouse/tests/validator_client_REMOTE_4094175.rs diff --git a/.github/workflows/test-suite.yml b/.github/workflows/test-suite.yml index 98ef2d38d45..413dd2b95dd 100644 --- a/.github/workflows/test-suite.yml +++ b/.github/workflows/test-suite.yml @@ -247,61 +247,8 @@ jobs: with: channel: stable cache-target: release -<<<<<<< HEAD - - name: Install Foundry (anvil) - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Run the beacon chain sim and go through the merge transition - run: cargo run --release --bin simulator eth1-sim --post-merge - no-eth1-simulator-ubuntu: - name: no-eth1-simulator-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Run the beacon chain sim without an eth1 connection - run: cargo run --release --bin simulator no-eth1-sim - fallback-simulator-ubuntu: - name: fallback-simulator-ubuntu - runs-on: ubuntu-latest - needs: cargo-fmt - steps: - - uses: actions/checkout@v3 - - name: Get latest version of stable Rust - run: rustup update stable - - name: Install Protoc - uses: arduino/setup-protoc@e52d9eb8f7b63115df1ac544a1376fdbf5a39612 - with: - repo-token: ${{ secrets.GITHUB_TOKEN }} - - name: Install Foundry (anvil) - uses: foundry-rs/foundry-toolchain@v1 - - name: Run the fallback simulator - run: cargo run --release --bin simulator fallback-sim - syncing-simulator-ubuntu: - name: syncing-simulator-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Install Foundry (anvil) - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Run the syncing simulator - run: cargo run --release --bin simulator syncing-sim -======= - name: Run a beacon chain sim which tests VC fallback behaviour run: cargo run --release --bin simulator fallback-sim ->>>>>>> unstable doppelganger-protection-test: name: doppelganger-protection-test runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "small"]') || 'ubuntu-latest' }} diff --git a/.github/workflows/test-suite.yml.orig b/.github/workflows/test-suite.yml.orig deleted file mode 100644 index 98ef2d38d45..00000000000 --- a/.github/workflows/test-suite.yml.orig +++ /dev/null @@ -1,475 +0,0 @@ -name: test-suite - -on: - push: - branches: - - stable - - staging - - trying - - 'pr/*' - pull_request: - merge_group: - -concurrency: - group: ${{ github.workflow }}-${{ github.ref }} - cancel-in-progress: true - -env: - # Deny warnings in CI - # Disable debug info (see https://github.com/sigp/lighthouse/issues/4005) - RUSTFLAGS: "-D warnings -C debuginfo=0" - # Prevent Github API rate limiting. - LIGHTHOUSE_GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - # Enable self-hosted runners for the sigp repo only. - SELF_HOSTED_RUNNERS: ${{ github.repository == 'sigp/lighthouse' }} - # Self-hosted runners need to reference a different host for `./watch` tests. - WATCH_HOST: ${{ github.repository == 'sigp/lighthouse' && 'host.docker.internal' || 'localhost' }} - # Disable incremental compilation - CARGO_INCREMENTAL: 0 - # Enable portable to prevent issues with caching `blst` for the wrong CPU type - TEST_FEATURES: portable -jobs: - target-branch-check: - name: target-branch-check - runs-on: ubuntu-latest - if: github.event_name == 'pull_request' || github.event_name == 'merge_group' - steps: - - name: Check that the pull request is not targeting the stable branch - run: test ${{ github.base_ref }} != "stable" - release-tests-ubuntu: - name: release-tests-ubuntu - # Use self-hosted runners only on the sigp repo. - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "large"]') || 'ubuntu-latest' }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - name: Install Foundry (anvil) - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Run tests in release - run: make nextest-release - - name: Show cache stats - if: env.SELF_HOSTED_RUNNERS == 'true' - run: sccache --show-stats - release-tests-windows: - name: release-tests-windows - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "windows", "CI"]') || 'windows-2019' }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - name: Install Foundry (anvil) - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Install make - if: env.SELF_HOSTED_RUNNERS == 'false' - run: choco install -y make -# - uses: KyleMayes/install-llvm-action@v1 -# if: env.SELF_HOSTED_RUNNERS == 'false' -# with: -# version: "16.0" -# directory: ${{ runner.temp }}/llvm - - name: Set LIBCLANG_PATH - run: echo "LIBCLANG_PATH=$((gcm clang).source -replace "clang.exe")" >> $env:GITHUB_ENV - - name: Run tests in release - run: make nextest-release - - name: Show cache stats - if: env.SELF_HOSTED_RUNNERS == 'true' - run: sccache --show-stats - beacon-chain-tests: - name: beacon-chain-tests - # Use self-hosted runners only on the sigp repo. - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "large"]') || 'ubuntu-latest' }} - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - - name: Run beacon_chain tests for all known forks - run: make test-beacon-chain - - name: Show cache stats - if: env.SELF_HOSTED_RUNNERS == 'true' - run: sccache --show-stats - op-pool-tests: - name: op-pool-tests - runs-on: ubuntu-latest - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - - name: Run operation_pool tests for all known forks - run: make test-op-pool - network-tests: - name: network-tests - runs-on: ubuntu-latest - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - - name: Run network tests for all known forks - run: make test-network - slasher-tests: - name: slasher-tests - runs-on: ubuntu-latest - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - - name: Run slasher tests for all supported backends - run: make test-slasher - debug-tests-ubuntu: - name: debug-tests-ubuntu - # Use self-hosted runners only on the sigp repo. - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "large"]') || 'ubuntu-latest' }} - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - bins: cargo-nextest - - name: Install Foundry (anvil) - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Run tests in debug - run: make nextest-debug - - name: Show cache stats - if: env.SELF_HOSTED_RUNNERS == 'true' - run: sccache --show-stats - state-transition-vectors-ubuntu: - name: state-transition-vectors-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Run state_transition_vectors in release. - run: make run-state-transition-tests - ef-tests-ubuntu: - name: ef-tests-ubuntu - # Use self-hosted runners only on the sigp repo. - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "small"]') || 'ubuntu-latest' }} - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - bins: cargo-nextest - - name: Run consensus-spec-tests with blst and fake_crypto - run: make nextest-ef - - name: Show cache stats - if: env.SELF_HOSTED_RUNNERS == 'true' - run: sccache --show-stats - dockerfile-ubuntu: - name: dockerfile-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Build the root Dockerfile - run: docker build --build-arg FEATURES=portable -t lighthouse:local . - - name: Test the built image - run: docker run -t lighthouse:local lighthouse --version - basic-simulator-ubuntu: - name: basic-simulator-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Run a basic beacon chain sim that starts from Bellatrix - run: cargo run --release --bin simulator basic-sim - fallback-simulator-ubuntu: - name: fallback-simulator-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release -<<<<<<< HEAD - - name: Install Foundry (anvil) - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Run the beacon chain sim and go through the merge transition - run: cargo run --release --bin simulator eth1-sim --post-merge - no-eth1-simulator-ubuntu: - name: no-eth1-simulator-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Run the beacon chain sim without an eth1 connection - run: cargo run --release --bin simulator no-eth1-sim - fallback-simulator-ubuntu: - name: fallback-simulator-ubuntu - runs-on: ubuntu-latest - needs: cargo-fmt - steps: - - uses: actions/checkout@v3 - - name: Get latest version of stable Rust - run: rustup update stable - - name: Install Protoc - uses: arduino/setup-protoc@e52d9eb8f7b63115df1ac544a1376fdbf5a39612 - with: - repo-token: ${{ secrets.GITHUB_TOKEN }} - - name: Install Foundry (anvil) - uses: foundry-rs/foundry-toolchain@v1 - - name: Run the fallback simulator - run: cargo run --release --bin simulator fallback-sim - syncing-simulator-ubuntu: - name: syncing-simulator-ubuntu - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Install Foundry (anvil) - uses: foundry-rs/foundry-toolchain@v1 - with: - version: nightly-ca67d15f4abd46394b324c50e21e66f306a1162d - - name: Run the syncing simulator - run: cargo run --release --bin simulator syncing-sim -======= - - name: Run a beacon chain sim which tests VC fallback behaviour - run: cargo run --release --bin simulator fallback-sim ->>>>>>> unstable - doppelganger-protection-test: - name: doppelganger-protection-test - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "small"]') || 'ubuntu-latest' }} - env: - # Enable portable to prevent issues with caching `blst` for the wrong CPU type - FEATURES: jemalloc,portable - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Install geth - if: env.SELF_HOSTED_RUNNERS == 'false' - run: | - sudo add-apt-repository -y ppa:ethereum/ethereum - sudo apt-get update - sudo apt-get install ethereum - - name: Install lighthouse - run: | - make - - name: Install lcli - # TODO: uncomment after the version of lcli in https://github.com/sigp/lighthouse/pull/5137 - # is installed on the runners - # if: env.SELF_HOSTED_RUNNERS == 'false' - run: make install-lcli - - name: Run the doppelganger protection failure test script - run: | - cd scripts/tests - ./doppelganger_protection.sh failure genesis.json - - name: Run the doppelganger protection success test script - run: | - cd scripts/tests - ./doppelganger_protection.sh success genesis.json - execution-engine-integration-ubuntu: - name: execution-engine-integration-ubuntu - runs-on: ${{ github.repository == 'sigp/lighthouse' && fromJson('["self-hosted", "linux", "CI", "small"]') || 'ubuntu-latest' }} - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - if: env.SELF_HOSTED_RUNNERS == 'false' - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - cache: false - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - name: Add go compiler to $PATH - if: env.SELF_HOSTED_RUNNERS == 'true' - run: echo "/usr/local/go/bin" >> $GITHUB_PATH - - name: Run exec engine integration tests in release - run: make test-exec-engine - check-code: - name: check-code - runs-on: ubuntu-latest - env: - CARGO_INCREMENTAL: 1 - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - components: rustfmt,clippy - bins: cargo-audit - - name: Check formatting with cargo fmt - run: make cargo-fmt - - name: Lint code for quality and style with Clippy - run: make lint - - name: Certify Cargo.lock freshness - run: git diff --exit-code Cargo.lock - - name: Typecheck benchmark code without running it - run: make check-benches - - name: Validate state_processing feature arbitrary-fuzz - run: make arbitrary-fuzz - - name: Run cargo audit - run: make audit-CI - - name: Run cargo vendor to make sure dependencies can be vendored for packaging, reproducibility and archival purpose - run: CARGO_HOME=$(readlink -f $HOME) make vendor - check-msrv: - name: check-msrv - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Install Rust at Minimum Supported Rust Version (MSRV) - run: | - metadata=$(cargo metadata --no-deps --format-version 1) - msrv=$(echo $metadata | jq -r '.packages | map(select(.name == "lighthouse")) | .[0].rust_version') - rustup override set $msrv - - name: Run cargo check - run: cargo check --workspace - cargo-udeps: - name: cargo-udeps - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of nightly Rust - uses: moonrepo/setup-rust@v1 - with: - channel: nightly - bins: cargo-udeps - cache: false - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - name: Create Cargo config dir - run: mkdir -p .cargo - - name: Install custom Cargo config - run: cp -f .github/custom/config.toml .cargo/config.toml - - name: Run cargo udeps to identify unused crates in the dependency graph - run: make udeps - env: - # Allow warnings on Nightly - RUSTFLAGS: "" - compile-with-beta-compiler: - name: compile-with-beta-compiler - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Install dependencies - run: sudo apt update && sudo apt install -y git gcc g++ make cmake pkg-config llvm-dev libclang-dev clang - - name: Use Rust beta - run: rustup override set beta - - name: Run make - run: make - cli-check: - name: cli-check - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - name: Get latest version of stable Rust - uses: moonrepo/setup-rust@v1 - with: - channel: stable - cache-target: release - - name: Run Makefile to trigger the bash script - run: make cli - # This job succeeds ONLY IF all others succeed. It is used by the merge queue to determine whether - # a PR is safe to merge. New jobs should be added here. - test-suite-success: - name: test-suite-success - runs-on: ubuntu-latest - needs: [ - 'target-branch-check', - 'release-tests-ubuntu', - 'release-tests-windows', - 'beacon-chain-tests', - 'op-pool-tests', - 'network-tests', - 'slasher-tests', - 'debug-tests-ubuntu', - 'state-transition-vectors-ubuntu', - 'ef-tests-ubuntu', - 'dockerfile-ubuntu', - 'basic-simulator-ubuntu', - 'fallback-simulator-ubuntu', - 'doppelganger-protection-test', - 'execution-engine-integration-ubuntu', - 'check-code', - 'check-msrv', - 'cargo-udeps', - 'compile-with-beta-compiler', - 'cli-check', - ] - steps: - - uses: actions/checkout@v4 - - name: Check that success job is dependent on all others - run: ./scripts/ci/check-success-job.sh ./.github/workflows/test-suite.yml test-suite-success diff --git a/lighthouse/tests/validator_client_BACKUP_4094175.rs b/lighthouse/tests/validator_client_BACKUP_4094175.rs deleted file mode 100644 index c785dc24fce..00000000000 --- a/lighthouse/tests/validator_client_BACKUP_4094175.rs +++ /dev/null @@ -1,705 +0,0 @@ -use validator_client::{config::DEFAULT_WEB3SIGNER_KEEP_ALIVE, ApiTopic, BeaconNodeSyncDistanceTiers, Config}; - -use crate::exec::CommandLineTestExec; -use bls::{Keypair, PublicKeyBytes}; -use std::fs::File; -use std::io::Write; -use std::net::IpAddr; -use std::path::PathBuf; -use std::process::Command; -use std::str::FromStr; -use std::string::ToString; -use std::time::Duration; -use tempfile::TempDir; -use types::{Address, Slot}; - -/// Returns the `lighthouse validator_client` command. -fn base_cmd() -> Command { - let lighthouse_bin = env!("CARGO_BIN_EXE_lighthouse"); - let path = lighthouse_bin - .parse::() - .expect("should parse CARGO_TARGET_DIR"); - - let mut cmd = Command::new(path); - cmd.arg("validator_client"); - cmd -} - -// Wrapper around `Command` for easier Command Line Testing. -struct CommandLineTest { - cmd: Command, -} -impl CommandLineTest { - fn new() -> CommandLineTest { - let base_cmd = base_cmd(); - CommandLineTest { cmd: base_cmd } - } -} - -impl CommandLineTestExec for CommandLineTest { - type Config = Config; - - fn cmd_mut(&mut self) -> &mut Command { - &mut self.cmd - } -} - -#[test] -fn datadir_flag() { - CommandLineTest::new() - .run() - .with_config_and_dir(|config, dir| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_and_secrets_dir_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validators-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_dir_alias_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validator-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn beacon_nodes_flag() { - CommandLineTest::new() - .flag( - "beacon-nodes", - Some("http://localhost:1001,https://project:secret@infura.io/"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes[0].full.to_string(), - "http://localhost:1001/" - ); - assert_eq!(config.beacon_nodes[0].to_string(), "http://localhost:1001/"); - assert_eq!( - config.beacon_nodes[1].full.to_string(), - "https://project:secret@infura.io/" - ); - assert_eq!(config.beacon_nodes[1].to_string(), "https://infura.io/"); - }); -} - -#[test] -fn disable_auto_discover_flag() { - CommandLineTest::new() - .flag("disable-auto-discover", None) - .run() - .with_config(|config| assert!(config.disable_auto_discover)); -} - -#[test] -fn init_slashing_protections_flag() { - CommandLineTest::new() - .flag("init-slashing-protection", None) - .run() - .with_config(|config| assert!(config.init_slashing_protection)); -} - -#[test] -fn use_long_timeouts_flag() { - CommandLineTest::new() - .flag("use-long-timeouts", None) - .run() - .with_config(|config| assert!(config.use_long_timeouts)); -} - -#[test] -fn beacon_nodes_tls_certs_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag( - "beacon-nodes-tls-certs", - Some( - vec![ - dir.path().join("certificate.crt").to_str().unwrap(), - dir.path().join("certificate2.crt").to_str().unwrap(), - ] - .join(",") - .as_str(), - ), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes_tls_certs, - Some(vec![ - dir.path().join("certificate.crt"), - dir.path().join("certificate2.crt") - ]) - ) - }); -} - -// Tests for Graffiti flags. -#[test] -fn graffiti_flag() { - CommandLineTest::new() - .flag("graffiti", Some("nice-graffiti")) - .run() - .with_config(|config| { - assert_eq!( - config.graffiti.unwrap().to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = "default:nice-graffiti"; - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - // Public key not present so load default. - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_with_pk_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = format!("{}:nice-graffiti", pubkeybytes.to_string()); - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} - -// Tests for suggested-fee-recipient flags. -#[test] -fn fee_recipient_flag() { - CommandLineTest::new() - .flag( - "suggested-fee-recipient", - Some("0x00000000219ab540356cbb839cbe05303d7705fa"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.fee_recipient, - Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap()) - ) - }); -} - -// Tests for HTTP flags. -#[test] -fn http_flag() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(config.http_api.enabled)); -} -#[test] -fn http_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("::1")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -#[should_panic] -fn missing_unencrypted_http_transport_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_port_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_port, 9090)); -} -#[test] -fn http_allow_origin_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_api.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn http_allow_origin_all_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_api.allow_origin, Some("*".to_string()))); -} -#[test] -fn http_allow_keystore_export_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.allow_keystore_export)); -} -#[test] -fn http_allow_keystore_export_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-keystore-export", None) - .run() - .with_config(|config| assert!(config.http_api.allow_keystore_export)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.store_passwords_in_secrets_dir)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-store-passwords-in-secrets-dir", None) - .run() - .with_config(|config| assert!(config.http_api.store_passwords_in_secrets_dir)); -} - -// Tests for Metrics flags. -#[test] -fn metrics_flag() { - CommandLineTest::new() - .flag("metrics", None) - .run() - .with_config(|config| assert!(config.http_metrics.enabled)); -} -#[test] -fn metrics_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("::1")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_port_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_port, 9090)); -} -#[test] -fn metrics_allow_origin_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_metrics.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn metrics_allow_origin_all_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allow_origin, Some("*".to_string()))); -} -#[test] -pub fn malloc_tuning_flag() { - CommandLineTest::new() - .flag("disable-malloc-tuning", None) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, false)); -} -#[test] -pub fn malloc_tuning_default() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, true)); -} -#[test] -fn doppelganger_protection_flag() { - CommandLineTest::new() - .flag("enable-doppelganger-protection", None) - .run() - .with_config(|config| assert!(config.enable_doppelganger_protection)); -} -#[test] -fn no_doppelganger_protection_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.enable_doppelganger_protection)); -} -#[test] -fn produce_block_v3_flag() { - CommandLineTest::new() - .flag("produce-block-v3", None) - .run() - .with_config(|config| assert!(config.produce_block_v3)); -} - -#[test] -fn no_produce_block_v3_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.produce_block_v3)); -} - -#[test] -fn no_gas_limit_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.gas_limit.is_none())); -} -#[test] -fn gas_limit_flag() { - CommandLineTest::new() - .flag("gas-limit", Some("600")) - .flag("builder-proposals", None) - .run() - .with_config(|config| assert_eq!(config.gas_limit, Some(600))); -} -#[test] -fn no_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.builder_proposals)); -} -#[test] -fn builder_proposals_flag() { - CommandLineTest::new() - .flag("builder-proposals", None) - .run() - .with_config(|config| assert!(config.builder_proposals)); -} -#[test] -fn builder_boost_factor_flag() { - CommandLineTest::new() - .flag("builder-boost-factor", Some("150")) - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, Some(150))); -} -#[test] -fn no_builder_boost_factor_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, None)); -} -#[test] -fn prefer_builder_proposals_flag() { - CommandLineTest::new() - .flag("prefer-builder-proposals", None) - .run() - .with_config(|config| assert!(config.prefer_builder_proposals)); -} -#[test] -fn no_prefer_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.prefer_builder_proposals)); -} -#[test] -fn no_builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.builder_registration_timestamp_override.is_none())); -} -#[test] -fn builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .flag("builder-registration-timestamp-override", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.builder_registration_timestamp_override, Some(100)) - }); -} -#[test] -fn monitoring_endpoint() { - CommandLineTest::new() - .flag("monitoring-endpoint", Some("http://example:8000")) - .flag("monitoring-endpoint-period", Some("30")) - .run() - .with_config(|config| { - let api_conf = config.monitoring_api.as_ref().unwrap(); - assert_eq!(api_conf.monitoring_endpoint.as_str(), "http://example:8000"); - assert_eq!(api_conf.update_period_secs, Some(30)); - }); -} -#[test] -fn disable_run_on_all_flag() { - CommandLineTest::new() - .flag("disable-run-on-all", None) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // --broadcast flag takes precedence - CommandLineTest::new() - .flag("disable-run-on-all", None) - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -#[test] -fn no_broadcast_flag() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Subscriptions]); - }); -} - -#[test] -fn broadcast_flag() { - // "none" variant - CommandLineTest::new() - .flag("broadcast", Some("none")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // "none" with other values is ignored - CommandLineTest::new() - .flag("broadcast", Some("none,sync-committee")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::SyncCommittee]); - }); - // Other valid variants - CommandLineTest::new() - .flag("broadcast", Some("blocks, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); - // Omitted "subscription" overrides default - CommandLineTest::new() - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -/// Tests for validator fallback flags. -#[test] -fn beacon_nodes_sync_tolerances_flag_default() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!( - config.beacon_node_fallback.sync_tolerances, - BeaconNodeSyncDistanceTiers::default() - ) - }); -} -#[test] -fn beacon_nodes_sync_tolerances_flag() { - CommandLineTest::new() - .flag("beacon-nodes-sync-tolerances", Some("4,4,4")) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_node_fallback.sync_tolerances, - BeaconNodeSyncDistanceTiers { - synced: Slot::new(4), - small: Slot::new(8), - medium: Slot::new(12), - } - ); - }); -} - -#[test] -#[should_panic(expected = "Unknown API topic")] -fn wrong_broadcast_flag() { - CommandLineTest::new() - .flag("broadcast", Some("foo, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); -} - -#[test] -fn latency_measurement_service() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", None) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("true")) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("false")) - .run() - .with_config(|config| { - assert!(!config.enable_latency_measurement_service); - }); -} - -#[test] -fn validator_registration_batch_size() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 500); - }); - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 100); - }); -} - -#[test] -#[should_panic] -fn validator_registration_batch_size_zero_value() { - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("0")) - .run(); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection_default() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_web3signer_slashing_protection); - }); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection() { - CommandLineTest::new() - .flag("disable-slashing-protection-web3signer", None) - .run() - .with_config(|config| { - assert!(!config.enable_web3signer_slashing_protection); - }); -} - -#[test] -fn validator_web3_signer_keep_alive_default() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!( - config.web3_signer_keep_alive_timeout, - DEFAULT_WEB3SIGNER_KEEP_ALIVE - ); - }); -} - -#[test] -fn validator_web3_signer_keep_alive_override() { - CommandLineTest::new() - .flag("web3-signer-keep-alive-timeout", Some("1000")) - .run() - .with_config(|config| { - assert_eq!( - config.web3_signer_keep_alive_timeout, - Some(Duration::from_secs(1)) - ); - }); -} diff --git a/lighthouse/tests/validator_client_BASE_4094175.rs b/lighthouse/tests/validator_client_BASE_4094175.rs deleted file mode 100644 index 764fd87ccdf..00000000000 --- a/lighthouse/tests/validator_client_BASE_4094175.rs +++ /dev/null @@ -1,655 +0,0 @@ -use validator_client::{ApiTopic, Config}; - -use crate::exec::CommandLineTestExec; -use bls::{Keypair, PublicKeyBytes}; -use std::fs::File; -use std::io::Write; -use std::net::IpAddr; -use std::path::PathBuf; -use std::process::Command; -use std::str::FromStr; -use std::string::ToString; -use tempfile::TempDir; -use types::Address; - -/// Returns the `lighthouse validator_client` command. -fn base_cmd() -> Command { - let lighthouse_bin = env!("CARGO_BIN_EXE_lighthouse"); - let path = lighthouse_bin - .parse::() - .expect("should parse CARGO_TARGET_DIR"); - - let mut cmd = Command::new(path); - cmd.arg("validator_client"); - cmd -} - -// Wrapper around `Command` for easier Command Line Testing. -struct CommandLineTest { - cmd: Command, -} -impl CommandLineTest { - fn new() -> CommandLineTest { - let base_cmd = base_cmd(); - CommandLineTest { cmd: base_cmd } - } -} - -impl CommandLineTestExec for CommandLineTest { - type Config = Config; - - fn cmd_mut(&mut self) -> &mut Command { - &mut self.cmd - } -} - -#[test] -fn datadir_flag() { - CommandLineTest::new() - .run() - .with_config_and_dir(|config, dir| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_and_secrets_dir_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validators-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_dir_alias_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validator-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn beacon_nodes_flag() { - CommandLineTest::new() - .flag( - "beacon-nodes", - Some("http://localhost:1001,https://project:secret@infura.io/"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes[0].full.to_string(), - "http://localhost:1001/" - ); - assert_eq!(config.beacon_nodes[0].to_string(), "http://localhost:1001/"); - assert_eq!( - config.beacon_nodes[1].full.to_string(), - "https://project:secret@infura.io/" - ); - assert_eq!(config.beacon_nodes[1].to_string(), "https://infura.io/"); - }); -} - -#[test] -fn disable_auto_discover_flag() { - CommandLineTest::new() - .flag("disable-auto-discover", None) - .run() - .with_config(|config| assert!(config.disable_auto_discover)); -} - -#[test] -fn init_slashing_protections_flag() { - CommandLineTest::new() - .flag("init-slashing-protection", None) - .run() - .with_config(|config| assert!(config.init_slashing_protection)); -} - -#[test] -fn use_long_timeouts_flag() { - CommandLineTest::new() - .flag("use-long-timeouts", None) - .run() - .with_config(|config| assert!(config.use_long_timeouts)); -} - -#[test] -fn beacon_nodes_tls_certs_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag( - "beacon-nodes-tls-certs", - Some( - vec![ - dir.path().join("certificate.crt").to_str().unwrap(), - dir.path().join("certificate2.crt").to_str().unwrap(), - ] - .join(",") - .as_str(), - ), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes_tls_certs, - Some(vec![ - dir.path().join("certificate.crt"), - dir.path().join("certificate2.crt") - ]) - ) - }); -} - -// Tests for Graffiti flags. -#[test] -fn graffiti_flag() { - CommandLineTest::new() - .flag("graffiti", Some("nice-graffiti")) - .run() - .with_config(|config| { - assert_eq!( - config.graffiti.unwrap().to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = "default:nice-graffiti"; - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - // Public key not present so load default. - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_with_pk_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = format!("{}:nice-graffiti", pubkeybytes.to_string()); - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} - -// Tests for suggested-fee-recipient flags. -#[test] -fn fee_recipient_flag() { - CommandLineTest::new() - .flag( - "suggested-fee-recipient", - Some("0x00000000219ab540356cbb839cbe05303d7705fa"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.fee_recipient, - Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap()) - ) - }); -} - -// Tests for HTTP flags. -#[test] -fn http_flag() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(config.http_api.enabled)); -} -#[test] -fn http_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("::1")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -#[should_panic] -fn missing_unencrypted_http_transport_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_port_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_port, 9090)); -} -#[test] -fn http_allow_origin_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_api.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn http_allow_origin_all_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_api.allow_origin, Some("*".to_string()))); -} -#[test] -fn http_allow_keystore_export_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.allow_keystore_export)); -} -#[test] -fn http_allow_keystore_export_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-keystore-export", None) - .run() - .with_config(|config| assert!(config.http_api.allow_keystore_export)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.store_passwords_in_secrets_dir)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-store-passwords-in-secrets-dir", None) - .run() - .with_config(|config| assert!(config.http_api.store_passwords_in_secrets_dir)); -} - -// Tests for Metrics flags. -#[test] -fn metrics_flag() { - CommandLineTest::new() - .flag("metrics", None) - .run() - .with_config(|config| assert!(config.http_metrics.enabled)); -} -#[test] -fn metrics_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("::1")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_port_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_port, 9090)); -} -#[test] -fn metrics_allow_origin_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_metrics.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn metrics_allow_origin_all_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allow_origin, Some("*".to_string()))); -} -#[test] -pub fn malloc_tuning_flag() { - CommandLineTest::new() - .flag("disable-malloc-tuning", None) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, false)); -} -#[test] -pub fn malloc_tuning_default() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, true)); -} -#[test] -fn doppelganger_protection_flag() { - CommandLineTest::new() - .flag("enable-doppelganger-protection", None) - .run() - .with_config(|config| assert!(config.enable_doppelganger_protection)); -} -#[test] -fn no_doppelganger_protection_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.enable_doppelganger_protection)); -} -#[test] -fn produce_block_v3_flag() { - CommandLineTest::new() - .flag("produce-block-v3", None) - .run() - .with_config(|config| assert!(config.produce_block_v3)); -} - -#[test] -fn no_produce_block_v3_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.produce_block_v3)); -} - -#[test] -fn no_gas_limit_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.gas_limit.is_none())); -} -#[test] -fn gas_limit_flag() { - CommandLineTest::new() - .flag("gas-limit", Some("600")) - .flag("builder-proposals", None) - .run() - .with_config(|config| assert_eq!(config.gas_limit, Some(600))); -} -#[test] -fn no_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.builder_proposals)); -} -#[test] -fn builder_proposals_flag() { - CommandLineTest::new() - .flag("builder-proposals", None) - .run() - .with_config(|config| assert!(config.builder_proposals)); -} -#[test] -fn builder_boost_factor_flag() { - CommandLineTest::new() - .flag("builder-boost-factor", Some("150")) - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, Some(150))); -} -#[test] -fn no_builder_boost_factor_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, None)); -} -#[test] -fn prefer_builder_proposals_flag() { - CommandLineTest::new() - .flag("prefer-builder-proposals", None) - .run() - .with_config(|config| assert!(config.prefer_builder_proposals)); -} -#[test] -fn no_prefer_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.prefer_builder_proposals)); -} -#[test] -fn no_builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.builder_registration_timestamp_override.is_none())); -} -#[test] -fn builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .flag("builder-registration-timestamp-override", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.builder_registration_timestamp_override, Some(100)) - }); -} -#[test] -fn monitoring_endpoint() { - CommandLineTest::new() - .flag("monitoring-endpoint", Some("http://example:8000")) - .flag("monitoring-endpoint-period", Some("30")) - .run() - .with_config(|config| { - let api_conf = config.monitoring_api.as_ref().unwrap(); - assert_eq!(api_conf.monitoring_endpoint.as_str(), "http://example:8000"); - assert_eq!(api_conf.update_period_secs, Some(30)); - }); -} - -#[test] -fn disable_run_on_all_flag() { - CommandLineTest::new() - .flag("disable-run-on-all", None) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // --broadcast flag takes precedence - CommandLineTest::new() - .flag("disable-run-on-all", None) - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -#[test] -fn no_broadcast_flag() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Subscriptions]); - }); -} - -#[test] -fn broadcast_flag() { - // "none" variant - CommandLineTest::new() - .flag("broadcast", Some("none")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // "none" with other values is ignored - CommandLineTest::new() - .flag("broadcast", Some("none,sync-committee")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::SyncCommittee]); - }); - // Other valid variants - CommandLineTest::new() - .flag("broadcast", Some("blocks, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); - // Omitted "subscription" overrides default - CommandLineTest::new() - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -#[test] -#[should_panic(expected = "Unknown API topic")] -fn wrong_broadcast_flag() { - CommandLineTest::new() - .flag("broadcast", Some("foo, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); -} - -#[test] -fn latency_measurement_service() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", None) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("true")) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("false")) - .run() - .with_config(|config| { - assert!(!config.enable_latency_measurement_service); - }); -} - -#[test] -fn validator_registration_batch_size() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 500); - }); - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 100); - }); -} - -#[test] -#[should_panic] -fn validator_registration_batch_size_zero_value() { - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("0")) - .run(); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection_default() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_web3signer_slashing_protection); - }); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection() { - CommandLineTest::new() - .flag("disable-slashing-protection-web3signer", None) - .run() - .with_config(|config| { - assert!(!config.enable_web3signer_slashing_protection); - }); -} diff --git a/lighthouse/tests/validator_client_LOCAL_4094175.rs b/lighthouse/tests/validator_client_LOCAL_4094175.rs deleted file mode 100644 index 285f3596d92..00000000000 --- a/lighthouse/tests/validator_client_LOCAL_4094175.rs +++ /dev/null @@ -1,681 +0,0 @@ -use validator_client::{ApiTopic, BeaconNodeSyncDistanceTiers, Config}; - -use crate::exec::CommandLineTestExec; -use bls::{Keypair, PublicKeyBytes}; -use std::fs::File; -use std::io::Write; -use std::net::IpAddr; -use std::path::PathBuf; -use std::process::Command; -use std::str::FromStr; -use std::string::ToString; -use tempfile::TempDir; -use types::{Address, Slot}; - -/// Returns the `lighthouse validator_client` command. -fn base_cmd() -> Command { - let lighthouse_bin = env!("CARGO_BIN_EXE_lighthouse"); - let path = lighthouse_bin - .parse::() - .expect("should parse CARGO_TARGET_DIR"); - - let mut cmd = Command::new(path); - cmd.arg("validator_client"); - cmd -} - -// Wrapper around `Command` for easier Command Line Testing. -struct CommandLineTest { - cmd: Command, -} -impl CommandLineTest { - fn new() -> CommandLineTest { - let base_cmd = base_cmd(); - CommandLineTest { cmd: base_cmd } - } -} - -impl CommandLineTestExec for CommandLineTest { - type Config = Config; - - fn cmd_mut(&mut self) -> &mut Command { - &mut self.cmd - } -} - -#[test] -fn datadir_flag() { - CommandLineTest::new() - .run() - .with_config_and_dir(|config, dir| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_and_secrets_dir_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validators-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_dir_alias_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validator-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn beacon_nodes_flag() { - CommandLineTest::new() - .flag( - "beacon-nodes", - Some("http://localhost:1001,https://project:secret@infura.io/"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes[0].full.to_string(), - "http://localhost:1001/" - ); - assert_eq!(config.beacon_nodes[0].to_string(), "http://localhost:1001/"); - assert_eq!( - config.beacon_nodes[1].full.to_string(), - "https://project:secret@infura.io/" - ); - assert_eq!(config.beacon_nodes[1].to_string(), "https://infura.io/"); - }); -} - -#[test] -fn disable_auto_discover_flag() { - CommandLineTest::new() - .flag("disable-auto-discover", None) - .run() - .with_config(|config| assert!(config.disable_auto_discover)); -} - -#[test] -fn init_slashing_protections_flag() { - CommandLineTest::new() - .flag("init-slashing-protection", None) - .run() - .with_config(|config| assert!(config.init_slashing_protection)); -} - -#[test] -fn use_long_timeouts_flag() { - CommandLineTest::new() - .flag("use-long-timeouts", None) - .run() - .with_config(|config| assert!(config.use_long_timeouts)); -} - -#[test] -fn beacon_nodes_tls_certs_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag( - "beacon-nodes-tls-certs", - Some( - vec![ - dir.path().join("certificate.crt").to_str().unwrap(), - dir.path().join("certificate2.crt").to_str().unwrap(), - ] - .join(",") - .as_str(), - ), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes_tls_certs, - Some(vec![ - dir.path().join("certificate.crt"), - dir.path().join("certificate2.crt") - ]) - ) - }); -} - -// Tests for Graffiti flags. -#[test] -fn graffiti_flag() { - CommandLineTest::new() - .flag("graffiti", Some("nice-graffiti")) - .run() - .with_config(|config| { - assert_eq!( - config.graffiti.unwrap().to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = "default:nice-graffiti"; - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - // Public key not present so load default. - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_with_pk_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = format!("{}:nice-graffiti", pubkeybytes.to_string()); - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} - -// Tests for suggested-fee-recipient flags. -#[test] -fn fee_recipient_flag() { - CommandLineTest::new() - .flag( - "suggested-fee-recipient", - Some("0x00000000219ab540356cbb839cbe05303d7705fa"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.fee_recipient, - Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap()) - ) - }); -} - -// Tests for HTTP flags. -#[test] -fn http_flag() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(config.http_api.enabled)); -} -#[test] -fn http_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("::1")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -#[should_panic] -fn missing_unencrypted_http_transport_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_port_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_port, 9090)); -} -#[test] -fn http_allow_origin_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_api.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn http_allow_origin_all_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_api.allow_origin, Some("*".to_string()))); -} -#[test] -fn http_allow_keystore_export_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.allow_keystore_export)); -} -#[test] -fn http_allow_keystore_export_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-keystore-export", None) - .run() - .with_config(|config| assert!(config.http_api.allow_keystore_export)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.store_passwords_in_secrets_dir)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-store-passwords-in-secrets-dir", None) - .run() - .with_config(|config| assert!(config.http_api.store_passwords_in_secrets_dir)); -} - -// Tests for Metrics flags. -#[test] -fn metrics_flag() { - CommandLineTest::new() - .flag("metrics", None) - .run() - .with_config(|config| assert!(config.http_metrics.enabled)); -} -#[test] -fn metrics_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("::1")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_port_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_port, 9090)); -} -#[test] -fn metrics_allow_origin_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_metrics.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn metrics_allow_origin_all_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allow_origin, Some("*".to_string()))); -} -#[test] -pub fn malloc_tuning_flag() { - CommandLineTest::new() - .flag("disable-malloc-tuning", None) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, false)); -} -#[test] -pub fn malloc_tuning_default() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, true)); -} -#[test] -fn doppelganger_protection_flag() { - CommandLineTest::new() - .flag("enable-doppelganger-protection", None) - .run() - .with_config(|config| assert!(config.enable_doppelganger_protection)); -} -#[test] -fn no_doppelganger_protection_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.enable_doppelganger_protection)); -} -#[test] -fn produce_block_v3_flag() { - CommandLineTest::new() - .flag("produce-block-v3", None) - .run() - .with_config(|config| assert!(config.produce_block_v3)); -} - -#[test] -fn no_produce_block_v3_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.produce_block_v3)); -} - -#[test] -fn no_gas_limit_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.gas_limit.is_none())); -} -#[test] -fn gas_limit_flag() { - CommandLineTest::new() - .flag("gas-limit", Some("600")) - .flag("builder-proposals", None) - .run() - .with_config(|config| assert_eq!(config.gas_limit, Some(600))); -} -#[test] -fn no_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.builder_proposals)); -} -#[test] -fn builder_proposals_flag() { - CommandLineTest::new() - .flag("builder-proposals", None) - .run() - .with_config(|config| assert!(config.builder_proposals)); -} -#[test] -fn builder_boost_factor_flag() { - CommandLineTest::new() - .flag("builder-boost-factor", Some("150")) - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, Some(150))); -} -#[test] -fn no_builder_boost_factor_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, None)); -} -#[test] -fn prefer_builder_proposals_flag() { - CommandLineTest::new() - .flag("prefer-builder-proposals", None) - .run() - .with_config(|config| assert!(config.prefer_builder_proposals)); -} -#[test] -fn no_prefer_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.prefer_builder_proposals)); -} -#[test] -fn no_builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.builder_registration_timestamp_override.is_none())); -} -#[test] -fn builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .flag("builder-registration-timestamp-override", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.builder_registration_timestamp_override, Some(100)) - }); -} -#[test] -fn monitoring_endpoint() { - CommandLineTest::new() - .flag("monitoring-endpoint", Some("http://example:8000")) - .flag("monitoring-endpoint-period", Some("30")) - .run() - .with_config(|config| { - let api_conf = config.monitoring_api.as_ref().unwrap(); - assert_eq!(api_conf.monitoring_endpoint.as_str(), "http://example:8000"); - assert_eq!(api_conf.update_period_secs, Some(30)); - }); -} -#[test] -fn disable_run_on_all_flag() { - CommandLineTest::new() - .flag("disable-run-on-all", None) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // --broadcast flag takes precedence - CommandLineTest::new() - .flag("disable-run-on-all", None) - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -#[test] -fn no_broadcast_flag() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Subscriptions]); - }); -} - -#[test] -fn broadcast_flag() { - // "none" variant - CommandLineTest::new() - .flag("broadcast", Some("none")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // "none" with other values is ignored - CommandLineTest::new() - .flag("broadcast", Some("none,sync-committee")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::SyncCommittee]); - }); - // Other valid variants - CommandLineTest::new() - .flag("broadcast", Some("blocks, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); - // Omitted "subscription" overrides default - CommandLineTest::new() - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -/// Tests for validator fallback flags. -#[test] -fn beacon_nodes_sync_tolerances_flag_default() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!( - config.beacon_node_fallback.sync_tolerances, - BeaconNodeSyncDistanceTiers::default() - ) - }); -} -#[test] -fn beacon_nodes_sync_tolerances_flag() { - CommandLineTest::new() - .flag("beacon-nodes-sync-tolerances", Some("4,4,4")) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_node_fallback.sync_tolerances, - BeaconNodeSyncDistanceTiers { - synced: Slot::new(4), - small: Slot::new(8), - medium: Slot::new(12), - } - ); - }); -} - -#[test] -#[should_panic(expected = "Unknown API topic")] -fn wrong_broadcast_flag() { - CommandLineTest::new() - .flag("broadcast", Some("foo, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); -} - -#[test] -fn latency_measurement_service() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", None) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("true")) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("false")) - .run() - .with_config(|config| { - assert!(!config.enable_latency_measurement_service); - }); -} - -#[test] -fn validator_registration_batch_size() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 500); - }); - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 100); - }); -} - -#[test] -#[should_panic] -fn validator_registration_batch_size_zero_value() { - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("0")) - .run(); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection_default() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_web3signer_slashing_protection); - }); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection() { - CommandLineTest::new() - .flag("disable-slashing-protection-web3signer", None) - .run() - .with_config(|config| { - assert!(!config.enable_web3signer_slashing_protection); - }); -} diff --git a/lighthouse/tests/validator_client_REMOTE_4094175.rs b/lighthouse/tests/validator_client_REMOTE_4094175.rs deleted file mode 100644 index cdf8fa15aaa..00000000000 --- a/lighthouse/tests/validator_client_REMOTE_4094175.rs +++ /dev/null @@ -1,679 +0,0 @@ -use validator_client::{config::DEFAULT_WEB3SIGNER_KEEP_ALIVE, ApiTopic, Config}; - -use crate::exec::CommandLineTestExec; -use bls::{Keypair, PublicKeyBytes}; -use std::fs::File; -use std::io::Write; -use std::net::IpAddr; -use std::path::PathBuf; -use std::process::Command; -use std::str::FromStr; -use std::string::ToString; -use std::time::Duration; -use tempfile::TempDir; -use types::Address; - -/// Returns the `lighthouse validator_client` command. -fn base_cmd() -> Command { - let lighthouse_bin = env!("CARGO_BIN_EXE_lighthouse"); - let path = lighthouse_bin - .parse::() - .expect("should parse CARGO_TARGET_DIR"); - - let mut cmd = Command::new(path); - cmd.arg("validator_client"); - cmd -} - -// Wrapper around `Command` for easier Command Line Testing. -struct CommandLineTest { - cmd: Command, -} -impl CommandLineTest { - fn new() -> CommandLineTest { - let base_cmd = base_cmd(); - CommandLineTest { cmd: base_cmd } - } -} - -impl CommandLineTestExec for CommandLineTest { - type Config = Config; - - fn cmd_mut(&mut self) -> &mut Command { - &mut self.cmd - } -} - -#[test] -fn datadir_flag() { - CommandLineTest::new() - .run() - .with_config_and_dir(|config, dir| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_and_secrets_dir_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validators-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn validators_dir_alias_flags() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag("validator-dir", dir.path().join("validators").to_str()) - .flag("secrets-dir", dir.path().join("secrets").to_str()) - .run_with_no_datadir() - .with_config(|config| { - assert_eq!(config.validator_dir, dir.path().join("validators")); - assert_eq!(config.secrets_dir, dir.path().join("secrets")); - }); -} - -#[test] -fn beacon_nodes_flag() { - CommandLineTest::new() - .flag( - "beacon-nodes", - Some("http://localhost:1001,https://project:secret@infura.io/"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes[0].full.to_string(), - "http://localhost:1001/" - ); - assert_eq!(config.beacon_nodes[0].to_string(), "http://localhost:1001/"); - assert_eq!( - config.beacon_nodes[1].full.to_string(), - "https://project:secret@infura.io/" - ); - assert_eq!(config.beacon_nodes[1].to_string(), "https://infura.io/"); - }); -} - -#[test] -fn disable_auto_discover_flag() { - CommandLineTest::new() - .flag("disable-auto-discover", None) - .run() - .with_config(|config| assert!(config.disable_auto_discover)); -} - -#[test] -fn init_slashing_protections_flag() { - CommandLineTest::new() - .flag("init-slashing-protection", None) - .run() - .with_config(|config| assert!(config.init_slashing_protection)); -} - -#[test] -fn use_long_timeouts_flag() { - CommandLineTest::new() - .flag("use-long-timeouts", None) - .run() - .with_config(|config| assert!(config.use_long_timeouts)); -} - -#[test] -fn beacon_nodes_tls_certs_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - CommandLineTest::new() - .flag( - "beacon-nodes-tls-certs", - Some( - vec![ - dir.path().join("certificate.crt").to_str().unwrap(), - dir.path().join("certificate2.crt").to_str().unwrap(), - ] - .join(",") - .as_str(), - ), - ) - .run() - .with_config(|config| { - assert_eq!( - config.beacon_nodes_tls_certs, - Some(vec![ - dir.path().join("certificate.crt"), - dir.path().join("certificate2.crt") - ]) - ) - }); -} - -// Tests for Graffiti flags. -#[test] -fn graffiti_flag() { - CommandLineTest::new() - .flag("graffiti", Some("nice-graffiti")) - .run() - .with_config(|config| { - assert_eq!( - config.graffiti.unwrap().to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = "default:nice-graffiti"; - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - // Public key not present so load default. - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} -#[test] -fn graffiti_file_with_pk_flag() { - let dir = TempDir::new().expect("Unable to create temporary directory"); - let mut file = File::create(dir.path().join("graffiti.txt")).expect("Unable to create file"); - let new_key = Keypair::random(); - let pubkeybytes = PublicKeyBytes::from(new_key.pk); - let contents = format!("{}:nice-graffiti", pubkeybytes.to_string()); - file.write_all(contents.as_bytes()) - .expect("Unable to write to file"); - CommandLineTest::new() - .flag( - "graffiti-file", - dir.path().join("graffiti.txt").as_os_str().to_str(), - ) - .run() - .with_config(|config| { - assert_eq!( - config - .graffiti_file - .clone() - .unwrap() - .load_graffiti(&pubkeybytes) - .unwrap() - .unwrap() - .to_string(), - "0x6e6963652d677261666669746900000000000000000000000000000000000000" - ) - }); -} - -// Tests for suggested-fee-recipient flags. -#[test] -fn fee_recipient_flag() { - CommandLineTest::new() - .flag( - "suggested-fee-recipient", - Some("0x00000000219ab540356cbb839cbe05303d7705fa"), - ) - .run() - .with_config(|config| { - assert_eq!( - config.fee_recipient, - Some(Address::from_str("0x00000000219ab540356cbb839cbe05303d7705fa").unwrap()) - ) - }); -} - -// Tests for HTTP flags. -#[test] -fn http_flag() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(config.http_api.enabled)); -} -#[test] -fn http_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("::1")) - .flag("unencrypted-http-transport", None) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -#[should_panic] -fn missing_unencrypted_http_transport_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("http", None) - .flag("http-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_addr, addr)); -} -#[test] -fn http_port_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_api.listen_port, 9090)); -} -#[test] -fn http_allow_origin_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_api.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn http_allow_origin_all_flag() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_api.allow_origin, Some("*".to_string()))); -} -#[test] -fn http_allow_keystore_export_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.allow_keystore_export)); -} -#[test] -fn http_allow_keystore_export_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-allow-keystore-export", None) - .run() - .with_config(|config| assert!(config.http_api.allow_keystore_export)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_default() { - CommandLineTest::new() - .flag("http", None) - .run() - .with_config(|config| assert!(!config.http_api.store_passwords_in_secrets_dir)); -} -#[test] -fn http_store_keystore_passwords_in_secrets_dir_present() { - CommandLineTest::new() - .flag("http", None) - .flag("http-store-passwords-in-secrets-dir", None) - .run() - .with_config(|config| assert!(config.http_api.store_passwords_in_secrets_dir)); -} - -// Tests for Metrics flags. -#[test] -fn metrics_flag() { - CommandLineTest::new() - .flag("metrics", None) - .run() - .with_config(|config| assert!(config.http_metrics.enabled)); -} -#[test] -fn metrics_address_flag() { - let addr = "127.0.0.99".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("127.0.0.99")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_address_ipv6_flag() { - let addr = "::1".parse::().unwrap(); - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-address", Some("::1")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_addr, addr)); -} -#[test] -fn metrics_port_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-port", Some("9090")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.listen_port, 9090)); -} -#[test] -fn metrics_allow_origin_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("http://localhost:9009")) - .run() - .with_config(|config| { - assert_eq!( - config.http_metrics.allow_origin, - Some("http://localhost:9009".to_string()) - ); - }); -} -#[test] -fn metrics_allow_origin_all_flag() { - CommandLineTest::new() - .flag("metrics", None) - .flag("metrics-allow-origin", Some("*")) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allow_origin, Some("*".to_string()))); -} -#[test] -pub fn malloc_tuning_flag() { - CommandLineTest::new() - .flag("disable-malloc-tuning", None) - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, false)); -} -#[test] -pub fn malloc_tuning_default() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.http_metrics.allocator_metrics_enabled, true)); -} -#[test] -fn doppelganger_protection_flag() { - CommandLineTest::new() - .flag("enable-doppelganger-protection", None) - .run() - .with_config(|config| assert!(config.enable_doppelganger_protection)); -} -#[test] -fn no_doppelganger_protection_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.enable_doppelganger_protection)); -} -#[test] -fn produce_block_v3_flag() { - CommandLineTest::new() - .flag("produce-block-v3", None) - .run() - .with_config(|config| assert!(config.produce_block_v3)); -} - -#[test] -fn no_produce_block_v3_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.produce_block_v3)); -} - -#[test] -fn no_gas_limit_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.gas_limit.is_none())); -} -#[test] -fn gas_limit_flag() { - CommandLineTest::new() - .flag("gas-limit", Some("600")) - .flag("builder-proposals", None) - .run() - .with_config(|config| assert_eq!(config.gas_limit, Some(600))); -} -#[test] -fn no_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.builder_proposals)); -} -#[test] -fn builder_proposals_flag() { - CommandLineTest::new() - .flag("builder-proposals", None) - .run() - .with_config(|config| assert!(config.builder_proposals)); -} -#[test] -fn builder_boost_factor_flag() { - CommandLineTest::new() - .flag("builder-boost-factor", Some("150")) - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, Some(150))); -} -#[test] -fn no_builder_boost_factor_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert_eq!(config.builder_boost_factor, None)); -} -#[test] -fn prefer_builder_proposals_flag() { - CommandLineTest::new() - .flag("prefer-builder-proposals", None) - .run() - .with_config(|config| assert!(config.prefer_builder_proposals)); -} -#[test] -fn no_prefer_builder_proposals_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(!config.prefer_builder_proposals)); -} -#[test] -fn no_builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .run() - .with_config(|config| assert!(config.builder_registration_timestamp_override.is_none())); -} -#[test] -fn builder_registration_timestamp_override_flag() { - CommandLineTest::new() - .flag("builder-registration-timestamp-override", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.builder_registration_timestamp_override, Some(100)) - }); -} -#[test] -fn monitoring_endpoint() { - CommandLineTest::new() - .flag("monitoring-endpoint", Some("http://example:8000")) - .flag("monitoring-endpoint-period", Some("30")) - .run() - .with_config(|config| { - let api_conf = config.monitoring_api.as_ref().unwrap(); - assert_eq!(api_conf.monitoring_endpoint.as_str(), "http://example:8000"); - assert_eq!(api_conf.update_period_secs, Some(30)); - }); -} - -#[test] -fn disable_run_on_all_flag() { - CommandLineTest::new() - .flag("disable-run-on-all", None) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // --broadcast flag takes precedence - CommandLineTest::new() - .flag("disable-run-on-all", None) - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -#[test] -fn no_broadcast_flag() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Subscriptions]); - }); -} - -#[test] -fn broadcast_flag() { - // "none" variant - CommandLineTest::new() - .flag("broadcast", Some("none")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![]); - }); - // "none" with other values is ignored - CommandLineTest::new() - .flag("broadcast", Some("none,sync-committee")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::SyncCommittee]); - }); - // Other valid variants - CommandLineTest::new() - .flag("broadcast", Some("blocks, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); - // Omitted "subscription" overrides default - CommandLineTest::new() - .flag("broadcast", Some("attestations")) - .run() - .with_config(|config| { - assert_eq!(config.broadcast_topics, vec![ApiTopic::Attestations]); - }); -} - -#[test] -#[should_panic(expected = "Unknown API topic")] -fn wrong_broadcast_flag() { - CommandLineTest::new() - .flag("broadcast", Some("foo, subscriptions")) - .run() - .with_config(|config| { - assert_eq!( - config.broadcast_topics, - vec![ApiTopic::Blocks, ApiTopic::Subscriptions], - ); - }); -} - -#[test] -fn latency_measurement_service() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", None) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("true")) - .run() - .with_config(|config| { - assert!(config.enable_latency_measurement_service); - }); - CommandLineTest::new() - .flag("latency-measurement-service", Some("false")) - .run() - .with_config(|config| { - assert!(!config.enable_latency_measurement_service); - }); -} - -#[test] -fn validator_registration_batch_size() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 500); - }); - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("100")) - .run() - .with_config(|config| { - assert_eq!(config.validator_registration_batch_size, 100); - }); -} - -#[test] -#[should_panic] -fn validator_registration_batch_size_zero_value() { - CommandLineTest::new() - .flag("validator-registration-batch-size", Some("0")) - .run(); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection_default() { - CommandLineTest::new().run().with_config(|config| { - assert!(config.enable_web3signer_slashing_protection); - }); -} - -#[test] -fn validator_disable_web3_signer_slashing_protection() { - CommandLineTest::new() - .flag("disable-slashing-protection-web3signer", None) - .run() - .with_config(|config| { - assert!(!config.enable_web3signer_slashing_protection); - }); -} - -#[test] -fn validator_web3_signer_keep_alive_default() { - CommandLineTest::new().run().with_config(|config| { - assert_eq!( - config.web3_signer_keep_alive_timeout, - DEFAULT_WEB3SIGNER_KEEP_ALIVE - ); - }); -} - -#[test] -fn validator_web3_signer_keep_alive_override() { - CommandLineTest::new() - .flag("web3-signer-keep-alive-timeout", Some("1000")) - .run() - .with_config(|config| { - assert_eq!( - config.web3_signer_keep_alive_timeout, - Some(Duration::from_secs(1)) - ); - }); -} From 7e3866aa5b7958ef5661bf014202e292869cd627 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 26 Apr 2024 17:44:20 +1000 Subject: [PATCH 32/50] Remove old commented code --- validator_client/src/beacon_node_fallback.rs | 9 --------- 1 file changed, 9 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 22e0ffa1400..76ecca6d4f7 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -48,15 +48,6 @@ pub struct Config { /// Disables publishing http api requests to all beacon nodes for select api calls. pub disable_run_on_all: bool, pub sync_tolerances: BeaconNodeSyncDistanceTiers, - // Sets the number of slots behind the head a beacon node is allowed to be to still be - // considered `synced`. - //pub sync_tolerance: Option, - // Sets the size of the range of the `small` sync distance tier. This range starts immediately - // after `sync_tolerance`. - //pub small_sync_distance_modifier: Option, - // Sets the size of the range of the `medium` sync distance tier. This range starts immediately - // after the `small` range. - //pub medium_sync_distance_modifier: Option, } /// Indicates a measurement of latency between the VC and a BN. From e348231d24897ce511234b03ef1dd716cb28d273 Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 5 Jun 2024 16:26:44 +1000 Subject: [PATCH 33/50] Update cli text --- book/src/help_vc.md | 223 ++++---------------------------------------- 1 file changed, 16 insertions(+), 207 deletions(-) diff --git a/book/src/help_vc.md b/book/src/help_vc.md index ccadc199070..e896c8fa57b 100644 --- a/book/src/help_vc.md +++ b/book/src/help_vc.md @@ -176,213 +176,23 @@ Options: Maximum number of idle connections to maintain per web3signer host. Default is unlimited. -<<<<<<< HEAD - --enable-doppelganger-protection - If this flag is set, Lighthouse will delay startup for three epochs and monitor for messages on the network - by any of the validators managed by this client. This will result in three (possibly four) epochs worth of - missed attestations. If an attestation is detected during this period, it means it is very likely that you - are running a second validator client with the same keys. This validator client will immediately shutdown if - this is detected in order to avoid potentially committing a slashable offense. Use this flag in order to - ENABLE this functionality, without this flag Lighthouse will begin attesting immediately. - --enable-high-validator-count-metrics - Enable per validator metrics for > 64 validators. Note: This flag is automatically enabled for <= 64 - validators. Enabling this metric for higher validator counts will lead to higher volume of prometheus - metrics being collected. - -h, --help Prints help information - --http Enable the RESTful HTTP API server. Disabled by default. - --http-allow-keystore-export - If present, allow access to the DELETE /lighthouse/keystores HTTP API method, which allows exporting - keystores and passwords to HTTP API consumers who have access to the API token. This method is useful for - exporting validators, however it should be used with caution since it exposes private key data to authorized - users. - --http-store-passwords-in-secrets-dir - If present, any validators created via the HTTP will have keystore passwords stored in the secrets-dir - rather than the validator definitions file. - --init-slashing-protection - If present, do not require the slashing protection database to exist before running. You SHOULD NOT use this - flag unless you're certain that a new slashing protection database is required. Usually, your database will - have been initialized when you imported your validator keys. If you misplace your database and then run with - this flag you risk being slashed. - --log-color Force outputting colors when emitting logs to the terminal. - --logfile-compress - If present, compress old log files. This can help reduce the space needed to store old logs. - - --logfile-no-restricted-perms - If present, log files will be generated as world-readable meaning they can be read by any user on the - machine. Note that logs can often contain sensitive information about your validator and so this flag should - be used with caution. For Windows users, the log file permissions will be inherited from the parent folder. - --metrics Enable the Prometheus metrics HTTP server. Disabled by default. - --prefer-builder-proposals - If this flag is set, Lighthouse will always prefer blocks constructed by builders, regardless of payload - value. - --produce-block-v3 - Enable block production via the block v3 endpoint for this validator client. This should only be enabled - when paired with a beacon node that has this endpoint implemented. This flag will be enabled by default in - future. - --unencrypted-http-transport - This is a safety flag to ensure that the user is aware that the http transport is unencrypted and using a - custom HTTP address is unsafe. - --use-long-timeouts - If present, the validator client will use longer timeouts for requests made to the beacon node. This flag is - generally not recommended, longer timeouts can cause missed duties when fallbacks are used. - -V, --version Prints version information - -OPTIONS: - --beacon-nodes - Comma-separated addresses to one or more beacon node HTTP APIs. Default is http://localhost:5052. - - --beacon-nodes-sync-tolerances - A comma-separated list of 3 values which sets the size of each sync distance range when determining the - health of each connected beacon node. The first value determines the `Synced` range. If a connected beacon - node is synced to within this number of slots it is considered 'Synced'. The second value determines the - `Small` sync distance range. This range starts immediately after the `Synced` range. The third value - determines the `Medium` sync distance range. This range starts immediately after the `Small` range. Any sync - distance value beyond that is considered `Large`. For example, a value of `8,8,48` would have ranges like - the following: `Synced`: 0..=8 `Small`: 9..=16 `Medium`: 17..=64 `Large`: 65.. These values are used to - determine what ordering beacon node fallbacks are used in. Generally, `Synced` nodes are preferred over - `Small` and so on. Nodes in the `Synced` range will tie-break based on their ordering in `--beacon-nodes`. - This ensures the primary beacon node is prioritised. [default: 8,8,48] - --beacon-nodes-tls-certs - Comma-separated paths to custom TLS certificates to use when connecting to a beacon node (and/or proposer - node). These certificates must be in PEM format and are used in addition to the OS trust store. Commas must - only be used as a delimiter, and must not be part of the certificate path. - --broadcast - Comma-separated list of beacon API topics to broadcast to all beacon nodes. Possible values are: none, - attestations, blocks, subscriptions, sync-committee. Default (when flag is omitted) is to broadcast - subscriptions only. - --builder-boost-factor - Defines the boost factor, a percentage multiplier to apply to the builder's payload value when choosing - between a builder payload header and payload from the local execution node. - --builder-registration-timestamp-override - This flag takes a unix timestamp value that will be used to override the timestamp used in the builder api - registration - -d, --datadir - Used to specify a custom root data directory for lighthouse keys and databases. Defaults to - $HOME/.lighthouse/{network} where network is the value of the `network` flag Note: Users should specify - separate custom datadirs for different networks. - --debug-level - Specifies the verbosity level used when emitting logs to the terminal. [default: info] [possible values: - info, debug, trace, warn, error, crit] - --gas-limit - The gas limit to be used in all builder proposals for all validators managed by this validator client. Note - this will not necessarily be used if the gas limit set here moves too far from the previous block's gas - limit. [default: 30,000,000] - --genesis-state-url - A URL of a beacon-API compatible server from which to download the genesis state. Checkpoint sync server - URLs can generally be used with this flag. If not supplied, a default URL or the --checkpoint-sync-url may - be used. If the genesis state is already included in this binary then this value will be ignored. - --genesis-state-url-timeout - The timeout in seconds for the request to --genesis-state-url. [default: 180] - - --graffiti - Specify your custom graffiti to be included in blocks. - - --graffiti-file - Specify a graffiti file to load validator graffitis from. - - --http-address
- Set the address for the HTTP address. The HTTP server is not encrypted and therefore it is unsafe to publish - on a public network. When this flag is used, it additionally requires the explicit use of the - `--unencrypted-http-transport` flag to ensure the user is aware of the risks involved. For access via the - Internet, users should apply transport-layer security like a HTTPS reverse-proxy or SSH tunnelling. - --http-allow-origin - Set the value of the Access-Control-Allow-Origin response HTTP header. Use * to allow any origin (not - recommended in production). If no value is supplied, the CORS allowed origin is set to the listen address of - this server (e.g., http://localhost:5062). - --http-port - Set the listen TCP port for the RESTful HTTP API server. - - --latency-measurement-service - Set to 'true' to enable a service that periodically attempts to measure latency to BNs. Set to 'false' to - disable. [default: true] - --log-format - Specifies the log format used when emitting logs to the terminal. [possible values: JSON] - - --logfile - File path where the log file will be stored. Once it grows to the value specified in `--logfile-max-size` a - new log file is generated where future logs are stored. Once the number of log files exceeds the value - specified in `--logfile-max-number` the oldest log file will be overwritten. - --logfile-debug-level - The verbosity level used when emitting logs to the log file. [default: debug] [possible values: info, - debug, trace, warn, error, crit] - --logfile-format - Specifies the log format used when emitting logs to the logfile. [possible values: DEFAULT, JSON] - - --logfile-max-number - The maximum number of log files that will be stored. If set to 0, background file logging is disabled. - [default: 5] - --logfile-max-size - The maximum size (in MB) each log file can grow to before rotating. If set to 0, background file logging is - disabled. [default: 200] - --metrics-address
- Set the listen address for the Prometheus metrics HTTP server. - - --metrics-allow-origin - Set the value of the Access-Control-Allow-Origin response HTTP header. Use * to allow any origin (not - recommended in production). If no value is supplied, the CORS allowed origin is set to the listen address of - this server (e.g., http://localhost:5064). - --metrics-port - Set the listen TCP port for the Prometheus metrics HTTP server. - - --monitoring-endpoint
- Enables the monitoring service for sending system metrics to a remote endpoint. This can be used to monitor - your setup on certain services (e.g. beaconcha.in). This flag sets the endpoint where the beacon node - metrics will be sent. Note: This will send information to a remote sever which may identify and associate - your validators, IP address and other personal information. Always use a HTTPS connection and never provide - an untrusted URL. - --monitoring-endpoint-period - Defines how many seconds to wait between each message sent to the monitoring-endpoint. Default: 60s - - --network - Name of the Eth2 chain Lighthouse will sync and follow. [possible values: mainnet, prater, goerli, gnosis, - chiado, sepolia, holesky] - --proposer-nodes - Comma-separated addresses to one or more beacon node HTTP APIs. These specify nodes that are used to send - beacon block proposals. A failure will revert back to the standard beacon nodes specified in --beacon-nodes. - --safe-slots-to-import-optimistically - Used to coordinate manual overrides of the SAFE_SLOTS_TO_IMPORT_OPTIMISTICALLY parameter. This flag should - only be used if the user has a clear understanding that the broad Ethereum community has elected to override - this parameter in the event of an attack at the PoS transition block. Incorrect use of this flag can cause - your node to possibly accept an invalid chain or sync more slowly. Be extremely careful with this flag. - --secrets-dir - The directory which contains the password to unlock the validator voting keypairs. Each password should be - contained in a file where the name is the 0x-prefixed hex representation of the validators voting public - key. Defaults to ~/.lighthouse/{network}/secrets. - --suggested-fee-recipient - Once the merge has happened, this address will receive transaction fees from blocks proposed by this - validator client. If a fee recipient is configured in the validator definitions it takes priority over this - value. - --terminal-block-hash-epoch-override - Used to coordinate manual overrides to the TERMINAL_BLOCK_HASH_ACTIVATION_EPOCH parameter. This flag should - only be used if the user has a clear understanding that the broad Ethereum community has elected to override - the terminal PoW block. Incorrect use of this flag will cause your node to experience a consensus failure. - Be extremely careful with this flag. - --terminal-block-hash-override - Used to coordinate manual overrides to the TERMINAL_BLOCK_HASH parameter. This flag should only be used if - the user has a clear understanding that the broad Ethereum community has elected to override the terminal - PoW block. Incorrect use of this flag will cause your node to experience a consensus failure. Be extremely - careful with this flag. - --terminal-total-difficulty-override - Used to coordinate manual overrides to the TERMINAL_TOTAL_DIFFICULTY parameter. Accepts a 256-bit decimal - integer (not a hex value). This flag should only be used if the user has a clear understanding that the - broad Ethereum community has elected to override the terminal difficulty. Incorrect use of this flag will - cause your node to experience a consensus failure. Be extremely careful with this flag. - -t, --testnet-dir - Path to directory containing eth2_testnet specs. Defaults to a hard-coded Lighthouse testnet. Only effective - if there is no existing database. - --validator-registration-batch-size - Defines the number of validators per validator/register_validator request sent to the BN. This value can be - reduced to avoid timeouts from builders. [default: 500] - --validators-dir - The directory which contains the validator keystores, deposit data for each validator along with the common - slashing protection database and the validator_definitions.yml - --web3-signer-keep-alive-timeout - Keep-alive timeout for each web3signer connection. Set to 'null' to never timeout [default: 20000] - - --web3-signer-max-idle-connections - Maximum number of idle connections to maintain per web3signer host. Default is unlimited. -======= Flags: + --beacon-nodes-sync-tolerances + A comma-separated list of 3 values which sets the size of each sync + distance range when determining the health of each connected beacon + node. The first value determines the `Synced` range. If a connected + beacon node is synced to within this number of slots it is considered + 'Synced'. The second value determines the `Small` sync distance range. + This range starts immediately after the `Synced` range. The third + value determines the `Medium` sync distance range. This range starts + immediately after the `Small` range. Any sync distance value beyond + that is considered `Large`. For example, a value of `8,8,48` would + have ranges like the following: `Synced`: 0..=8 `Small`: 9..=16 + `Medium`: 17..=64 `Large`: 65.. These values are used to determine + what ordering beacon node fallbacks are used in. Generally, `Synced` + nodes are preferred over `Small` and so on. Nodes in the `Synced` + range will tie-break based on their ordering in `--beacon-nodes`. This + ensures the primary beacon node is prioritised. [default: 8,8,48] --builder-proposals If this flag is set, Lighthouse will query the Beacon Node for only block headers during proposals and will sign over headers. Useful for @@ -482,7 +292,6 @@ Flags: If present, the validator client will use longer timeouts for requests made to the beacon node. This flag is generally not recommended, longer timeouts can cause missed duties when fallbacks are used. ->>>>>>> unstable ``` From 5c8dc2186535676403049575cb30e6d6c5559fac Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 6 Jun 2024 16:06:57 +1000 Subject: [PATCH 34/50] Silence candidate errors when pre-genesis --- testing/simulator/src/cli.rs | 2 +- validator_client/src/beacon_node_fallback.rs | 24 +++++++++++++------- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/testing/simulator/src/cli.rs b/testing/simulator/src/cli.rs index a82c8b85775..db77712e51b 100644 --- a/testing/simulator/src/cli.rs +++ b/testing/simulator/src/cli.rs @@ -77,7 +77,7 @@ pub fn cli_app() -> Command { ) .arg( Arg::new("vc-count") - .short('c') + .short('n') .long("vc-count") .action(ArgAction::Set) .default_value("3") diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 76ecca6d4f7..020fb9a33d3 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -129,8 +129,9 @@ impl fmt::Display for Errors { } /// Reasons why a candidate might not be ready. -#[derive(Debug, Clone, Copy, Deserialize, Serialize)] +#[derive(Debug, Clone, Copy, PartialEq, Deserialize, Serialize)] pub enum CandidateError { + PreGenesis, Uninitialized, Offline, Incompatible, @@ -210,7 +211,12 @@ impl CandidateBeaconNode { if let Some(slot_clock) = slot_clock { match check_node_health(&self.beacon_node, log).await { Ok((head, is_optimistic, el_offline)) => { - let slot_clock_head = slot_clock.now().ok_or(CandidateError::Uninitialized)?; + let Some(slot_clock_head) = slot_clock.now() else { + match slot_clock.is_prior_to_genesis() { + Some(true) => return Err(CandidateError::PreGenesis), + _ => return Err(CandidateError::Uninitialized), + } + }; if head > slot_clock_head + FUTURE_SLOT_TOLERANCE { return Err(CandidateError::TimeDiscrepancy); @@ -457,12 +463,14 @@ impl BeaconNodeFallback { for (result, node) in results { if let Err(e) = result { - warn!( - self.log, - "A connected beacon node errored during routine health check."; - "error" => ?e, - "endpoint" => node, - ); + if *e != CandidateError::PreGenesis { + warn!( + self.log, + "A connected beacon node errored during routine health check."; + "error" => ?e, + "endpoint" => node, + ); + } } } From c373891a960582f2462cac3f662681571a25b3ea Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 16 Jul 2024 21:00:06 +1000 Subject: [PATCH 35/50] Retry on failure --- validator_client/src/beacon_node_fallback.rs | 28 +++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 020fb9a33d3..f6acbf66371 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -564,15 +564,20 @@ impl BeaconNodeFallback { }}; } - // First pass: try `func` on all synced and ready candidates. - // - // This ensures that we always choose a synced node if it is available. + // First pass: try `func` on all candidates. Candidate order has already been set in + // `update_all_candidates`. This ensures the most suitable node is always tried first. let candidates = self.candidates.read().await; for candidate in candidates.iter() { try_func!(candidate); } - // There were no candidates already ready and we were unable to make any of them ready. + // Second pass. No candidates returned successfully. Try again with the same order. + // This will duplicate errors. + for candidate in candidates.iter() { + try_func!(candidate); + } + + // No candidates returned successfully. Err(Errors(errors)) } @@ -592,6 +597,7 @@ impl BeaconNodeFallback { R: Future>, { let mut results = vec![]; + let mut to_retry = vec![]; // Run `func` using a `candidate`, returning the value or capturing errors. // @@ -610,20 +616,28 @@ impl BeaconNodeFallback { $candidate.beacon_node.to_string(), Error::RequestFailed(e), ))); + to_retry.push($candidate); inc_counter_vec(&ENDPOINT_ERRORS, &[$candidate.beacon_node.as_ref()]); } } }}; } - // First pass: try `func` on all synced and ready candidates. - // - // This ensures that we always choose a synced node if it is available. + // First pass: try `func` on all candidates. Candidate order has already been set in + // `update_all_candidates`. This ensures the most suitable node is always tried first. let candidates = self.candidates.read().await; for candidate in candidates.iter() { try_func!(candidate); } + if !to_retry.is_empty() { + // Second pass. Some candidates did not return successfully. Try them again. + // Errors will still be shown for a node if it required a retry. + for candidate in to_retry.clone().iter() { + try_func!(candidate); + } + } + let errors: Vec<_> = results.into_iter().filter_map(|res| res.err()).collect(); if !errors.is_empty() { From 453e003f926e27c09b5a04be7c18f7f6efd8f99f Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 24 Jul 2024 19:40:14 +1000 Subject: [PATCH 36/50] Remove disable_run_on_all --- validator_client/src/beacon_node_fallback.rs | 2 -- validator_client/src/config.rs | 3 --- 2 files changed, 5 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index f6acbf66371..c02264a6cb2 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -45,8 +45,6 @@ const FUTURE_SLOT_TOLERANCE: Slot = Slot::new(1); // Configuration for the Beacon Node fallback. #[derive(Copy, Clone, Debug, Default, Serialize, Deserialize)] pub struct Config { - /// Disables publishing http api requests to all beacon nodes for select api calls. - pub disable_run_on_all: bool, pub sync_tolerances: BeaconNodeSyncDistanceTiers, } diff --git a/validator_client/src/config.rs b/validator_client/src/config.rs index 99eb7711068..d8b912b4da7 100644 --- a/validator_client/src/config.rs +++ b/validator_client/src/config.rs @@ -270,9 +270,6 @@ impl Config { /* * Beacon node fallback */ - - config.beacon_node_fallback.disable_run_on_all = cli_args.get_flag("disable-run-on-all"); - if let Some(sync_tolerance) = cli_args.get_one::("beacon-nodes-sync-tolerances") { config.beacon_node_fallback.sync_tolerances = BeaconNodeSyncDistanceTiers::from_str(sync_tolerance)?; From 500dbe54fbfdac81c14ab059cd96fc7bba53401c Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 24 Jul 2024 19:58:09 +1000 Subject: [PATCH 37/50] Remove unused error variant --- validator_client/src/beacon_node_fallback.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index c02264a6cb2..76f16ce392a 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -94,8 +94,6 @@ pub fn start_fallback_updater_service( #[derive(Debug)] pub enum Error { - /// The node was unavailable and we didn't attempt to contact it. - Unavailable(CandidateError), /// We attempted to contact the node but it failed. RequestFailed(T), } @@ -104,7 +102,6 @@ impl Error { pub fn request_failure(&self) -> Option<&T> { match self { Error::RequestFailed(e) => Some(e), - _ => None, } } } From a63ec8fdae035acb31f55e7378fd22230448cc38 Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 24 Jul 2024 19:59:40 +1000 Subject: [PATCH 38/50] Fix out of date comment --- validator_client/src/beacon_node_health.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index d2e515de16e..3a4b7cf01ad 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -30,7 +30,7 @@ pub enum SyncDistanceTier { } /// Contains the different sync distance tiers which are determined at runtime by the -/// `sync_tolerance` CLI flag and the `sync_distance_modifier` flags. +/// `beacon-nodes-sync-tolerances` flag. #[derive(Copy, Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] pub struct BeaconNodeSyncDistanceTiers { pub synced: SyncDistance, From 16534849c77040ea2b1b3bf47b19be72db7c60ee Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 25 Jul 2024 20:56:36 +1000 Subject: [PATCH 39/50] Remove unnecessary as_u64 --- validator_client/src/beacon_node_health.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/validator_client/src/beacon_node_health.rs b/validator_client/src/beacon_node_health.rs index 3a4b7cf01ad..1783bb312cf 100644 --- a/validator_client/src/beacon_node_health.rs +++ b/validator_client/src/beacon_node_health.rs @@ -77,11 +77,11 @@ impl BeaconNodeSyncDistanceTiers { /// Takes a given sync distance and determines its tier based on the `sync_tolerance` defined by /// the CLI. pub fn compute_distance_tier(&self, distance: SyncDistance) -> SyncDistanceTier { - if distance.as_u64() <= self.synced.as_u64() { + if distance <= self.synced { SyncDistanceTier::Synced - } else if distance <= self.small.as_u64() { + } else if distance <= self.small { SyncDistanceTier::Small - } else if distance <= self.medium.as_u64() { + } else if distance <= self.medium { SyncDistanceTier::Medium } else { SyncDistanceTier::Large From 9c206d5d6ed70e9c3e626a39a0c2165bfe3c4aac Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 25 Jul 2024 21:00:42 +1000 Subject: [PATCH 40/50] Remove more out of date comments --- validator_client/src/beacon_node_fallback.rs | 7 ------- 1 file changed, 7 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 76f16ce392a..509dc968f59 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -519,10 +519,6 @@ impl BeaconNodeFallback { /// Run `func` against each candidate in `self`, returning immediately if a result is found. /// Otherwise, return all the errors encountered along the way. - /// - /// First this function will try all nodes with a suitable status. If no candidates are suitable - /// or all the requests fail, it will try updating the status of all unsuitable nodes and - /// re-running `func` again. pub async fn first_success(&self, func: F) -> Result> where F: Fn(BeaconNodeHttpClient) -> R, @@ -579,9 +575,6 @@ impl BeaconNodeFallback { /// Run `func` against all candidates in `self`, collecting the result of `func` against each /// candidate. /// - /// First this function will try all nodes with a suitable status. If no candidates are suitable - /// it will try updating the status of all unsuitable nodes and re-running `func` again. - /// /// Note: This function returns `Ok(())` if `func` returned successfully on all beacon nodes. /// It returns a list of errors along with the beacon node id that failed for `func`. /// Since this ignores the actual result of `func`, this function should only be used for beacon From 526a894a2b8ea283f8daed3329c12e93dfe6e7b4 Mon Sep 17 00:00:00 2001 From: Mac L Date: Mon, 19 Aug 2024 22:15:33 +1000 Subject: [PATCH 41/50] Use tokio RwLock and remove parking_lot --- validator_client/src/beacon_node_fallback.rs | 103 ++++++++++--------- validator_client/src/block_service.rs | 10 +- validator_client/src/doppelganger_service.rs | 76 +++++++------- validator_client/src/http_api/mod.rs | 4 +- 4 files changed, 105 insertions(+), 88 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 509dc968f59..25183f706d4 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -11,7 +11,6 @@ use crate::http_metrics::metrics::{inc_counter_vec, ENDPOINT_ERRORS, ENDPOINT_RE use environment::RuntimeContext; use eth2::BeaconNodeHttpClient; use futures::future; -use parking_lot::RwLock as PLRwLock; use serde::{Deserialize, Serialize}; use slog::{debug, error, warn, Logger}; use slot_clock::SlotClock; @@ -142,11 +141,11 @@ pub struct CandidateInfo { /// Represents a `BeaconNodeHttpClient` inside a `BeaconNodeFallback` that may or may not be used /// for a query. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct CandidateBeaconNode { pub index: usize, pub beacon_node: BeaconNodeHttpClient, - pub health: PLRwLock>, + pub health: Arc>>, _phantom: PhantomData, } @@ -158,37 +157,20 @@ impl PartialEq for CandidateBeaconNode { impl Eq for CandidateBeaconNode {} -impl Ord for CandidateBeaconNode { - fn cmp(&self, other: &Self) -> Ordering { - match (&(self.health()), &(other.health())) { - (Err(_), Err(_)) => Ordering::Equal, - (Err(_), _) => Ordering::Greater, - (_, Err(_)) => Ordering::Less, - (Ok(health_1), Ok(health_2)) => health_1.cmp(health_2), - } - } -} - -impl PartialOrd for CandidateBeaconNode { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - impl CandidateBeaconNode { /// Instantiate a new node. pub fn new(beacon_node: BeaconNodeHttpClient, index: usize) -> Self { Self { index, beacon_node, - health: PLRwLock::new(Err(CandidateError::Uninitialized)), + health: Arc::new(RwLock::new(Err(CandidateError::Uninitialized))), _phantom: PhantomData, } } /// Returns the health of `self`. - pub fn health(&self) -> Result { - *self.health.read() + pub async fn health(&self) -> Result { + *self.health.read().await } pub async fn refresh_health( @@ -199,7 +181,7 @@ impl CandidateBeaconNode { log: &Logger, ) -> Result<(), CandidateError> { if let Err(e) = self.is_compatible(spec, log).await { - *self.health.write() = Err(e); + *self.health.write().await = Err(e); return Err(e); } @@ -240,12 +222,12 @@ impl CandidateBeaconNode { distance_tiers, ); - *self.health.write() = Ok(new_health); + *self.health.write().await = Ok(new_health); Ok(()) } Err(e) => { // Set the health as `Err` which is sorted last in the list. - *self.health.write() = Err(e); + *self.health.write().await = Err(e); Err(e) } } @@ -388,7 +370,7 @@ impl BeaconNodeFallback { pub async fn num_available(&self) -> usize { let mut n = 0; for candidate in self.candidates.read().await.iter() { - match candidate.health() { + match candidate.health().await { Ok(_) | Err(CandidateError::Uninitialized) => n += 1, Err(_) => continue, } @@ -405,9 +387,9 @@ impl BeaconNodeFallback { let mut num_synced = 0; for candidate in candidates.iter() { - let health = candidate.health(); + let health = candidate.health().await; - match candidate.health() { + match candidate.health().await { Ok(health) => { if self .distance_tiers @@ -471,9 +453,8 @@ impl BeaconNodeFallback { drop(candidates); - // Sort the list to put the healthiest candidate first. - let mut write = self.candidates.write().await; - write.sort(); + let mut candidates = self.candidates.write().await; + sort_nodes_by_health(&mut candidates).await; } /// Concurrently send a request to all candidates (regardless of @@ -519,7 +500,7 @@ impl BeaconNodeFallback { /// Run `func` against each candidate in `self`, returning immediately if a result is found. /// Otherwise, return all the errors encountered along the way. - pub async fn first_success(&self, func: F) -> Result> + pub async fn first_success<'a, F, O, Err, R>(&'a self, func: F) -> Result> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, @@ -562,8 +543,8 @@ impl BeaconNodeFallback { try_func!(candidate); } - // Second pass. No candidates returned successfully. Try again with the same order. - // This will duplicate errors. + //// Second pass. No candidates returned successfully. Try again with the same order. + //// This will duplicate errors. for candidate in candidates.iter() { try_func!(candidate); } @@ -579,7 +560,7 @@ impl BeaconNodeFallback { /// It returns a list of errors along with the beacon node id that failed for `func`. /// Since this ignores the actual result of `func`, this function should only be used for beacon /// node calls whose results we do not care about, only that they completed successfully. - pub async fn broadcast(&self, func: F) -> Result<(), Errors> + pub async fn broadcast<'a, F, O, Err, R>(&'a self, func: F) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, @@ -637,7 +618,11 @@ impl BeaconNodeFallback { /// Call `func` on first beacon node that returns success or on all beacon nodes /// depending on the `topic` and configuration. - pub async fn request(&self, topic: ApiTopic, func: F) -> Result<(), Errors> + pub async fn request<'a, F, Err, R>( + &'a self, + topic: ApiTopic, + func: F, + ) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, @@ -652,6 +637,32 @@ impl BeaconNodeFallback { } } +/// Helper functions to allow sorting candidate nodes by health. +async fn sort_nodes_by_health(nodes: &mut Vec>) { + // Fetch all health values. + let health_results: Vec> = + future::join_all(nodes.iter().map(|node| node.health())).await; + + // Pair health results with their indices. + let mut indices_with_health: Vec<(usize, Result)> = + health_results.into_iter().enumerate().collect(); + + // Sort indices based on their health. + indices_with_health.sort_by(|a, b| match (&a.1, &b.1) { + (Ok(health_a), Ok(health_b)) => health_a.cmp(health_b), + (Err(_), Ok(_)) => Ordering::Greater, + (Ok(_), Err(_)) => Ordering::Less, + (Err(_), Err(_)) => Ordering::Equal, + }); + + // Reorder candidates based on the sorted indices. + let sorted_nodes: Vec> = indices_with_health + .into_iter() + .map(|(index, _)| nodes[index].clone()) + .collect(); + *nodes = sorted_nodes; +} + /// Serves as a cue for `BeaconNodeFallback` to tell which requests need to be broadcasted. #[derive(Clone, Copy, Debug, PartialEq, Deserialize, Serialize, EnumString, EnumVariantNames)] #[strum(serialize_all = "kebab-case")] @@ -691,8 +702,8 @@ mod tests { .eq(all.into_iter())); } - #[test] - fn check_candidate_order() { + #[tokio::test] + async fn check_candidate_order() { // These fields is irrelvant for sorting. They are set to arbitrary values. let head = Slot::new(99); let optimistic_status = IsOptimistic::No; @@ -773,12 +784,12 @@ mod tests { health_tier: BeaconNodeHealthTier::new(4, Slot::new(10), small), }; - *candidate_1.health.write() = Ok(health_1); - *candidate_2.health.write() = Ok(health_2); - *candidate_3.health.write() = Ok(health_3); - *candidate_4.health.write() = Ok(health_4); - *candidate_5.health.write() = Ok(health_5); - *candidate_6.health.write() = Ok(health_6); + *candidate_1.health.write().await = Ok(health_1); + *candidate_2.health.write().await = Ok(health_2); + *candidate_3.health.write().await = Ok(health_3); + *candidate_4.health.write().await = Ok(health_4); + *candidate_5.health.write().await = Ok(health_5); + *candidate_6.health.write().await = Ok(health_6); let mut candidates = vec![ candidate_3, @@ -797,7 +808,7 @@ mod tests { expected_candidate_6, ]; - candidates.sort(); + sort_nodes_by_health(&mut candidates).await; assert_eq!(candidates, expected_candidates); } diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index fff8b0d0f35..b7a9df4c5d1 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -140,7 +140,10 @@ pub struct ProposerFallback { impl ProposerFallback { // Try `func` on `self.proposer_nodes` first. If that doesn't work, try `self.beacon_nodes`. - pub async fn request_proposers_first(&self, func: F) -> Result<(), Errors> + pub async fn request_proposers_first<'a, F, Err, R>( + &'a self, + func: F, + ) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R + Clone, R: Future>, @@ -162,7 +165,10 @@ impl ProposerFallback { } // Try `func` on `self.beacon_nodes` first. If that doesn't work, try `self.proposer_nodes`. - pub async fn request_proposers_last(&self, func: F) -> Result> + pub async fn request_proposers_last<'a, F, O, Err, R>( + &'a self, + func: F, + ) -> Result> where F: Fn(BeaconNodeHttpClient) -> R + Clone, R: Future>, diff --git a/validator_client/src/doppelganger_service.rs b/validator_client/src/doppelganger_service.rs index 94cc9f9724c..42a2eb84d3e 100644 --- a/validator_client/src/doppelganger_service.rs +++ b/validator_client/src/doppelganger_service.rs @@ -174,25 +174,25 @@ async fn beacon_node_liveness<'a, T: 'static + SlotClock, E: EthSpec>( } else { // Request the previous epoch liveness state from the beacon node. beacon_nodes - .first_success(|beacon_node| async { - let owned_beacon_node = beacon_node.clone(); - drop(beacon_node); - - owned_beacon_node - .post_validator_liveness_epoch(previous_epoch, &validator_indices) - .await - .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) - .map(|result| { - result - .data - .into_iter() - .map(|response| LivenessResponseData { - index: response.index, - epoch: previous_epoch, - is_live: response.is_live, - }) - .collect() - }) + .first_success(|beacon_node| { + let validator_indices_ref = &validator_indices; + async move { + beacon_node + .post_validator_liveness_epoch(previous_epoch, validator_indices_ref) + .await + .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) + .map(|result| { + result + .data + .into_iter() + .map(|response| LivenessResponseData { + index: response.index, + epoch: previous_epoch, + is_live: response.is_live, + }) + .collect() + }) + } }) .await .unwrap_or_else(|e| { @@ -210,25 +210,25 @@ async fn beacon_node_liveness<'a, T: 'static + SlotClock, E: EthSpec>( // Request the current epoch liveness state from the beacon node. let current_epoch_responses = beacon_nodes - .first_success(|beacon_node| async { - let owned_beacon_node = beacon_node.clone(); - drop(beacon_node); - - owned_beacon_node - .post_validator_liveness_epoch(current_epoch, &validator_indices) - .await - .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) - .map(|result| { - result - .data - .into_iter() - .map(|response| LivenessResponseData { - index: response.index, - epoch: current_epoch, - is_live: response.is_live, - }) - .collect() - }) + .first_success(|beacon_node| { + let validator_indices_ref = &validator_indices; + async move { + beacon_node + .post_validator_liveness_epoch(current_epoch, validator_indices_ref) + .await + .map_err(|e| format!("Failed query for validator liveness: {:?}", e)) + .map(|result| { + result + .data + .into_iter() + .map(|response| LivenessResponseData { + index: response.index, + epoch: current_epoch, + is_live: response.is_live, + }) + .collect() + }) + } }) .await .unwrap_or_else(|e| { diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index 2625baa5c98..950d4f9f048 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -424,14 +424,14 @@ pub fn serve( for node in &*block_filter.beacon_nodes.candidates.read().await { result.insert( (node.index, node.beacon_node.to_string()), - *node.health.read(), + *node.health.read().await, ); } if let Some(proposer_nodes) = &block_filter.proposer_nodes { for node in &*proposer_nodes.candidates.read().await { result.insert( (node.index, node.beacon_node.to_string()), - *node.health.read(), + *node.health.read().await, ); } } From 2de68ed83cb91d9c66ec589f31c10a9864d0ec81 Mon Sep 17 00:00:00 2001 From: Mac L Date: Tue, 20 Aug 2024 00:35:57 +1000 Subject: [PATCH 42/50] Formatting --- validator_client/src/block_service.rs | 46 ++++++++++++-------------- validator_client/src/duties_service.rs | 21 +++++------- 2 files changed, 31 insertions(+), 36 deletions(-) diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index e421a71e4aa..713c9d164dc 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -480,30 +480,28 @@ impl BlockService { // Try the proposer nodes last, since it's likely that they don't have a // great view of attestations on the network. let unsigned_block = proposer_fallback - .request_proposers_last( - |beacon_node| async move { - let _get_timer = metrics::start_timer_vec( - &metrics::BLOCK_SERVICE_TIMES, - &[metrics::BEACON_BLOCK_HTTP_GET], - ); - Self::get_validator_block( - &beacon_node, - slot, - randao_reveal_ref, - graffiti, - proposer_index, - builder_boost_factor, - log, - ) - .await - .map_err(|e| { - BlockError::Recoverable(format!( - "Error from beacon node when producing block: {:?}", - e - )) - }) - }, - ) + .request_proposers_last(|beacon_node| async move { + let _get_timer = metrics::start_timer_vec( + &metrics::BLOCK_SERVICE_TIMES, + &[metrics::BEACON_BLOCK_HTTP_GET], + ); + Self::get_validator_block( + &beacon_node, + slot, + randao_reveal_ref, + graffiti, + proposer_index, + builder_boost_factor, + log, + ) + .await + .map_err(|e| { + BlockError::Recoverable(format!( + "Error from beacon node when producing block: {:?}", + e + )) + }) + }) .await?; self_ref diff --git a/validator_client/src/duties_service.rs b/validator_client/src/duties_service.rs index 10551a654ea..c19eac8876f 100644 --- a/validator_client/src/duties_service.rs +++ b/validator_client/src/duties_service.rs @@ -740,18 +740,15 @@ async fn poll_beacon_attesters( let subscriptions_ref = &subscriptions; let subscription_result = duties_service .beacon_nodes - .request( - ApiTopic::Subscriptions, - |beacon_node| async move { - let _timer = metrics::start_timer_vec( - &metrics::DUTIES_SERVICE_TIMES, - &[metrics::SUBSCRIPTIONS_HTTP_POST], - ); - beacon_node - .post_validator_beacon_committee_subscriptions(subscriptions_ref) - .await - }, - ) + .request(ApiTopic::Subscriptions, |beacon_node| async move { + let _timer = metrics::start_timer_vec( + &metrics::DUTIES_SERVICE_TIMES, + &[metrics::SUBSCRIPTIONS_HTTP_POST], + ); + beacon_node + .post_validator_beacon_committee_subscriptions(subscriptions_ref) + .await + }) .await; if subscription_result.as_ref().is_ok() { debug!( From 4bf2f59161e84711c912cf33172f845bae84136d Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 22 Aug 2024 23:05:33 +1000 Subject: [PATCH 43/50] Ensure nodes are still added to total when not available --- validator_client/src/beacon_node_fallback.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index aef674446be..fe825c757ac 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -407,7 +407,7 @@ impl BeaconNodeFallback { num_available += 1; } Err(CandidateError::Uninitialized) => num_available += 1, - Err(_) => continue, + Err(_) => (), } candidate_info.push(CandidateInfo { From 4c55b033faf23442fdfe2f1ab56c8533670b22a4 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 23 Aug 2024 00:41:11 +1000 Subject: [PATCH 44/50] Allow VC to detect when BN comes online --- validator_client/src/beacon_node_fallback.rs | 21 ++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index fe825c757ac..cf3f6c71f5a 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -195,14 +195,18 @@ impl CandidateBeaconNode { match check_node_health(&self.beacon_node, log).await { Ok((head, is_optimistic, el_offline)) => { let Some(slot_clock_head) = slot_clock.now() else { - match slot_clock.is_prior_to_genesis() { - Some(true) => return Err(CandidateError::PreGenesis), - _ => return Err(CandidateError::Uninitialized), - } + let e = match slot_clock.is_prior_to_genesis() { + Some(true) => CandidateError::PreGenesis, + _ => CandidateError::Uninitialized, + }; + *self.health.write().await = Err(e); + return Err(e); }; if head > slot_clock_head + FUTURE_SLOT_TOLERANCE { - return Err(CandidateError::TimeDiscrepancy); + let e = CandidateError::TimeDiscrepancy; + *self.health.write().await = Err(e); + return Err(e); } let sync_distance = slot_clock_head.saturating_sub(head); @@ -239,8 +243,9 @@ impl CandidateBeaconNode { } } else { // Slot clock will only be `None` at startup. - // Assume compatible nodes are available. - Ok(()) + let e = CandidateError::Uninitialized; + *self.health.write().await = Err(e); + Err(e) } } @@ -449,7 +454,7 @@ impl BeaconNodeFallback { if *e != CandidateError::PreGenesis { warn!( self.log, - "A connected beacon node errored during routine health check."; + "A connected beacon node errored during routine health check"; "error" => ?e, "endpoint" => node, ); From 5cc2c132efdb19421eaa4aee886fb981a10d3360 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 23 Aug 2024 04:04:27 +1000 Subject: [PATCH 45/50] Fix ui endpoint --- validator_client/src/beacon_node_fallback.rs | 48 +++++++++++++++++--- validator_client/src/http_api/mod.rs | 34 ++++++++------ validator_client/src/notifier.rs | 8 ++-- 3 files changed, 66 insertions(+), 24 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index cf3f6c71f5a..514e3642100 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -11,7 +11,7 @@ use crate::http_metrics::metrics::{inc_counter_vec, ENDPOINT_ERRORS, ENDPOINT_RE use environment::RuntimeContext; use eth2::BeaconNodeHttpClient; use futures::future; -use serde::{Deserialize, Serialize}; +use serde::{ser::SerializeStruct, Deserialize, Serialize, Serializer}; use slog::{debug, error, warn, Logger}; use slot_clock::SlotClock; use std::cmp::Ordering; @@ -138,11 +138,47 @@ pub enum CandidateError { TimeDiscrepancy, } -#[derive(Debug, Clone)] +impl std::fmt::Display for CandidateError { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + CandidateError::PreGenesis => write!(f, "PreGenesis"), + CandidateError::Uninitialized => write!(f, "Uninitialized"), + CandidateError::Offline => write!(f, "Offline"), + CandidateError::Incompatible => write!(f, "Incompatible"), + CandidateError::TimeDiscrepancy => write!(f, "TimeDiscrepancy"), + } + } +} + +#[derive(Debug, Clone, Deserialize)] pub struct CandidateInfo { pub index: usize, - pub node: String, - pub health: Option, + pub endpoint: String, + pub health: Result, +} + +impl Serialize for CandidateInfo { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + let mut state = serializer.serialize_struct("CandidateInfo", 2)?; + + state.serialize_field("index", &self.index)?; + state.serialize_field("endpoint", &self.endpoint)?; + + // Serialize either the health or the error field based on the Result + match &self.health { + Ok(health) => { + state.serialize_field("health", health)?; + } + Err(e) => { + state.serialize_field("error", &e.to_string())?; + } + } + + state.end() + } } /// Represents a `BeaconNodeHttpClient` inside a `BeaconNodeFallback` that may or may not be used @@ -417,8 +453,8 @@ impl BeaconNodeFallback { candidate_info.push(CandidateInfo { index: candidate.index, - node: candidate.beacon_node.to_string(), - health: health.ok(), + endpoint: candidate.beacon_node.to_string(), + health, }); } diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index 950d4f9f048..cffbcd62a5c 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -8,8 +8,7 @@ mod tests; pub mod test_utils; -use crate::beacon_node_fallback::CandidateError; -use crate::beacon_node_health::BeaconNodeHealth; +use crate::beacon_node_fallback::CandidateInfo; use crate::http_api::graffiti::{delete_graffiti, get_graffiti, set_graffiti}; use crate::http_api::create_signed_voluntary_exit::create_signed_voluntary_exit; @@ -419,21 +418,28 @@ pub fn serve( .and(warp::path::end()) .and(block_service_filter.clone()) .then(|block_filter: BlockService| async move { - let mut result: HashMap<(usize, String), Result> = - HashMap::new(); + let mut result: HashMap> = HashMap::new(); + + let mut beacon_nodes = Vec::new(); for node in &*block_filter.beacon_nodes.candidates.read().await { - result.insert( - (node.index, node.beacon_node.to_string()), - *node.health.read().await, - ); + beacon_nodes.push(CandidateInfo { + index: node.index, + endpoint: node.beacon_node.to_string(), + health: *node.health.read().await, + }); } - if let Some(proposer_nodes) = &block_filter.proposer_nodes { - for node in &*proposer_nodes.candidates.read().await { - result.insert( - (node.index, node.beacon_node.to_string()), - *node.health.read().await, - ); + result.insert("beacon_nodes".to_string(), beacon_nodes); + + if let Some(proposer_nodes_list) = &block_filter.proposer_nodes { + let mut proposer_nodes = Vec::new(); + for node in &*proposer_nodes_list.candidates.read().await { + proposer_nodes.push(CandidateInfo { + index: node.index, + endpoint: node.beacon_node.to_string(), + health: *node.health.read().await, + }); } + result.insert("proposer_nodes".to_string(), proposer_nodes); } blocking_json_task(move || Ok(api_types::GenericResponse::from(result))).await diff --git a/validator_client/src/notifier.rs b/validator_client/src/notifier.rs index 1e524ceae87..00d7b14de7c 100644 --- a/validator_client/src/notifier.rs +++ b/validator_client/src/notifier.rs @@ -59,7 +59,7 @@ async fn notify( if num_synced > 0 { let primary = candidate_info .first() - .map(|candidate| candidate.node.as_str()) + .map(|candidate| candidate.endpoint.as_str()) .unwrap_or("None"); info!( log, @@ -85,13 +85,13 @@ async fn notify( } for info in candidate_info { - if let Some(health) = info.health { + if let Ok(health) = info.health { debug!( log, "Beacon node info"; "status" => "Connected", "index" => info.index, - "endpoint" => info.node, + "endpoint" => info.endpoint, "head_slot" => %health.head, "is_optimistic" => ?health.optimistic_status, "execution_engine_status" => ?health.execution_status, @@ -103,7 +103,7 @@ async fn notify( "Beacon node info"; "status" => "Disconnected", "index" => info.index, - "endpoint" => info.node, + "endpoint" => info.endpoint, ); } } From b5445a09e9f0942ecd561575367f0d1f120415f6 Mon Sep 17 00:00:00 2001 From: Mac L Date: Fri, 23 Aug 2024 23:45:41 +1000 Subject: [PATCH 46/50] Don't have block_service as an Option --- validator_client/src/http_api/mod.rs | 12 ++---------- validator_client/src/http_api/test_utils.rs | 5 ++++- validator_client/src/http_api/tests.rs | 5 ++++- validator_client/src/lib.rs | 2 +- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index cffbcd62a5c..f98f3c7db58 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -73,7 +73,7 @@ impl From for Error { pub struct Context { pub task_executor: TaskExecutor, pub api_secret: ApiSecret, - pub block_service: Option>, + pub block_service: BlockService, pub validator_store: Option>>, pub validator_dir: Option, pub secrets_dir: Option, @@ -172,15 +172,7 @@ pub fn serve( }; let inner_block_service = ctx.block_service.clone(); - let block_service_filter = warp::any() - .map(move || inner_block_service.clone()) - .and_then(|block_service: Option<_>| async move { - block_service.ok_or_else(|| { - warp_utils::reject::custom_not_found( - "block service is not initialized.".to_string(), - ) - }) - }); + let block_service_filter = warp::any().map(move || inner_block_service.clone()); let inner_validator_store = ctx.validator_store.clone(); let validator_store_filter = warp::any() diff --git a/validator_client/src/http_api/test_utils.rs b/validator_client/src/http_api/test_utils.rs index d135dfc2720..8885160c6c2 100644 --- a/validator_client/src/http_api/test_utils.rs +++ b/validator_client/src/http_api/test_utils.rs @@ -1,5 +1,6 @@ use crate::doppelganger_service::DoppelgangerService; use crate::key_cache::{KeyCache, CACHE_FILENAME}; +use crate::BlockServiceBuilder; use crate::{ http_api::{ApiSecret, Config as HttpConfig, Context}, initialized_validators::{InitializedValidators, OnDecryptFailure}, @@ -127,7 +128,9 @@ impl ApiTester { let context = Arc::new(Context { task_executor: test_runtime.task_executor.clone(), api_secret, - block_service: None, + block_service: BlockServiceBuilder::new() + .build() + .expect("Should build block service"), validator_dir: Some(validator_dir.path().into()), secrets_dir: Some(secrets_dir.path().into()), validator_store: Some(validator_store.clone()), diff --git a/validator_client/src/http_api/tests.rs b/validator_client/src/http_api/tests.rs index f6da4f76e7a..a64aa3ad5ba 100644 --- a/validator_client/src/http_api/tests.rs +++ b/validator_client/src/http_api/tests.rs @@ -4,6 +4,7 @@ mod keystores; use crate::doppelganger_service::DoppelgangerService; +use crate::BlockServiceBuilder; use crate::{ http_api::{ApiSecret, Config as HttpConfig, Context}, initialized_validators::InitializedValidators, @@ -115,7 +116,9 @@ impl ApiTester { let context = Arc::new(Context { task_executor: test_runtime.task_executor.clone(), api_secret, - block_service: None, + block_service: BlockServiceBuilder::new() + .build() + .expect("Should build block service"), validator_dir: Some(validator_dir.path().into()), secrets_dir: Some(secrets_dir.path().into()), validator_store: Some(validator_store.clone()), diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index 9a02ffdefb3..548fe0d6788 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -572,7 +572,7 @@ impl ProductionValidatorClient { let ctx = Arc::new(http_api::Context { task_executor: self.context.executor.clone(), api_secret, - block_service: Some(self.block_service.clone()), + block_service: self.block_service.clone(), validator_store: Some(self.validator_store.clone()), validator_dir: Some(self.config.validator_dir.clone()), secrets_dir: Some(self.config.secrets_dir.clone()), From 7eab573909976b6ee5798840f04e9639ce764854 Mon Sep 17 00:00:00 2001 From: Michael Sproul Date: Mon, 2 Sep 2024 16:32:29 +1000 Subject: [PATCH 47/50] Clean up lifetimes and futures --- validator_client/src/beacon_node_fallback.rs | 109 +++++++++---------- validator_client/src/block_service.rs | 10 +- 2 files changed, 51 insertions(+), 68 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 514e3642100..4896d868e90 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -547,59 +547,66 @@ impl BeaconNodeFallback { /// Run `func` against each candidate in `self`, returning immediately if a result is found. /// Otherwise, return all the errors encountered along the way. - pub async fn first_success<'a, F, O, Err, R>(&'a self, func: F) -> Result> + pub async fn first_success(&self, func: F) -> Result> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, Err: Debug, { - let mut errors = vec![]; - let log = &self.log.clone(); - - // Run `func` using a `candidate`, returning the value or capturing errors. - // - // We use a macro instead of a closure here since it is not trivial to move `func` into a - // closure. - macro_rules! try_func { - ($candidate: ident) => {{ - inc_counter_vec(&ENDPOINT_REQUESTS, &[$candidate.beacon_node.as_ref()]); - - // There exists a race condition where `func` may be called when the candidate is - // actually not ready. We deem this an acceptable inefficiency. - match func($candidate.beacon_node.clone()).await { - Ok(val) => return Ok(val), - Err(e) => { - debug!( - log, - "Request to beacon node failed"; - "node" => $candidate.beacon_node.to_string(), - "error" => ?e, - ); - - errors.push(($candidate.beacon_node.to_string(), Error::RequestFailed(e))); - inc_counter_vec(&ENDPOINT_ERRORS, &[$candidate.beacon_node.as_ref()]); - } - } - }}; - } - // First pass: try `func` on all candidates. Candidate order has already been set in // `update_all_candidates`. This ensures the most suitable node is always tried first. let candidates = self.candidates.read().await; + let mut errors = vec![]; for candidate in candidates.iter() { - try_func!(candidate); + match Self::run_on_candidate(candidate, &func, &self.log).await { + Ok(val) => return Ok(val), + Err(e) => errors.push(e), + } } - //// Second pass. No candidates returned successfully. Try again with the same order. - //// This will duplicate errors. + // Second pass. No candidates returned successfully. Try again with the same order. + // This will duplicate errors. for candidate in candidates.iter() { - try_func!(candidate); + match Self::run_on_candidate(candidate, &func, &self.log).await { + Ok(val) => return Ok(val), + Err(e) => errors.push(e), + } } // No candidates returned successfully. Err(Errors(errors)) } + /// Run the future `func` on `candidate` while reporting metrics. + async fn run_on_candidate( + candidate: &CandidateBeaconNode, + func: F, + log: &Logger, + ) -> Result)> + where + F: Fn(BeaconNodeHttpClient) -> R, + R: Future>, + Err: Debug, + { + inc_counter_vec(&ENDPOINT_REQUESTS, &[candidate.beacon_node.as_ref()]); + + // There exists a race condition where `func` may be called when the candidate is + // actually not ready. We deem this an acceptable inefficiency. + match func(candidate.beacon_node.clone()).await { + Ok(val) => Ok(val), + Err(e) => { + debug!( + log, + "Request to beacon node failed"; + "node" => %candidate.beacon_node, + "error" => ?e, + ); + inc_counter_vec(&ENDPOINT_ERRORS, &[candidate.beacon_node.as_ref()]); + Err((candidate.beacon_node.to_string(), Error::RequestFailed(e))) + } + } + } + /// Run `func` against all candidates in `self`, collecting the result of `func` against each /// candidate. /// @@ -607,33 +614,19 @@ impl BeaconNodeFallback { /// It returns a list of errors along with the beacon node id that failed for `func`. /// Since this ignores the actual result of `func`, this function should only be used for beacon /// node calls whose results we do not care about, only that they completed successfully. - pub async fn broadcast<'a, F, O, Err, R>(&'a self, func: F) -> Result<(), Errors> + pub async fn broadcast(&self, func: F) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, + Err: Debug, { - // Run `func` using a `candidate`, returning the value or capturing errors. - let run_on_candidate = |candidate: CandidateBeaconNode| async { - //inc_counter_vec(&ENDPOINT_REQUESTS, &[candidate.beacon_node.as_ref()]); - - // There exists a race condition where `func` may be called when the candidate is - // actually not ready. We deem this an acceptable inefficiency. - match func(candidate.beacon_node.clone()).await { - Ok(val) => Ok(val), - Err(e) => { - //inc_counter_vec(&ENDPOINT_ERRORS, &[candidate.beacon_node.as_ref()]); - drop(candidate); - Err(("Placeholder".to_string(), Error::RequestFailed(e))) - //Err((candidate.beacon_node.to_string(), Error::RequestFailed(e))) - } - } - }; - // Run `func` on all candidates. + let candidates = self.candidates.read().await; let mut futures = vec![]; - let candidates = self.candidates.read().await.clone(); + + // Run `func` using a `candidate`, returning the value or capturing errors. for candidate in candidates.iter() { - futures.push(run_on_candidate(candidate.clone())); + futures.push(Self::run_on_candidate(candidate, &func, &self.log)); } let results = future::join_all(futures).await; @@ -648,11 +641,7 @@ impl BeaconNodeFallback { /// Call `func` on first beacon node that returns success or on all beacon nodes /// depending on the `topic` and configuration. - pub async fn request<'a, F, Err, R>( - &'a self, - topic: ApiTopic, - func: F, - ) -> Result<(), Errors> + pub async fn request(&self, topic: ApiTopic, func: F) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R, R: Future>, diff --git a/validator_client/src/block_service.rs b/validator_client/src/block_service.rs index 713c9d164dc..665eaf0a0f7 100644 --- a/validator_client/src/block_service.rs +++ b/validator_client/src/block_service.rs @@ -140,10 +140,7 @@ pub struct ProposerFallback { impl ProposerFallback { // Try `func` on `self.proposer_nodes` first. If that doesn't work, try `self.beacon_nodes`. - pub async fn request_proposers_first<'a, F, Err, R>( - &'a self, - func: F, - ) -> Result<(), Errors> + pub async fn request_proposers_first(&self, func: F) -> Result<(), Errors> where F: Fn(BeaconNodeHttpClient) -> R + Clone, R: Future>, @@ -165,10 +162,7 @@ impl ProposerFallback { } // Try `func` on `self.beacon_nodes` first. If that doesn't work, try `self.proposer_nodes`. - pub async fn request_proposers_last<'a, F, O, Err, R>( - &'a self, - func: F, - ) -> Result> + pub async fn request_proposers_last(&self, func: F) -> Result> where F: Fn(BeaconNodeHttpClient) -> R + Clone, R: Future>, From fff8637c344d27740d4c14042aff6e9e61c8a94a Mon Sep 17 00:00:00 2001 From: Mac L Date: Wed, 11 Sep 2024 18:51:27 +1000 Subject: [PATCH 48/50] Revert "Don't have block_service as an Option" This reverts commit b5445a09e9f0942ecd561575367f0d1f120415f6. --- validator_client/src/http_api/mod.rs | 12 ++++++++++-- validator_client/src/http_api/test_utils.rs | 5 +---- validator_client/src/http_api/tests.rs | 5 +---- validator_client/src/lib.rs | 2 +- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/validator_client/src/http_api/mod.rs b/validator_client/src/http_api/mod.rs index f98f3c7db58..cffbcd62a5c 100644 --- a/validator_client/src/http_api/mod.rs +++ b/validator_client/src/http_api/mod.rs @@ -73,7 +73,7 @@ impl From for Error { pub struct Context { pub task_executor: TaskExecutor, pub api_secret: ApiSecret, - pub block_service: BlockService, + pub block_service: Option>, pub validator_store: Option>>, pub validator_dir: Option, pub secrets_dir: Option, @@ -172,7 +172,15 @@ pub fn serve( }; let inner_block_service = ctx.block_service.clone(); - let block_service_filter = warp::any().map(move || inner_block_service.clone()); + let block_service_filter = warp::any() + .map(move || inner_block_service.clone()) + .and_then(|block_service: Option<_>| async move { + block_service.ok_or_else(|| { + warp_utils::reject::custom_not_found( + "block service is not initialized.".to_string(), + ) + }) + }); let inner_validator_store = ctx.validator_store.clone(); let validator_store_filter = warp::any() diff --git a/validator_client/src/http_api/test_utils.rs b/validator_client/src/http_api/test_utils.rs index 8885160c6c2..d135dfc2720 100644 --- a/validator_client/src/http_api/test_utils.rs +++ b/validator_client/src/http_api/test_utils.rs @@ -1,6 +1,5 @@ use crate::doppelganger_service::DoppelgangerService; use crate::key_cache::{KeyCache, CACHE_FILENAME}; -use crate::BlockServiceBuilder; use crate::{ http_api::{ApiSecret, Config as HttpConfig, Context}, initialized_validators::{InitializedValidators, OnDecryptFailure}, @@ -128,9 +127,7 @@ impl ApiTester { let context = Arc::new(Context { task_executor: test_runtime.task_executor.clone(), api_secret, - block_service: BlockServiceBuilder::new() - .build() - .expect("Should build block service"), + block_service: None, validator_dir: Some(validator_dir.path().into()), secrets_dir: Some(secrets_dir.path().into()), validator_store: Some(validator_store.clone()), diff --git a/validator_client/src/http_api/tests.rs b/validator_client/src/http_api/tests.rs index a64aa3ad5ba..f6da4f76e7a 100644 --- a/validator_client/src/http_api/tests.rs +++ b/validator_client/src/http_api/tests.rs @@ -4,7 +4,6 @@ mod keystores; use crate::doppelganger_service::DoppelgangerService; -use crate::BlockServiceBuilder; use crate::{ http_api::{ApiSecret, Config as HttpConfig, Context}, initialized_validators::InitializedValidators, @@ -116,9 +115,7 @@ impl ApiTester { let context = Arc::new(Context { task_executor: test_runtime.task_executor.clone(), api_secret, - block_service: BlockServiceBuilder::new() - .build() - .expect("Should build block service"), + block_service: None, validator_dir: Some(validator_dir.path().into()), secrets_dir: Some(secrets_dir.path().into()), validator_store: Some(validator_store.clone()), diff --git a/validator_client/src/lib.rs b/validator_client/src/lib.rs index 548fe0d6788..9a02ffdefb3 100644 --- a/validator_client/src/lib.rs +++ b/validator_client/src/lib.rs @@ -572,7 +572,7 @@ impl ProductionValidatorClient { let ctx = Arc::new(http_api::Context { task_executor: self.context.executor.clone(), api_secret, - block_service: self.block_service.clone(), + block_service: Some(self.block_service.clone()), validator_store: Some(self.validator_store.clone()), validator_dir: Some(self.config.validator_dir.clone()), secrets_dir: Some(self.config.secrets_dir.clone()), From d66b06ac0a5f0e57479b237fc7359ce7d76b22bc Mon Sep 17 00:00:00 2001 From: Mac L Date: Thu, 3 Oct 2024 08:00:35 +1000 Subject: [PATCH 49/50] Improve rwlock sanitation using clones --- validator_client/src/beacon_node_fallback.rs | 60 +++++++++++++++----- 1 file changed, 47 insertions(+), 13 deletions(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 7343bf80c38..47b197d58df 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -436,7 +436,7 @@ impl BeaconNodeFallback { for candidate in candidates.iter() { let health = candidate.health().await; - match candidate.health().await { + match health { Ok(health) => { if self .distance_tiers @@ -510,8 +510,9 @@ impl BeaconNodeFallback { pub async fn measure_latency(&self) -> Vec { let candidates = self.candidates.read().await; let futures: Vec<_> = candidates - .iter() - .map(|candidate| async { + .clone() + .into_iter() + .map(|candidate| async move { let beacon_node_id = candidate.beacon_node.to_string(); // The `node/version` endpoint is used since I imagine it would // require the least processing in the BN and therefore measure @@ -528,6 +529,7 @@ impl BeaconNodeFallback { (beacon_node_id, response_instant) }) .collect(); + drop(candidates); let request_instant = Instant::now(); @@ -553,12 +555,25 @@ impl BeaconNodeFallback { R: Future>, Err: Debug, { + let mut errors = vec![]; + // First pass: try `func` on all candidates. Candidate order has already been set in // `update_all_candidates`. This ensures the most suitable node is always tried first. let candidates = self.candidates.read().await; - let mut errors = vec![]; + let mut futures = vec![]; + + // Run `func` using a `candidate`, returning the value or capturing errors. for candidate in candidates.iter() { - match Self::run_on_candidate(candidate, &func, &self.log).await { + futures.push(Self::run_on_candidate( + candidate.beacon_node.clone(), + &func, + &self.log, + )); + } + drop(candidates); + + for future in futures { + match future.await { Ok(val) => return Ok(val), Err(e) => errors.push(e), } @@ -566,8 +581,21 @@ impl BeaconNodeFallback { // Second pass. No candidates returned successfully. Try again with the same order. // This will duplicate errors. + let candidates = self.candidates.read().await; + let mut futures = vec![]; + + // Run `func` using a `candidate`, returning the value or capturing errors. for candidate in candidates.iter() { - match Self::run_on_candidate(candidate, &func, &self.log).await { + futures.push(Self::run_on_candidate( + candidate.beacon_node.clone(), + &func, + &self.log, + )); + } + drop(candidates); + + for future in futures { + match future.await { Ok(val) => return Ok(val), Err(e) => errors.push(e), } @@ -579,7 +607,7 @@ impl BeaconNodeFallback { /// Run the future `func` on `candidate` while reporting metrics. async fn run_on_candidate( - candidate: &CandidateBeaconNode, + candidate: BeaconNodeHttpClient, func: F, log: &Logger, ) -> Result)> @@ -588,21 +616,21 @@ impl BeaconNodeFallback { R: Future>, Err: Debug, { - inc_counter_vec(&ENDPOINT_REQUESTS, &[candidate.beacon_node.as_ref()]); + inc_counter_vec(&ENDPOINT_REQUESTS, &[candidate.as_ref()]); // There exists a race condition where `func` may be called when the candidate is // actually not ready. We deem this an acceptable inefficiency. - match func(candidate.beacon_node.clone()).await { + match func(candidate.clone()).await { Ok(val) => Ok(val), Err(e) => { debug!( log, "Request to beacon node failed"; - "node" => %candidate.beacon_node, + "node" => %candidate, "error" => ?e, ); - inc_counter_vec(&ENDPOINT_ERRORS, &[candidate.beacon_node.as_ref()]); - Err((candidate.beacon_node.to_string(), Error::RequestFailed(e))) + inc_counter_vec(&ENDPOINT_ERRORS, &[candidate.as_ref()]); + Err((candidate.to_string(), Error::RequestFailed(e))) } } } @@ -626,8 +654,14 @@ impl BeaconNodeFallback { // Run `func` using a `candidate`, returning the value or capturing errors. for candidate in candidates.iter() { - futures.push(Self::run_on_candidate(candidate, &func, &self.log)); + futures.push(Self::run_on_candidate( + candidate.beacon_node.clone(), + &func, + &self.log, + )); } + drop(candidates); + let results = future::join_all(futures).await; let errors: Vec<_> = results.into_iter().filter_map(|res| res.err()).collect(); From 30ddfa600d31a8041192488d7a9101a122cd809b Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 3 Oct 2024 14:50:01 +1000 Subject: [PATCH 50/50] Drop read lock immediately by cloning the vec. --- validator_client/src/beacon_node_fallback.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/validator_client/src/beacon_node_fallback.rs b/validator_client/src/beacon_node_fallback.rs index 47b197d58df..e5fe419983a 100644 --- a/validator_client/src/beacon_node_fallback.rs +++ b/validator_client/src/beacon_node_fallback.rs @@ -467,7 +467,9 @@ impl BeaconNodeFallback { /// low quality responses. To route around this it's best to poll all connected beacon nodes. /// A previous implementation of this function polled only the unavailable BNs. pub async fn update_all_candidates(&self) { - let candidates = self.candidates.read().await; + // Clone the vec, so we release the read lock immediately. + // `candidate.health` is behind an Arc, so this would still allow us to mutate the values. + let candidates = self.candidates.read().await.clone(); let mut futures = Vec::with_capacity(candidates.len()); let mut nodes = Vec::with_capacity(candidates.len());