From b97fe50f0717fac88cbecf6d6e134f0782b96af6 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 10:47:11 -0400 Subject: [PATCH 01/39] Adding networking configurations into ChainSpec --- consensus/types/src/chain_spec.rs | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 2b25cc1d593..889ee110331 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -173,6 +173,18 @@ pub struct ChainSpec { pub subnets_per_node: u8, pub epochs_per_subnet_subscription: u64, attestation_subnet_extra_bits: u8, + // Shift networking configuration into configs + pub gossip_max_size: usize, + pub gossip_max_size_post_merge: usize, + pub max_request_blocks: u64, + // min_epochs_for_block_requests?? + pub max_chunk_size: usize, + pub ttfb_timeout: u64, + pub resp_timeout: u64, + // attestation_propagation_slot_range?? + // maximum_gossip_clock_disparity?? + // message_domain_invalid_snappy?? + pub message_domain_valid_snappy: [u8; 4], /* * Application params @@ -632,6 +644,13 @@ impl ChainSpec { epochs_per_random_subnet_subscription: 256, epochs_per_subnet_subscription: 256, attestation_subnet_extra_bits: 6, + gossip_max_size: 1_048_576, + gossip_max_size_post_merge: 10 * 1_048_576, + max_request_blocks: 1024, + max_chunk_size: 1_048_576, + ttfb_timeout: 5, + resp_timeout: 10, + message_domain_valid_snappy: [1, 0, 0, 0], /* * Application specific @@ -859,6 +878,13 @@ impl ChainSpec { epochs_per_random_subnet_subscription: 256, epochs_per_subnet_subscription: 256, attestation_subnet_extra_bits: 6, + gossip_max_size: 1_048_576, + gossip_max_size_post_merge: 10 * 1_048_576, + max_request_blocks: 1024, + max_chunk_size: 1_048_576, + ttfb_timeout: 5, + resp_timeout: 10, + message_domain_valid_snappy: [1, 0, 0, 0], /* * Application specific From 18c02d2a6bf832c6b14a0f79d404b3b426841b99 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 15:24:29 -0400 Subject: [PATCH 02/39] Finishing gossip_max_size --- beacon_node/lighthouse_network/src/config.rs | 18 +++++++----------- .../lighthouse_network/src/service/mod.rs | 2 +- beacon_node/src/config.rs | 2 +- consensus/types/src/chain_spec.rs | 3 --- 4 files changed, 9 insertions(+), 16 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 5a11890a263..86f4fd7df4c 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -17,12 +17,7 @@ use std::net::{Ipv4Addr, Ipv6Addr}; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; -use types::{ForkContext, ForkName}; - -/// The maximum transmit size of gossip messages in bytes pre-merge. -const GOSSIP_MAX_SIZE: usize = 1_048_576; // 1M -/// The maximum transmit size of gossip messages in bytes post-merge. -const GOSSIP_MAX_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M +use types::{ForkContext, ForkName, EthSpec}; /// The cache time is set to accommodate the circulation time of an attestation. /// @@ -44,11 +39,12 @@ pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); const MESSAGE_DOMAIN_VALID_SNAPPY: [u8; 4] = [1, 0, 0, 0]; /// The maximum size of gossip messages. -pub fn gossip_max_size(is_merge_enabled: bool) -> usize { +pub fn gossip_max_size(is_merge_enabled: bool) -> usize { + let gossip_max_size = TSpec::default_spec().gossip_max_size; if is_merge_enabled { - GOSSIP_MAX_SIZE_POST_MERGE + 10 * gossip_max_size } else { - GOSSIP_MAX_SIZE + gossip_max_size } } @@ -408,7 +404,7 @@ impl From for NetworkLoad { } /// Return a Lighthouse specific `GossipsubConfig` where the `message_id_fn` depends on the current fork. -pub fn gossipsub_config(network_load: u8, fork_context: Arc) -> GossipsubConfig { +pub fn gossipsub_config(network_load: u8, fork_context: Arc) -> GossipsubConfig { // The function used to generate a gossipsub message id // We use the first 8 bytes of SHA256(topic, data) for content addressing let fast_gossip_message_id = |message: &RawGossipsubMessage| { @@ -454,7 +450,7 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc) -> Gos let load = NetworkLoad::from(network_load); GossipsubConfigBuilder::default() - .max_transmit_size(gossip_max_size(is_merge_enabled)) + .max_transmit_size(gossip_max_size::(is_merge_enabled)) .heartbeat_interval(load.heartbeat_interval) .mesh_n(load.mesh_n) .mesh_n_low(load.mesh_n_low) diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index f815e3bd36b..cfd00bcb604 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -239,7 +239,7 @@ impl Network { max_subscriptions_per_request: 150, // 148 in theory = (64 attestation + 4 sync committee + 6 core topics) * 2 }; - config.gs_config = gossipsub_config(config.network_load, ctx.fork_context.clone()); + config.gs_config = gossipsub_config::(config.network_load, ctx.fork_context.clone()); // If metrics are enabled for gossipsub build the configuration let gossipsub_metrics = ctx diff --git a/beacon_node/src/config.rs b/beacon_node/src/config.rs index 6f626bee8d8..38616efc767 100644 --- a/beacon_node/src/config.rs +++ b/beacon_node/src/config.rs @@ -576,7 +576,7 @@ pub fn get_config( } client_config.chain.max_network_size = - lighthouse_network::gossip_max_size(spec.bellatrix_fork_epoch.is_some()); + lighthouse_network::gossip_max_size::(spec.bellatrix_fork_epoch.is_some()); if cli_args.is_present("slasher") { let slasher_dir = if let Some(slasher_dir) = cli_args.value_of("slasher-dir") { diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 889ee110331..52b9dc38bfa 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -175,7 +175,6 @@ pub struct ChainSpec { attestation_subnet_extra_bits: u8, // Shift networking configuration into configs pub gossip_max_size: usize, - pub gossip_max_size_post_merge: usize, pub max_request_blocks: u64, // min_epochs_for_block_requests?? pub max_chunk_size: usize, @@ -645,7 +644,6 @@ impl ChainSpec { epochs_per_subnet_subscription: 256, attestation_subnet_extra_bits: 6, gossip_max_size: 1_048_576, - gossip_max_size_post_merge: 10 * 1_048_576, max_request_blocks: 1024, max_chunk_size: 1_048_576, ttfb_timeout: 5, @@ -879,7 +877,6 @@ impl ChainSpec { epochs_per_subnet_subscription: 256, attestation_subnet_extra_bits: 6, gossip_max_size: 1_048_576, - gossip_max_size_post_merge: 10 * 1_048_576, max_request_blocks: 1024, max_chunk_size: 1_048_576, ttfb_timeout: 5, From 3c770a7a55ddd394e686b3f40448ecda306985ce Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 16:32:59 -0400 Subject: [PATCH 03/39] Finishing max_request_blocks? --- beacon_node/lighthouse_network/src/rpc/config.rs | 4 ++-- beacon_node/lighthouse_network/src/rpc/methods.rs | 1 - beacon_node/lighthouse_network/src/rpc/mod.rs | 4 ++-- beacon_node/lighthouse_network/src/rpc/protocol.rs | 4 ++-- .../network/src/beacon_processor/worker/rpc_methods.rs | 4 ++-- beacon_node/network/src/sync/manager.rs | 3 +-- 6 files changed, 9 insertions(+), 11 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/config.rs b/beacon_node/lighthouse_network/src/rpc/config.rs index bea0929fb0b..c158f405dae 100644 --- a/beacon_node/lighthouse_network/src/rpc/config.rs +++ b/beacon_node/lighthouse_network/src/rpc/config.rs @@ -4,7 +4,7 @@ use std::{ time::Duration, }; -use super::{methods, rate_limiter::Quota, Protocol}; +use super::{rate_limiter::Quota, Protocol}; use serde_derive::{Deserialize, Serialize}; @@ -75,7 +75,7 @@ impl OutboundRateLimiterConfig { pub const DEFAULT_STATUS_QUOTA: Quota = Quota::n_every(5, 15); pub const DEFAULT_GOODBYE_QUOTA: Quota = Quota::one_every(10); pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota = - Quota::n_every(methods::MAX_REQUEST_BLOCKS, 10); + Quota::n_every(1024, 10); pub const DEFAULT_BLOCKS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10); } diff --git a/beacon_node/lighthouse_network/src/rpc/methods.rs b/beacon_node/lighthouse_network/src/rpc/methods.rs index 5da595c3db7..87b6254f492 100644 --- a/beacon_node/lighthouse_network/src/rpc/methods.rs +++ b/beacon_node/lighthouse_network/src/rpc/methods.rs @@ -18,7 +18,6 @@ use types::{ /// Maximum number of blocks in a single request. pub type MaxRequestBlocks = U1024; -pub const MAX_REQUEST_BLOCKS: u64 = 1024; /// Maximum length of error message. pub type MaxErrorLen = U256; diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 31569b820b1..6298e6df1ca 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -27,7 +27,7 @@ pub(crate) use protocol::{InboundRequest, RPCProtocol}; pub use handler::SubstreamId; pub use methods::{ BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, LightClientBootstrapRequest, - MaxRequestBlocks, RPCResponseErrorCode, ResponseTermination, StatusMessage, MAX_REQUEST_BLOCKS, + MaxRequestBlocks, RPCResponseErrorCode, ResponseTermination, StatusMessage, }; pub(crate) use outbound::OutboundRequest; pub use protocol::{max_rpc_size, Protocol, RPCError}; @@ -140,7 +140,7 @@ impl RPC { .one_every(Protocol::LightClientBootstrap, Duration::from_secs(10)) .n_every( Protocol::BlocksByRange, - methods::MAX_REQUEST_BLOCKS, + TSpec::default_spec().max_request_blocks, Duration::from_secs(10), ) .n_every(Protocol::BlocksByRoot, 128, Duration::from_secs(10)) diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index a8423e47b0b..a895da44a4b 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -2,7 +2,7 @@ use super::methods::*; use crate::rpc::{ codec::{base::BaseInboundCodec, ssz_snappy::SSZSnappyInboundCodec, InboundCodec}, methods::{MaxErrorLen, ResponseTermination, MAX_ERROR_LEN}, - MaxRequestBlocks, MAX_REQUEST_BLOCKS, + MaxRequestBlocks, }; use futures::future::BoxFuture; use futures::prelude::{AsyncRead, AsyncWrite}; @@ -90,7 +90,7 @@ lazy_static! { pub static ref BLOCKS_BY_ROOT_REQUEST_MAX: usize = VariableList::::from(vec![ Hash256::zero(); - MAX_REQUEST_BLOCKS + MainnetEthSpec::default_spec().max_request_blocks as usize ]) .as_ssz_bytes() diff --git a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs b/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs index 81b163bf7ee..3974c3d39ac 100644 --- a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs @@ -297,8 +297,8 @@ impl Worker { ); // Should not send more than max request blocks - if req.count > MAX_REQUEST_BLOCKS { - req.count = MAX_REQUEST_BLOCKS; + if req.count > T::EthSpec::default_spec().max_request_blocks { + req.count = T::EthSpec::default_spec().max_request_blocks; } let forwards_block_root_iter = match self diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 230c883a937..db1c0d5eab7 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -43,7 +43,6 @@ use crate::service::NetworkMessage; use crate::status::ToStatusMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, EngineState}; use futures::StreamExt; -use lighthouse_network::rpc::methods::MAX_REQUEST_BLOCKS; use lighthouse_network::types::{NetworkGlobals, SyncState}; use lighthouse_network::SyncInfo; use lighthouse_network::{PeerAction, PeerId}; @@ -192,7 +191,7 @@ pub fn spawn( log: slog::Logger, ) -> mpsc::UnboundedSender> { assert!( - MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, + T::EthSpec::default_spec().max_request_blocks >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, "Max blocks that can be requested in a single batch greater than max allowed blocks in a single request" ); // generate the message channel From e01ab7b59cca5b0cce4a382ee1a0fdf717ddce4f Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 16:59:59 -0400 Subject: [PATCH 04/39] Finishing max_chunk_size --- beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs | 4 ++-- beacon_node/lighthouse_network/src/rpc/protocol.rs | 4 +--- .../network/src/beacon_processor/worker/rpc_methods.rs | 5 +++-- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index 28fea40a20d..b8347b1aa15 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -661,7 +661,7 @@ mod tests { use std::sync::Arc; use types::{ BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EmptyBlock, Epoch, - ForkContext, FullPayload, Hash256, Signature, SignedBeaconBlock, Slot, + ForkContext, FullPayload, Hash256, Signature, SignedBeaconBlock, Slot, MainnetEthSpec, }; use snap::write::FrameEncoder; @@ -1522,7 +1522,7 @@ mod tests { let mut dst = BytesMut::with_capacity(1024); // Insert length-prefix - uvi_codec.encode(MAX_RPC_SIZE + 1, &mut dst).unwrap(); + uvi_codec.encode(MainnetEthSpec::default_spec().max_chunk_size + 1, &mut dst).unwrap(); // Insert snappy stream identifier dst.extend_from_slice(stream_identifier); diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index a895da44a4b..ceffae081bd 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -109,8 +109,6 @@ lazy_static! { .len(); } -/// The maximum bytes that can be sent across the RPC pre-merge. -pub(crate) const MAX_RPC_SIZE: usize = 1_048_576; // 1M /// The maximum bytes that can be sent across the RPC post-merge. pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M pub(crate) const MAX_RPC_SIZE_POST_CAPELLA: usize = 10 * 1_048_576; // 10M @@ -125,7 +123,7 @@ const REQUEST_TIMEOUT: u64 = 15; /// Returns the maximum bytes that can be sent across the RPC. pub fn max_rpc_size(fork_context: &ForkContext) -> usize { match fork_context.current_fork() { - ForkName::Altair | ForkName::Base => MAX_RPC_SIZE, + ForkName::Altair | ForkName::Base => MainnetEthSpec::default_spec().max_chunk_size, ForkName::Merge => MAX_RPC_SIZE_POST_MERGE, ForkName::Capella => MAX_RPC_SIZE_POST_CAPELLA, } diff --git a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs b/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs index 3974c3d39ac..fe1d12f9ded 100644 --- a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs @@ -297,8 +297,9 @@ impl Worker { ); // Should not send more than max request blocks - if req.count > T::EthSpec::default_spec().max_request_blocks { - req.count = T::EthSpec::default_spec().max_request_blocks; + let max_request_blocks = T::EthSpec::default_spec().max_request_blocks; + if req.count > max_request_blocks { + req.count = max_request_blocks; } let forwards_block_root_iter = match self From e8f86cdfc29d6698f19e3acbd05ffd1bdea1fa9b Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 17:07:58 -0400 Subject: [PATCH 05/39] Finishing ttfb_timeout --- beacon_node/lighthouse_network/src/rpc/protocol.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index ceffae081bd..8b76b640720 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -114,8 +114,6 @@ pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M pub(crate) const MAX_RPC_SIZE_POST_CAPELLA: usize = 10 * 1_048_576; // 10M /// The protocol prefix the RPC protocol id. const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req"; -/// Time allowed for the first byte of a request to arrive before we time out (Time To First Byte). -const TTFB_TIMEOUT: u64 = 5; /// The number of seconds to wait for the first bytes of a request once a protocol has been /// established before the stream is terminated. const REQUEST_TIMEOUT: u64 = 15; @@ -412,7 +410,7 @@ where } }; let mut timed_socket = TimeoutStream::new(socket); - timed_socket.set_read_timeout(Some(Duration::from_secs(TTFB_TIMEOUT))); + timed_socket.set_read_timeout(Some(Duration::from_secs(TSpec::default_spec().ttfb_timeout))); let socket = Framed::new(Box::pin(timed_socket), codec); From 2cccf6a5d02f6ae988a123c49d93cf2574597631 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 17:13:06 -0400 Subject: [PATCH 06/39] Finishing resp_timeout --- beacon_node/lighthouse_network/src/rpc/handler.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index a1743c15fb6..3dce47582ab 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -31,9 +31,6 @@ use tokio::time::{sleep_until, Instant as TInstant, Sleep}; use tokio_util::time::{delay_queue, DelayQueue}; use types::{EthSpec, ForkContext}; -/// The time (in seconds) before a substream that is awaiting a response from the user times out. -pub const RESPONSE_TIMEOUT: u64 = 10; - /// The number of times to retry an outbound upgrade in the case of IO errors. const IO_ERROR_RETRIES: u8 = 3; @@ -351,7 +348,7 @@ where // new outbound request. Store the stream and tag the output. let delay_key = self.outbound_substreams_delay.insert( self.current_outbound_substream_id, - Duration::from_secs(RESPONSE_TIMEOUT), + Duration::from_secs(TSpec::default_spec().resp_timeout), ); let awaiting_stream = OutboundSubstreamState::RequestPendingResponse { substream: Box::new(out), @@ -402,7 +399,7 @@ where // Store the stream and tag the output. let delay_key = self.inbound_substreams_delay.insert( self.current_inbound_substream_id, - Duration::from_secs(RESPONSE_TIMEOUT), + Duration::from_secs(TSpec::default_spec().resp_timeout), ); let awaiting_stream = InboundState::Idle(substream); self.inbound_substreams.insert( @@ -714,7 +711,7 @@ where // Each chunk is allowed RESPONSE_TIMEOUT to be sent. if let Some(ref delay_key) = info.delay_key { self.inbound_substreams_delay - .reset(delay_key, Duration::from_secs(RESPONSE_TIMEOUT)); + .reset(delay_key, Duration::from_secs(TSpec::default_spec().resp_timeout)); } // The stream may be currently idle. Attempt to process more @@ -848,7 +845,7 @@ where }; substream_entry.remaining_chunks = Some(remaining_chunks); self.outbound_substreams_delay - .reset(delay_key, Duration::from_secs(RESPONSE_TIMEOUT)); + .reset(delay_key, Duration::from_secs(TSpec::default_spec().resp_timeout)); } } else { // either this is a single response request or this response closes the From 72f92afc03532ca7fd765bc7b6dc23b191cbf95b Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 17:17:56 -0400 Subject: [PATCH 07/39] Finishing message_domain_valid_snappy --- beacon_node/lighthouse_network/src/config.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 86f4fd7df4c..33a6b138497 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -36,7 +36,6 @@ pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); // We treat uncompressed messages as invalid and never use the INVALID_SNAPPY_DOMAIN as in the // specification. We leave it here for posterity. // const MESSAGE_DOMAIN_INVALID_SNAPPY: [u8; 4] = [0, 0, 0, 0]; -const MESSAGE_DOMAIN_VALID_SNAPPY: [u8; 4] = [1, 0, 0, 0]; /// The maximum size of gossip messages. pub fn gossip_max_size(is_merge_enabled: bool) -> usize { @@ -442,7 +441,7 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc Date: Wed, 21 Jun 2023 17:48:07 -0400 Subject: [PATCH 08/39] Finishing maximum_gossip_clock_disparity? --- .../beacon_chain/src/attestation_verification.rs | 10 +++++----- beacon_node/beacon_chain/src/beacon_chain.rs | 5 ----- beacon_node/beacon_chain/src/block_verification.rs | 4 ++-- beacon_node/beacon_chain/src/lib.rs | 2 +- .../src/light_client_finality_update_verification.rs | 6 +++--- .../light_client_optimistic_update_verification.rs | 6 +++--- .../beacon_chain/src/sync_committee_verification.rs | 12 ++++++------ consensus/types/src/chain_spec.rs | 1 - 8 files changed, 20 insertions(+), 26 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index 04f601fad97..b4ac5e2f8a0 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -35,7 +35,7 @@ mod batch; use crate::{ - beacon_chain::{MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT}, + beacon_chain::VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, metrics, observed_aggregates::ObserveOutcome, observed_attesters::Error as ObservedAttestersError, @@ -53,7 +53,7 @@ use state_processing::{ signed_aggregate_selection_proof_signature_set, signed_aggregate_signature_set, }, }; -use std::borrow::Cow; +use std::{borrow::Cow, time::Duration}; use strum::AsRefStr; use tree_hash::TreeHash; use types::{ @@ -1035,9 +1035,9 @@ pub fn verify_propagation_slot_range( attestation: &Attestation, ) -> Result<(), Error> { let attestation_slot = attestation.data.slot; - + let maximum_gossip_clock_disparity = Duration::from_millis(E::default_spec().maximum_gossip_clock_disparity_millis); let latest_permissible_slot = slot_clock - .now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_future_tolerance(maximum_gossip_clock_disparity) .ok_or(BeaconChainError::UnableToReadSlot)?; if attestation_slot > latest_permissible_slot { return Err(Error::FutureSlot { @@ -1048,7 +1048,7 @@ pub fn verify_propagation_slot_range( // Taking advantage of saturating subtraction on `Slot`. let earliest_permissible_slot = slot_clock - .now_with_past_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_past_tolerance(maximum_gossip_clock_disparity) .ok_or(BeaconChainError::UnableToReadSlot)? - E::slots_per_epoch(); if attestation_slot < earliest_permissible_slot { diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 2fa04304f59..9f7fe4004de 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -207,11 +207,6 @@ pub enum OverrideForkchoiceUpdate { AlreadyApplied, } -/// The accepted clock drift for nodes gossiping blocks and attestations. See: -/// -/// https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/p2p-interface.md#configuration -pub const MAXIMUM_GOSSIP_CLOCK_DISPARITY: Duration = Duration::from_millis(500); - #[derive(Debug, PartialEq)] pub enum AttestationProcessingOutcome { Processed, diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index ca4df864db9..defeac3f32e 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -58,7 +58,7 @@ use crate::validator_pubkey_cache::ValidatorPubkeyCache; use crate::{ beacon_chain::{ BeaconForkChoice, ForkChoiceError, BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, - MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, + VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, }, metrics, BeaconChain, BeaconChainError, BeaconChainTypes, }; @@ -697,7 +697,7 @@ impl GossipVerifiedBlock { // Do not gossip or process blocks from future slots. let present_slot_with_tolerance = chain .slot_clock - .now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) .ok_or(BeaconChainError::UnableToReadSlot)?; if block.slot() > present_slot_with_tolerance { return Err(BlockError::FutureSlot { diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index be1522a3b80..c5d719b3fdb 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -54,7 +54,7 @@ pub use self::beacon_chain::{ AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BeaconStore, ChainSegmentResult, CountUnrealized, ForkChoiceError, OverrideForkchoiceUpdate, ProduceBlockVerification, StateSkipConfig, WhenSlotSkipped, INVALID_FINALIZED_MERGE_TRANSITION_BLOCK_SHUTDOWN_REASON, - INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON, MAXIMUM_GOSSIP_CLOCK_DISPARITY, + INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON, }; pub use self::beacon_snapshot::BeaconSnapshot; pub use self::chain_config::ChainConfig; diff --git a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs index 7c431ebccca..db6db018699 100644 --- a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs @@ -1,12 +1,12 @@ use crate::{ - beacon_chain::MAXIMUM_GOSSIP_CLOCK_DISPARITY, BeaconChain, BeaconChainError, BeaconChainTypes, + BeaconChain, BeaconChainError, BeaconChainTypes, }; use derivative::Derivative; use slot_clock::SlotClock; use std::time::Duration; use strum::AsRefStr; use types::{ - light_client_update::Error as LightClientUpdateError, LightClientFinalityUpdate, Slot, + light_client_update::Error as LightClientUpdateError, LightClientFinalityUpdate, Slot, EthSpec, }; /// Returned when a light client finality update was not successfully verified. It might not have been verified for @@ -103,7 +103,7 @@ impl VerifiedLightClientFinalityUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp + MAXIMUM_GOSSIP_CLOCK_DISPARITY < time + one_third_slot_duration + if seen_timestamp + Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis) < time + one_third_slot_duration { return Err(Error::TooEarly); } diff --git a/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs b/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs index 20d7181808a..51f95e274a2 100644 --- a/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs @@ -1,5 +1,5 @@ use crate::{ - beacon_chain::MAXIMUM_GOSSIP_CLOCK_DISPARITY, BeaconChain, BeaconChainError, BeaconChainTypes, + BeaconChain, BeaconChainError, BeaconChainTypes, }; use derivative::Derivative; use eth2::types::Hash256; @@ -7,7 +7,7 @@ use slot_clock::SlotClock; use std::time::Duration; use strum::AsRefStr; use types::{ - light_client_update::Error as LightClientUpdateError, LightClientOptimisticUpdate, Slot, + light_client_update::Error as LightClientUpdateError, LightClientOptimisticUpdate, Slot, EthSpec, }; /// Returned when a light client optimistic update was not successfully verified. It might not have been verified for @@ -103,7 +103,7 @@ impl VerifiedLightClientOptimisticUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp + MAXIMUM_GOSSIP_CLOCK_DISPARITY < time + one_third_slot_duration + if seen_timestamp + Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis) < time + one_third_slot_duration { return Err(Error::TooEarly); } diff --git a/beacon_node/beacon_chain/src/sync_committee_verification.rs b/beacon_node/beacon_chain/src/sync_committee_verification.rs index 14cdc2400d8..8c107c16af9 100644 --- a/beacon_node/beacon_chain/src/sync_committee_verification.rs +++ b/beacon_node/beacon_chain/src/sync_committee_verification.rs @@ -28,7 +28,7 @@ use crate::observed_attesters::SlotSubcommitteeIndex; use crate::{ - beacon_chain::{MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT}, + beacon_chain::VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, metrics, observed_aggregates::ObserveOutcome, BeaconChain, BeaconChainError, BeaconChainTypes, @@ -43,7 +43,7 @@ use state_processing::signature_sets::{ sync_committee_contribution_signature_set_from_pubkeys, sync_committee_message_set_from_pubkeys, }; -use std::borrow::Cow; +use std::{borrow::Cow, time::Duration}; use std::collections::HashMap; use strum::AsRefStr; use tree_hash::TreeHash; @@ -53,7 +53,7 @@ use types::sync_committee::Error as SyncCommitteeError; use types::{ sync_committee_contribution::Error as ContributionError, AggregateSignature, BeaconStateError, EthSpec, Hash256, SignedContributionAndProof, Slot, SyncCommitteeContribution, - SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, + SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, MainnetEthSpec, }; /// Returned when a sync committee contribution was not successfully verified. It might not have been verified for @@ -560,9 +560,9 @@ pub fn verify_propagation_slot_range( sync_contribution: &U, ) -> Result<(), Error> { let message_slot = sync_contribution.get_slot(); - + let maximum_gossip_clock_disparity = Duration::from_millis(MainnetEthSpec::default_spec().maximum_gossip_clock_disparity_millis); let latest_permissible_slot = slot_clock - .now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_future_tolerance(maximum_gossip_clock_disparity) .ok_or(BeaconChainError::UnableToReadSlot)?; if message_slot > latest_permissible_slot { return Err(Error::FutureSlot { @@ -572,7 +572,7 @@ pub fn verify_propagation_slot_range( } let earliest_permissible_slot = slot_clock - .now_with_past_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_past_tolerance(maximum_gossip_clock_disparity) .ok_or(BeaconChainError::UnableToReadSlot)?; if message_slot < earliest_permissible_slot { diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 52b9dc38bfa..33cabceee9e 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -181,7 +181,6 @@ pub struct ChainSpec { pub ttfb_timeout: u64, pub resp_timeout: u64, // attestation_propagation_slot_range?? - // maximum_gossip_clock_disparity?? // message_domain_invalid_snappy?? pub message_domain_valid_snappy: [u8; 4], From 5e28c2ebd38f8a907da9d50c9aa7ca4a5cdbe302 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 21 Jun 2023 17:52:55 -0400 Subject: [PATCH 09/39] Finishing maximum_gossip_clock_disparity --- beacon_node/http_api/src/attester_duties.rs | 5 +++-- beacon_node/http_api/src/proposer_duties.rs | 6 +++--- beacon_node/http_api/src/sync_committees.rs | 5 +++-- .../network/src/beacon_processor/work_reprocessing_queue.rs | 4 ++-- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/beacon_node/http_api/src/attester_duties.rs b/beacon_node/http_api/src/attester_duties.rs index 5c3e420839d..e19b6c27295 100644 --- a/beacon_node/http_api/src/attester_duties.rs +++ b/beacon_node/http_api/src/attester_duties.rs @@ -2,7 +2,7 @@ use crate::state_id::StateId; use beacon_chain::{ - BeaconChain, BeaconChainError, BeaconChainTypes, MAXIMUM_GOSSIP_CLOCK_DISPARITY, + BeaconChain, BeaconChainError, BeaconChainTypes, }; use eth2::types::{self as api_types}; use slot_clock::SlotClock; @@ -10,6 +10,7 @@ use state_processing::state_advance::partial_state_advance; use types::{ AttestationDuty, BeaconState, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256, RelativeEpoch, }; +use std::time::Duration; /// The struct that is returned to the requesting HTTP client. type ApiDuties = api_types::DutiesResponse>; @@ -32,7 +33,7 @@ pub fn attester_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/proposer_duties.rs b/beacon_node/http_api/src/proposer_duties.rs index 7e946b89e72..6a6a8bc8853 100644 --- a/beacon_node/http_api/src/proposer_duties.rs +++ b/beacon_node/http_api/src/proposer_duties.rs @@ -3,13 +3,13 @@ use crate::state_id::StateId; use beacon_chain::{ beacon_proposer_cache::{compute_proposer_duties_from_head, ensure_state_is_in_epoch}, - BeaconChain, BeaconChainError, BeaconChainTypes, MAXIMUM_GOSSIP_CLOCK_DISPARITY, + BeaconChain, BeaconChainError, BeaconChainTypes, }; use eth2::types::{self as api_types}; use safe_arith::SafeArith; use slog::{debug, Logger}; use slot_clock::SlotClock; -use std::cmp::Ordering; +use std::{cmp::Ordering, time::Duration}; use types::{CloneConfig, Epoch, EthSpec, Hash256, Slot}; /// The struct that is returned to the requesting HTTP client. @@ -33,7 +33,7 @@ pub fn proposer_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/sync_committees.rs b/beacon_node/http_api/src/sync_committees.rs index c728fbeb14e..9ce25a2caed 100644 --- a/beacon_node/http_api/src/sync_committees.rs +++ b/beacon_node/http_api/src/sync_committees.rs @@ -6,7 +6,7 @@ use beacon_chain::sync_committee_verification::{ }; use beacon_chain::{ validator_monitor::timestamp_now, BeaconChain, BeaconChainError, BeaconChainTypes, - StateSkipConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY, + StateSkipConfig, }; use eth2::types::{self as api_types}; use lighthouse_network::PubsubMessage; @@ -15,6 +15,7 @@ use slog::{debug, error, warn, Logger}; use slot_clock::SlotClock; use std::cmp::max; use std::collections::HashMap; +use std::time::Duration; use tokio::sync::mpsc::UnboundedSender; use types::{ slot_data::SlotData, BeaconStateError, Epoch, EthSpec, SignedContributionAndProof, @@ -85,7 +86,7 @@ fn duties_from_state_load( let current_epoch = chain.epoch()?; let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY) + .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) .ok_or(BeaconChainError::UnableToReadSlot)? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 427be6d5138..ef5d745bd27 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -14,7 +14,7 @@ use super::MAX_SCHEDULED_WORK_QUEUE_LEN; use crate::beacon_processor::{ChainSegmentProcessId, Work, WorkEvent}; use crate::metrics; use crate::sync::manager::BlockProcessType; -use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; +use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; use fnv::FnvHashMap; use futures::task::Poll; use futures::{Stream, StreamExt}; @@ -383,7 +383,7 @@ pub fn spawn_reprocess_scheduler( ) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); // Basic sanity check. - assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < MAXIMUM_GOSSIP_CLOCK_DISPARITY); + assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)); let mut queue = ReprocessQueue { work_reprocessing_rx, From 563ce17c13da75f250b1aa8e487d2c653df6ce74 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 22 Jun 2023 07:38:57 -0400 Subject: [PATCH 10/39] Finishing for the PR --- beacon_node/lighthouse_network/src/config.rs | 8 ++------ consensus/types/src/chain_spec.rs | 13 ++++++++----- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 33a6b138497..fd90960f0f7 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -33,17 +33,13 @@ use types::{ForkContext, ForkName, EthSpec}; /// another 500ms for "fudge factor". pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); -// We treat uncompressed messages as invalid and never use the INVALID_SNAPPY_DOMAIN as in the -// specification. We leave it here for posterity. -// const MESSAGE_DOMAIN_INVALID_SNAPPY: [u8; 4] = [0, 0, 0, 0]; - /// The maximum size of gossip messages. pub fn gossip_max_size(is_merge_enabled: bool) -> usize { let gossip_max_size = TSpec::default_spec().gossip_max_size; if is_merge_enabled { - 10 * gossip_max_size - } else { gossip_max_size + } else { + gossip_max_size / 10 } } diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 33cabceee9e..890a8240932 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -176,12 +176,11 @@ pub struct ChainSpec { // Shift networking configuration into configs pub gossip_max_size: usize, pub max_request_blocks: u64, - // min_epochs_for_block_requests?? + pub min_epochs_for_block_requests: usize, pub max_chunk_size: usize, pub ttfb_timeout: u64, pub resp_timeout: u64, - // attestation_propagation_slot_range?? - // message_domain_invalid_snappy?? + pub message_domain_invalid_snappy: [u8; 4], pub message_domain_valid_snappy: [u8; 4], /* @@ -642,11 +641,13 @@ impl ChainSpec { epochs_per_random_subnet_subscription: 256, epochs_per_subnet_subscription: 256, attestation_subnet_extra_bits: 6, - gossip_max_size: 1_048_576, + gossip_max_size: 10_485_760, max_request_blocks: 1024, + min_epochs_for_block_requests: 33024, max_chunk_size: 1_048_576, ttfb_timeout: 5, resp_timeout: 10, + message_domain_invalid_snappy: [0, 0, 0, 0], message_domain_valid_snappy: [1, 0, 0, 0], /* @@ -875,11 +876,13 @@ impl ChainSpec { epochs_per_random_subnet_subscription: 256, epochs_per_subnet_subscription: 256, attestation_subnet_extra_bits: 6, - gossip_max_size: 1_048_576, + gossip_max_size: 10_485_760, max_request_blocks: 1024, + min_epochs_for_block_requests: 33024, max_chunk_size: 1_048_576, ttfb_timeout: 5, resp_timeout: 10, + message_domain_invalid_snappy: [0, 0, 0, 0], message_domain_valid_snappy: [1, 0, 0, 0], /* From 47d1ae77466af0d8705e8da370c89385d62eb3a2 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 22 Jun 2023 13:54:46 -0400 Subject: [PATCH 11/39] use chain instead of calling default_spec() function --- .../src/light_client_finality_update_verification.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs index db6db018699..62e71d7b852 100644 --- a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs @@ -6,7 +6,7 @@ use slot_clock::SlotClock; use std::time::Duration; use strum::AsRefStr; use types::{ - light_client_update::Error as LightClientUpdateError, LightClientFinalityUpdate, Slot, EthSpec, + light_client_update::Error as LightClientUpdateError, LightClientFinalityUpdate, Slot, }; /// Returned when a light client finality update was not successfully verified. It might not have been verified for @@ -103,7 +103,7 @@ impl VerifiedLightClientFinalityUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp + Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis) < time + one_third_slot_duration + if seen_timestamp + Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis) < time + one_third_slot_duration { return Err(Error::TooEarly); } From 0724ae07e5457dbe235f6eff6739c790bfcd1bb2 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 22 Jun 2023 14:32:51 -0400 Subject: [PATCH 12/39] use chain instead of default_spec() --- beacon_node/http_api/src/attester_duties.rs | 2 +- beacon_node/http_api/src/proposer_duties.rs | 2 +- beacon_node/http_api/src/sync_committees.rs | 2 +- beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs | 4 ++-- .../network/src/beacon_processor/worker/rpc_methods.rs | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/beacon_node/http_api/src/attester_duties.rs b/beacon_node/http_api/src/attester_duties.rs index e19b6c27295..b641f1cd07a 100644 --- a/beacon_node/http_api/src/attester_duties.rs +++ b/beacon_node/http_api/src/attester_duties.rs @@ -33,7 +33,7 @@ pub fn attester_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis)) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/proposer_duties.rs b/beacon_node/http_api/src/proposer_duties.rs index 6a6a8bc8853..be858be4239 100644 --- a/beacon_node/http_api/src/proposer_duties.rs +++ b/beacon_node/http_api/src/proposer_duties.rs @@ -33,7 +33,7 @@ pub fn proposer_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis)) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/sync_committees.rs b/beacon_node/http_api/src/sync_committees.rs index 9ce25a2caed..f698dba3106 100644 --- a/beacon_node/http_api/src/sync_committees.rs +++ b/beacon_node/http_api/src/sync_committees.rs @@ -86,7 +86,7 @@ fn duties_from_state_load( let current_epoch = chain.epoch()?; let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis)) .ok_or(BeaconChainError::UnableToReadSlot)? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index 9204fda802f..a48eecb35cb 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -616,7 +616,7 @@ mod tests { use std::sync::Arc; use types::{ BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EmptyBlock, Epoch, - ForkContext, FullPayload, Hash256, Signature, SignedBeaconBlock, Slot, MainnetEthSpec, + ForkContext, FullPayload, Hash256, Signature, SignedBeaconBlock, Slot, }; use snap::write::FrameEncoder; @@ -1399,7 +1399,7 @@ mod tests { let mut dst = BytesMut::with_capacity(1024); // Insert length-prefix - uvi_codec.encode(MainnetEthSpec::default_spec().max_chunk_size + 1, &mut dst).unwrap(); + uvi_codec.encode(Spec::default_spec().max_chunk_size + 1, &mut dst).unwrap(); // Insert snappy stream identifier dst.extend_from_slice(stream_identifier); diff --git a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs b/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs index 6930498d9e8..40314129bd4 100644 --- a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs @@ -297,7 +297,7 @@ impl Worker { ); // Should not send more than max request blocks - let max_request_blocks = T::EthSpec::default_spec().max_request_blocks; + let max_request_blocks = self.chain.spec.max_request_blocks; if *req.count() > max_request_blocks { *req.count_mut() = max_request_blocks; } From a0f3b39ce7bfd44d13bd31000a424dc75aa82748 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Fri, 23 Jun 2023 18:44:36 -0400 Subject: [PATCH 13/39] Adding gossip_max_size function ChainSpec object --- beacon_node/lighthouse_network/src/config.rs | 10 +++++----- beacon_node/lighthouse_network/src/service/mod.rs | 2 +- beacon_node/src/config.rs | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 701d9a5e8bb..ccc1ed11627 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -17,7 +17,7 @@ use std::net::{Ipv4Addr, Ipv6Addr}; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; -use types::{ForkContext, ForkName, EthSpec}; +use types::{ForkContext, ForkName, EthSpec, ChainSpec}; /// The cache time is set to accommodate the circulation time of an attestation. /// @@ -34,8 +34,8 @@ use types::{ForkContext, ForkName, EthSpec}; pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); /// The maximum size of gossip messages. -pub fn gossip_max_size(is_merge_enabled: bool) -> usize { - let gossip_max_size = TSpec::default_spec().gossip_max_size; +pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { + let gossip_max_size = spec.gossip_max_size; if is_merge_enabled { gossip_max_size } else { @@ -407,7 +407,7 @@ impl From for NetworkLoad { } /// Return a Lighthouse specific `GossipsubConfig` where the `message_id_fn` depends on the current fork. -pub fn gossipsub_config(network_load: u8, fork_context: Arc) -> GossipsubConfig { +pub fn gossipsub_config(network_load: u8, fork_context: Arc, spec: &ChainSpec) -> GossipsubConfig { // The function used to generate a gossipsub message id // We use the first 8 bytes of SHA256(topic, data) for content addressing let fast_gossip_message_id = |message: &RawGossipsubMessage| { @@ -453,7 +453,7 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc(is_merge_enabled)) + .max_transmit_size(gossip_max_size::(is_merge_enabled, spec)) .heartbeat_interval(load.heartbeat_interval) .mesh_n(load.mesh_n) .mesh_n_low(load.mesh_n_low) diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index 074aa2cb360..2ea788e72d3 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -239,7 +239,7 @@ impl Network { max_subscriptions_per_request: 150, // 148 in theory = (64 attestation + 4 sync committee + 6 core topics) * 2 }; - config.gs_config = gossipsub_config::(config.network_load, ctx.fork_context.clone()); + config.gs_config = gossipsub_config::(config.network_load, ctx.fork_context.clone(), ctx.chain_spec); // If metrics are enabled for gossipsub build the configuration let gossipsub_metrics = ctx diff --git a/beacon_node/src/config.rs b/beacon_node/src/config.rs index b3046a629f9..b2ab933c937 100644 --- a/beacon_node/src/config.rs +++ b/beacon_node/src/config.rs @@ -576,7 +576,7 @@ pub fn get_config( } client_config.chain.max_network_size = - lighthouse_network::gossip_max_size::(spec.bellatrix_fork_epoch.is_some()); + lighthouse_network::gossip_max_size::(spec.bellatrix_fork_epoch.is_some(), spec); if cli_args.is_present("slasher") { let slasher_dir = if let Some(slasher_dir) = cli_args.value_of("slasher-dir") { From 816ebececc4771ed8356d2677848b077865a6248 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Fri, 23 Jun 2023 20:00:33 -0400 Subject: [PATCH 14/39] Adding ChainSpec object to other parts --- .../src/attestation_verification.rs | 18 +++++++++--------- .../beacon_chain/src/block_verification.rs | 4 +++- beacon_node/beacon_chain/src/lib.rs | 4 ++-- ...ight_client_finality_update_verification.rs | 8 ++++---- ...ht_client_optimistic_update_verification.rs | 10 +++++----- .../src/sync_committee_verification.rs | 15 +++++++-------- beacon_node/http_api/src/attester_duties.rs | 10 +++++----- beacon_node/http_api/src/proposer_duties.rs | 4 +++- beacon_node/http_api/src/sync_committees.rs | 4 +++- beacon_node/http_api/tests/tests.rs | 6 +++--- beacon_node/lighthouse_network/src/config.rs | 17 +++++++++++++---- .../src/rpc/codec/ssz_snappy.rs | 4 +++- .../lighthouse_network/src/rpc/config.rs | 3 +-- .../lighthouse_network/src/rpc/handler.rs | 12 ++++++++---- .../lighthouse_network/src/rpc/protocol.rs | 4 +++- .../lighthouse_network/src/service/mod.rs | 6 +++++- .../network/src/beacon_processor/tests.rs | 6 +++--- .../work_reprocessing_queue.rs | 7 ++++++- .../beacon_processor/worker/gossip_methods.rs | 2 ++ beacon_node/network/src/sync/manager.rs | 2 +- 20 files changed, 89 insertions(+), 57 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index b4ac5e2f8a0..e04a8d99677 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -35,10 +35,8 @@ mod batch; use crate::{ - beacon_chain::VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, - metrics, - observed_aggregates::ObserveOutcome, - observed_attesters::Error as ObservedAttestersError, + beacon_chain::VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, metrics, + observed_aggregates::ObserveOutcome, observed_attesters::Error as ObservedAttestersError, BeaconChain, BeaconChainError, BeaconChainTypes, }; use bls::verify_signature_sets; @@ -57,8 +55,8 @@ use std::{borrow::Cow, time::Duration}; use strum::AsRefStr; use tree_hash::TreeHash; use types::{ - Attestation, BeaconCommittee, CommitteeIndex, Epoch, EthSpec, Hash256, IndexedAttestation, - SelectionProof, SignedAggregateAndProof, Slot, SubnetId, + Attestation, BeaconCommittee, ChainSpec, CommitteeIndex, Epoch, EthSpec, Hash256, + IndexedAttestation, SelectionProof, SignedAggregateAndProof, Slot, SubnetId, }; pub use batch::{batch_verify_aggregated_attestations, batch_verify_unaggregated_attestations}; @@ -454,7 +452,7 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { // MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance). // // We do not queue future attestations for later processing. - verify_propagation_slot_range(&chain.slot_clock, attestation)?; + verify_propagation_slot_range(&chain.slot_clock, attestation, &chain.spec)?; // Check the attestation's epoch matches its target. if attestation.data.slot.epoch(T::EthSpec::slots_per_epoch()) @@ -718,7 +716,7 @@ impl<'a, T: BeaconChainTypes> IndexedUnaggregatedAttestation<'a, T> { // MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance). // // We do not queue future attestations for later processing. - verify_propagation_slot_range(&chain.slot_clock, attestation)?; + verify_propagation_slot_range(&chain.slot_clock, attestation, &chain.spec)?; // Check to ensure that the attestation is "unaggregated". I.e., it has exactly one // aggregation bit set. @@ -1033,9 +1031,11 @@ fn verify_head_block_is_known( pub fn verify_propagation_slot_range( slot_clock: &S, attestation: &Attestation, + spec: &ChainSpec, ) -> Result<(), Error> { let attestation_slot = attestation.data.slot; - let maximum_gossip_clock_disparity = Duration::from_millis(E::default_spec().maximum_gossip_clock_disparity_millis); + let maximum_gossip_clock_disparity = + Duration::from_millis(spec.maximum_gossip_clock_disparity_millis); let latest_permissible_slot = slot_clock .now_with_future_tolerance(maximum_gossip_clock_disparity) .ok_or(BeaconChainError::UnableToReadSlot)?; diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 8cd7d13f8e6..2ce979305a0 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -695,7 +695,9 @@ impl GossipVerifiedBlock { // Do not gossip or process blocks from future slots. let present_slot_with_tolerance = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis( + chain.spec.maximum_gossip_clock_disparity_millis, + )) .ok_or(BeaconChainError::UnableToReadSlot)?; if block.slot() > present_slot_with_tolerance { return Err(BlockError::FutureSlot { diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index b72fb9055b1..f1a0d159636 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -52,8 +52,8 @@ pub mod validator_pubkey_cache; pub use self::beacon_chain::{ AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BeaconStore, ChainSegmentResult, - ForkChoiceError, OverrideForkchoiceUpdate, ProduceBlockVerification, - StateSkipConfig, WhenSlotSkipped, INVALID_FINALIZED_MERGE_TRANSITION_BLOCK_SHUTDOWN_REASON, + ForkChoiceError, OverrideForkchoiceUpdate, ProduceBlockVerification, StateSkipConfig, + WhenSlotSkipped, INVALID_FINALIZED_MERGE_TRANSITION_BLOCK_SHUTDOWN_REASON, INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON, }; pub use self::beacon_snapshot::BeaconSnapshot; diff --git a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs index 62e71d7b852..c17202ae513 100644 --- a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs @@ -1,6 +1,4 @@ -use crate::{ - BeaconChain, BeaconChainError, BeaconChainTypes, -}; +use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; use derivative::Derivative; use slot_clock::SlotClock; use std::time::Duration; @@ -103,7 +101,9 @@ impl VerifiedLightClientFinalityUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp + Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis) < time + one_third_slot_duration + if seen_timestamp + + Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis) + < time + one_third_slot_duration { return Err(Error::TooEarly); } diff --git a/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs b/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs index 51f95e274a2..cb79be6c5b8 100644 --- a/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs @@ -1,13 +1,11 @@ -use crate::{ - BeaconChain, BeaconChainError, BeaconChainTypes, -}; +use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; use derivative::Derivative; use eth2::types::Hash256; use slot_clock::SlotClock; use std::time::Duration; use strum::AsRefStr; use types::{ - light_client_update::Error as LightClientUpdateError, LightClientOptimisticUpdate, Slot, EthSpec, + light_client_update::Error as LightClientUpdateError, LightClientOptimisticUpdate, Slot, }; /// Returned when a light client optimistic update was not successfully verified. It might not have been verified for @@ -103,7 +101,9 @@ impl VerifiedLightClientOptimisticUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp + Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis) < time + one_third_slot_duration + if seen_timestamp + + Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis) + < time + one_third_slot_duration { return Err(Error::TooEarly); } diff --git a/beacon_node/beacon_chain/src/sync_committee_verification.rs b/beacon_node/beacon_chain/src/sync_committee_verification.rs index 8c107c16af9..741f3fe8066 100644 --- a/beacon_node/beacon_chain/src/sync_committee_verification.rs +++ b/beacon_node/beacon_chain/src/sync_committee_verification.rs @@ -28,10 +28,8 @@ use crate::observed_attesters::SlotSubcommitteeIndex; use crate::{ - beacon_chain::VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, - metrics, - observed_aggregates::ObserveOutcome, - BeaconChain, BeaconChainError, BeaconChainTypes, + beacon_chain::VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, metrics, + observed_aggregates::ObserveOutcome, BeaconChain, BeaconChainError, BeaconChainTypes, }; use bls::{verify_signature_sets, PublicKeyBytes}; use derivative::Derivative; @@ -43,8 +41,8 @@ use state_processing::signature_sets::{ sync_committee_contribution_signature_set_from_pubkeys, sync_committee_message_set_from_pubkeys, }; -use std::{borrow::Cow, time::Duration}; use std::collections::HashMap; +use std::{borrow::Cow, time::Duration}; use strum::AsRefStr; use tree_hash::TreeHash; use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT; @@ -52,8 +50,8 @@ use types::slot_data::SlotData; use types::sync_committee::Error as SyncCommitteeError; use types::{ sync_committee_contribution::Error as ContributionError, AggregateSignature, BeaconStateError, - EthSpec, Hash256, SignedContributionAndProof, Slot, SyncCommitteeContribution, - SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, MainnetEthSpec, + EthSpec, Hash256, MainnetEthSpec, SignedContributionAndProof, Slot, SyncCommitteeContribution, + SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, }; /// Returned when a sync committee contribution was not successfully verified. It might not have been verified for @@ -560,7 +558,8 @@ pub fn verify_propagation_slot_range( sync_contribution: &U, ) -> Result<(), Error> { let message_slot = sync_contribution.get_slot(); - let maximum_gossip_clock_disparity = Duration::from_millis(MainnetEthSpec::default_spec().maximum_gossip_clock_disparity_millis); + let maximum_gossip_clock_disparity = + Duration::from_millis(MainnetEthSpec::default_spec().maximum_gossip_clock_disparity_millis); let latest_permissible_slot = slot_clock .now_with_future_tolerance(maximum_gossip_clock_disparity) .ok_or(BeaconChainError::UnableToReadSlot)?; diff --git a/beacon_node/http_api/src/attester_duties.rs b/beacon_node/http_api/src/attester_duties.rs index b641f1cd07a..5abc4badb32 100644 --- a/beacon_node/http_api/src/attester_duties.rs +++ b/beacon_node/http_api/src/attester_duties.rs @@ -1,16 +1,14 @@ //! Contains the handler for the `GET validator/duties/attester/{epoch}` endpoint. use crate::state_id::StateId; -use beacon_chain::{ - BeaconChain, BeaconChainError, BeaconChainTypes, -}; +use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes}; use eth2::types::{self as api_types}; use slot_clock::SlotClock; use state_processing::state_advance::partial_state_advance; +use std::time::Duration; use types::{ AttestationDuty, BeaconState, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256, RelativeEpoch, }; -use std::time::Duration; /// The struct that is returned to the requesting HTTP client. type ApiDuties = api_types::DutiesResponse>; @@ -33,7 +31,9 @@ pub fn attester_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis( + chain.spec.maximum_gossip_clock_disparity_millis, + )) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/proposer_duties.rs b/beacon_node/http_api/src/proposer_duties.rs index be858be4239..27c1bac19c1 100644 --- a/beacon_node/http_api/src/proposer_duties.rs +++ b/beacon_node/http_api/src/proposer_duties.rs @@ -33,7 +33,9 @@ pub fn proposer_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis( + chain.spec.maximum_gossip_clock_disparity_millis, + )) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/sync_committees.rs b/beacon_node/http_api/src/sync_committees.rs index f698dba3106..62af7579e7d 100644 --- a/beacon_node/http_api/src/sync_committees.rs +++ b/beacon_node/http_api/src/sync_committees.rs @@ -86,7 +86,9 @@ fn duties_from_state_load( let current_epoch = chain.epoch()?; let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis)) + .now_with_future_tolerance(Duration::from_millis( + chain.spec.maximum_gossip_clock_disparity_millis, + )) .ok_or(BeaconChainError::UnableToReadSlot)? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index a6c49ddaeef..bc884e571df 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -1,7 +1,7 @@ use beacon_chain::test_utils::RelativeSyncCommittee; use beacon_chain::{ test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType}, - BeaconChain, StateSkipConfig, WhenSlotSkipped, MAXIMUM_GOSSIP_CLOCK_DISPARITY, + BeaconChain, StateSkipConfig, WhenSlotSkipped, }; use environment::null_logger; use eth2::{ @@ -2259,7 +2259,7 @@ impl ApiTester { .unwrap(); self.chain.slot_clock.set_current_time( - current_epoch_start - MAXIMUM_GOSSIP_CLOCK_DISPARITY - Duration::from_millis(1), + current_epoch_start - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis) - Duration::from_millis(1), ); let dependent_root = self @@ -2298,7 +2298,7 @@ impl ApiTester { self.chain .slot_clock - .set_current_time(current_epoch_start - MAXIMUM_GOSSIP_CLOCK_DISPARITY); + .set_current_time(current_epoch_start - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis)); self.client .get_validator_duties_proposer(current_epoch) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index ccc1ed11627..28c4fbf1236 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -17,7 +17,7 @@ use std::net::{Ipv4Addr, Ipv6Addr}; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; -use types::{ForkContext, ForkName, EthSpec, ChainSpec}; +use types::{ChainSpec, EthSpec, ForkContext, ForkName}; /// The cache time is set to accommodate the circulation time of an attestation. /// @@ -34,7 +34,7 @@ use types::{ForkContext, ForkName, EthSpec, ChainSpec}; pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); /// The maximum size of gossip messages. -pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { +pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { let gossip_max_size = spec.gossip_max_size; if is_merge_enabled { gossip_max_size @@ -407,7 +407,11 @@ impl From for NetworkLoad { } /// Return a Lighthouse specific `GossipsubConfig` where the `message_id_fn` depends on the current fork. -pub fn gossipsub_config(network_load: u8, fork_context: Arc, spec: &ChainSpec) -> GossipsubConfig { +pub fn gossipsub_config( + network_load: u8, + fork_context: Arc, + spec: &ChainSpec, +) -> GossipsubConfig { // The function used to generate a gossipsub message id // We use the first 8 bytes of SHA256(topic, data) for content addressing let fast_gossip_message_id = |message: &RawGossipsubMessage| { @@ -445,7 +449,12 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc Network { max_subscriptions_per_request: 150, // 148 in theory = (64 attestation + 4 sync committee + 6 core topics) * 2 }; - config.gs_config = gossipsub_config::(config.network_load, ctx.fork_context.clone(), ctx.chain_spec); + config.gs_config = gossipsub_config::( + config.network_load, + ctx.fork_context.clone(), + ctx.chain_spec, + ); // If metrics are enabled for gossipsub build the configuration let gossipsub_metrics = ctx diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index b93e83ad785..e72c80af335 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -9,7 +9,7 @@ use crate::{service::NetworkMessage, sync::SyncMessage}; use beacon_chain::test_utils::{ AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, }; -use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; +use beacon_chain::{BeaconChain, ChainConfig}; use lighthouse_network::{ discv5::enr::{CombinedKey, EnrBuilder}, rpc::methods::{MetaData, MetaDataV2}, @@ -478,7 +478,7 @@ async fn import_gossip_block_acceptably_early() { rig.chain .slot_clock - .set_current_time(slot_start - MAXIMUM_GOSSIP_CLOCK_DISPARITY); + .set_current_time(slot_start - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis)); assert_eq!( rig.chain.slot().unwrap(), @@ -527,7 +527,7 @@ async fn import_gossip_block_unacceptably_early() { rig.chain .slot_clock - .set_current_time(slot_start - MAXIMUM_GOSSIP_CLOCK_DISPARITY - Duration::from_millis(1)); + .set_current_time(slot_start - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis) - Duration::from_millis(1)); assert_eq!( rig.chain.slot().unwrap(), diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index ef5d745bd27..c64dc212d0a 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -383,7 +383,12 @@ pub fn spawn_reprocess_scheduler( ) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); // Basic sanity check. - assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < Duration::from_millis(T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis)); + assert!( + ADDITIONAL_QUEUED_BLOCK_DELAY + < Duration::from_millis( + T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis + ) + ); let mut queue = ReprocessQueue { work_reprocessing_rx, diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index e3cff00103b..76bf2485e35 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -1652,6 +1652,7 @@ impl Worker { attestation_verification::verify_propagation_slot_range( seen_clock, failed_att.attestation(), + &self.chain.spec, ); // Only penalize the peer if it would have been invalid at the moment we received @@ -2498,6 +2499,7 @@ impl Worker { let is_timely = attestation_verification::verify_propagation_slot_range( &self.chain.slot_clock, attestation, + &self.chain.spec, ) .is_ok(); diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index d874d6f6f94..28e2949b310 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -191,7 +191,7 @@ pub fn spawn( log: slog::Logger, ) -> mpsc::UnboundedSender> { assert!( - T::EthSpec::default_spec().max_request_blocks >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, + beacon_chain.spec.max_request_blocks >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, "Max blocks that can be requested in a single batch greater than max allowed blocks in a single request" ); // generate the message channel From b7e009c7bc7871ed745c5b3b751af34cecbb23d0 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 27 Jun 2023 05:34:47 -0400 Subject: [PATCH 15/39] Adding ChainSpec object to other parts --- beacon_node/http_api/tests/tests.rs | 11 ++++--- .../lighthouse_network/src/rpc/handler.rs | 15 ++++++---- beacon_node/lighthouse_network/src/rpc/mod.rs | 7 ++++- .../lighthouse_network/src/service/mod.rs | 4 ++- .../lighthouse_network/tests/common.rs | 17 +++++++---- .../lighthouse_network/tests/rpc_tests.rs | 29 ++++++++++++++----- .../network/src/beacon_processor/tests.rs | 14 +++++---- beacon_node/network/src/service.rs | 9 ++++-- 8 files changed, 74 insertions(+), 32 deletions(-) diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index bc884e571df..4a5bdca0ce7 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -2259,7 +2259,9 @@ impl ApiTester { .unwrap(); self.chain.slot_clock.set_current_time( - current_epoch_start - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis) - Duration::from_millis(1), + current_epoch_start + - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis) + - Duration::from_millis(1), ); let dependent_root = self @@ -2296,9 +2298,10 @@ impl ApiTester { "should not get attester duties outside of tolerance" ); - self.chain - .slot_clock - .set_current_time(current_epoch_start - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis)); + self.chain.slot_clock.set_current_time( + current_epoch_start + - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis), + ); self.client .get_validator_duties_proposer(current_epoch) diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index 2ade1aa0fc0..9470ff20601 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -29,7 +29,7 @@ use std::{ }; use tokio::time::{sleep_until, Instant as TInstant, Sleep}; use tokio_util::time::{delay_queue, DelayQueue}; -use types::{EthSpec, ForkContext}; +use types::{ChainSpec, EthSpec, ForkContext}; /// The number of times to retry an outbound upgrade in the case of IO errors. const IO_ERROR_RETRIES: u8 = 3; @@ -128,6 +128,9 @@ where /// Logger for handling RPC streams log: slog::Logger, + + /// ChainSpec for networking constants + chain_spec: ChainSpec, } enum HandlerState { @@ -210,6 +213,7 @@ where listen_protocol: SubstreamProtocol, ()>, fork_context: Arc, log: &slog::Logger, + spec: &ChainSpec, ) -> Self { RPCHandler { listen_protocol, @@ -228,6 +232,7 @@ where fork_context, waker: None, log: log.clone(), + chain_spec: spec.clone(), } } @@ -348,7 +353,7 @@ where // new outbound request. Store the stream and tag the output. let delay_key = self.outbound_substreams_delay.insert( self.current_outbound_substream_id, - Duration::from_secs(TSpec::default_spec().resp_timeout), + Duration::from_secs(self.chain_spec.resp_timeout), ); let awaiting_stream = OutboundSubstreamState::RequestPendingResponse { substream: Box::new(out), @@ -399,7 +404,7 @@ where // Store the stream and tag the output. let delay_key = self.inbound_substreams_delay.insert( self.current_inbound_substream_id, - Duration::from_secs(TSpec::default_spec().resp_timeout), + Duration::from_secs(self.chain_spec.resp_timeout), ); let awaiting_stream = InboundState::Idle(substream); self.inbound_substreams.insert( @@ -712,7 +717,7 @@ where if let Some(ref delay_key) = info.delay_key { self.inbound_substreams_delay.reset( delay_key, - Duration::from_secs(TSpec::default_spec().resp_timeout), + Duration::from_secs(self.chain_spec.resp_timeout), ); } @@ -848,7 +853,7 @@ where substream_entry.remaining_chunks = Some(remaining_chunks); self.outbound_substreams_delay.reset( delay_key, - Duration::from_secs(TSpec::default_spec().resp_timeout), + Duration::from_secs(self.chain_spec.resp_timeout), ); } } else { diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 0724730fb38..6c3350878b2 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -17,7 +17,7 @@ use slog::{crit, debug, o}; use std::marker::PhantomData; use std::sync::Arc; use std::task::{Context, Poll}; -use types::{EthSpec, ForkContext}; +use types::{ChainSpec, EthSpec, ForkContext}; pub(crate) use handler::HandlerErr; pub(crate) use methods::{MetaData, MetaDataV1, MetaDataV2, Ping, RPCCodedResponse, RPCResponse}; @@ -120,6 +120,8 @@ pub struct RPC { enable_light_client_server: bool, /// Slog logger for RPC behaviour. log: slog::Logger, + /// ChainSpec for networking constants + chain_spec: ChainSpec, } impl RPC { @@ -129,6 +131,7 @@ impl RPC { inbound_rate_limiter_config: Option, outbound_rate_limiter_config: Option, log: slog::Logger, + spec: ChainSpec, ) -> Self { let log = log.new(o!("service" => "libp2p_rpc")); @@ -149,6 +152,7 @@ impl RPC { fork_context, enable_light_client_server, log, + chain_spec: spec, } } @@ -223,6 +227,7 @@ where ), self.fork_context.clone(), &self.log, + &self.chain_spec, ) } diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index 1421489b94a..e3da83280c3 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -41,10 +41,10 @@ use std::{ sync::Arc, task::{Context, Poll}, }; -use types::ForkName; use types::{ consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext, Slot, SubnetId, }; +use types::{ChainSpec, ForkName}; use utils::{build_transport, strip_peer_id, Context as ServiceContext, MAX_CONNECTIONS_PER_PEER}; pub mod api_types; @@ -143,6 +143,7 @@ impl Network { executor: task_executor::TaskExecutor, ctx: ServiceContext<'_>, log: &slog::Logger, + spec: &ChainSpec, ) -> error::Result<(Self, Arc>)> { let log = log.new(o!("service"=> "libp2p")); let mut config = ctx.config.clone(); @@ -273,6 +274,7 @@ impl Network { config.inbound_rate_limiter_config.clone(), config.outbound_rate_limiter_config.clone(), log.clone(), + spec.clone(), ); let discovery = { diff --git a/beacon_node/lighthouse_network/tests/common.rs b/beacon_node/lighthouse_network/tests/common.rs index 64714cbc0a8..439a39c0be7 100644 --- a/beacon_node/lighthouse_network/tests/common.rs +++ b/beacon_node/lighthouse_network/tests/common.rs @@ -94,6 +94,7 @@ pub async fn build_libp2p_instance( boot_nodes: Vec, log: slog::Logger, fork_name: ForkName, + spec: ChainSpec, ) -> Libp2pInstance { let port = unused_tcp4_port().unwrap(); let config = build_config(port, boot_nodes); @@ -106,11 +107,11 @@ pub async fn build_libp2p_instance( config: &config, enr_fork_id: EnrForkId::default(), fork_context: Arc::new(fork_context(fork_name)), - chain_spec: &ChainSpec::minimal(), + chain_spec: &spec, gossipsub_registry: None, }; Libp2pInstance( - LibP2PService::new(executor, libp2p_context, &log) + LibP2PService::new(executor, libp2p_context, &log, &spec) .await .expect("should build libp2p instance") .0, @@ -130,12 +131,15 @@ pub async fn build_node_pair( rt: Weak, log: &slog::Logger, fork_name: ForkName, + spec: &ChainSpec, ) -> (Libp2pInstance, Libp2pInstance) { let sender_log = log.new(o!("who" => "sender")); let receiver_log = log.new(o!("who" => "receiver")); - let mut sender = build_libp2p_instance(rt.clone(), vec![], sender_log, fork_name).await; - let mut receiver = build_libp2p_instance(rt, vec![], receiver_log, fork_name).await; + let mut sender = + build_libp2p_instance(rt.clone(), vec![], sender_log, fork_name, spec.clone()).await; + let mut receiver = + build_libp2p_instance(rt, vec![], receiver_log, fork_name, spec.clone()).await; let receiver_multiaddr = receiver.local_enr().multiaddr()[1].clone(); @@ -180,10 +184,13 @@ pub async fn build_linear( log: slog::Logger, n: usize, fork_name: ForkName, + spec: &ChainSpec, ) -> Vec { let mut nodes = Vec::with_capacity(n); for _ in 0..n { - nodes.push(build_libp2p_instance(rt.clone(), vec![], log.clone(), fork_name).await); + nodes.push( + build_libp2p_instance(rt.clone(), vec![], log.clone(), fork_name, spec.clone()).await, + ); } let multiaddrs: Vec = nodes diff --git a/beacon_node/lighthouse_network/tests/rpc_tests.rs b/beacon_node/lighthouse_network/tests/rpc_tests.rs index 656df0c4a16..fff836dec3b 100644 --- a/beacon_node/lighthouse_network/tests/rpc_tests.rs +++ b/beacon_node/lighthouse_network/tests/rpc_tests.rs @@ -57,10 +57,12 @@ fn test_status_rpc() { let log = common::build_log(log_level, enable_logging); + let spec = E::default_spec(); + rt.block_on(async { // get sender/receiver let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base, &spec).await; // Dummy STATUS RPC message let rpc_request = Request::Status(StatusMessage { @@ -149,10 +151,12 @@ fn test_blocks_by_range_chunked_rpc() { let rt = Arc::new(Runtime::new().unwrap()); + let spec = E::default_spec(); + rt.block_on(async { // get sender/receiver let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Merge).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Merge, &spec).await; // BlocksByRange Request let rpc_request = Request::BlocksByRange(BlocksByRangeRequest::new(0, messages_to_send)); @@ -273,10 +277,12 @@ fn test_blocks_by_range_over_limit() { let rt = Arc::new(Runtime::new().unwrap()); + let spec = E::default_spec(); + rt.block_on(async { // get sender/receiver let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Merge).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Merge, &spec).await; // BlocksByRange Request let rpc_request = Request::BlocksByRange(BlocksByRangeRequest::new(0, messages_to_send)); @@ -355,10 +361,12 @@ fn test_blocks_by_range_chunked_rpc_terminates_correctly() { let rt = Arc::new(Runtime::new().unwrap()); + let spec = E::default_spec(); + rt.block_on(async { // get sender/receiver let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base, &spec).await; // BlocksByRange Request let rpc_request = Request::BlocksByRange(BlocksByRangeRequest::new(0, messages_to_send)); @@ -475,10 +483,12 @@ fn test_blocks_by_range_single_empty_rpc() { let log = common::build_log(log_level, enable_logging); let rt = Arc::new(Runtime::new().unwrap()); + let spec = E::default_spec(); + rt.block_on(async { // get sender/receiver let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base, &spec).await; // BlocksByRange Request let rpc_request = Request::BlocksByRange(BlocksByRangeRequest::new(0, 10)); @@ -579,7 +589,7 @@ fn test_blocks_by_root_chunked_rpc() { // get sender/receiver rt.block_on(async { let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Merge).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Merge, &spec).await; // BlocksByRoot Request let rpc_request = @@ -706,7 +716,7 @@ fn test_blocks_by_root_chunked_rpc_terminates_correctly() { // get sender/receiver rt.block_on(async { let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base, &spec).await; // BlocksByRoot Request let rpc_request = @@ -833,10 +843,13 @@ fn test_goodbye_rpc() { let log = common::build_log(log_level, enable_logging); let rt = Arc::new(Runtime::new().unwrap()); + + let spec = E::default_spec(); + // get sender/receiver rt.block_on(async { let (mut sender, mut receiver) = - common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base).await; + common::build_node_pair(Arc::downgrade(&rt), &log, ForkName::Base, &spec).await; // build the sender future let sender_future = async { diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index e72c80af335..18b405f4535 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -476,9 +476,9 @@ async fn import_gossip_block_acceptably_early() { .start_of(rig.next_block.slot()) .unwrap(); - rig.chain - .slot_clock - .set_current_time(slot_start - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis)); + rig.chain.slot_clock.set_current_time( + slot_start - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis), + ); assert_eq!( rig.chain.slot().unwrap(), @@ -525,9 +525,11 @@ async fn import_gossip_block_unacceptably_early() { .start_of(rig.next_block.slot()) .unwrap(); - rig.chain - .slot_clock - .set_current_time(slot_start - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis) - Duration::from_millis(1)); + rig.chain.slot_clock.set_current_time( + slot_start + - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis) + - Duration::from_millis(1), + ); assert_eq!( rig.chain.slot().unwrap(), diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index 2c919233fca..a15ad50ee53 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -281,8 +281,13 @@ impl NetworkService { }; // launch libp2p service - let (mut libp2p, network_globals) = - Network::new(executor.clone(), service_context, &network_log).await?; + let (mut libp2p, network_globals) = Network::new( + executor.clone(), + service_context, + &network_log, + &beacon_chain.spec, + ) + .await?; // Repopulate the DHT with stored ENR's if discovery is not disabled. if !config.disable_discovery { From d3d9b8f7bff5219ba5b31bcda71f90fe5e4a913b Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 27 Jun 2023 16:49:12 -0400 Subject: [PATCH 16/39] Add the networking constants to Config object --- beacon_node/lighthouse_network/src/config.rs | 4 +- .../src/rpc/codec/ssz_snappy.rs | 2 +- .../lighthouse_network/src/rpc/protocol.rs | 2 +- consensus/types/src/chain_spec.rs | 64 ++++++++++++++++--- consensus/types/src/subnet_id.rs | 4 +- 5 files changed, 61 insertions(+), 15 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 28c4fbf1236..53a1b693ad2 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -37,9 +37,9 @@ pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { let gossip_max_size = spec.gossip_max_size; if is_merge_enabled { - gossip_max_size + gossip_max_size as usize } else { - gossip_max_size / 10 + (gossip_max_size / 10) as usize } } diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index a5990ffe105..cfa3baa5f01 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -1400,7 +1400,7 @@ mod tests { // Insert length-prefix uvi_codec - .encode(Spec::default_spec().max_chunk_size + 1, &mut dst) + .encode(Spec::default_spec().max_chunk_size as usize + 1, &mut dst) .unwrap(); // Insert snappy stream identifier diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index f7c97ac043c..cddf380ada3 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -121,7 +121,7 @@ const REQUEST_TIMEOUT: u64 = 15; /// Returns the maximum bytes that can be sent across the RPC. pub fn max_rpc_size(fork_context: &ForkContext) -> usize { match fork_context.current_fork() { - ForkName::Altair | ForkName::Base => MainnetEthSpec::default_spec().max_chunk_size, + ForkName::Altair | ForkName::Base => MainnetEthSpec::default_spec().max_chunk_size as usize, ForkName::Merge => MAX_RPC_SIZE_POST_MERGE, ForkName::Capella => MAX_RPC_SIZE_POST_CAPELLA, } diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index e6b0cac24e8..4e5adea5788 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -172,14 +172,15 @@ pub struct ChainSpec { pub epochs_per_subnet_subscription: u64, pub attestation_subnet_extra_bits: u8, // Shift networking configuration into configs - pub gossip_max_size: usize, + pub gossip_max_size: u64, pub max_request_blocks: u64, - pub min_epochs_for_block_requests: usize, - pub max_chunk_size: usize, + pub min_epochs_for_block_requests: u64, + pub max_chunk_size: u64, pub ttfb_timeout: u64, pub resp_timeout: u64, pub message_domain_invalid_snappy: [u8; 4], pub message_domain_valid_snappy: [u8; 4], + pub attestation_subnet_prefix_bits: u32, /* * Application params @@ -460,12 +461,6 @@ impl ChainSpec { Hash256::from(domain) } - #[allow(clippy::integer_arithmetic)] - pub const fn attestation_subnet_prefix_bits(&self) -> u32 { - let attestation_subnet_count_bits = self.attestation_subnet_count.ilog2(); - self.attestation_subnet_extra_bits as u32 + attestation_subnet_count_bits - } - /// Returns a `ChainSpec` compatible with the Ethereum Foundation specification. pub fn mainnet() -> Self { Self { @@ -635,6 +630,7 @@ impl ChainSpec { resp_timeout: 10, message_domain_invalid_snappy: [0, 0, 0, 0], message_domain_valid_snappy: [1, 0, 0, 0], + attestation_subnet_prefix_bits: 6, /* * Application specific @@ -868,6 +864,7 @@ impl ChainSpec { resp_timeout: 10, message_domain_invalid_snappy: [0, 0, 0, 0], message_domain_valid_snappy: [1, 0, 0, 0], + attestation_subnet_prefix_bits: 6, /* * Application specific @@ -978,6 +975,25 @@ pub struct Config { #[serde(with = "serde_utils::quoted_u64")] deposit_network_id: u64, deposit_contract_address: Address, + + #[serde(with = "serde_utils::quoted_u64")] + gossip_max_size: u64, + #[serde(with = "serde_utils::quoted_u64")] + max_request_blocks: u64, + #[serde(with = "serde_utils::quoted_u64")] + min_epochs_for_block_requests: u64, + #[serde(with = "serde_utils::quoted_u64")] + max_chunk_size: u64, + #[serde(with = "serde_utils::quoted_u64")] + ttfb_timeout: u64, + #[serde(with = "serde_utils::quoted_u64")] + resp_timeout: u64, + #[serde(with = "serde_utils::bytes_4_hex")] + message_domain_invalid_snappy: [u8; 4], + #[serde(with = "serde_utils::bytes_4_hex")] + message_domain_valid_snappy: [u8; 4], + #[serde(with = "serde_utils::quoted_u32")] + attestation_subnet_prefix_bits: u32, } fn default_bellatrix_fork_version() -> [u8; 4] { @@ -1113,6 +1129,16 @@ impl Config { deposit_chain_id: spec.deposit_chain_id, deposit_network_id: spec.deposit_network_id, deposit_contract_address: spec.deposit_contract_address, + + gossip_max_size: spec.gossip_max_size, + max_request_blocks: spec.max_request_blocks, + min_epochs_for_block_requests: spec.min_epochs_for_block_requests, + max_chunk_size: spec.max_chunk_size, + ttfb_timeout: spec.ttfb_timeout, + resp_timeout: spec.resp_timeout, + message_domain_invalid_snappy: spec.message_domain_invalid_snappy, + message_domain_valid_snappy: spec.message_domain_valid_snappy, + attestation_subnet_prefix_bits: spec.attestation_subnet_prefix_bits, } } @@ -1157,6 +1183,15 @@ impl Config { deposit_chain_id, deposit_network_id, deposit_contract_address, + gossip_max_size, + max_request_blocks, + min_epochs_for_block_requests, + max_chunk_size, + ttfb_timeout, + resp_timeout, + message_domain_invalid_snappy, + message_domain_valid_snappy, + attestation_subnet_prefix_bits, } = self; if preset_base != T::spec_name().to_string().as_str() { @@ -1194,6 +1229,15 @@ impl Config { terminal_block_hash, terminal_block_hash_activation_epoch, safe_slots_to_import_optimistically, + gossip_max_size, + max_request_blocks, + min_epochs_for_block_requests, + max_chunk_size, + ttfb_timeout, + resp_timeout, + message_domain_invalid_snappy, + message_domain_valid_snappy, + attestation_subnet_prefix_bits, ..chain_spec.clone() }) } @@ -1432,6 +1476,8 @@ mod yaml_tests { DEPOSIT_CHAIN_ID: 1 DEPOSIT_NETWORK_ID: 1 DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa + GOSSIP_MAX_SIZE: 10485760 + MAX_REQUEST_BLOCKS: 1024 "#; let chain_spec: Config = serde_yaml::from_str(spec).unwrap(); diff --git a/consensus/types/src/subnet_id.rs b/consensus/types/src/subnet_id.rs index 6793fe55740..3662ac8de5c 100644 --- a/consensus/types/src/subnet_id.rs +++ b/consensus/types/src/subnet_id.rs @@ -84,7 +84,7 @@ impl SubnetId { let subscription_duration = spec.epochs_per_subnet_subscription; let node_id_prefix = - (node_id >> (256 - spec.attestation_subnet_prefix_bits() as usize)).as_usize(); + (node_id >> (256 - spec.attestation_subnet_prefix_bits as usize)).as_usize(); // NOTE: The as_u64() panics if the number is larger than u64::max_value(). This cannot be // true as spec.epochs_per_subnet_subscription is a u64. @@ -99,7 +99,7 @@ impl SubnetId { let permutation_seed = ethereum_hashing::hash(&int_to_bytes::int_to_bytes8(subscription_event_idx)); - let num_subnets = 1 << spec.attestation_subnet_prefix_bits(); + let num_subnets = 1 << spec.attestation_subnet_prefix_bits; let permutated_prefix = compute_shuffled_index( node_id_prefix, num_subnets, From 7090979ee3e915b83932a05d3cf8bec13de50e0c Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 28 Jun 2023 10:57:58 -0400 Subject: [PATCH 17/39] Add ChainSpec object to max_rpc_size function --- .../lighthouse_network/src/rpc/codec/base.rs | 15 ++- .../src/rpc/codec/ssz_snappy.rs | 118 ++++++++++++++---- .../lighthouse_network/src/rpc/handler.rs | 2 +- beacon_node/lighthouse_network/src/rpc/mod.rs | 2 +- .../lighthouse_network/src/rpc/protocol.rs | 6 +- .../lighthouse_network/tests/rpc_tests.rs | 23 ++-- 6 files changed, 122 insertions(+), 44 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/codec/base.rs b/beacon_node/lighthouse_network/src/rpc/codec/base.rs index d568f27897e..79282e53f45 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/base.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/base.rs @@ -217,9 +217,12 @@ mod tests { let snappy_protocol_id = ProtocolId::new(SupportedProtocol::StatusV1, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(ForkName::Base)); + + let chain_spec = Spec::default_spec(); + let mut snappy_outbound_codec = SSZSnappyOutboundCodec::::new( snappy_protocol_id, - max_rpc_size(&fork_context), + max_rpc_size(&fork_context, &chain_spec), fork_context, ); @@ -251,9 +254,12 @@ mod tests { let snappy_protocol_id = ProtocolId::new(SupportedProtocol::StatusV1, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(ForkName::Base)); + + let chain_spec = Spec::default_spec(); + let mut snappy_outbound_codec = SSZSnappyOutboundCodec::::new( snappy_protocol_id, - max_rpc_size(&fork_context), + max_rpc_size(&fork_context, &chain_spec), fork_context, ); @@ -279,7 +285,10 @@ mod tests { // Response limits let fork_context = Arc::new(fork_context(ForkName::Base)); - let max_rpc_size = max_rpc_size(&fork_context); + + let chain_spec = Spec::default_spec(); + + let max_rpc_size = max_rpc_size(&fork_context, &chain_spec); let limit = protocol_id.rpc_response_limits::(&fork_context); let mut max = encode_len(limit.max + 1); let mut codec = SSZSnappyOutboundCodec::::new( diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index cfa3baa5f01..45607c01952 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -615,8 +615,8 @@ mod tests { }; use std::sync::Arc; use types::{ - BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EmptyBlock, Epoch, - ForkContext, FullPayload, Hash256, Signature, SignedBeaconBlock, Slot, + BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, ChainSpec, EmptyBlock, + Epoch, ForkContext, FullPayload, Hash256, Signature, SignedBeaconBlock, Slot, }; use snap::write::FrameEncoder; @@ -658,7 +658,7 @@ mod tests { } /// Merge block with length < max_rpc_size. - fn merge_block_small(fork_context: &ForkContext) -> SignedBeaconBlock { + fn merge_block_small(fork_context: &ForkContext, spec: &ChainSpec) -> SignedBeaconBlock { let mut block: BeaconBlockMerge<_, FullPayload> = BeaconBlockMerge::empty(&Spec::default_spec()); let tx = VariableList::from(vec![0; 1024]); @@ -667,14 +667,14 @@ mod tests { block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context)); + assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, &spec)); SignedBeaconBlock::from_block(block, Signature::empty()) } /// Merge block with length > MAX_RPC_SIZE. /// The max limit for a merge block is in the order of ~16GiB which wouldn't fit in memory. /// Hence, we generate a merge block just greater than `MAX_RPC_SIZE` to test rejection on the rpc layer. - fn merge_block_large(fork_context: &ForkContext) -> SignedBeaconBlock { + fn merge_block_large(fork_context: &ForkContext, spec: &ChainSpec) -> SignedBeaconBlock { let mut block: BeaconBlockMerge<_, FullPayload> = BeaconBlockMerge::empty(&Spec::default_spec()); let tx = VariableList::from(vec![0; 1024]); @@ -683,7 +683,7 @@ mod tests { block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() > max_rpc_size(fork_context)); + assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, &spec)); SignedBeaconBlock::from_block(block, Signature::empty()) } @@ -737,10 +737,11 @@ mod tests { protocol: SupportedProtocol, message: RPCCodedResponse, fork_name: ForkName, + spec: &ChainSpec, ) -> Result { let snappy_protocol_id = ProtocolId::new(protocol, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context); + let max_packet_size = max_rpc_size(&fork_context, &spec); let mut buf = BytesMut::new(); let mut snappy_inbound_codec = @@ -783,10 +784,11 @@ mod tests { protocol: SupportedProtocol, message: &mut BytesMut, fork_name: ForkName, + spec: &ChainSpec, ) -> Result>, RPCError> { let snappy_protocol_id = ProtocolId::new(protocol, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context); + let max_packet_size = max_rpc_size(&fork_context, spec); let mut snappy_outbound_codec = SSZSnappyOutboundCodec::::new(snappy_protocol_id, max_packet_size, fork_context); // decode message just as snappy message @@ -798,15 +800,20 @@ mod tests { protocol: SupportedProtocol, message: RPCCodedResponse, fork_name: ForkName, + spec: &ChainSpec, ) -> Result>, RPCError> { - let mut encoded = encode_response(protocol, message, fork_name)?; - decode_response(protocol, &mut encoded, fork_name) + let mut encoded = encode_response(protocol, message, fork_name, spec)?; + decode_response(protocol, &mut encoded, fork_name, spec) } /// Verifies that requests we send are encoded in a way that we would correctly decode too. - fn encode_then_decode_request(req: OutboundRequest, fork_name: ForkName) { + fn encode_then_decode_request( + req: OutboundRequest, + fork_name: ForkName, + spec: &ChainSpec, + ) { let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context); + let max_packet_size = max_rpc_size(&fork_context, spec); let protocol = ProtocolId::new(req.versioned_protocol(), Encoding::SSZSnappy); // Encode a request we send let mut buf = BytesMut::new(); @@ -851,11 +858,14 @@ mod tests { // Test RPCResponse encoding/decoding for V1 messages #[test] fn test_encode_then_decode_v1() { + let chain_spec = Spec::default_spec(); + assert_eq!( encode_then_decode_response( SupportedProtocol::StatusV1, RPCCodedResponse::Success(RPCResponse::Status(status_message())), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::Status(status_message()))) ); @@ -865,6 +875,7 @@ mod tests { SupportedProtocol::PingV1, RPCCodedResponse::Success(RPCResponse::Pong(ping_message())), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::Pong(ping_message()))) ); @@ -874,6 +885,7 @@ mod tests { SupportedProtocol::BlocksByRangeV1, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRange(Arc::new( empty_base_block() @@ -886,6 +898,7 @@ mod tests { SupportedProtocol::BlocksByRangeV1, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(altair_block()))), ForkName::Altair, + &chain_spec, ) .unwrap_err(), RPCError::SSZDecodeError(_) @@ -898,6 +911,7 @@ mod tests { SupportedProtocol::BlocksByRootV1, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRoot( Arc::new(empty_base_block()) @@ -910,6 +924,7 @@ mod tests { SupportedProtocol::BlocksByRootV1, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(altair_block()))), ForkName::Altair, + &chain_spec, ) .unwrap_err(), RPCError::SSZDecodeError(_) @@ -922,6 +937,7 @@ mod tests { SupportedProtocol::MetaDataV1, RPCCodedResponse::Success(RPCResponse::MetaData(metadata())), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::MetaData(metadata()))), ); @@ -932,6 +948,7 @@ mod tests { SupportedProtocol::MetaDataV1, RPCCodedResponse::Success(RPCResponse::MetaData(metadata_v2())), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::MetaData(metadata()))), ); @@ -940,11 +957,14 @@ mod tests { // Test RPCResponse encoding/decoding for V1 messages #[test] fn test_encode_then_decode_v2() { + let chain_spec = Spec::default_spec(); + assert_eq!( encode_then_decode_response( SupportedProtocol::BlocksByRangeV2, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRange(Arc::new( empty_base_block() @@ -959,6 +979,7 @@ mod tests { SupportedProtocol::BlocksByRangeV2, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))), ForkName::Altair, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRange(Arc::new( empty_base_block() @@ -970,12 +991,13 @@ mod tests { SupportedProtocol::BlocksByRangeV2, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(altair_block()))), ForkName::Altair, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRange(Arc::new(altair_block())))) ); - let merge_block_small = merge_block_small(&fork_context(ForkName::Merge)); - let merge_block_large = merge_block_large(&fork_context(ForkName::Merge)); + let merge_block_small = merge_block_small(&fork_context(ForkName::Merge), &chain_spec); + let merge_block_large = merge_block_large(&fork_context(ForkName::Merge), &chain_spec); assert_eq!( encode_then_decode_response( @@ -984,6 +1006,7 @@ mod tests { merge_block_small.clone() ))), ForkName::Merge, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRange(Arc::new( merge_block_small.clone() @@ -1000,6 +1023,7 @@ mod tests { SupportedProtocol::BlocksByRangeV2, &mut encoded, ForkName::Merge, + &chain_spec, ) .unwrap_err(), RPCError::InvalidData(_) @@ -1012,6 +1036,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRoot( Arc::new(empty_base_block()) @@ -1026,6 +1051,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))), ForkName::Altair, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRoot( Arc::new(empty_base_block()) @@ -1037,6 +1063,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(altair_block()))), ForkName::Altair, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRoot(Arc::new(altair_block())))) ); @@ -1048,6 +1075,7 @@ mod tests { merge_block_small.clone() ))), ForkName::Merge, + &chain_spec, ), Ok(Some(RPCResponse::BlocksByRoot(Arc::new(merge_block_small)))) ); @@ -1062,6 +1090,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, &mut encoded, ForkName::Merge, + &chain_spec, ) .unwrap_err(), RPCError::InvalidData(_) @@ -1075,6 +1104,7 @@ mod tests { SupportedProtocol::MetaDataV2, RPCCodedResponse::Success(RPCResponse::MetaData(metadata())), ForkName::Base, + &chain_spec, ), Ok(Some(RPCResponse::MetaData(metadata_v2()))) ); @@ -1084,6 +1114,7 @@ mod tests { SupportedProtocol::MetaDataV2, RPCCodedResponse::Success(RPCResponse::MetaData(metadata_v2())), ForkName::Altair, + &chain_spec, ), Ok(Some(RPCResponse::MetaData(metadata_v2()))) ); @@ -1094,11 +1125,14 @@ mod tests { fn test_context_bytes_v2() { let fork_context = fork_context(ForkName::Altair); + let chain_spec = Spec::default_spec(); + // Removing context bytes for v2 messages should error let mut encoded_bytes = encode_response( SupportedProtocol::BlocksByRangeV2, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))), ForkName::Base, + &chain_spec, ) .unwrap(); @@ -1108,7 +1142,8 @@ mod tests { decode_response( SupportedProtocol::BlocksByRangeV2, &mut encoded_bytes, - ForkName::Base + ForkName::Base, + &chain_spec, ) .unwrap_err(), RPCError::ErrorResponse(RPCResponseErrorCode::InvalidRequest, _), @@ -1118,6 +1153,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))), ForkName::Base, + &chain_spec, ) .unwrap(); @@ -1127,7 +1163,8 @@ mod tests { decode_response( SupportedProtocol::BlocksByRangeV2, &mut encoded_bytes, - ForkName::Base + ForkName::Base, + &chain_spec, ) .unwrap_err(), RPCError::ErrorResponse(RPCResponseErrorCode::InvalidRequest, _), @@ -1138,6 +1175,7 @@ mod tests { SupportedProtocol::BlocksByRangeV2, RPCCodedResponse::Success(RPCResponse::BlocksByRange(Arc::new(empty_base_block()))), ForkName::Altair, + &chain_spec, ) .unwrap(); @@ -1150,7 +1188,8 @@ mod tests { decode_response( SupportedProtocol::BlocksByRangeV2, &mut wrong_fork_bytes, - ForkName::Altair + ForkName::Altair, + &chain_spec, ) .unwrap_err(), RPCError::SSZDecodeError(_), @@ -1161,6 +1200,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(altair_block()))), ForkName::Altair, + &chain_spec, ) .unwrap(); @@ -1172,7 +1212,8 @@ mod tests { decode_response( SupportedProtocol::BlocksByRangeV2, &mut wrong_fork_bytes, - ForkName::Altair + ForkName::Altair, + &chain_spec, ) .unwrap_err(), RPCError::SSZDecodeError(_), @@ -1186,6 +1227,7 @@ mod tests { SupportedProtocol::MetaDataV2, RPCCodedResponse::Success(RPCResponse::MetaData(metadata())), ForkName::Altair, + &chain_spec, ) .unwrap(), ); @@ -1193,7 +1235,8 @@ mod tests { assert!(decode_response( SupportedProtocol::MetaDataV2, &mut encoded_bytes, - ForkName::Altair + ForkName::Altair, + &chain_spec, ) .is_err()); @@ -1202,6 +1245,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))), ForkName::Altair, + &chain_spec, ) .unwrap(); @@ -1213,7 +1257,8 @@ mod tests { decode_response( SupportedProtocol::BlocksByRangeV2, &mut wrong_fork_bytes, - ForkName::Altair + ForkName::Altair, + &chain_spec, ) .unwrap_err(), RPCError::ErrorResponse(RPCResponseErrorCode::InvalidRequest, _), @@ -1224,6 +1269,7 @@ mod tests { SupportedProtocol::BlocksByRootV2, RPCCodedResponse::Success(RPCResponse::BlocksByRoot(Arc::new(empty_base_block()))), ForkName::Altair, + &chain_spec, ) .unwrap(); @@ -1233,7 +1279,8 @@ mod tests { decode_response( SupportedProtocol::BlocksByRangeV2, &mut part, - ForkName::Altair + ForkName::Altair, + &chain_spec, ), Ok(None) ) @@ -1252,9 +1299,12 @@ mod tests { OutboundRequest::MetaData(MetadataRequest::new_v1()), OutboundRequest::MetaData(MetadataRequest::new_v2()), ]; + + let chain_spec = Spec::default_spec(); + for req in requests.iter() { for fork_name in ForkName::list_all() { - encode_then_decode_request(req.clone(), fork_name); + encode_then_decode_request(req.clone(), fork_name, &chain_spec); } } } @@ -1308,9 +1358,16 @@ mod tests { assert_eq!(writer.get_ref().len(), 42); dst.extend_from_slice(writer.get_ref()); + let chain_spec = Spec::default_spec(); // 10 (for stream identifier) + 80 + 42 = 132 > `max_compressed_len`. Hence, decoding should fail with `InvalidData`. assert!(matches!( - decode_response(SupportedProtocol::StatusV1, &mut dst, ForkName::Base).unwrap_err(), + decode_response( + SupportedProtocol::StatusV1, + &mut dst, + ForkName::Base, + &chain_spec + ) + .unwrap_err(), RPCError::InvalidData(_) )); } @@ -1365,12 +1422,15 @@ mod tests { assert_eq!(writer.get_ref().len(), 8103); dst.extend_from_slice(writer.get_ref()); + let chain_spec = Spec::default_spec(); + // 10 (for stream identifier) + 176156 + 8103 = 184269 > `max_compressed_len`. Hence, decoding should fail with `InvalidData`. assert!(matches!( decode_response( SupportedProtocol::BlocksByRangeV2, &mut dst, - ForkName::Altair + ForkName::Altair, + &chain_spec, ) .unwrap_err(), RPCError::InvalidData(_) @@ -1412,8 +1472,16 @@ mod tests { writer.flush().unwrap(); dst.extend_from_slice(writer.get_ref()); + let chain_spec = Spec::default_spec(); + assert!(matches!( - decode_response(SupportedProtocol::StatusV1, &mut dst, ForkName::Base).unwrap_err(), + decode_response( + SupportedProtocol::StatusV1, + &mut dst, + ForkName::Base, + &chain_spec + ) + .unwrap_err(), RPCError::InvalidData(_) )); } diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index 9470ff20601..c67d62e2ed3 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -973,7 +973,7 @@ where OutboundRequestContainer { req: req.clone(), fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context), + max_rpc_size: max_rpc_size(&self.fork_context, &self.chain_spec), }, (), ) diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 6c3350878b2..8d5a493efbb 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -219,7 +219,7 @@ where SubstreamProtocol::new( RPCProtocol { fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context), + max_rpc_size: max_rpc_size(&self.fork_context, &self.chain_spec), enable_light_client_server: self.enable_light_client_server, phantom: PhantomData, }, diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index cddf380ada3..22ad9e69868 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -22,7 +22,7 @@ use tokio_util::{ }; use types::{ BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockCapella, BeaconBlockMerge, - EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, + ChainSpec, EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, SignedBeaconBlock, }; @@ -119,9 +119,9 @@ const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req"; const REQUEST_TIMEOUT: u64 = 15; /// Returns the maximum bytes that can be sent across the RPC. -pub fn max_rpc_size(fork_context: &ForkContext) -> usize { +pub fn max_rpc_size(fork_context: &ForkContext, spec: &ChainSpec) -> usize { match fork_context.current_fork() { - ForkName::Altair | ForkName::Base => MainnetEthSpec::default_spec().max_chunk_size as usize, + ForkName::Altair | ForkName::Base => spec.max_chunk_size as usize, ForkName::Merge => MAX_RPC_SIZE_POST_MERGE, ForkName::Capella => MAX_RPC_SIZE_POST_CAPELLA, } diff --git a/beacon_node/lighthouse_network/tests/rpc_tests.rs b/beacon_node/lighthouse_network/tests/rpc_tests.rs index fff836dec3b..155fea710cb 100644 --- a/beacon_node/lighthouse_network/tests/rpc_tests.rs +++ b/beacon_node/lighthouse_network/tests/rpc_tests.rs @@ -9,8 +9,9 @@ use std::time::Duration; use tokio::runtime::Runtime; use tokio::time::sleep; use types::{ - BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EmptyBlock, Epoch, EthSpec, - ForkContext, ForkName, Hash256, MinimalEthSpec, Signature, SignedBeaconBlock, Slot, + BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, ChainSpec, EmptyBlock, + Epoch, EthSpec, ForkContext, ForkName, Hash256, MinimalEthSpec, Signature, SignedBeaconBlock, + Slot, }; mod common; @@ -18,30 +19,30 @@ mod common; type E = MinimalEthSpec; /// Merge block with length < max_rpc_size. -fn merge_block_small(fork_context: &ForkContext) -> BeaconBlock { - let mut block = BeaconBlockMerge::::empty(&E::default_spec()); +fn merge_block_small(fork_context: &ForkContext, spec: &ChainSpec) -> BeaconBlock { + let mut block = BeaconBlockMerge::::empty(spec); let tx = VariableList::from(vec![0; 1024]); let txs = VariableList::from(std::iter::repeat(tx).take(5000).collect::>()); block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context)); + assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, spec)); block } /// Merge block with length > MAX_RPC_SIZE. /// The max limit for a merge block is in the order of ~16GiB which wouldn't fit in memory. /// Hence, we generate a merge block just greater than `MAX_RPC_SIZE` to test rejection on the rpc layer. -fn merge_block_large(fork_context: &ForkContext) -> BeaconBlock { - let mut block = BeaconBlockMerge::::empty(&E::default_spec()); +fn merge_block_large(fork_context: &ForkContext, spec: &ChainSpec) -> BeaconBlock { + let mut block = BeaconBlockMerge::::empty(spec); let tx = VariableList::from(vec![0; 1024]); let txs = VariableList::from(std::iter::repeat(tx).take(100000).collect::>()); block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() > max_rpc_size(fork_context)); + assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, spec)); block } @@ -172,7 +173,7 @@ fn test_blocks_by_range_chunked_rpc() { let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty()); let rpc_response_altair = Response::BlocksByRange(Some(Arc::new(signed_full_block))); - let full_block = merge_block_small(&common::fork_context(ForkName::Merge)); + let full_block = merge_block_small(&common::fork_context(ForkName::Merge), &spec); let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty()); let rpc_response_merge_small = Response::BlocksByRange(Some(Arc::new(signed_full_block))); @@ -288,7 +289,7 @@ fn test_blocks_by_range_over_limit() { let rpc_request = Request::BlocksByRange(BlocksByRangeRequest::new(0, messages_to_send)); // BlocksByRange Response - let full_block = merge_block_large(&common::fork_context(ForkName::Merge)); + let full_block = merge_block_large(&common::fork_context(ForkName::Merge), &spec); let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty()); let rpc_response_merge_large = Response::BlocksByRange(Some(Arc::new(signed_full_block))); @@ -611,7 +612,7 @@ fn test_blocks_by_root_chunked_rpc() { let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty()); let rpc_response_altair = Response::BlocksByRoot(Some(Arc::new(signed_full_block))); - let full_block = merge_block_small(&common::fork_context(ForkName::Merge)); + let full_block = merge_block_small(&common::fork_context(ForkName::Merge), &spec); let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty()); let rpc_response_merge_small = Response::BlocksByRoot(Some(Arc::new(signed_full_block))); From eec5813f4f85ad392ec989d93883f1e28bf9df68 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 28 Jun 2023 12:50:46 -0400 Subject: [PATCH 18/39] remove the needless borrows --- beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index 45607c01952..a6f2a567459 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -667,7 +667,7 @@ mod tests { block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, &spec)); + assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, spec)); SignedBeaconBlock::from_block(block, Signature::empty()) } @@ -683,7 +683,7 @@ mod tests { block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, &spec)); + assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, spec)); SignedBeaconBlock::from_block(block, Signature::empty()) } @@ -741,7 +741,7 @@ mod tests { ) -> Result { let snappy_protocol_id = ProtocolId::new(protocol, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context, &spec); + let max_packet_size = max_rpc_size(&fork_context, spec); let mut buf = BytesMut::new(); let mut snappy_inbound_codec = From 6ca24e94f02cbccdc9428f894772ab4231807739 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 28 Jun 2023 16:24:55 -0400 Subject: [PATCH 19/39] Fixing some test errors --- .../built_in_network_configs/mainnet/config.yaml | 13 ++++++++++++- consensus/types/src/chain_spec.rs | 7 +++++++ .../environment/tests/testnet_dir/config.yaml | 15 +++++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml index 7b26b30a6ce..7333f0bc267 100644 --- a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml @@ -89,4 +89,15 @@ DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa # Network # --------------------------------------------------------------- -SUBNETS_PER_NODE: 2 \ No newline at end of file +SUBNETS_PER_NODE: 2 +GOSSIP_MAX_SIZE: 10485760 +MAX_REQUEST_BLOCKS: 1024 +MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 +MAX_CHUNK_SIZE: 10485760 +TTFB_TIMEOUT: 5 +RESP_TIMEOUT: 10 +MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 +MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 +ATTESTATION_SUBNET_COUNT: 64 +ATTESTATION_SUBNET_EXTRA_BITS: 0 +ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 4e5adea5788..e761b5df4cb 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -1478,6 +1478,13 @@ mod yaml_tests { DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 + MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 + MAX_CHUNK_SIZE: 1048576 + TTFB_TIMEOUT: 5 + RESP_TIMEOUT: 10 + MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 + MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 + ATTESTATION_SUBNET_PREFIX_BITS: 6 "#; let chain_spec: Config = serde_yaml::from_str(spec).unwrap(); diff --git a/lighthouse/environment/tests/testnet_dir/config.yaml b/lighthouse/environment/tests/testnet_dir/config.yaml index 33aa8ad165d..faf63433ce0 100644 --- a/lighthouse/environment/tests/testnet_dir/config.yaml +++ b/lighthouse/environment/tests/testnet_dir/config.yaml @@ -81,3 +81,18 @@ PROPOSER_SCORE_BOOST: 40 DEPOSIT_CHAIN_ID: 1 DEPOSIT_NETWORK_ID: 1 DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa + +# Network +# --------------------------------------------------------------- +SUBNETS_PER_NODE: 2 +GOSSIP_MAX_SIZE: 10485760 +MAX_REQUEST_BLOCKS: 1024 +MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 +MAX_CHUNK_SIZE: 10485760 +TTFB_TIMEOUT: 5 +RESP_TIMEOUT: 10 +MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 +MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 +ATTESTATION_SUBNET_COUNT: 64 +ATTESTATION_SUBNET_EXTRA_BITS: 0 +ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file From 8f3b9a4cba67544c2f3aea58e014d61170d0fe96 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 28 Jun 2023 18:04:21 -0400 Subject: [PATCH 20/39] Fix the other tests --- .vscode/settings.json | 5 +++++ .../built_in_network_configs/gnosis/config.yaml | 13 ++++++++++++- .../built_in_network_configs/mainnet/config.yaml | 2 +- .../built_in_network_configs/prater/config.yaml | 13 ++++++++++++- .../built_in_network_configs/sepolia/config.yaml | 13 ++++++++++++- .../environment/tests/testnet_dir/config.yaml | 2 +- 6 files changed, 43 insertions(+), 5 deletions(-) create mode 100644 .vscode/settings.json diff --git a/.vscode/settings.json b/.vscode/settings.json new file mode 100644 index 00000000000..7a1a9e637ac --- /dev/null +++ b/.vscode/settings.json @@ -0,0 +1,5 @@ +{ + "rust-analyzer.linkedProjects": [ + "./beacon_node/network/Cargo.toml" + ] +} \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml index 95ca9d01080..113796017a0 100644 --- a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml @@ -89,4 +89,15 @@ DEPOSIT_CONTRACT_ADDRESS: 0x0B98057eA310F4d31F2a452B414647007d1645d9 # Network # --------------------------------------------------------------- -SUBNETS_PER_NODE: 4 \ No newline at end of file +SUBNETS_PER_NODE: 4 +GOSSIP_MAX_SIZE: 10485760 +MAX_REQUEST_BLOCKS: 1024 +MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 +MAX_CHUNK_SIZE: 1048576 +TTFB_TIMEOUT: 5 +RESP_TIMEOUT: 10 +MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 +MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 +ATTESTATION_SUBNET_COUNT: 64 +ATTESTATION_SUBNET_EXTRA_BITS: 0 +ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml index 7333f0bc267..ee091c0998b 100644 --- a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml @@ -93,7 +93,7 @@ SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 10485760 +MAX_CHUNK_SIZE: 1048576 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 diff --git a/common/eth2_network_config/built_in_network_configs/prater/config.yaml b/common/eth2_network_config/built_in_network_configs/prater/config.yaml index 63b3d45db9a..4a3482a6bba 100644 --- a/common/eth2_network_config/built_in_network_configs/prater/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/prater/config.yaml @@ -89,4 +89,15 @@ DEPOSIT_CONTRACT_ADDRESS: 0xff50ed3d0ec03aC01D4C79aAd74928BFF48a7b2b # Network # --------------------------------------------------------------- -SUBNETS_PER_NODE: 2 \ No newline at end of file +SUBNETS_PER_NODE: 2 +GOSSIP_MAX_SIZE: 10485760 +MAX_REQUEST_BLOCKS: 1024 +MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 +MAX_CHUNK_SIZE: 1048576 +TTFB_TIMEOUT: 5 +RESP_TIMEOUT: 10 +MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 +MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 +ATTESTATION_SUBNET_COUNT: 64 +ATTESTATION_SUBNET_EXTRA_BITS: 0 +ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file diff --git a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml index 8489f085f4c..fe61e0b26eb 100644 --- a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml @@ -77,4 +77,15 @@ DEPOSIT_CONTRACT_ADDRESS: 0x7f02C3E3c98b133055B8B348B2Ac625669Ed295D # Network # --------------------------------------------------------------- -SUBNETS_PER_NODE: 2 \ No newline at end of file +SUBNETS_PER_NODE: 2 +GOSSIP_MAX_SIZE: 10485760 +MAX_REQUEST_BLOCKS: 1024 +MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 +MAX_CHUNK_SIZE: 1048576 +TTFB_TIMEOUT: 5 +RESP_TIMEOUT: 10 +MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 +MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 +ATTESTATION_SUBNET_COUNT: 64 +ATTESTATION_SUBNET_EXTRA_BITS: 0 +ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file diff --git a/lighthouse/environment/tests/testnet_dir/config.yaml b/lighthouse/environment/tests/testnet_dir/config.yaml index faf63433ce0..ad69a3aa253 100644 --- a/lighthouse/environment/tests/testnet_dir/config.yaml +++ b/lighthouse/environment/tests/testnet_dir/config.yaml @@ -88,7 +88,7 @@ SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 10485760 +MAX_CHUNK_SIZE: 1048576 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 From e003f20077a948095cfeb8166a3ecfd68c454ab4 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 29 Jun 2023 10:34:01 -0400 Subject: [PATCH 21/39] Resolve the some reviews --- .vscode/settings.json | 5 ----- .../beacon_chain/src/attestation_verification.rs | 8 +++----- .../src/sync_committee_verification.rs | 16 ++++++++-------- beacon_node/lighthouse_network/src/config.rs | 4 ++-- .../lighthouse_network/src/rpc/codec/base.rs | 6 +++--- .../src/rpc/codec/ssz_snappy.rs | 10 +++++----- .../lighthouse_network/src/rpc/handler.rs | 16 ++++++++-------- beacon_node/lighthouse_network/src/rpc/mod.rs | 3 ++- .../lighthouse_network/src/rpc/protocol.rs | 11 +++++------ .../lighthouse_network/tests/rpc_tests.rs | 4 ++-- .../beacon_processor/worker/gossip_methods.rs | 1 + consensus/types/src/chain_spec.rs | 13 +++++++++++++ 12 files changed, 52 insertions(+), 45 deletions(-) delete mode 100644 .vscode/settings.json diff --git a/.vscode/settings.json b/.vscode/settings.json deleted file mode 100644 index 7a1a9e637ac..00000000000 --- a/.vscode/settings.json +++ /dev/null @@ -1,5 +0,0 @@ -{ - "rust-analyzer.linkedProjects": [ - "./beacon_node/network/Cargo.toml" - ] -} \ No newline at end of file diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index e04a8d99677..c9530fb051f 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -51,7 +51,7 @@ use state_processing::{ signed_aggregate_selection_proof_signature_set, signed_aggregate_signature_set, }, }; -use std::{borrow::Cow, time::Duration}; +use std::borrow::Cow; use strum::AsRefStr; use tree_hash::TreeHash; use types::{ @@ -1034,10 +1034,8 @@ pub fn verify_propagation_slot_range( spec: &ChainSpec, ) -> Result<(), Error> { let attestation_slot = attestation.data.slot; - let maximum_gossip_clock_disparity = - Duration::from_millis(spec.maximum_gossip_clock_disparity_millis); let latest_permissible_slot = slot_clock - .now_with_future_tolerance(maximum_gossip_clock_disparity) + .now_with_future_tolerance(spec.clone().maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if attestation_slot > latest_permissible_slot { return Err(Error::FutureSlot { @@ -1048,7 +1046,7 @@ pub fn verify_propagation_slot_range( // Taking advantage of saturating subtraction on `Slot`. let earliest_permissible_slot = slot_clock - .now_with_past_tolerance(maximum_gossip_clock_disparity) + .now_with_past_tolerance(spec.clone().maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)? - E::slots_per_epoch(); if attestation_slot < earliest_permissible_slot { diff --git a/beacon_node/beacon_chain/src/sync_committee_verification.rs b/beacon_node/beacon_chain/src/sync_committee_verification.rs index 741f3fe8066..bbfe25d057e 100644 --- a/beacon_node/beacon_chain/src/sync_committee_verification.rs +++ b/beacon_node/beacon_chain/src/sync_committee_verification.rs @@ -41,8 +41,9 @@ use state_processing::signature_sets::{ sync_committee_contribution_signature_set_from_pubkeys, sync_committee_message_set_from_pubkeys, }; +use types::ChainSpec; use std::collections::HashMap; -use std::{borrow::Cow, time::Duration}; +use std::borrow::Cow; use strum::AsRefStr; use tree_hash::TreeHash; use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT; @@ -50,7 +51,7 @@ use types::slot_data::SlotData; use types::sync_committee::Error as SyncCommitteeError; use types::{ sync_committee_contribution::Error as ContributionError, AggregateSignature, BeaconStateError, - EthSpec, Hash256, MainnetEthSpec, SignedContributionAndProof, Slot, SyncCommitteeContribution, + EthSpec, Hash256, SignedContributionAndProof, Slot, SyncCommitteeContribution, SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, }; @@ -285,7 +286,7 @@ impl VerifiedSyncContribution { let subcommittee_index = contribution.subcommittee_index as usize; // Ensure sync committee contribution is within the MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance. - verify_propagation_slot_range(&chain.slot_clock, contribution)?; + verify_propagation_slot_range(&chain.slot_clock, contribution, &chain.spec)?; // Validate subcommittee index. if contribution.subcommittee_index >= SYNC_COMMITTEE_SUBNET_COUNT { @@ -440,7 +441,7 @@ impl VerifiedSyncCommitteeMessage { // MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance). // // We do not queue future sync committee messages for later processing. - verify_propagation_slot_range(&chain.slot_clock, &sync_message)?; + verify_propagation_slot_range(&chain.slot_clock, &sync_message, &chain.spec)?; // Ensure the `subnet_id` is valid for the given validator. let pubkey = chain @@ -556,12 +557,11 @@ impl VerifiedSyncCommitteeMessage { pub fn verify_propagation_slot_range( slot_clock: &S, sync_contribution: &U, + spec: &ChainSpec, ) -> Result<(), Error> { let message_slot = sync_contribution.get_slot(); - let maximum_gossip_clock_disparity = - Duration::from_millis(MainnetEthSpec::default_spec().maximum_gossip_clock_disparity_millis); let latest_permissible_slot = slot_clock - .now_with_future_tolerance(maximum_gossip_clock_disparity) + .now_with_future_tolerance(spec.clone().maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if message_slot > latest_permissible_slot { return Err(Error::FutureSlot { @@ -571,7 +571,7 @@ pub fn verify_propagation_slot_range( } let earliest_permissible_slot = slot_clock - .now_with_past_tolerance(maximum_gossip_clock_disparity) + .now_with_past_tolerance(spec.clone().maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if message_slot < earliest_permissible_slot { diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 53a1b693ad2..67164ed1178 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -444,13 +444,13 @@ pub fn gossipsub_config( } } } - + let cloned_spec = spec.clone(); let is_merge_enabled = fork_context.fork_exists(ForkName::Merge); let gossip_message_id = move |message: &GossipsubMessage| { MessageId::from( &Sha256::digest( prefix( - TSpec::default_spec().message_domain_valid_snappy, + cloned_spec.message_domain_valid_snappy, message, fork_context.clone(), ) diff --git a/beacon_node/lighthouse_network/src/rpc/codec/base.rs b/beacon_node/lighthouse_network/src/rpc/codec/base.rs index 79282e53f45..943d4a3bce2 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/base.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/base.rs @@ -222,7 +222,7 @@ mod tests { let mut snappy_outbound_codec = SSZSnappyOutboundCodec::::new( snappy_protocol_id, - max_rpc_size(&fork_context, &chain_spec), + max_rpc_size(&fork_context, chain_spec.max_chunk_size as usize), fork_context, ); @@ -259,7 +259,7 @@ mod tests { let mut snappy_outbound_codec = SSZSnappyOutboundCodec::::new( snappy_protocol_id, - max_rpc_size(&fork_context, &chain_spec), + max_rpc_size(&fork_context, chain_spec.max_chunk_size as usize), fork_context, ); @@ -288,7 +288,7 @@ mod tests { let chain_spec = Spec::default_spec(); - let max_rpc_size = max_rpc_size(&fork_context, &chain_spec); + let max_rpc_size = max_rpc_size(&fork_context, chain_spec.max_chunk_size as usize); let limit = protocol_id.rpc_response_limits::(&fork_context); let mut max = encode_len(limit.max + 1); let mut codec = SSZSnappyOutboundCodec::::new( diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index a6f2a567459..c6fd422007c 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -667,7 +667,7 @@ mod tests { block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, spec)); + assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, spec.max_chunk_size as usize)); SignedBeaconBlock::from_block(block, Signature::empty()) } @@ -683,7 +683,7 @@ mod tests { block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, spec)); + assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, spec.max_chunk_size as usize)); SignedBeaconBlock::from_block(block, Signature::empty()) } @@ -741,7 +741,7 @@ mod tests { ) -> Result { let snappy_protocol_id = ProtocolId::new(protocol, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context, spec); + let max_packet_size = max_rpc_size(&fork_context, spec.max_chunk_size as usize); let mut buf = BytesMut::new(); let mut snappy_inbound_codec = @@ -788,7 +788,7 @@ mod tests { ) -> Result>, RPCError> { let snappy_protocol_id = ProtocolId::new(protocol, Encoding::SSZSnappy); let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context, spec); + let max_packet_size = max_rpc_size(&fork_context, spec.max_chunk_size as usize); let mut snappy_outbound_codec = SSZSnappyOutboundCodec::::new(snappy_protocol_id, max_packet_size, fork_context); // decode message just as snappy message @@ -813,7 +813,7 @@ mod tests { spec: &ChainSpec, ) { let fork_context = Arc::new(fork_context(fork_name)); - let max_packet_size = max_rpc_size(&fork_context, spec); + let max_packet_size = max_rpc_size(&fork_context, spec.max_chunk_size as usize); let protocol = ProtocolId::new(req.versioned_protocol(), Encoding::SSZSnappy); // Encode a request we send let mut buf = BytesMut::new(); diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index c67d62e2ed3..129871ef0eb 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -129,8 +129,8 @@ where /// Logger for handling RPC streams log: slog::Logger, - /// ChainSpec for networking constants - chain_spec: ChainSpec, + /// resp_timeout for networking constants + resp_timeout: Duration, } enum HandlerState { @@ -232,7 +232,7 @@ where fork_context, waker: None, log: log.clone(), - chain_spec: spec.clone(), + resp_timeout: spec.clone().resp_timeout(), } } @@ -353,7 +353,7 @@ where // new outbound request. Store the stream and tag the output. let delay_key = self.outbound_substreams_delay.insert( self.current_outbound_substream_id, - Duration::from_secs(self.chain_spec.resp_timeout), + self.resp_timeout, ); let awaiting_stream = OutboundSubstreamState::RequestPendingResponse { substream: Box::new(out), @@ -404,7 +404,7 @@ where // Store the stream and tag the output. let delay_key = self.inbound_substreams_delay.insert( self.current_inbound_substream_id, - Duration::from_secs(self.chain_spec.resp_timeout), + self.resp_timeout, ); let awaiting_stream = InboundState::Idle(substream); self.inbound_substreams.insert( @@ -717,7 +717,7 @@ where if let Some(ref delay_key) = info.delay_key { self.inbound_substreams_delay.reset( delay_key, - Duration::from_secs(self.chain_spec.resp_timeout), + self.resp_timeout, ); } @@ -853,7 +853,7 @@ where substream_entry.remaining_chunks = Some(remaining_chunks); self.outbound_substreams_delay.reset( delay_key, - Duration::from_secs(self.chain_spec.resp_timeout), + self.resp_timeout, ); } } else { @@ -973,7 +973,7 @@ where OutboundRequestContainer { req: req.clone(), fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context, &self.chain_spec), + max_rpc_size: max_rpc_size(&self.fork_context, self.listen_protocol.upgrade().max_rpc_size), }, (), ) diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 8d5a493efbb..8f13468de15 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -219,9 +219,10 @@ where SubstreamProtocol::new( RPCProtocol { fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context, &self.chain_spec), + max_rpc_size: max_rpc_size(&self.fork_context, self.chain_spec.max_chunk_size as usize), enable_light_client_server: self.enable_light_client_server, phantom: PhantomData, + ttfb_timeout: self.chain_spec.clone().ttfb_timeout(), }, (), ), diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index 22ad9e69868..552414b1237 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -22,7 +22,7 @@ use tokio_util::{ }; use types::{ BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockCapella, BeaconBlockMerge, - ChainSpec, EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, + EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, SignedBeaconBlock, }; @@ -119,9 +119,9 @@ const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req"; const REQUEST_TIMEOUT: u64 = 15; /// Returns the maximum bytes that can be sent across the RPC. -pub fn max_rpc_size(fork_context: &ForkContext, spec: &ChainSpec) -> usize { +pub fn max_rpc_size(fork_context: &ForkContext, max_chunk_size: usize) -> usize { match fork_context.current_fork() { - ForkName::Altair | ForkName::Base => spec.max_chunk_size as usize, + ForkName::Altair | ForkName::Base => max_chunk_size, ForkName::Merge => MAX_RPC_SIZE_POST_MERGE, ForkName::Capella => MAX_RPC_SIZE_POST_CAPELLA, } @@ -258,6 +258,7 @@ pub struct RPCProtocol { pub max_rpc_size: usize, pub enable_light_client_server: bool, pub phantom: PhantomData, + pub ttfb_timeout: Duration, } impl UpgradeInfo for RPCProtocol { @@ -443,9 +444,7 @@ where } }; let mut timed_socket = TimeoutStream::new(socket); - timed_socket.set_read_timeout(Some(Duration::from_secs( - TSpec::default_spec().ttfb_timeout, - ))); + timed_socket.set_read_timeout(Some(self.ttfb_timeout)); let socket = Framed::new(Box::pin(timed_socket), codec); diff --git a/beacon_node/lighthouse_network/tests/rpc_tests.rs b/beacon_node/lighthouse_network/tests/rpc_tests.rs index 155fea710cb..05fa5ab8542 100644 --- a/beacon_node/lighthouse_network/tests/rpc_tests.rs +++ b/beacon_node/lighthouse_network/tests/rpc_tests.rs @@ -27,7 +27,7 @@ fn merge_block_small(fork_context: &ForkContext, spec: &ChainSpec) -> BeaconBloc block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, spec)); + assert!(block.ssz_bytes_len() <= max_rpc_size(fork_context, spec.max_chunk_size as usize)); block } @@ -42,7 +42,7 @@ fn merge_block_large(fork_context: &ForkContext, spec: &ChainSpec) -> BeaconBloc block.body.execution_payload.execution_payload.transactions = txs; let block = BeaconBlock::Merge(block); - assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, spec)); + assert!(block.ssz_bytes_len() > max_rpc_size(fork_context, spec.max_chunk_size as usize)); block } diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 76bf2485e35..40623ae4845 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -2187,6 +2187,7 @@ impl Worker { sync_committee_verification::verify_propagation_slot_range( seen_clock, &sync_committee_message_slot, + &self.chain.spec, ); hindsight_verification.is_err() }; diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index e761b5df4cb..42e919cd700 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -6,6 +6,7 @@ use serde_derive::Deserialize; use serde_utils::quoted_u64::MaybeQuoted; use std::fs::File; use std::path::Path; +use std::time::Duration; use tree_hash::TreeHash; /// Each of the BLS signature domains. @@ -461,6 +462,18 @@ impl ChainSpec { Hash256::from(domain) } + pub fn maximum_gossip_clock_disparity(self) -> Duration { + Duration::from_millis(self.maximum_gossip_clock_disparity_millis) + } + + pub fn ttfb_timeout(self) -> Duration { + Duration::from_secs(self.ttfb_timeout) + } + + pub fn resp_timeout(self) -> Duration { + Duration::from_secs(self.resp_timeout) + } + /// Returns a `ChainSpec` compatible with the Ethereum Foundation specification. pub fn mainnet() -> Self { Self { From 49074a3c044ed5703ac71aba93c274323860b4db Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 29 Jun 2023 10:36:42 -0400 Subject: [PATCH 22/39] fmt --- .../src/sync_committee_verification.rs | 4 +-- .../lighthouse_network/src/rpc/handler.rs | 31 +++++++++---------- beacon_node/lighthouse_network/src/rpc/mod.rs | 5 ++- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/beacon_node/beacon_chain/src/sync_committee_verification.rs b/beacon_node/beacon_chain/src/sync_committee_verification.rs index bbfe25d057e..5633853be7c 100644 --- a/beacon_node/beacon_chain/src/sync_committee_verification.rs +++ b/beacon_node/beacon_chain/src/sync_committee_verification.rs @@ -41,14 +41,14 @@ use state_processing::signature_sets::{ sync_committee_contribution_signature_set_from_pubkeys, sync_committee_message_set_from_pubkeys, }; -use types::ChainSpec; -use std::collections::HashMap; use std::borrow::Cow; +use std::collections::HashMap; use strum::AsRefStr; use tree_hash::TreeHash; use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT; use types::slot_data::SlotData; use types::sync_committee::Error as SyncCommitteeError; +use types::ChainSpec; use types::{ sync_committee_contribution::Error as ContributionError, AggregateSignature, BeaconStateError, EthSpec, Hash256, SignedContributionAndProof, Slot, SyncCommitteeContribution, diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index 129871ef0eb..beb97e788aa 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -351,10 +351,9 @@ where let expected_responses = request.expected_responses(); if expected_responses > 0 { // new outbound request. Store the stream and tag the output. - let delay_key = self.outbound_substreams_delay.insert( - self.current_outbound_substream_id, - self.resp_timeout, - ); + let delay_key = self + .outbound_substreams_delay + .insert(self.current_outbound_substream_id, self.resp_timeout); let awaiting_stream = OutboundSubstreamState::RequestPendingResponse { substream: Box::new(out), request, @@ -402,10 +401,9 @@ where if expected_responses > 0 { if self.inbound_substreams.len() < MAX_INBOUND_SUBSTREAMS { // Store the stream and tag the output. - let delay_key = self.inbound_substreams_delay.insert( - self.current_inbound_substream_id, - self.resp_timeout, - ); + let delay_key = self + .inbound_substreams_delay + .insert(self.current_inbound_substream_id, self.resp_timeout); let awaiting_stream = InboundState::Idle(substream); self.inbound_substreams.insert( self.current_inbound_substream_id, @@ -715,10 +713,8 @@ where // If this substream has not ended, we reset the timer. // Each chunk is allowed RESPONSE_TIMEOUT to be sent. if let Some(ref delay_key) = info.delay_key { - self.inbound_substreams_delay.reset( - delay_key, - self.resp_timeout, - ); + self.inbound_substreams_delay + .reset(delay_key, self.resp_timeout); } // The stream may be currently idle. Attempt to process more @@ -851,10 +847,8 @@ where request, }; substream_entry.remaining_chunks = Some(remaining_chunks); - self.outbound_substreams_delay.reset( - delay_key, - self.resp_timeout, - ); + self.outbound_substreams_delay + .reset(delay_key, self.resp_timeout); } } else { // either this is a single response request or this response closes the @@ -973,7 +967,10 @@ where OutboundRequestContainer { req: req.clone(), fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context, self.listen_protocol.upgrade().max_rpc_size), + max_rpc_size: max_rpc_size( + &self.fork_context, + self.listen_protocol.upgrade().max_rpc_size, + ), }, (), ) diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 8f13468de15..f7b1edfb66b 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -219,7 +219,10 @@ where SubstreamProtocol::new( RPCProtocol { fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context, self.chain_spec.max_chunk_size as usize), + max_rpc_size: max_rpc_size( + &self.fork_context, + self.chain_spec.max_chunk_size as usize, + ), enable_light_client_server: self.enable_light_client_server, phantom: PhantomData, ttfb_timeout: self.chain_spec.clone().ttfb_timeout(), From ba59277a53e85cceec8160b00e8b728ae779d865 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 29 Jun 2023 13:01:57 -0400 Subject: [PATCH 23/39] change self instead of &self for removing clone and unnecessary consuming the ChainSpec --- beacon_node/beacon_chain/src/attestation_verification.rs | 4 ++-- beacon_node/beacon_chain/src/sync_committee_verification.rs | 4 ++-- beacon_node/lighthouse_network/src/rpc/handler.rs | 2 +- beacon_node/lighthouse_network/src/rpc/mod.rs | 2 +- consensus/types/src/chain_spec.rs | 6 +++--- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index c9530fb051f..8ff1c88cdfc 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -1035,7 +1035,7 @@ pub fn verify_propagation_slot_range( ) -> Result<(), Error> { let attestation_slot = attestation.data.slot; let latest_permissible_slot = slot_clock - .now_with_future_tolerance(spec.clone().maximum_gossip_clock_disparity()) + .now_with_future_tolerance(spec.maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if attestation_slot > latest_permissible_slot { return Err(Error::FutureSlot { @@ -1046,7 +1046,7 @@ pub fn verify_propagation_slot_range( // Taking advantage of saturating subtraction on `Slot`. let earliest_permissible_slot = slot_clock - .now_with_past_tolerance(spec.clone().maximum_gossip_clock_disparity()) + .now_with_past_tolerance(spec.maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)? - E::slots_per_epoch(); if attestation_slot < earliest_permissible_slot { diff --git a/beacon_node/beacon_chain/src/sync_committee_verification.rs b/beacon_node/beacon_chain/src/sync_committee_verification.rs index 5633853be7c..eacf0cb60e2 100644 --- a/beacon_node/beacon_chain/src/sync_committee_verification.rs +++ b/beacon_node/beacon_chain/src/sync_committee_verification.rs @@ -561,7 +561,7 @@ pub fn verify_propagation_slot_range( ) -> Result<(), Error> { let message_slot = sync_contribution.get_slot(); let latest_permissible_slot = slot_clock - .now_with_future_tolerance(spec.clone().maximum_gossip_clock_disparity()) + .now_with_future_tolerance(spec.maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if message_slot > latest_permissible_slot { return Err(Error::FutureSlot { @@ -571,7 +571,7 @@ pub fn verify_propagation_slot_range( } let earliest_permissible_slot = slot_clock - .now_with_past_tolerance(spec.clone().maximum_gossip_clock_disparity()) + .now_with_past_tolerance(spec.maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if message_slot < earliest_permissible_slot { diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index beb97e788aa..f9cbebaa012 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -232,7 +232,7 @@ where fork_context, waker: None, log: log.clone(), - resp_timeout: spec.clone().resp_timeout(), + resp_timeout: spec.resp_timeout(), } } diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index f7b1edfb66b..82bd346df6c 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -225,7 +225,7 @@ where ), enable_light_client_server: self.enable_light_client_server, phantom: PhantomData, - ttfb_timeout: self.chain_spec.clone().ttfb_timeout(), + ttfb_timeout: self.chain_spec.ttfb_timeout(), }, (), ), diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 42e919cd700..d68ea0f7ac1 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -462,15 +462,15 @@ impl ChainSpec { Hash256::from(domain) } - pub fn maximum_gossip_clock_disparity(self) -> Duration { + pub fn maximum_gossip_clock_disparity(&self) -> Duration { Duration::from_millis(self.maximum_gossip_clock_disparity_millis) } - pub fn ttfb_timeout(self) -> Duration { + pub fn ttfb_timeout(&self) -> Duration { Duration::from_secs(self.ttfb_timeout) } - pub fn resp_timeout(self) -> Duration { + pub fn resp_timeout(&self) -> Duration { Duration::from_secs(self.resp_timeout) } From 7ce90059aa940b185fdb446326faf935e3186f48 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Mon, 10 Jul 2023 14:40:13 -0400 Subject: [PATCH 24/39] Changes after review --- .../beacon_chain/src/block_verification.rs | 4 +-- ...ght_client_finality_update_verification.rs | 3 +-- ...t_client_optimistic_update_verification.rs | 3 +-- beacon_node/http_api/src/attester_duties.rs | 5 +--- beacon_node/http_api/src/proposer_duties.rs | 6 ++--- beacon_node/http_api/src/sync_committees.rs | 5 +--- beacon_node/http_api/tests/tests.rs | 5 ++-- beacon_node/lighthouse_network/src/config.rs | 9 ++----- .../src/rpc/codec/ssz_snappy.rs | 6 ++--- .../lighthouse_network/src/rpc/handler.rs | 8 +++--- beacon_node/lighthouse_network/src/rpc/mod.rs | 26 +++++++++++-------- .../lighthouse_network/src/service/mod.rs | 7 ++--- .../lighthouse_network/tests/common.rs | 16 +++++------- .../network/src/beacon_processor/mod.rs | 1 + .../work_reprocessing_queue.rs | 8 ++---- beacon_node/network/src/service.rs | 9 ++----- 16 files changed, 48 insertions(+), 73 deletions(-) diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 2ce979305a0..59da9e5f8d9 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -695,9 +695,7 @@ impl GossipVerifiedBlock { // Do not gossip or process blocks from future slots. let present_slot_with_tolerance = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis( - chain.spec.maximum_gossip_clock_disparity_millis, - )) + .now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)?; if block.slot() > present_slot_with_tolerance { return Err(BlockError::FutureSlot { diff --git a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs index c17202ae513..638d2b4012e 100644 --- a/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_finality_update_verification.rs @@ -101,8 +101,7 @@ impl VerifiedLightClientFinalityUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp - + Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis) + if seen_timestamp + chain.spec.maximum_gossip_clock_disparity() < time + one_third_slot_duration { return Err(Error::TooEarly); diff --git a/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs b/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs index cb79be6c5b8..2d1a5cf97cf 100644 --- a/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs +++ b/beacon_node/beacon_chain/src/light_client_optimistic_update_verification.rs @@ -101,8 +101,7 @@ impl VerifiedLightClientOptimisticUpdate { // verify that enough time has passed for the block to have been propagated match start_time { Some(time) => { - if seen_timestamp - + Duration::from_millis(chain.spec.maximum_gossip_clock_disparity_millis) + if seen_timestamp + chain.spec.maximum_gossip_clock_disparity() < time + one_third_slot_duration { return Err(Error::TooEarly); diff --git a/beacon_node/http_api/src/attester_duties.rs b/beacon_node/http_api/src/attester_duties.rs index 5abc4badb32..aad405d56ba 100644 --- a/beacon_node/http_api/src/attester_duties.rs +++ b/beacon_node/http_api/src/attester_duties.rs @@ -5,7 +5,6 @@ use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes}; use eth2::types::{self as api_types}; use slot_clock::SlotClock; use state_processing::state_advance::partial_state_advance; -use std::time::Duration; use types::{ AttestationDuty, BeaconState, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256, RelativeEpoch, }; @@ -31,9 +30,7 @@ pub fn attester_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis( - chain.spec.maximum_gossip_clock_disparity_millis, - )) + .now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity()) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/proposer_duties.rs b/beacon_node/http_api/src/proposer_duties.rs index 27c1bac19c1..708df39b4d6 100644 --- a/beacon_node/http_api/src/proposer_duties.rs +++ b/beacon_node/http_api/src/proposer_duties.rs @@ -9,7 +9,7 @@ use eth2::types::{self as api_types}; use safe_arith::SafeArith; use slog::{debug, Logger}; use slot_clock::SlotClock; -use std::{cmp::Ordering, time::Duration}; +use std::cmp::Ordering; use types::{CloneConfig, Epoch, EthSpec, Hash256, Slot}; /// The struct that is returned to the requesting HTTP client. @@ -33,9 +33,7 @@ pub fn proposer_duties( // will equal `current_epoch + 1` let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis( - chain.spec.maximum_gossip_clock_disparity_millis, - )) + .now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity()) .ok_or_else(|| warp_utils::reject::custom_server_error("unable to read slot clock".into()))? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/src/sync_committees.rs b/beacon_node/http_api/src/sync_committees.rs index 62af7579e7d..a0e2ce8ad47 100644 --- a/beacon_node/http_api/src/sync_committees.rs +++ b/beacon_node/http_api/src/sync_committees.rs @@ -15,7 +15,6 @@ use slog::{debug, error, warn, Logger}; use slot_clock::SlotClock; use std::cmp::max; use std::collections::HashMap; -use std::time::Duration; use tokio::sync::mpsc::UnboundedSender; use types::{ slot_data::SlotData, BeaconStateError, Epoch, EthSpec, SignedContributionAndProof, @@ -86,9 +85,7 @@ fn duties_from_state_load( let current_epoch = chain.epoch()?; let tolerant_current_epoch = chain .slot_clock - .now_with_future_tolerance(Duration::from_millis( - chain.spec.maximum_gossip_clock_disparity_millis, - )) + .now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity()) .ok_or(BeaconChainError::UnableToReadSlot)? .epoch(T::EthSpec::slots_per_epoch()); diff --git a/beacon_node/http_api/tests/tests.rs b/beacon_node/http_api/tests/tests.rs index 4a5bdca0ce7..b034c74e512 100644 --- a/beacon_node/http_api/tests/tests.rs +++ b/beacon_node/http_api/tests/tests.rs @@ -2260,7 +2260,7 @@ impl ApiTester { self.chain.slot_clock.set_current_time( current_epoch_start - - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis) + - self.chain.spec.maximum_gossip_clock_disparity() - Duration::from_millis(1), ); @@ -2299,8 +2299,7 @@ impl ApiTester { ); self.chain.slot_clock.set_current_time( - current_epoch_start - - Duration::from_millis(self.chain.spec.maximum_gossip_clock_disparity_millis), + current_epoch_start - self.chain.spec.maximum_gossip_clock_disparity(), ); self.client diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 67164ed1178..e41659c396e 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -444,17 +444,12 @@ pub fn gossipsub_config( } } } - let cloned_spec = spec.clone(); + let message_domain_valid_snappy = spec.message_domain_valid_snappy; let is_merge_enabled = fork_context.fork_exists(ForkName::Merge); let gossip_message_id = move |message: &GossipsubMessage| { MessageId::from( &Sha256::digest( - prefix( - cloned_spec.message_domain_valid_snappy, - message, - fork_context.clone(), - ) - .as_slice(), + prefix(message_domain_valid_snappy, message, fork_context.clone()).as_slice(), )[..20], ) }; diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index c6fd422007c..f1d94da7ece 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -1458,9 +1458,11 @@ mod tests { let mut uvi_codec: Uvi = Uvi::default(); let mut dst = BytesMut::with_capacity(1024); + let chain_spec = Spec::default_spec(); + // Insert length-prefix uvi_codec - .encode(Spec::default_spec().max_chunk_size as usize + 1, &mut dst) + .encode(chain_spec.max_chunk_size as usize + 1, &mut dst) .unwrap(); // Insert snappy stream identifier @@ -1472,8 +1474,6 @@ mod tests { writer.flush().unwrap(); dst.extend_from_slice(writer.get_ref()); - let chain_spec = Spec::default_spec(); - assert!(matches!( decode_response( SupportedProtocol::StatusV1, diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index f9cbebaa012..1101011daef 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -29,7 +29,7 @@ use std::{ }; use tokio::time::{sleep_until, Instant as TInstant, Sleep}; use tokio_util::time::{delay_queue, DelayQueue}; -use types::{ChainSpec, EthSpec, ForkContext}; +use types::{EthSpec, ForkContext}; /// The number of times to retry an outbound upgrade in the case of IO errors. const IO_ERROR_RETRIES: u8 = 3; @@ -129,7 +129,7 @@ where /// Logger for handling RPC streams log: slog::Logger, - /// resp_timeout for networking constants + /// Timeout that will me used for inbound and outbound responses. resp_timeout: Duration, } @@ -213,7 +213,7 @@ where listen_protocol: SubstreamProtocol, ()>, fork_context: Arc, log: &slog::Logger, - spec: &ChainSpec, + resp_timeout: Duration, ) -> Self { RPCHandler { listen_protocol, @@ -232,7 +232,7 @@ where fork_context, waker: None, log: log.clone(), - resp_timeout: spec.resp_timeout(), + resp_timeout: resp_timeout, } } diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 82bd346df6c..a03534f755f 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -17,7 +17,8 @@ use slog::{crit, debug, o}; use std::marker::PhantomData; use std::sync::Arc; use std::task::{Context, Poll}; -use types::{ChainSpec, EthSpec, ForkContext}; +use std::time::Duration; +use types::{EthSpec, ForkContext}; pub(crate) use handler::HandlerErr; pub(crate) use methods::{MetaData, MetaDataV1, MetaDataV2, Ping, RPCCodedResponse, RPCResponse}; @@ -120,8 +121,10 @@ pub struct RPC { enable_light_client_server: bool, /// Slog logger for RPC behaviour. log: slog::Logger, - /// ChainSpec for networking constants - chain_spec: ChainSpec, + + max_chunk_size: usize, + ttfb_timeout: Duration, + resp_timeout: Duration, } impl RPC { @@ -131,7 +134,9 @@ impl RPC { inbound_rate_limiter_config: Option, outbound_rate_limiter_config: Option, log: slog::Logger, - spec: ChainSpec, + max_chunk_size: usize, + ttfb_timeout: Duration, + resp_timeout: Duration, ) -> Self { let log = log.new(o!("service" => "libp2p_rpc")); @@ -152,7 +157,9 @@ impl RPC { fork_context, enable_light_client_server, log, - chain_spec: spec, + max_chunk_size: max_chunk_size, + ttfb_timeout: ttfb_timeout, + resp_timeout: resp_timeout, } } @@ -219,19 +226,16 @@ where SubstreamProtocol::new( RPCProtocol { fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size( - &self.fork_context, - self.chain_spec.max_chunk_size as usize, - ), + max_rpc_size: max_rpc_size(&self.fork_context, self.max_chunk_size), enable_light_client_server: self.enable_light_client_server, phantom: PhantomData, - ttfb_timeout: self.chain_spec.ttfb_timeout(), + ttfb_timeout: self.ttfb_timeout, }, (), ), self.fork_context.clone(), &self.log, - &self.chain_spec, + self.resp_timeout, ) } diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index e3da83280c3..24eb39df3cb 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -41,10 +41,10 @@ use std::{ sync::Arc, task::{Context, Poll}, }; +use types::ForkName; use types::{ consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext, Slot, SubnetId, }; -use types::{ChainSpec, ForkName}; use utils::{build_transport, strip_peer_id, Context as ServiceContext, MAX_CONNECTIONS_PER_PEER}; pub mod api_types; @@ -143,7 +143,6 @@ impl Network { executor: task_executor::TaskExecutor, ctx: ServiceContext<'_>, log: &slog::Logger, - spec: &ChainSpec, ) -> error::Result<(Self, Arc>)> { let log = log.new(o!("service"=> "libp2p")); let mut config = ctx.config.clone(); @@ -274,7 +273,9 @@ impl Network { config.inbound_rate_limiter_config.clone(), config.outbound_rate_limiter_config.clone(), log.clone(), - spec.clone(), + ctx.chain_spec.max_chunk_size as usize, + ctx.chain_spec.ttfb_timeout(), + ctx.chain_spec.resp_timeout(), ); let discovery = { diff --git a/beacon_node/lighthouse_network/tests/common.rs b/beacon_node/lighthouse_network/tests/common.rs index 439a39c0be7..50ae5576239 100644 --- a/beacon_node/lighthouse_network/tests/common.rs +++ b/beacon_node/lighthouse_network/tests/common.rs @@ -94,7 +94,7 @@ pub async fn build_libp2p_instance( boot_nodes: Vec, log: slog::Logger, fork_name: ForkName, - spec: ChainSpec, + spec: &ChainSpec, ) -> Libp2pInstance { let port = unused_tcp4_port().unwrap(); let config = build_config(port, boot_nodes); @@ -107,11 +107,11 @@ pub async fn build_libp2p_instance( config: &config, enr_fork_id: EnrForkId::default(), fork_context: Arc::new(fork_context(fork_name)), - chain_spec: &spec, + chain_spec: spec, gossipsub_registry: None, }; Libp2pInstance( - LibP2PService::new(executor, libp2p_context, &log, &spec) + LibP2PService::new(executor, libp2p_context, &log) .await .expect("should build libp2p instance") .0, @@ -136,10 +136,8 @@ pub async fn build_node_pair( let sender_log = log.new(o!("who" => "sender")); let receiver_log = log.new(o!("who" => "receiver")); - let mut sender = - build_libp2p_instance(rt.clone(), vec![], sender_log, fork_name, spec.clone()).await; - let mut receiver = - build_libp2p_instance(rt, vec![], receiver_log, fork_name, spec.clone()).await; + let mut sender = build_libp2p_instance(rt.clone(), vec![], sender_log, fork_name, spec).await; + let mut receiver = build_libp2p_instance(rt, vec![], receiver_log, fork_name, spec).await; let receiver_multiaddr = receiver.local_enr().multiaddr()[1].clone(); @@ -188,9 +186,7 @@ pub async fn build_linear( ) -> Vec { let mut nodes = Vec::with_capacity(n); for _ in 0..n { - nodes.push( - build_libp2p_instance(rt.clone(), vec![], log.clone(), fork_name, spec.clone()).await, - ); + nodes.push(build_libp2p_instance(rt.clone(), vec![], log.clone(), fork_name, spec).await); } let multiaddrs: Vec = nodes diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 26d2c19b519..cd5cb5ba8d4 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -1090,6 +1090,7 @@ impl BeaconProcessor { &self.executor, chain.slot_clock.clone(), self.log.clone(), + chain.spec.maximum_gossip_clock_disparity(), ); let executor = self.executor.clone(); diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index c64dc212d0a..8c468b0e749 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -380,15 +380,11 @@ pub fn spawn_reprocess_scheduler( executor: &TaskExecutor, slot_clock: T::SlotClock, log: Logger, + maximum_gossip_clock_disparity: Duration, ) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); // Basic sanity check. - assert!( - ADDITIONAL_QUEUED_BLOCK_DELAY - < Duration::from_millis( - T::EthSpec::default_spec().maximum_gossip_clock_disparity_millis - ) - ); + assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < maximum_gossip_clock_disparity); let mut queue = ReprocessQueue { work_reprocessing_rx, diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index a15ad50ee53..2c919233fca 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -281,13 +281,8 @@ impl NetworkService { }; // launch libp2p service - let (mut libp2p, network_globals) = Network::new( - executor.clone(), - service_context, - &network_log, - &beacon_chain.spec, - ) - .await?; + let (mut libp2p, network_globals) = + Network::new(executor.clone(), service_context, &network_log).await?; // Repopulate the DHT with stored ENR's if discovery is not disabled. if !config.disable_discovery { From 2c403d94bccc571ab1db9177162294d88ea41f99 Mon Sep 17 00:00:00 2001 From: Diva M Date: Mon, 10 Jul 2023 15:15:26 -0500 Subject: [PATCH 25/39] patch merge --- beacon_node/beacon_processor/src/lib.rs | 5 ++++- .../beacon_processor/src/work_reprocessing_queue.rs | 5 ++++- beacon_node/client/src/builder.rs | 1 + .../network/src/network_beacon_processor/mod.rs | 13 +------------ 4 files changed, 10 insertions(+), 14 deletions(-) diff --git a/beacon_node/beacon_processor/src/lib.rs b/beacon_node/beacon_processor/src/lib.rs index 88066f2a305..ddea76eeedf 100644 --- a/beacon_node/beacon_processor/src/lib.rs +++ b/beacon_node/beacon_processor/src/lib.rs @@ -61,7 +61,7 @@ use std::time::Duration; use task_executor::TaskExecutor; use tokio::sync::mpsc; use tokio::sync::mpsc::error::TrySendError; -use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, Slot, SubnetId}; +use types::{Attestation, ChainSpec, EthSpec, Hash256, SignedAggregateAndProof, Slot, SubnetId}; use work_reprocessing_queue::IgnoredRpcBlock; mod metrics; @@ -661,6 +661,7 @@ impl BeaconProcessor { work_reprocessing_rx: mpsc::Receiver, work_journal_tx: Option>, slot_clock: S, + spec: &ChainSpec, ) { // Used by workers to communicate that they are finished a task. let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); @@ -717,12 +718,14 @@ impl BeaconProcessor { // receive them back once they are ready (`ready_work_rx`). let (ready_work_tx, ready_work_rx) = mpsc::channel::(MAX_SCHEDULED_WORK_QUEUE_LEN); + spawn_reprocess_scheduler( ready_work_tx, work_reprocessing_rx, &self.executor, slot_clock, self.log.clone(), + spec, ); let executor = self.executor.clone(); diff --git a/beacon_node/beacon_processor/src/work_reprocessing_queue.rs b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs index 608f634d537..c05a25fc069 100644 --- a/beacon_node/beacon_processor/src/work_reprocessing_queue.rs +++ b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs @@ -29,7 +29,7 @@ use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; -use types::{EthSpec, Hash256, Slot}; +use types::{ChainSpec, EthSpec, Hash256, Slot}; const TASK_NAME: &str = "beacon_processor_reprocess_queue"; const GOSSIP_BLOCKS: &str = "gossip_blocks"; @@ -361,7 +361,10 @@ pub fn spawn_reprocess_scheduler( executor: &TaskExecutor, slot_clock: S, log: Logger, + spec: &ChainSpec, ) { + // Sanity check + assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < spec.maximum_gossip_clock_disparity()); let mut queue = ReprocessQueue { work_reprocessing_rx, ready_work_tx, diff --git a/beacon_node/client/src/builder.rs b/beacon_node/client/src/builder.rs index f3ed7f65a90..7966ce4c58c 100644 --- a/beacon_node/client/src/builder.rs +++ b/beacon_node/client/src/builder.rs @@ -792,6 +792,7 @@ where self.work_reprocessing_rx, None, beacon_chain.slot_clock.clone(), + beacon_chain.spec, ); } diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 7f0ef1fb817..48dc0f5e491 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -576,15 +576,4 @@ impl NetworkBeaconProcessor> { (network_beacon_processor, beacon_processor_receive) } -} - -#[cfg(test)] -mod test { - #[test] - fn queued_block_delay_is_sane() { - assert!( - beacon_processor::work_reprocessing_queue::ADDITIONAL_QUEUED_BLOCK_DELAY - < beacon_chain::MAXIMUM_GOSSIP_CLOCK_DISPARITY - ); - } -} +} \ No newline at end of file From 819ff45dc56994415d0b545ed6df9dc51eca10aa Mon Sep 17 00:00:00 2001 From: Diva M Date: Mon, 10 Jul 2023 15:15:49 -0500 Subject: [PATCH 26/39] fmt --- beacon_node/network/src/network_beacon_processor/mod.rs | 2 +- beacon_node/network/src/network_beacon_processor/tests.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 48dc0f5e491..db83bfc1645 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -576,4 +576,4 @@ impl NetworkBeaconProcessor> { (network_beacon_processor, beacon_processor_receive) } -} \ No newline at end of file +} diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 86d0412c506..dbbc9d8dd5e 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -962,4 +962,4 @@ async fn test_backfill_sync_processing_rate_limiting_disabled() { Duration::from_millis(100), ) .await; -} \ No newline at end of file +} From 965ca75887a2a978ac6bcd7becb51e438206a4bb Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Mon, 10 Jul 2023 18:45:28 -0400 Subject: [PATCH 27/39] fixing test errors --- beacon_node/client/src/builder.rs | 2 +- .../lighthouse_network/src/rpc/handler.rs | 2 +- beacon_node/lighthouse_network/src/rpc/mod.rs | 29 ++++++++++--------- .../lighthouse_network/src/service/mod.rs | 9 ++++-- .../src/network_beacon_processor/tests.rs | 13 ++++----- 5 files changed, 30 insertions(+), 25 deletions(-) diff --git a/beacon_node/client/src/builder.rs b/beacon_node/client/src/builder.rs index 7966ce4c58c..10ef6bf1fa3 100644 --- a/beacon_node/client/src/builder.rs +++ b/beacon_node/client/src/builder.rs @@ -792,7 +792,7 @@ where self.work_reprocessing_rx, None, beacon_chain.slot_clock.clone(), - beacon_chain.spec, + &beacon_chain.spec, ); } diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index 1101011daef..08d09f53059 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -232,7 +232,7 @@ where fork_context, waker: None, log: log.clone(), - resp_timeout: resp_timeout, + resp_timeout, } } diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index a03534f755f..0c266b25d65 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -108,6 +108,12 @@ pub struct RPCMessage { type BehaviourAction = NetworkBehaviourAction, RPCHandler>; +pub struct NetworkParams { + pub max_chunk_size: usize, + pub ttfb_timeout: Duration, + pub resp_timeout: Duration, +} + /// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level /// logic. pub struct RPC { @@ -121,10 +127,8 @@ pub struct RPC { enable_light_client_server: bool, /// Slog logger for RPC behaviour. log: slog::Logger, - - max_chunk_size: usize, - ttfb_timeout: Duration, - resp_timeout: Duration, + /// Networking constant values + network_params: NetworkParams, } impl RPC { @@ -134,9 +138,7 @@ impl RPC { inbound_rate_limiter_config: Option, outbound_rate_limiter_config: Option, log: slog::Logger, - max_chunk_size: usize, - ttfb_timeout: Duration, - resp_timeout: Duration, + network_params: NetworkParams, ) -> Self { let log = log.new(o!("service" => "libp2p_rpc")); @@ -157,9 +159,7 @@ impl RPC { fork_context, enable_light_client_server, log, - max_chunk_size: max_chunk_size, - ttfb_timeout: ttfb_timeout, - resp_timeout: resp_timeout, + network_params, } } @@ -226,16 +226,19 @@ where SubstreamProtocol::new( RPCProtocol { fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size(&self.fork_context, self.max_chunk_size), + max_rpc_size: max_rpc_size( + &self.fork_context, + self.network_params.max_chunk_size, + ), enable_light_client_server: self.enable_light_client_server, phantom: PhantomData, - ttfb_timeout: self.ttfb_timeout, + ttfb_timeout: self.network_params.ttfb_timeout, }, (), ), self.fork_context.clone(), &self.log, - self.resp_timeout, + self.network_params.resp_timeout, ) } diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index 24eb39df3cb..97bdb2cb27b 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -267,15 +267,18 @@ impl Network { (gossipsub, update_gossipsub_scores) }; + let network_params = NetworkParams { + max_chunk_size: ctx.chain_spec.max_chunk_size as usize, + ttfb_timeout: ctx.chain_spec.ttfb_timeout(), + resp_timeout: ctx.chain_spec.resp_timeout(), + }; let eth2_rpc = RPC::new( ctx.fork_context.clone(), config.enable_light_client_server, config.inbound_rate_limiter_config.clone(), config.outbound_rate_limiter_config.clone(), log.clone(), - ctx.chain_spec.max_chunk_size as usize, - ctx.chain_spec.ttfb_timeout(), - ctx.chain_spec.resp_timeout(), + network_params, ); let discovery = { diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index dbbc9d8dd5e..47dc5498d60 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -11,7 +11,7 @@ use crate::{ use beacon_chain::test_utils::{ AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, }; -use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; +use beacon_chain::{BeaconChain, ChainConfig}; use beacon_processor::{work_reprocessing_queue::*, *}; use lighthouse_network::{ discv5::enr::{CombinedKey, EnrBuilder}, @@ -229,6 +229,7 @@ impl TestRig { work_reprocessing_rx, Some(work_journal_tx), harness.chain.slot_clock.clone(), + &chain.spec, ); Self { @@ -503,9 +504,9 @@ async fn import_gossip_block_acceptably_early() { .start_of(rig.next_block.slot()) .unwrap(); - rig.chain.slot_clock.set_current_time( - slot_start - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis), - ); + rig.chain + .slot_clock + .set_current_time(slot_start - rig.chain.spec.maximum_gossip_clock_disparity()); assert_eq!( rig.chain.slot().unwrap(), @@ -553,9 +554,7 @@ async fn import_gossip_block_unacceptably_early() { .unwrap(); rig.chain.slot_clock.set_current_time( - slot_start - - Duration::from_millis(rig.chain.spec.maximum_gossip_clock_disparity_millis) - - Duration::from_millis(1), + slot_start - rig.chain.spec.maximum_gossip_clock_disparity() - Duration::from_millis(1), ); assert_eq!( From d4ea7add9c94ed184b5f9191dcf422658b0f1c7a Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 11 Jul 2023 19:09:20 -0400 Subject: [PATCH 28/39] Add sanity check to spawn_reprocess_scheduler function --- beacon_node/beacon_processor/src/lib.rs | 11 ++++++----- .../beacon_processor/src/work_reprocessing_queue.rs | 11 +++++++---- beacon_node/client/src/builder.rs | 4 ++-- .../network/src/network_beacon_processor/tests.rs | 6 ++++-- 4 files changed, 19 insertions(+), 13 deletions(-) diff --git a/beacon_node/beacon_processor/src/lib.rs b/beacon_node/beacon_processor/src/lib.rs index ddea76eeedf..297c4868db7 100644 --- a/beacon_node/beacon_processor/src/lib.rs +++ b/beacon_node/beacon_processor/src/lib.rs @@ -61,7 +61,7 @@ use std::time::Duration; use task_executor::TaskExecutor; use tokio::sync::mpsc; use tokio::sync::mpsc::error::TrySendError; -use types::{Attestation, ChainSpec, EthSpec, Hash256, SignedAggregateAndProof, Slot, SubnetId}; +use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, Slot, SubnetId}; use work_reprocessing_queue::IgnoredRpcBlock; mod metrics; @@ -661,8 +661,8 @@ impl BeaconProcessor { work_reprocessing_rx: mpsc::Receiver, work_journal_tx: Option>, slot_clock: S, - spec: &ChainSpec, - ) { + maximum_gossip_clock_disparity: Duration, + ) -> Result<(), String> { // Used by workers to communicate that they are finished a task. let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); @@ -725,8 +725,8 @@ impl BeaconProcessor { &self.executor, slot_clock, self.log.clone(), - spec, - ); + maximum_gossip_clock_disparity, + )?; let executor = self.executor.clone(); @@ -1206,6 +1206,7 @@ impl BeaconProcessor { // Spawn on the core executor. executor.spawn(manager_future, MANAGER_TASK_NAME); + Ok(()) } /// Spawns a blocking worker thread to process some `Work`. diff --git a/beacon_node/beacon_processor/src/work_reprocessing_queue.rs b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs index c05a25fc069..0127bbd1b00 100644 --- a/beacon_node/beacon_processor/src/work_reprocessing_queue.rs +++ b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs @@ -29,7 +29,7 @@ use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; -use types::{ChainSpec, EthSpec, Hash256, Slot}; +use types::{EthSpec, Hash256, Slot}; const TASK_NAME: &str = "beacon_processor_reprocess_queue"; const GOSSIP_BLOCKS: &str = "gossip_blocks"; @@ -361,10 +361,12 @@ pub fn spawn_reprocess_scheduler( executor: &TaskExecutor, slot_clock: S, log: Logger, - spec: &ChainSpec, -) { + maximum_gossip_clock_disparity: Duration, +) -> Result<(), String> { // Sanity check - assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < spec.maximum_gossip_clock_disparity()); + if ADDITIONAL_QUEUED_BLOCK_DELAY > maximum_gossip_clock_disparity { + return Err("The block delay and gossip disparity don't match.".to_string()); + } let mut queue = ReprocessQueue { work_reprocessing_rx, ready_work_tx, @@ -403,6 +405,7 @@ pub fn spawn_reprocess_scheduler( }, TASK_NAME, ); + Ok(()) } impl ReprocessQueue { diff --git a/beacon_node/client/src/builder.rs b/beacon_node/client/src/builder.rs index 10ef6bf1fa3..fdd1e6af739 100644 --- a/beacon_node/client/src/builder.rs +++ b/beacon_node/client/src/builder.rs @@ -792,8 +792,8 @@ where self.work_reprocessing_rx, None, beacon_chain.slot_clock.clone(), - &beacon_chain.spec, - ); + beacon_chain.spec.maximum_gossip_clock_disparity(), + )?; } let state_advance_context = runtime_context.service_context("state_advance".into()); diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 47dc5498d60..2429024112d 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -215,7 +215,7 @@ impl TestRig { }; let network_beacon_processor = Arc::new(network_beacon_processor); - BeaconProcessor { + let beacon_processor = BeaconProcessor { network_globals, executor, max_workers: cmp::max(1, num_cpus::get()), @@ -229,9 +229,11 @@ impl TestRig { work_reprocessing_rx, Some(work_journal_tx), harness.chain.slot_clock.clone(), - &chain.spec, + chain.spec.maximum_gossip_clock_disparity(), ); + assert!(!beacon_processor.is_err()); + Self { chain, next_block: Arc::new(next_block), From 24ef86dd2f1c8514492eb3ac7c9ef039b7e044fe Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 11 Jul 2023 19:11:28 -0400 Subject: [PATCH 29/39] fmt --- beacon_node/network/src/network_beacon_processor/tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 2429024112d..dbe93de1ea9 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -233,7 +233,7 @@ impl TestRig { ); assert!(!beacon_processor.is_err()); - + Self { chain, next_block: Arc::new(next_block), From 75fcfbf289df5cc0184ebdade72fe88c0c8950ff Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Mon, 17 Jul 2023 14:57:23 -0400 Subject: [PATCH 30/39] Remove unnecessary comment --- consensus/types/src/chain_spec.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 596f48da8c3..0f6a0f05e4d 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -172,7 +172,6 @@ pub struct ChainSpec { pub subnets_per_node: u8, pub epochs_per_subnet_subscription: u64, pub attestation_subnet_extra_bits: u8, - // Shift networking configuration into configs pub gossip_max_size: u64, pub max_request_blocks: u64, pub min_epochs_for_block_requests: u64, From 0e67472364a1657f4ede4488123251716ca464d3 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Mon, 17 Jul 2023 21:21:03 -0400 Subject: [PATCH 31/39] fixes after review --- beacon_node/lighthouse_network/src/config.rs | 4 +-- .../lighthouse_network/src/rpc/protocol.rs | 11 +++---- beacon_node/src/config.rs | 2 +- .../gnosis/config.yaml | 2 +- .../mainnet/config.yaml | 2 +- .../prater/config.yaml | 2 +- .../sepolia/config.yaml | 2 +- consensus/types/src/chain_spec.rs | 31 +++++++++---------- .../environment/tests/testnet_dir/config.yaml | 2 +- 9 files changed, 27 insertions(+), 31 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index e41659c396e..748ee8bf279 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -34,7 +34,7 @@ use types::{ChainSpec, EthSpec, ForkContext, ForkName}; pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); /// The maximum size of gossip messages. -pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { +pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { let gossip_max_size = spec.gossip_max_size; if is_merge_enabled { gossip_max_size as usize @@ -457,7 +457,7 @@ pub fn gossipsub_config( let load = NetworkLoad::from(network_load); GossipsubConfigBuilder::default() - .max_transmit_size(gossip_max_size::(is_merge_enabled, spec)) + .max_transmit_size(gossip_max_size(is_merge_enabled, spec)) .heartbeat_interval(load.heartbeat_interval) .mesh_n(load.mesh_n) .mesh_n_low(load.mesh_n_low) diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index 552414b1237..6b537be0ea0 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -72,7 +72,7 @@ lazy_static! { /// The `BeaconBlockMerge` block has an `ExecutionPayload` field which has a max size ~16 GiB for future proofing. /// We calculate the value from its fields instead of constructing the block and checking the length. /// Note: This is only the theoretical upper bound. We further bound the max size we receive over the network - /// with `MAX_RPC_SIZE_POST_MERGE`. + /// with `max_chunk_size`. pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize = // Size of a full altair block *SIGNED_BEACON_BLOCK_ALTAIR_MAX @@ -109,9 +109,6 @@ lazy_static! { .len(); } -/// The maximum bytes that can be sent across the RPC post-merge. -pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M -pub(crate) const MAX_RPC_SIZE_POST_CAPELLA: usize = 10 * 1_048_576; // 10M /// The protocol prefix the RPC protocol id. const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req"; /// The number of seconds to wait for the first bytes of a request once a protocol has been @@ -121,9 +118,9 @@ const REQUEST_TIMEOUT: u64 = 15; /// Returns the maximum bytes that can be sent across the RPC. pub fn max_rpc_size(fork_context: &ForkContext, max_chunk_size: usize) -> usize { match fork_context.current_fork() { - ForkName::Altair | ForkName::Base => max_chunk_size, - ForkName::Merge => MAX_RPC_SIZE_POST_MERGE, - ForkName::Capella => MAX_RPC_SIZE_POST_CAPELLA, + ForkName::Altair | ForkName::Base => max_chunk_size / 10, + ForkName::Merge => max_chunk_size, + ForkName::Capella => max_chunk_size, } } diff --git a/beacon_node/src/config.rs b/beacon_node/src/config.rs index 2a0298ab094..a83496fc1f2 100644 --- a/beacon_node/src/config.rs +++ b/beacon_node/src/config.rs @@ -582,7 +582,7 @@ pub fn get_config( } client_config.chain.max_network_size = - lighthouse_network::gossip_max_size::(spec.bellatrix_fork_epoch.is_some(), spec); + lighthouse_network::gossip_max_size(spec.bellatrix_fork_epoch.is_some(), spec); if cli_args.is_present("slasher") { let slasher_dir = if let Some(slasher_dir) = cli_args.value_of("slasher-dir") { diff --git a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml index 85fb341de08..7cc64ad5557 100644 --- a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml @@ -93,7 +93,7 @@ SUBNETS_PER_NODE: 4 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 1048576 +MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 diff --git a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml index ee091c0998b..7333f0bc267 100644 --- a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml @@ -93,7 +93,7 @@ SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 1048576 +MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 diff --git a/common/eth2_network_config/built_in_network_configs/prater/config.yaml b/common/eth2_network_config/built_in_network_configs/prater/config.yaml index 4a3482a6bba..62882c6e8ac 100644 --- a/common/eth2_network_config/built_in_network_configs/prater/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/prater/config.yaml @@ -93,7 +93,7 @@ SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 1048576 +MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 diff --git a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml index fe61e0b26eb..359f10e9aea 100644 --- a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml @@ -81,7 +81,7 @@ SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 1048576 +MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 93b344099cc..21955f7037c 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -171,7 +171,6 @@ pub struct ChainSpec { pub attestation_subnet_count: u64, pub subnets_per_node: u8, pub epochs_per_subnet_subscription: u64, - pub attestation_subnet_extra_bits: u8, pub gossip_max_size: u64, pub max_request_blocks: u64, pub min_epochs_for_block_requests: u64, @@ -180,7 +179,8 @@ pub struct ChainSpec { pub resp_timeout: u64, pub message_domain_invalid_snappy: [u8; 4], pub message_domain_valid_snappy: [u8; 4], - pub attestation_subnet_prefix_bits: u32, + pub attestation_subnet_extra_bits: u8, + pub attestation_subnet_prefix_bits: u8, /* * Application params @@ -461,12 +461,6 @@ impl ChainSpec { Hash256::from(domain) } - #[allow(clippy::arithmetic_side_effects)] - pub const fn attestation_subnet_prefix_bits(&self) -> u32 { - let attestation_subnet_count_bits = self.attestation_subnet_count.ilog2(); - self.attestation_subnet_extra_bits as u32 + attestation_subnet_count_bits - } - pub fn maximum_gossip_clock_disparity(&self) -> Duration { Duration::from_millis(self.maximum_gossip_clock_disparity_millis) } @@ -639,17 +633,16 @@ impl ChainSpec { maximum_gossip_clock_disparity_millis: 500, target_aggregators_per_committee: 16, epochs_per_subnet_subscription: 256, - attestation_subnet_extra_bits: 6, gossip_max_size: 10_485_760, max_request_blocks: 1024, min_epochs_for_block_requests: 33024, - max_chunk_size: 1_048_576, + max_chunk_size: 10_485_760, ttfb_timeout: 5, resp_timeout: 10, message_domain_invalid_snappy: [0, 0, 0, 0], message_domain_valid_snappy: [1, 0, 0, 0], + attestation_subnet_extra_bits: 0, attestation_subnet_prefix_bits: 6, - /* * Application specific */ @@ -873,15 +866,15 @@ impl ChainSpec { maximum_gossip_clock_disparity_millis: 500, target_aggregators_per_committee: 16, epochs_per_subnet_subscription: 256, - attestation_subnet_extra_bits: 6, gossip_max_size: 10_485_760, max_request_blocks: 1024, min_epochs_for_block_requests: 33024, - max_chunk_size: 1_048_576, + max_chunk_size: 10_485_760, ttfb_timeout: 5, resp_timeout: 10, message_domain_invalid_snappy: [0, 0, 0, 0], message_domain_valid_snappy: [1, 0, 0, 0], + attestation_subnet_extra_bits: 0, attestation_subnet_prefix_bits: 6, /* @@ -1010,8 +1003,10 @@ pub struct Config { message_domain_invalid_snappy: [u8; 4], #[serde(with = "serde_utils::bytes_4_hex")] message_domain_valid_snappy: [u8; 4], - #[serde(with = "serde_utils::quoted_u32")] - attestation_subnet_prefix_bits: u32, + #[serde(with = "serde_utils::quoted_u8")] + attestation_subnet_extra_bits: u8, + #[serde(with = "serde_utils::quoted_u8")] + attestation_subnet_prefix_bits: u8, } fn default_bellatrix_fork_version() -> [u8; 4] { @@ -1156,6 +1151,7 @@ impl Config { resp_timeout: spec.resp_timeout, message_domain_invalid_snappy: spec.message_domain_invalid_snappy, message_domain_valid_snappy: spec.message_domain_valid_snappy, + attestation_subnet_extra_bits: spec.attestation_subnet_extra_bits, attestation_subnet_prefix_bits: spec.attestation_subnet_prefix_bits, } } @@ -1209,6 +1205,7 @@ impl Config { resp_timeout, message_domain_invalid_snappy, message_domain_valid_snappy, + attestation_subnet_extra_bits, attestation_subnet_prefix_bits, } = self; @@ -1255,6 +1252,7 @@ impl Config { resp_timeout, message_domain_invalid_snappy, message_domain_valid_snappy, + attestation_subnet_extra_bits, attestation_subnet_prefix_bits, ..chain_spec.clone() }) @@ -1497,11 +1495,12 @@ mod yaml_tests { GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 - MAX_CHUNK_SIZE: 1048576 + MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 + ATTESTATION_SUBNET_EXTRA_BITS: 6 ATTESTATION_SUBNET_PREFIX_BITS: 6 "#; diff --git a/lighthouse/environment/tests/testnet_dir/config.yaml b/lighthouse/environment/tests/testnet_dir/config.yaml index ad69a3aa253..faf63433ce0 100644 --- a/lighthouse/environment/tests/testnet_dir/config.yaml +++ b/lighthouse/environment/tests/testnet_dir/config.yaml @@ -88,7 +88,7 @@ SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 -MAX_CHUNK_SIZE: 1048576 +MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 From 821dab252230cf2b0991fb734bed42fe0c4a379a Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Mon, 17 Jul 2023 21:23:36 -0400 Subject: [PATCH 32/39] Set ATTESTATION_SUBNET_EXTRA_BITS value to 0 --- consensus/types/src/chain_spec.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index 21955f7037c..f54b0b8b967 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -1500,7 +1500,7 @@ mod yaml_tests { RESP_TIMEOUT: 10 MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 - ATTESTATION_SUBNET_EXTRA_BITS: 6 + ATTESTATION_SUBNET_EXTRA_BITS: 0 ATTESTATION_SUBNET_PREFIX_BITS: 6 "#; From 95025969f36c0271798fea1e456b6e7605145489 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 18 Jul 2023 08:35:08 -0400 Subject: [PATCH 33/39] clippy --- beacon_node/lighthouse_network/src/config.rs | 16 +++++++++------- .../lighthouse_network/src/service/mod.rs | 2 +- beacon_node/src/config.rs | 6 ++++-- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index 748ee8bf279..722531ce44c 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -17,7 +17,7 @@ use std::net::{Ipv4Addr, Ipv6Addr}; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; -use types::{ChainSpec, EthSpec, ForkContext, ForkName}; +use types::{ChainSpec, ForkContext, ForkName}; /// The cache time is set to accommodate the circulation time of an attestation. /// @@ -34,12 +34,11 @@ use types::{ChainSpec, EthSpec, ForkContext, ForkName}; pub const DUPLICATE_CACHE_TIME: Duration = Duration::from_secs(33 * 12 + 1); /// The maximum size of gossip messages. -pub fn gossip_max_size(is_merge_enabled: bool, spec: &ChainSpec) -> usize { - let gossip_max_size = spec.gossip_max_size; +pub fn gossip_max_size(is_merge_enabled: bool, gossip_max_size: usize) -> usize { if is_merge_enabled { - gossip_max_size as usize + gossip_max_size } else { - (gossip_max_size / 10) as usize + gossip_max_size / 10 } } @@ -407,7 +406,7 @@ impl From for NetworkLoad { } /// Return a Lighthouse specific `GossipsubConfig` where the `message_id_fn` depends on the current fork. -pub fn gossipsub_config( +pub fn gossipsub_config( network_load: u8, fork_context: Arc, spec: &ChainSpec, @@ -457,7 +456,10 @@ pub fn gossipsub_config( let load = NetworkLoad::from(network_load); GossipsubConfigBuilder::default() - .max_transmit_size(gossip_max_size(is_merge_enabled, spec)) + .max_transmit_size(gossip_max_size( + is_merge_enabled, + spec.gossip_max_size as usize, + )) .heartbeat_interval(load.heartbeat_interval) .mesh_n(load.mesh_n) .mesh_n_low(load.mesh_n_low) diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index 97bdb2cb27b..73160aed806 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -239,7 +239,7 @@ impl Network { max_subscriptions_per_request: 150, // 148 in theory = (64 attestation + 4 sync committee + 6 core topics) * 2 }; - config.gs_config = gossipsub_config::( + config.gs_config = gossipsub_config( config.network_load, ctx.fork_context.clone(), ctx.chain_spec, diff --git a/beacon_node/src/config.rs b/beacon_node/src/config.rs index a83496fc1f2..c16b1675a9f 100644 --- a/beacon_node/src/config.rs +++ b/beacon_node/src/config.rs @@ -581,8 +581,10 @@ pub fn get_config( }; } - client_config.chain.max_network_size = - lighthouse_network::gossip_max_size(spec.bellatrix_fork_epoch.is_some(), spec); + client_config.chain.max_network_size = lighthouse_network::gossip_max_size( + spec.bellatrix_fork_epoch.is_some(), + spec.gossip_max_size as usize, + ); if cli_args.is_present("slasher") { let slasher_dir = if let Some(slasher_dir) = cli_args.value_of("slasher-dir") { From 3608b1a7dee06d4f099da775c2de15cf8a74ae9f Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Thu, 27 Jul 2023 12:31:45 +1000 Subject: [PATCH 34/39] Add serde default impls to `Config` --- Cargo.lock | 5 +- consensus/types/Cargo.toml | 1 + consensus/types/src/chain_spec.rs | 150 ++++++++++++++++++++---------- 3 files changed, 103 insertions(+), 53 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4bb3ab99e28..91b2cdc52fe 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5958,9 +5958,9 @@ dependencies = [ [[package]] name = "paste" -version = "1.0.12" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f746c4065a8fa3fe23974dd82f15431cc8d40779821001404d10d2e79ca7d79" +checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" [[package]] name = "pbkdf2" @@ -8951,6 +8951,7 @@ dependencies = [ "merkle_proof", "metastruct", "parking_lot 0.12.1", + "paste", "rand 0.8.5", "rand_xorshift", "rayon", diff --git a/consensus/types/Cargo.toml b/consensus/types/Cargo.toml index ba15f6d4885..f030f2e97a6 100644 --- a/consensus/types/Cargo.toml +++ b/consensus/types/Cargo.toml @@ -60,6 +60,7 @@ beacon_chain = { path = "../../beacon_node/beacon_chain" } eth2_interop_keypairs = { path = "../../common/eth2_interop_keypairs" } state_processing = { path = "../state_processing" } tokio = "1.14.0" +paste = "1.0.14" [features] default = ["sqlite", "legacy-arith"] diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index f54b0b8b967..a7e23cb8478 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -633,16 +633,16 @@ impl ChainSpec { maximum_gossip_clock_disparity_millis: 500, target_aggregators_per_committee: 16, epochs_per_subnet_subscription: 256, - gossip_max_size: 10_485_760, - max_request_blocks: 1024, - min_epochs_for_block_requests: 33024, - max_chunk_size: 10_485_760, - ttfb_timeout: 5, - resp_timeout: 10, - message_domain_invalid_snappy: [0, 0, 0, 0], - message_domain_valid_snappy: [1, 0, 0, 0], - attestation_subnet_extra_bits: 0, - attestation_subnet_prefix_bits: 6, + gossip_max_size: default_gossip_max_size(), + max_request_blocks: default_max_request_blocks(), + min_epochs_for_block_requests: default_min_epochs_for_block_requests(), + max_chunk_size: default_max_chunk_size(), + ttfb_timeout: default_ttfb_timeout(), + resp_timeout: default_resp_timeout(), + message_domain_invalid_snappy: default_message_domain_invalid_snappy(), + message_domain_valid_snappy: default_message_domain_valid_snappy(), + attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(), + attestation_subnet_prefix_bits: default_attestation_subnet_prefix_bits(), /* * Application specific */ @@ -866,16 +866,16 @@ impl ChainSpec { maximum_gossip_clock_disparity_millis: 500, target_aggregators_per_committee: 16, epochs_per_subnet_subscription: 256, - gossip_max_size: 10_485_760, - max_request_blocks: 1024, - min_epochs_for_block_requests: 33024, - max_chunk_size: 10_485_760, - ttfb_timeout: 5, - resp_timeout: 10, - message_domain_invalid_snappy: [0, 0, 0, 0], - message_domain_valid_snappy: [1, 0, 0, 0], - attestation_subnet_extra_bits: 0, - attestation_subnet_prefix_bits: 6, + gossip_max_size: default_gossip_max_size(), + max_request_blocks: default_max_request_blocks(), + min_epochs_for_block_requests: default_min_epochs_for_block_requests(), + max_chunk_size: default_max_chunk_size(), + ttfb_timeout: default_ttfb_timeout(), + resp_timeout: default_resp_timeout(), + message_domain_invalid_snappy: default_message_domain_invalid_snappy(), + message_domain_valid_snappy: default_message_domain_valid_snappy(), + attestation_subnet_extra_bits: default_attestation_subnet_extra_bits(), + attestation_subnet_prefix_bits: default_attestation_subnet_prefix_bits(), /* * Application specific @@ -987,24 +987,34 @@ pub struct Config { deposit_network_id: u64, deposit_contract_address: Address, + #[serde(default = "default_gossip_max_size")] #[serde(with = "serde_utils::quoted_u64")] gossip_max_size: u64, + #[serde(default = "default_max_request_blocks")] #[serde(with = "serde_utils::quoted_u64")] max_request_blocks: u64, + #[serde(default = "default_min_epochs_for_block_requests")] #[serde(with = "serde_utils::quoted_u64")] min_epochs_for_block_requests: u64, + #[serde(default = "default_max_chunk_size")] #[serde(with = "serde_utils::quoted_u64")] max_chunk_size: u64, + #[serde(default = "default_ttfb_timeout")] #[serde(with = "serde_utils::quoted_u64")] ttfb_timeout: u64, + #[serde(default = "default_resp_timeout")] #[serde(with = "serde_utils::quoted_u64")] resp_timeout: u64, + #[serde(default = "default_message_domain_invalid_snappy")] #[serde(with = "serde_utils::bytes_4_hex")] message_domain_invalid_snappy: [u8; 4], + #[serde(default = "default_message_domain_valid_snappy")] #[serde(with = "serde_utils::bytes_4_hex")] message_domain_valid_snappy: [u8; 4], + #[serde(default = "default_attestation_subnet_extra_bits")] #[serde(with = "serde_utils::quoted_u8")] attestation_subnet_extra_bits: u8, + #[serde(default = "default_attestation_subnet_prefix_bits")] #[serde(with = "serde_utils::quoted_u8")] attestation_subnet_prefix_bits: u8, } @@ -1047,6 +1057,46 @@ fn default_subnets_per_node() -> u8 { 2u8 } +const fn default_gossip_max_size() -> u64 { + 10485760 +} + +const fn default_max_request_blocks() -> u64 { + 1024 +} + +const fn default_min_epochs_for_block_requests() -> u64 { + 33024 +} + +const fn default_max_chunk_size() -> u64 { + 10485760 +} + +const fn default_ttfb_timeout() -> u64 { + 5 +} + +const fn default_resp_timeout() -> u64 { + 10 +} + +const fn default_message_domain_invalid_snappy() -> [u8; 4] { + [0, 0, 0, 0] +} + +const fn default_message_domain_valid_snappy() -> [u8; 4] { + [1, 0, 0, 0] +} + +const fn default_attestation_subnet_extra_bits() -> u8 { + 0 +} + +const fn default_attestation_subnet_prefix_bits() -> u8 { + 6 +} + impl Default for Config { fn default() -> Self { let chain_spec = MainnetEthSpec::default_spec(); @@ -1391,6 +1441,7 @@ mod tests { #[cfg(test)] mod yaml_tests { use super::*; + use paste::paste; use tempfile::NamedTempFile; #[test] @@ -1492,42 +1543,39 @@ mod yaml_tests { DEPOSIT_CHAIN_ID: 1 DEPOSIT_NETWORK_ID: 1 DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa - GOSSIP_MAX_SIZE: 10485760 - MAX_REQUEST_BLOCKS: 1024 - MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 - MAX_CHUNK_SIZE: 10485760 - TTFB_TIMEOUT: 5 - RESP_TIMEOUT: 10 - MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 - MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 - ATTESTATION_SUBNET_EXTRA_BITS: 0 - ATTESTATION_SUBNET_PREFIX_BITS: 6 "#; let chain_spec: Config = serde_yaml::from_str(spec).unwrap(); - assert_eq!( - chain_spec.terminal_total_difficulty, - default_terminal_total_difficulty() - ); - assert_eq!( - chain_spec.terminal_block_hash, - default_terminal_block_hash() - ); - assert_eq!( - chain_spec.terminal_block_hash_activation_epoch, - default_terminal_block_hash_activation_epoch() - ); - assert_eq!( - chain_spec.safe_slots_to_import_optimistically, - default_safe_slots_to_import_optimistically() - ); - assert_eq!(chain_spec.bellatrix_fork_epoch, None); + // Asserts that `chain_spec.$name` and `default_$name()` are equal. + macro_rules! check_default { + ($name: ident) => { + paste! { + assert_eq!( + chain_spec.$name, + [](), + "{} does not match default", stringify!($name)); + } + }; + } - assert_eq!( - chain_spec.bellatrix_fork_version, - default_bellatrix_fork_version() - ); + check_default!(terminal_total_difficulty); + check_default!(terminal_block_hash); + check_default!(terminal_block_hash_activation_epoch); + check_default!(safe_slots_to_import_optimistically); + check_default!(bellatrix_fork_version); + check_default!(gossip_max_size); + check_default!(max_request_blocks); + check_default!(min_epochs_for_block_requests); + check_default!(max_chunk_size); + check_default!(ttfb_timeout); + check_default!(resp_timeout); + check_default!(message_domain_invalid_snappy); + check_default!(message_domain_valid_snappy); + check_default!(attestation_subnet_extra_bits); + check_default!(attestation_subnet_prefix_bits); + + assert_eq!(chain_spec.bellatrix_fork_epoch, None); } #[test] From 82fc995fcb09f987d696305d5a9db8ebfd73aa89 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Thu, 27 Jul 2023 08:44:26 -0400 Subject: [PATCH 35/39] Changes after review --- beacon_node/beacon_processor/src/work_reprocessing_queue.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_processor/src/work_reprocessing_queue.rs b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs index 0127bbd1b00..9191509d39f 100644 --- a/beacon_node/beacon_processor/src/work_reprocessing_queue.rs +++ b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs @@ -364,7 +364,7 @@ pub fn spawn_reprocess_scheduler( maximum_gossip_clock_disparity: Duration, ) -> Result<(), String> { // Sanity check - if ADDITIONAL_QUEUED_BLOCK_DELAY > maximum_gossip_clock_disparity { + if ADDITIONAL_QUEUED_BLOCK_DELAY >= maximum_gossip_clock_disparity { return Err("The block delay and gossip disparity don't match.".to_string()); } let mut queue = ReprocessQueue { From 7c61007b7c2bcdc42f23507e28ff5568ba44350e Mon Sep 17 00:00:00 2001 From: Age Manning Date: Tue, 1 Aug 2023 16:12:07 +1000 Subject: [PATCH 36/39] Diva's patch. Seems to add subnet prefix back also --- beacon_node/lighthouse_network/src/rpc/config.rs | 5 +++-- beacon_node/lighthouse_network/src/rpc/methods.rs | 1 + beacon_node/lighthouse_network/src/rpc/mod.rs | 2 +- beacon_node/lighthouse_network/src/rpc/protocol.rs | 4 ++-- .../src/network_beacon_processor/rpc_methods.rs | 5 ++--- beacon_node/network/src/sync/manager.rs | 3 ++- .../built_in_network_configs/gnosis/config.yaml | 3 +-- .../built_in_network_configs/mainnet/config.yaml | 3 +-- .../built_in_network_configs/prater/config.yaml | 3 +-- .../built_in_network_configs/sepolia/config.yaml | 3 +-- consensus/types/src/chain_spec.rs | 14 -------------- .../environment/tests/testnet_dir/config.yaml | 3 +-- 12 files changed, 16 insertions(+), 33 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/config.rs b/beacon_node/lighthouse_network/src/rpc/config.rs index b1e824e5e8e..a0f3acaf766 100644 --- a/beacon_node/lighthouse_network/src/rpc/config.rs +++ b/beacon_node/lighthouse_network/src/rpc/config.rs @@ -4,7 +4,7 @@ use std::{ time::Duration, }; -use super::{rate_limiter::Quota, Protocol}; +use super::{methods, rate_limiter::Quota, Protocol}; use serde_derive::{Deserialize, Serialize}; @@ -97,7 +97,8 @@ impl RateLimiterConfig { pub const DEFAULT_META_DATA_QUOTA: Quota = Quota::n_every(2, 5); pub const DEFAULT_STATUS_QUOTA: Quota = Quota::n_every(5, 15); pub const DEFAULT_GOODBYE_QUOTA: Quota = Quota::one_every(10); - pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota = Quota::n_every(1024, 10); + pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota = + Quota::n_every(methods::MAX_REQUEST_BLOCKS, 10); pub const DEFAULT_BLOCKS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10); pub const DEFAULT_LIGHT_CLIENT_BOOTSTRAP_QUOTA: Quota = Quota::one_every(10); } diff --git a/beacon_node/lighthouse_network/src/rpc/methods.rs b/beacon_node/lighthouse_network/src/rpc/methods.rs index 52efc35431e..af0ba2510bf 100644 --- a/beacon_node/lighthouse_network/src/rpc/methods.rs +++ b/beacon_node/lighthouse_network/src/rpc/methods.rs @@ -20,6 +20,7 @@ use types::{ /// Maximum number of blocks in a single request. pub type MaxRequestBlocks = U1024; +pub const MAX_REQUEST_BLOCKS: u64 = 1024; /// Maximum length of error message. pub type MaxErrorLen = U256; diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 0c266b25d65..96b1cbd5818 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -27,7 +27,7 @@ pub(crate) use protocol::{InboundRequest, RPCProtocol}; pub use handler::SubstreamId; pub use methods::{ BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, LightClientBootstrapRequest, - MaxRequestBlocks, RPCResponseErrorCode, ResponseTermination, StatusMessage, + MaxRequestBlocks, RPCResponseErrorCode, ResponseTermination, StatusMessage, MAX_REQUEST_BLOCKS, }; pub(crate) use outbound::OutboundRequest; pub use protocol::{max_rpc_size, Protocol, RPCError}; diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index 6b537be0ea0..2641d55af7d 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -2,7 +2,7 @@ use super::methods::*; use crate::rpc::{ codec::{base::BaseInboundCodec, ssz_snappy::SSZSnappyInboundCodec, InboundCodec}, methods::{MaxErrorLen, ResponseTermination, MAX_ERROR_LEN}, - MaxRequestBlocks, + MaxRequestBlocks, MAX_REQUEST_BLOCKS, }; use futures::future::BoxFuture; use futures::prelude::{AsyncRead, AsyncWrite}; @@ -90,7 +90,7 @@ lazy_static! { pub static ref BLOCKS_BY_ROOT_REQUEST_MAX: usize = VariableList::::from(vec![ Hash256::zero(); - MainnetEthSpec::default_spec().max_request_blocks + MAX_REQUEST_BLOCKS as usize ]) .as_ssz_bytes() diff --git a/beacon_node/network/src/network_beacon_processor/rpc_methods.rs b/beacon_node/network/src/network_beacon_processor/rpc_methods.rs index ad9fe54a256..19b0a60a43e 100644 --- a/beacon_node/network/src/network_beacon_processor/rpc_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/rpc_methods.rs @@ -297,9 +297,8 @@ impl NetworkBeaconProcessor { ); // Should not send more than max request blocks - let max_request_blocks = self.chain.spec.max_request_blocks; - if *req.count() > max_request_blocks { - *req.count_mut() = max_request_blocks; + if *req.count() > MAX_REQUEST_BLOCKS { + *req.count_mut() = MAX_REQUEST_BLOCKS; } let forwards_block_root_iter = match self diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index ae4ff295d28..72542752c51 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -43,6 +43,7 @@ use crate::service::NetworkMessage; use crate::status::ToStatusMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, EngineState}; use futures::StreamExt; +use lighthouse_network::rpc::methods::MAX_REQUEST_BLOCKS; use lighthouse_network::types::{NetworkGlobals, SyncState}; use lighthouse_network::SyncInfo; use lighthouse_network::{PeerAction, PeerId}; @@ -188,7 +189,7 @@ pub fn spawn( log: slog::Logger, ) { assert!( - beacon_chain.spec.max_request_blocks >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, + MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, "Max blocks that can be requested in a single batch greater than max allowed blocks in a single request" ); diff --git a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml index 7cc64ad5557..8e7a9dd07a4 100644 --- a/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/gnosis/config.yaml @@ -91,7 +91,6 @@ DEPOSIT_CONTRACT_ADDRESS: 0x0B98057eA310F4d31F2a452B414647007d1645d9 # --------------------------------------------------------------- SUBNETS_PER_NODE: 4 GOSSIP_MAX_SIZE: 10485760 -MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 @@ -100,4 +99,4 @@ MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 ATTESTATION_SUBNET_COUNT: 64 ATTESTATION_SUBNET_EXTRA_BITS: 0 -ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file +ATTESTATION_SUBNET_PREFIX_BITS: 6 diff --git a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml index 7333f0bc267..98984f3b7db 100644 --- a/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/mainnet/config.yaml @@ -91,7 +91,6 @@ DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa # --------------------------------------------------------------- SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 -MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 @@ -100,4 +99,4 @@ MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 ATTESTATION_SUBNET_COUNT: 64 ATTESTATION_SUBNET_EXTRA_BITS: 0 -ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file +ATTESTATION_SUBNET_PREFIX_BITS: 6 diff --git a/common/eth2_network_config/built_in_network_configs/prater/config.yaml b/common/eth2_network_config/built_in_network_configs/prater/config.yaml index 62882c6e8ac..a0dd85fec07 100644 --- a/common/eth2_network_config/built_in_network_configs/prater/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/prater/config.yaml @@ -91,7 +91,6 @@ DEPOSIT_CONTRACT_ADDRESS: 0xff50ed3d0ec03aC01D4C79aAd74928BFF48a7b2b # --------------------------------------------------------------- SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 -MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 @@ -100,4 +99,4 @@ MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 ATTESTATION_SUBNET_COUNT: 64 ATTESTATION_SUBNET_EXTRA_BITS: 0 -ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file +ATTESTATION_SUBNET_PREFIX_BITS: 6 diff --git a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml index 359f10e9aea..e3674cf7df5 100644 --- a/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml +++ b/common/eth2_network_config/built_in_network_configs/sepolia/config.yaml @@ -79,7 +79,6 @@ DEPOSIT_CONTRACT_ADDRESS: 0x7f02C3E3c98b133055B8B348B2Ac625669Ed295D # --------------------------------------------------------------- SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 -MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 @@ -88,4 +87,4 @@ MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 ATTESTATION_SUBNET_COUNT: 64 ATTESTATION_SUBNET_EXTRA_BITS: 0 -ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file +ATTESTATION_SUBNET_PREFIX_BITS: 6 diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index a7e23cb8478..a13d3116d8b 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -172,7 +172,6 @@ pub struct ChainSpec { pub subnets_per_node: u8, pub epochs_per_subnet_subscription: u64, pub gossip_max_size: u64, - pub max_request_blocks: u64, pub min_epochs_for_block_requests: u64, pub max_chunk_size: u64, pub ttfb_timeout: u64, @@ -634,7 +633,6 @@ impl ChainSpec { target_aggregators_per_committee: 16, epochs_per_subnet_subscription: 256, gossip_max_size: default_gossip_max_size(), - max_request_blocks: default_max_request_blocks(), min_epochs_for_block_requests: default_min_epochs_for_block_requests(), max_chunk_size: default_max_chunk_size(), ttfb_timeout: default_ttfb_timeout(), @@ -867,7 +865,6 @@ impl ChainSpec { target_aggregators_per_committee: 16, epochs_per_subnet_subscription: 256, gossip_max_size: default_gossip_max_size(), - max_request_blocks: default_max_request_blocks(), min_epochs_for_block_requests: default_min_epochs_for_block_requests(), max_chunk_size: default_max_chunk_size(), ttfb_timeout: default_ttfb_timeout(), @@ -990,9 +987,6 @@ pub struct Config { #[serde(default = "default_gossip_max_size")] #[serde(with = "serde_utils::quoted_u64")] gossip_max_size: u64, - #[serde(default = "default_max_request_blocks")] - #[serde(with = "serde_utils::quoted_u64")] - max_request_blocks: u64, #[serde(default = "default_min_epochs_for_block_requests")] #[serde(with = "serde_utils::quoted_u64")] min_epochs_for_block_requests: u64, @@ -1061,10 +1055,6 @@ const fn default_gossip_max_size() -> u64 { 10485760 } -const fn default_max_request_blocks() -> u64 { - 1024 -} - const fn default_min_epochs_for_block_requests() -> u64 { 33024 } @@ -1194,7 +1184,6 @@ impl Config { deposit_contract_address: spec.deposit_contract_address, gossip_max_size: spec.gossip_max_size, - max_request_blocks: spec.max_request_blocks, min_epochs_for_block_requests: spec.min_epochs_for_block_requests, max_chunk_size: spec.max_chunk_size, ttfb_timeout: spec.ttfb_timeout, @@ -1248,7 +1237,6 @@ impl Config { deposit_network_id, deposit_contract_address, gossip_max_size, - max_request_blocks, min_epochs_for_block_requests, max_chunk_size, ttfb_timeout, @@ -1295,7 +1283,6 @@ impl Config { terminal_block_hash_activation_epoch, safe_slots_to_import_optimistically, gossip_max_size, - max_request_blocks, min_epochs_for_block_requests, max_chunk_size, ttfb_timeout, @@ -1565,7 +1552,6 @@ mod yaml_tests { check_default!(safe_slots_to_import_optimistically); check_default!(bellatrix_fork_version); check_default!(gossip_max_size); - check_default!(max_request_blocks); check_default!(min_epochs_for_block_requests); check_default!(max_chunk_size); check_default!(ttfb_timeout); diff --git a/lighthouse/environment/tests/testnet_dir/config.yaml b/lighthouse/environment/tests/testnet_dir/config.yaml index faf63433ce0..b98145163c4 100644 --- a/lighthouse/environment/tests/testnet_dir/config.yaml +++ b/lighthouse/environment/tests/testnet_dir/config.yaml @@ -86,7 +86,6 @@ DEPOSIT_CONTRACT_ADDRESS: 0x00000000219ab540356cBB839Cbe05303d7705Fa # --------------------------------------------------------------- SUBNETS_PER_NODE: 2 GOSSIP_MAX_SIZE: 10485760 -MAX_REQUEST_BLOCKS: 1024 MIN_EPOCHS_FOR_BLOCK_REQUESTS: 33024 MAX_CHUNK_SIZE: 10485760 TTFB_TIMEOUT: 5 @@ -95,4 +94,4 @@ MESSAGE_DOMAIN_INVALID_SNAPPY: 0x00000000 MESSAGE_DOMAIN_VALID_SNAPPY: 0x01000000 ATTESTATION_SUBNET_COUNT: 64 ATTESTATION_SUBNET_EXTRA_BITS: 0 -ATTESTATION_SUBNET_PREFIX_BITS: 6 \ No newline at end of file +ATTESTATION_SUBNET_PREFIX_BITS: 6 From 165a1c2cad4a1d7b24b286839bfbb27ecff73dee Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 1 Aug 2023 11:48:38 -0400 Subject: [PATCH 37/39] change after review --- beacon_node/lighthouse_network/src/rpc/handler.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index 08d09f53059..7709270732e 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -967,10 +967,7 @@ where OutboundRequestContainer { req: req.clone(), fork_context: self.fork_context.clone(), - max_rpc_size: max_rpc_size( - &self.fork_context, - self.listen_protocol.upgrade().max_rpc_size, - ), + max_rpc_size: self.listen_protocol().upgrade().max_rpc_size, }, (), ) From a5368b08be9702ffefc736b0426acd84279641f4 Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Tue, 1 Aug 2023 11:50:52 -0400 Subject: [PATCH 38/39] change after review --- beacon_node/lighthouse_network/src/rpc/handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index 7709270732e..822d48b9671 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -3,7 +3,7 @@ use super::methods::{GoodbyeReason, RPCCodedResponse, RPCResponseErrorCode, ResponseTermination}; use super::outbound::OutboundRequestContainer; -use super::protocol::{max_rpc_size, InboundRequest, Protocol, RPCError, RPCProtocol}; +use super::protocol::{InboundRequest, Protocol, RPCError, RPCProtocol}; use super::{RPCReceived, RPCSend, ReqId}; use crate::rpc::outbound::{OutboundFramed, OutboundRequest}; use crate::rpc::protocol::InboundFramed; From 7fa9a176963a8b585dd67aed30e0df3914caae2c Mon Sep 17 00:00:00 2001 From: armaganyildirak Date: Wed, 2 Aug 2023 10:45:36 -0400 Subject: [PATCH 39/39] fmt --- beacon_node/lighthouse_network/src/config.rs | 11 +++++++++-- beacon_node/lighthouse_network/src/rpc/handler.rs | 14 ++++++-------- beacon_node/lighthouse_network/src/rpc/mod.rs | 14 ++++++++++++-- beacon_node/lighthouse_network/src/service/mod.rs | 2 +- 4 files changed, 28 insertions(+), 13 deletions(-) diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index fab1ee44897..0ab7c03e7f6 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -408,7 +408,11 @@ impl From for NetworkLoad { } /// Return a Lighthouse specific `GossipsubConfig` where the `message_id_fn` depends on the current fork. -pub fn gossipsub_config(network_load: u8, fork_context: Arc, gossipsub_config_params: GossipsubConfigParams) -> gossipsub::Config { +pub fn gossipsub_config( + network_load: u8, + fork_context: Arc, + gossipsub_config_params: GossipsubConfigParams, +) -> gossipsub::Config { // The function used to generate a gossipsub message id // We use the first 8 bytes of SHA256(topic, data) for content addressing let fast_gossip_message_id = |message: &gossipsub::RawMessage| { @@ -454,7 +458,10 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc, gossip let load = NetworkLoad::from(network_load); gossipsub::ConfigBuilder::default() - .max_transmit_size(gossip_max_size(is_merge_enabled, gossipsub_config_params.gossip_max_size)) + .max_transmit_size(gossip_max_size( + is_merge_enabled, + gossipsub_config_params.gossip_max_size, + )) .heartbeat_interval(load.heartbeat_interval) .mesh_n(load.mesh_n) .mesh_n_low(load.mesh_n_low) diff --git a/beacon_node/lighthouse_network/src/rpc/handler.rs b/beacon_node/lighthouse_network/src/rpc/handler.rs index f36bd3e0d6a..36a5abc0863 100644 --- a/beacon_node/lighthouse_network/src/rpc/handler.rs +++ b/beacon_node/lighthouse_network/src/rpc/handler.rs @@ -896,10 +896,9 @@ where if expected_responses > 0 { if self.inbound_substreams.len() < MAX_INBOUND_SUBSTREAMS { // Store the stream and tag the output. - let delay_key = self.inbound_substreams_delay.insert( - self.current_inbound_substream_id, - self.resp_timeout, - ); + let delay_key = self + .inbound_substreams_delay + .insert(self.current_inbound_substream_id, self.resp_timeout); let awaiting_stream = InboundState::Idle(substream); self.inbound_substreams.insert( self.current_inbound_substream_id, @@ -961,10 +960,9 @@ where let expected_responses = request.expected_responses(); if expected_responses > 0 { // new outbound request. Store the stream and tag the output. - let delay_key = self.outbound_substreams_delay.insert( - self.current_outbound_substream_id, - self.resp_timeout, - ); + let delay_key = self + .outbound_substreams_delay + .insert(self.current_outbound_substream_id, self.resp_timeout); let awaiting_stream = OutboundSubstreamState::RequestPendingResponse { substream: Box::new(substream), request, diff --git a/beacon_node/lighthouse_network/src/rpc/mod.rs b/beacon_node/lighthouse_network/src/rpc/mod.rs index 358586fdce6..14f77e4ba23 100644 --- a/beacon_node/lighthouse_network/src/rpc/mod.rs +++ b/beacon_node/lighthouse_network/src/rpc/mod.rs @@ -241,7 +241,12 @@ where // NOTE: this is needed because PeerIds have interior mutability. let peer_repr = peer_id.to_string(); let log = self.log.new(slog::o!("peer_id" => peer_repr)); - let handler = RPCHandler::new(protocol, self.fork_context.clone(), &log, self.network_params.resp_timeout); + let handler = RPCHandler::new( + protocol, + self.fork_context.clone(), + &log, + self.network_params.resp_timeout, + ); Ok(handler) } @@ -267,7 +272,12 @@ where // NOTE: this is needed because PeerIds have interior mutability. let peer_repr = peer_id.to_string(); let log = self.log.new(slog::o!("peer_id" => peer_repr)); - let handler = RPCHandler::new(protocol, self.fork_context.clone(), &log, self.network_params.resp_timeout); + let handler = RPCHandler::new( + protocol, + self.fork_context.clone(), + &log, + self.network_params.resp_timeout, + ); Ok(handler) } diff --git a/beacon_node/lighthouse_network/src/service/mod.rs b/beacon_node/lighthouse_network/src/service/mod.rs index c2b3d87b32e..63e5bcbff66 100644 --- a/beacon_node/lighthouse_network/src/service/mod.rs +++ b/beacon_node/lighthouse_network/src/service/mod.rs @@ -1,6 +1,6 @@ use self::behaviour::Behaviour; use self::gossip_cache::GossipCache; -use crate::config::{gossipsub_config, NetworkLoad, GossipsubConfigParams}; +use crate::config::{gossipsub_config, GossipsubConfigParams, NetworkLoad}; use crate::discovery::{ subnet_predicate, DiscoveredPeers, Discovery, FIND_NODE_QUERY_CLOSEST_PEERS, };