Separate committee subscriptions queue (#3508)
## Issue Addressed NA ## Proposed Changes As we've seen on Prater, there seems to be a correlation between these messages ``` WARN Not enough time for a discovery search subnet_id: ExactSubnet { subnet_id: SubnetId(19), slot: Slot(3742336) }, service: attestation_service ``` ... and nodes falling 20-30 slots behind the head for short periods. These nodes are running ~20k Prater validators. After running some metrics, I can see that the `network_recv` channel is processing ~250k `AttestationSubscribe` messages per minute. It occurred to me that perhaps the `AttestationSubscribe` messages are "washing out" the `SendRequest` and `SendResponse` messages. In this PR I separate the `AttestationSubscribe` and `SyncCommitteeSubscribe` messages into their own queue so the `tokio::select!` in the `NetworkService` can still process the other messages in the `network_recv` channel without necessarily having to clear all the subscription messages first. ~~I've also added filter to the HTTP API to prevent duplicate subscriptions going to the network service.~~ ## Additional Info - Currently being tested on Prater
This commit is contained in:
parent
ebd0e0e2d9
commit
661307dce1
@ -22,7 +22,7 @@ use execution_layer::ExecutionLayer;
|
|||||||
use genesis::{interop_genesis_state, Eth1GenesisService, DEFAULT_ETH1_BLOCK_HASH};
|
use genesis::{interop_genesis_state, Eth1GenesisService, DEFAULT_ETH1_BLOCK_HASH};
|
||||||
use lighthouse_network::{prometheus_client::registry::Registry, NetworkGlobals};
|
use lighthouse_network::{prometheus_client::registry::Registry, NetworkGlobals};
|
||||||
use monitoring_api::{MonitoringHttpClient, ProcessType};
|
use monitoring_api::{MonitoringHttpClient, ProcessType};
|
||||||
use network::{NetworkConfig, NetworkMessage, NetworkService};
|
use network::{NetworkConfig, NetworkSenders, NetworkService};
|
||||||
use slasher::Slasher;
|
use slasher::Slasher;
|
||||||
use slasher_service::SlasherService;
|
use slasher_service::SlasherService;
|
||||||
use slog::{debug, info, warn, Logger};
|
use slog::{debug, info, warn, Logger};
|
||||||
@ -31,7 +31,7 @@ use std::path::{Path, PathBuf};
|
|||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
use timer::spawn_timer;
|
use timer::spawn_timer;
|
||||||
use tokio::sync::{mpsc::UnboundedSender, oneshot};
|
use tokio::sync::oneshot;
|
||||||
use types::{
|
use types::{
|
||||||
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec,
|
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec,
|
||||||
ExecutionBlockHash, Hash256, SignedBeaconBlock,
|
ExecutionBlockHash, Hash256, SignedBeaconBlock,
|
||||||
@ -66,7 +66,7 @@ pub struct ClientBuilder<T: BeaconChainTypes> {
|
|||||||
beacon_chain: Option<Arc<BeaconChain<T>>>,
|
beacon_chain: Option<Arc<BeaconChain<T>>>,
|
||||||
eth1_service: Option<Eth1Service>,
|
eth1_service: Option<Eth1Service>,
|
||||||
network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
|
network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
|
||||||
network_send: Option<UnboundedSender<NetworkMessage<T::EthSpec>>>,
|
network_senders: Option<NetworkSenders<T::EthSpec>>,
|
||||||
gossipsub_registry: Option<Registry>,
|
gossipsub_registry: Option<Registry>,
|
||||||
db_path: Option<PathBuf>,
|
db_path: Option<PathBuf>,
|
||||||
freezer_db_path: Option<PathBuf>,
|
freezer_db_path: Option<PathBuf>,
|
||||||
@ -98,7 +98,7 @@ where
|
|||||||
beacon_chain: None,
|
beacon_chain: None,
|
||||||
eth1_service: None,
|
eth1_service: None,
|
||||||
network_globals: None,
|
network_globals: None,
|
||||||
network_send: None,
|
network_senders: None,
|
||||||
gossipsub_registry: None,
|
gossipsub_registry: None,
|
||||||
db_path: None,
|
db_path: None,
|
||||||
freezer_db_path: None,
|
freezer_db_path: None,
|
||||||
@ -397,7 +397,7 @@ where
|
|||||||
> = Arc::new(http_api::Context {
|
> = Arc::new(http_api::Context {
|
||||||
config: self.http_api_config.clone(),
|
config: self.http_api_config.clone(),
|
||||||
chain: None,
|
chain: None,
|
||||||
network_tx: None,
|
network_senders: None,
|
||||||
network_globals: None,
|
network_globals: None,
|
||||||
eth1_service: Some(genesis_service.eth1_service.clone()),
|
eth1_service: Some(genesis_service.eth1_service.clone()),
|
||||||
log: context.log().clone(),
|
log: context.log().clone(),
|
||||||
@ -481,7 +481,7 @@ where
|
|||||||
None
|
None
|
||||||
};
|
};
|
||||||
|
|
||||||
let (network_globals, network_send) = NetworkService::start(
|
let (network_globals, network_senders) = NetworkService::start(
|
||||||
beacon_chain,
|
beacon_chain,
|
||||||
config,
|
config,
|
||||||
context.executor,
|
context.executor,
|
||||||
@ -493,7 +493,7 @@ where
|
|||||||
.map_err(|e| format!("Failed to start network: {:?}", e))?;
|
.map_err(|e| format!("Failed to start network: {:?}", e))?;
|
||||||
|
|
||||||
self.network_globals = Some(network_globals);
|
self.network_globals = Some(network_globals);
|
||||||
self.network_send = Some(network_send);
|
self.network_senders = Some(network_senders);
|
||||||
self.gossipsub_registry = gossipsub_registry;
|
self.gossipsub_registry = gossipsub_registry;
|
||||||
|
|
||||||
Ok(self)
|
Ok(self)
|
||||||
@ -537,16 +537,16 @@ where
|
|||||||
.beacon_chain
|
.beacon_chain
|
||||||
.clone()
|
.clone()
|
||||||
.ok_or("slasher service requires a beacon chain")?;
|
.ok_or("slasher service requires a beacon chain")?;
|
||||||
let network_send = self
|
let network_senders = self
|
||||||
.network_send
|
.network_senders
|
||||||
.clone()
|
.clone()
|
||||||
.ok_or("slasher service requires a network sender")?;
|
.ok_or("slasher service requires network senders")?;
|
||||||
let context = self
|
let context = self
|
||||||
.runtime_context
|
.runtime_context
|
||||||
.as_ref()
|
.as_ref()
|
||||||
.ok_or("slasher requires a runtime_context")?
|
.ok_or("slasher requires a runtime_context")?
|
||||||
.service_context("slasher_service_ctxt".into());
|
.service_context("slasher_service_ctxt".into());
|
||||||
SlasherService::new(beacon_chain, network_send).run(&context.executor)
|
SlasherService::new(beacon_chain, network_senders.network_send()).run(&context.executor)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Start the explorer client which periodically sends beacon
|
/// Start the explorer client which periodically sends beacon
|
||||||
@ -616,7 +616,7 @@ where
|
|||||||
let ctx = Arc::new(http_api::Context {
|
let ctx = Arc::new(http_api::Context {
|
||||||
config: self.http_api_config.clone(),
|
config: self.http_api_config.clone(),
|
||||||
chain: self.beacon_chain.clone(),
|
chain: self.beacon_chain.clone(),
|
||||||
network_tx: self.network_send.clone(),
|
network_senders: self.network_senders.clone(),
|
||||||
network_globals: self.network_globals.clone(),
|
network_globals: self.network_globals.clone(),
|
||||||
eth1_service: self.eth1_service.clone(),
|
eth1_service: self.eth1_service.clone(),
|
||||||
log: log.clone(),
|
log: log.clone(),
|
||||||
|
@ -25,11 +25,10 @@ use beacon_chain::{
|
|||||||
BeaconChainTypes, ProduceBlockVerification, WhenSlotSkipped,
|
BeaconChainTypes, ProduceBlockVerification, WhenSlotSkipped,
|
||||||
};
|
};
|
||||||
pub use block_id::BlockId;
|
pub use block_id::BlockId;
|
||||||
use eth2::types::ValidatorStatus;
|
use eth2::types::{self as api_types, EndpointVersion, ValidatorId, ValidatorStatus};
|
||||||
use eth2::types::{self as api_types, EndpointVersion, ValidatorId};
|
|
||||||
use lighthouse_network::{types::SyncState, EnrExt, NetworkGlobals, PeerId, PubsubMessage};
|
use lighthouse_network::{types::SyncState, EnrExt, NetworkGlobals, PeerId, PubsubMessage};
|
||||||
use lighthouse_version::version_with_platform;
|
use lighthouse_version::version_with_platform;
|
||||||
use network::NetworkMessage;
|
use network::{NetworkMessage, NetworkSenders, ValidatorSubscriptionMessage};
|
||||||
use serde::{Deserialize, Serialize};
|
use serde::{Deserialize, Serialize};
|
||||||
use slog::{crit, debug, error, info, warn, Logger};
|
use slog::{crit, debug, error, info, warn, Logger};
|
||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
@ -42,7 +41,7 @@ use std::net::{IpAddr, Ipv4Addr, SocketAddr};
|
|||||||
use std::path::PathBuf;
|
use std::path::PathBuf;
|
||||||
use std::pin::Pin;
|
use std::pin::Pin;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use tokio::sync::mpsc::UnboundedSender;
|
use tokio::sync::mpsc::{Sender, UnboundedSender};
|
||||||
use tokio_stream::{wrappers::BroadcastStream, StreamExt};
|
use tokio_stream::{wrappers::BroadcastStream, StreamExt};
|
||||||
use types::{
|
use types::{
|
||||||
Attestation, AttestationData, AttesterSlashing, BeaconStateError, BlindedPayload,
|
Attestation, AttestationData, AttesterSlashing, BeaconStateError, BlindedPayload,
|
||||||
@ -93,7 +92,7 @@ pub struct TlsConfig {
|
|||||||
pub struct Context<T: BeaconChainTypes> {
|
pub struct Context<T: BeaconChainTypes> {
|
||||||
pub config: Config,
|
pub config: Config,
|
||||||
pub chain: Option<Arc<BeaconChain<T>>>,
|
pub chain: Option<Arc<BeaconChain<T>>>,
|
||||||
pub network_tx: Option<UnboundedSender<NetworkMessage<T::EthSpec>>>,
|
pub network_senders: Option<NetworkSenders<T::EthSpec>>,
|
||||||
pub network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
|
pub network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
|
||||||
pub eth1_service: Option<eth1::Service>,
|
pub eth1_service: Option<eth1::Service>,
|
||||||
pub log: Logger,
|
pub log: Logger,
|
||||||
@ -337,14 +336,35 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
});
|
});
|
||||||
|
|
||||||
// Create a `warp` filter that provides access to the network sender channel.
|
// Create a `warp` filter that provides access to the network sender channel.
|
||||||
let inner_ctx = ctx.clone();
|
let network_tx = ctx
|
||||||
let network_tx_filter = warp::any()
|
.network_senders
|
||||||
.map(move || inner_ctx.network_tx.clone())
|
.as_ref()
|
||||||
.and_then(|network_tx| async move {
|
.map(|senders| senders.network_send());
|
||||||
match network_tx {
|
let network_tx_filter =
|
||||||
Some(network_tx) => Ok(network_tx),
|
warp::any()
|
||||||
|
.map(move || network_tx.clone())
|
||||||
|
.and_then(|network_tx| async move {
|
||||||
|
match network_tx {
|
||||||
|
Some(network_tx) => Ok(network_tx),
|
||||||
|
None => Err(warp_utils::reject::custom_not_found(
|
||||||
|
"The networking stack has not yet started (network_tx).".to_string(),
|
||||||
|
)),
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// Create a `warp` filter that provides access to the network attestation subscription channel.
|
||||||
|
let validator_subscriptions_tx = ctx
|
||||||
|
.network_senders
|
||||||
|
.as_ref()
|
||||||
|
.map(|senders| senders.validator_subscription_send());
|
||||||
|
let validator_subscription_tx_filter = warp::any()
|
||||||
|
.map(move || validator_subscriptions_tx.clone())
|
||||||
|
.and_then(|validator_subscriptions_tx| async move {
|
||||||
|
match validator_subscriptions_tx {
|
||||||
|
Some(validator_subscriptions_tx) => Ok(validator_subscriptions_tx),
|
||||||
None => Err(warp_utils::reject::custom_not_found(
|
None => Err(warp_utils::reject::custom_not_found(
|
||||||
"The networking stack has not yet started.".to_string(),
|
"The networking stack has not yet started (validator_subscription_tx)."
|
||||||
|
.to_string(),
|
||||||
)),
|
)),
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
@ -2345,7 +2365,7 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
.and(not_while_syncing_filter.clone())
|
.and(not_while_syncing_filter.clone())
|
||||||
.and(chain_filter.clone())
|
.and(chain_filter.clone())
|
||||||
.and(warp::body::json())
|
.and(warp::body::json())
|
||||||
.and(network_tx_filter.clone())
|
.and(network_tx_filter)
|
||||||
.and(log_filter.clone())
|
.and(log_filter.clone())
|
||||||
.and_then(
|
.and_then(
|
||||||
|chain: Arc<BeaconChain<T>>,
|
|chain: Arc<BeaconChain<T>>,
|
||||||
@ -2370,12 +2390,14 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
.and(warp::path("beacon_committee_subscriptions"))
|
.and(warp::path("beacon_committee_subscriptions"))
|
||||||
.and(warp::path::end())
|
.and(warp::path::end())
|
||||||
.and(warp::body::json())
|
.and(warp::body::json())
|
||||||
.and(network_tx_filter.clone())
|
.and(validator_subscription_tx_filter.clone())
|
||||||
.and(chain_filter.clone())
|
.and(chain_filter.clone())
|
||||||
|
.and(log_filter.clone())
|
||||||
.and_then(
|
.and_then(
|
||||||
|subscriptions: Vec<api_types::BeaconCommitteeSubscription>,
|
|subscriptions: Vec<api_types::BeaconCommitteeSubscription>,
|
||||||
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
|
validator_subscription_tx: Sender<ValidatorSubscriptionMessage>,
|
||||||
chain: Arc<BeaconChain<T>>| {
|
chain: Arc<BeaconChain<T>>,
|
||||||
|
log: Logger| {
|
||||||
blocking_json_task(move || {
|
blocking_json_task(move || {
|
||||||
for subscription in &subscriptions {
|
for subscription in &subscriptions {
|
||||||
chain
|
chain
|
||||||
@ -2383,7 +2405,7 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
.write()
|
.write()
|
||||||
.auto_register_local_validator(subscription.validator_index);
|
.auto_register_local_validator(subscription.validator_index);
|
||||||
|
|
||||||
let subscription = api_types::ValidatorSubscription {
|
let validator_subscription = api_types::ValidatorSubscription {
|
||||||
validator_index: subscription.validator_index,
|
validator_index: subscription.validator_index,
|
||||||
attestation_committee_index: subscription.committee_index,
|
attestation_committee_index: subscription.committee_index,
|
||||||
slot: subscription.slot,
|
slot: subscription.slot,
|
||||||
@ -2391,12 +2413,20 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
is_aggregator: subscription.is_aggregator,
|
is_aggregator: subscription.is_aggregator,
|
||||||
};
|
};
|
||||||
|
|
||||||
publish_network_message(
|
let message = ValidatorSubscriptionMessage::AttestationSubscribe {
|
||||||
&network_tx,
|
subscriptions: vec![validator_subscription],
|
||||||
NetworkMessage::AttestationSubscribe {
|
};
|
||||||
subscriptions: vec![subscription],
|
if let Err(e) = validator_subscription_tx.try_send(message) {
|
||||||
},
|
warn!(
|
||||||
)?;
|
log,
|
||||||
|
"Unable to process committee subscriptions";
|
||||||
|
"info" => "the host may be overloaded or resource-constrained",
|
||||||
|
"error" => ?e,
|
||||||
|
);
|
||||||
|
return Err(warp_utils::reject::custom_server_error(
|
||||||
|
"unable to queue subscription, host may be overloaded or shutting down".to_string(),
|
||||||
|
));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
@ -2581,12 +2611,15 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
.and(warp::path("sync_committee_subscriptions"))
|
.and(warp::path("sync_committee_subscriptions"))
|
||||||
.and(warp::path::end())
|
.and(warp::path::end())
|
||||||
.and(warp::body::json())
|
.and(warp::body::json())
|
||||||
.and(network_tx_filter)
|
.and(validator_subscription_tx_filter)
|
||||||
.and(chain_filter.clone())
|
.and(chain_filter.clone())
|
||||||
|
.and(log_filter.clone())
|
||||||
.and_then(
|
.and_then(
|
||||||
|subscriptions: Vec<types::SyncCommitteeSubscription>,
|
|subscriptions: Vec<types::SyncCommitteeSubscription>,
|
||||||
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
|
validator_subscription_tx: Sender<ValidatorSubscriptionMessage>,
|
||||||
chain: Arc<BeaconChain<T>>| {
|
chain: Arc<BeaconChain<T>>,
|
||||||
|
log: Logger
|
||||||
|
| {
|
||||||
blocking_json_task(move || {
|
blocking_json_task(move || {
|
||||||
for subscription in subscriptions {
|
for subscription in subscriptions {
|
||||||
chain
|
chain
|
||||||
@ -2594,12 +2627,20 @@ pub fn serve<T: BeaconChainTypes>(
|
|||||||
.write()
|
.write()
|
||||||
.auto_register_local_validator(subscription.validator_index);
|
.auto_register_local_validator(subscription.validator_index);
|
||||||
|
|
||||||
publish_network_message(
|
let message = ValidatorSubscriptionMessage::SyncCommitteeSubscribe {
|
||||||
&network_tx,
|
|
||||||
NetworkMessage::SyncCommitteeSubscribe {
|
|
||||||
subscriptions: vec![subscription],
|
subscriptions: vec![subscription],
|
||||||
},
|
};
|
||||||
)?;
|
if let Err(e) = validator_subscription_tx.try_send(message) {
|
||||||
|
warn!(
|
||||||
|
log,
|
||||||
|
"Unable to process sync subscriptions";
|
||||||
|
"info" => "the host may be overloaded or resource-constrained",
|
||||||
|
"error" => ?e
|
||||||
|
);
|
||||||
|
return Err(warp_utils::reject::custom_server_error(
|
||||||
|
"unable to queue subscription, host may be overloaded or shutting down".to_string(),
|
||||||
|
));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
|
@ -11,14 +11,14 @@ use lighthouse_network::{
|
|||||||
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield, SyncState},
|
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield, SyncState},
|
||||||
ConnectedPoint, Enr, NetworkGlobals, PeerId, PeerManager,
|
ConnectedPoint, Enr, NetworkGlobals, PeerId, PeerManager,
|
||||||
};
|
};
|
||||||
use network::NetworkMessage;
|
use network::{NetworkReceivers, NetworkSenders};
|
||||||
use sensitive_url::SensitiveUrl;
|
use sensitive_url::SensitiveUrl;
|
||||||
use slog::Logger;
|
use slog::Logger;
|
||||||
use std::future::Future;
|
use std::future::Future;
|
||||||
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
|
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
use tokio::sync::{mpsc, oneshot};
|
use tokio::sync::oneshot;
|
||||||
use types::{ChainSpec, EthSpec};
|
use types::{ChainSpec, EthSpec};
|
||||||
|
|
||||||
pub const TCP_PORT: u16 = 42;
|
pub const TCP_PORT: u16 = 42;
|
||||||
@ -30,7 +30,7 @@ pub const EXTERNAL_ADDR: &str = "/ip4/0.0.0.0/tcp/9000";
|
|||||||
pub struct InteractiveTester<E: EthSpec> {
|
pub struct InteractiveTester<E: EthSpec> {
|
||||||
pub harness: BeaconChainHarness<EphemeralHarnessType<E>>,
|
pub harness: BeaconChainHarness<EphemeralHarnessType<E>>,
|
||||||
pub client: BeaconNodeHttpClient,
|
pub client: BeaconNodeHttpClient,
|
||||||
pub network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
pub network_rx: NetworkReceivers<E>,
|
||||||
_server_shutdown: oneshot::Sender<()>,
|
_server_shutdown: oneshot::Sender<()>,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -41,7 +41,7 @@ pub struct ApiServer<E: EthSpec, SFut: Future<Output = ()>> {
|
|||||||
pub server: SFut,
|
pub server: SFut,
|
||||||
pub listening_socket: SocketAddr,
|
pub listening_socket: SocketAddr,
|
||||||
pub shutdown_tx: oneshot::Sender<()>,
|
pub shutdown_tx: oneshot::Sender<()>,
|
||||||
pub network_rx: tokio::sync::mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
pub network_rx: NetworkReceivers<E>,
|
||||||
pub local_enr: Enr,
|
pub local_enr: Enr,
|
||||||
pub external_peer_id: PeerId,
|
pub external_peer_id: PeerId,
|
||||||
}
|
}
|
||||||
@ -97,7 +97,7 @@ pub async fn create_api_server_on_port<T: BeaconChainTypes>(
|
|||||||
log: Logger,
|
log: Logger,
|
||||||
port: u16,
|
port: u16,
|
||||||
) -> ApiServer<T::EthSpec, impl Future<Output = ()>> {
|
) -> ApiServer<T::EthSpec, impl Future<Output = ()>> {
|
||||||
let (network_tx, network_rx) = mpsc::unbounded_channel();
|
let (network_senders, network_receivers) = NetworkSenders::new();
|
||||||
|
|
||||||
// Default metadata
|
// Default metadata
|
||||||
let meta_data = MetaData::V2(MetaDataV2 {
|
let meta_data = MetaData::V2(MetaDataV2 {
|
||||||
@ -146,7 +146,7 @@ pub async fn create_api_server_on_port<T: BeaconChainTypes>(
|
|||||||
spec_fork_name: None,
|
spec_fork_name: None,
|
||||||
},
|
},
|
||||||
chain: Some(chain.clone()),
|
chain: Some(chain.clone()),
|
||||||
network_tx: Some(network_tx),
|
network_senders: Some(network_senders),
|
||||||
network_globals: Some(network_globals),
|
network_globals: Some(network_globals),
|
||||||
eth1_service: Some(eth1_service),
|
eth1_service: Some(eth1_service),
|
||||||
log,
|
log,
|
||||||
@ -163,7 +163,7 @@ pub async fn create_api_server_on_port<T: BeaconChainTypes>(
|
|||||||
server,
|
server,
|
||||||
listening_socket,
|
listening_socket,
|
||||||
shutdown_tx,
|
shutdown_tx,
|
||||||
network_rx,
|
network_rx: network_receivers,
|
||||||
local_enr: enr,
|
local_enr: enr,
|
||||||
external_peer_id: peer_id,
|
external_peer_id: peer_id,
|
||||||
}
|
}
|
||||||
|
@ -17,14 +17,14 @@ use futures::stream::{Stream, StreamExt};
|
|||||||
use futures::FutureExt;
|
use futures::FutureExt;
|
||||||
use http_api::{BlockId, StateId};
|
use http_api::{BlockId, StateId};
|
||||||
use lighthouse_network::{Enr, EnrExt, PeerId};
|
use lighthouse_network::{Enr, EnrExt, PeerId};
|
||||||
use network::NetworkMessage;
|
use network::NetworkReceivers;
|
||||||
use proto_array::ExecutionStatus;
|
use proto_array::ExecutionStatus;
|
||||||
use sensitive_url::SensitiveUrl;
|
use sensitive_url::SensitiveUrl;
|
||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
use state_processing::per_slot_processing;
|
use state_processing::per_slot_processing;
|
||||||
use std::convert::TryInto;
|
use std::convert::TryInto;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use tokio::sync::{mpsc, oneshot};
|
use tokio::sync::oneshot;
|
||||||
use tokio::time::Duration;
|
use tokio::time::Duration;
|
||||||
use tree_hash::TreeHash;
|
use tree_hash::TreeHash;
|
||||||
use types::application_domain::ApplicationDomain;
|
use types::application_domain::ApplicationDomain;
|
||||||
@ -65,7 +65,7 @@ struct ApiTester {
|
|||||||
proposer_slashing: ProposerSlashing,
|
proposer_slashing: ProposerSlashing,
|
||||||
voluntary_exit: SignedVoluntaryExit,
|
voluntary_exit: SignedVoluntaryExit,
|
||||||
_server_shutdown: oneshot::Sender<()>,
|
_server_shutdown: oneshot::Sender<()>,
|
||||||
network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
network_rx: NetworkReceivers<E>,
|
||||||
local_enr: Enr,
|
local_enr: Enr,
|
||||||
external_peer_id: PeerId,
|
external_peer_id: PeerId,
|
||||||
mock_builder: Option<Arc<TestingBuilder<E>>>,
|
mock_builder: Option<Arc<TestingBuilder<E>>>,
|
||||||
@ -899,7 +899,7 @@ impl ApiTester {
|
|||||||
self.client.post_beacon_blocks(next_block).await.unwrap();
|
self.client.post_beacon_blocks(next_block).await.unwrap();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"valid blocks should be sent to network"
|
"valid blocks should be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -913,7 +913,7 @@ impl ApiTester {
|
|||||||
assert!(self.client.post_beacon_blocks(&next_block).await.is_err());
|
assert!(self.client.post_beacon_blocks(&next_block).await.is_err());
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"invalid blocks should be sent to network"
|
"invalid blocks should be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1041,7 +1041,7 @@ impl ApiTester {
|
|||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"valid attestation should be sent to network"
|
"valid attestation should be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1078,7 +1078,7 @@ impl ApiTester {
|
|||||||
}
|
}
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"if some attestations are valid, we should send them to the network"
|
"if some attestations are valid, we should send them to the network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1108,7 +1108,7 @@ impl ApiTester {
|
|||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"valid attester slashing should be sent to network"
|
"valid attester slashing should be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1125,7 +1125,7 @@ impl ApiTester {
|
|||||||
.unwrap_err();
|
.unwrap_err();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().now_or_never().is_none(),
|
self.network_rx.network_recv.recv().now_or_never().is_none(),
|
||||||
"invalid attester slashing should not be sent to network"
|
"invalid attester slashing should not be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1154,7 +1154,7 @@ impl ApiTester {
|
|||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"valid proposer slashing should be sent to network"
|
"valid proposer slashing should be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1171,7 +1171,7 @@ impl ApiTester {
|
|||||||
.unwrap_err();
|
.unwrap_err();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().now_or_never().is_none(),
|
self.network_rx.network_recv.recv().now_or_never().is_none(),
|
||||||
"invalid proposer slashing should not be sent to network"
|
"invalid proposer slashing should not be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1200,7 +1200,7 @@ impl ApiTester {
|
|||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().await.is_some(),
|
self.network_rx.network_recv.recv().await.is_some(),
|
||||||
"valid exit should be sent to network"
|
"valid exit should be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -1217,7 +1217,7 @@ impl ApiTester {
|
|||||||
.unwrap_err();
|
.unwrap_err();
|
||||||
|
|
||||||
assert!(
|
assert!(
|
||||||
self.network_rx.recv().now_or_never().is_none(),
|
self.network_rx.network_recv.recv().now_or_never().is_none(),
|
||||||
"invalid exit should not be sent to network"
|
"invalid exit should not be sent to network"
|
||||||
);
|
);
|
||||||
|
|
||||||
@ -2351,7 +2351,7 @@ impl ApiTester {
|
|||||||
.await
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
assert!(self.network_rx.recv().await.is_some());
|
assert!(self.network_rx.network_recv.recv().await.is_some());
|
||||||
|
|
||||||
self
|
self
|
||||||
}
|
}
|
||||||
@ -2366,7 +2366,7 @@ impl ApiTester {
|
|||||||
.await
|
.await
|
||||||
.unwrap_err();
|
.unwrap_err();
|
||||||
|
|
||||||
assert!(self.network_rx.recv().now_or_never().is_none());
|
assert!(self.network_rx.network_recv.recv().now_or_never().is_none());
|
||||||
|
|
||||||
self
|
self
|
||||||
}
|
}
|
||||||
@ -2385,7 +2385,11 @@ impl ApiTester {
|
|||||||
.await
|
.await
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
self.network_rx.recv().now_or_never().unwrap();
|
self.network_rx
|
||||||
|
.validator_subscription_recv
|
||||||
|
.recv()
|
||||||
|
.now_or_never()
|
||||||
|
.unwrap();
|
||||||
|
|
||||||
self
|
self
|
||||||
}
|
}
|
||||||
|
@ -18,4 +18,6 @@ mod subnet_service;
|
|||||||
mod sync;
|
mod sync;
|
||||||
|
|
||||||
pub use lighthouse_network::NetworkConfig;
|
pub use lighthouse_network::NetworkConfig;
|
||||||
pub use service::{NetworkMessage, NetworkService};
|
pub use service::{
|
||||||
|
NetworkMessage, NetworkReceivers, NetworkSenders, NetworkService, ValidatorSubscriptionMessage,
|
||||||
|
};
|
||||||
|
@ -252,6 +252,20 @@ lazy_static! {
|
|||||||
"Gossipsub sync_committee errors per error type",
|
"Gossipsub sync_committee errors per error type",
|
||||||
&["type"]
|
&["type"]
|
||||||
);
|
);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Network queue metrics
|
||||||
|
*/
|
||||||
|
pub static ref NETWORK_RECEIVE_EVENTS: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||||
|
"network_receive_events",
|
||||||
|
"Count of events received by the channel to the network service",
|
||||||
|
&["type"]
|
||||||
|
);
|
||||||
|
pub static ref NETWORK_RECEIVE_TIMES: Result<HistogramVec> = try_create_histogram_vec(
|
||||||
|
"network_receive_times",
|
||||||
|
"Time taken for network to handle an event sent to the network service.",
|
||||||
|
&["type"]
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
lazy_static! {
|
lazy_static! {
|
||||||
|
@ -26,6 +26,7 @@ use lighthouse_network::{
|
|||||||
use slog::{crit, debug, error, info, o, trace, warn};
|
use slog::{crit, debug, error, info, o, trace, warn};
|
||||||
use std::{net::SocketAddr, pin::Pin, sync::Arc, time::Duration};
|
use std::{net::SocketAddr, pin::Pin, sync::Arc, time::Duration};
|
||||||
use store::HotColdDB;
|
use store::HotColdDB;
|
||||||
|
use strum::IntoStaticStr;
|
||||||
use task_executor::ShutdownReason;
|
use task_executor::ShutdownReason;
|
||||||
use tokio::sync::mpsc;
|
use tokio::sync::mpsc;
|
||||||
use tokio::time::Sleep;
|
use tokio::time::Sleep;
|
||||||
@ -42,6 +43,9 @@ const METRIC_UPDATE_INTERVAL: u64 = 5;
|
|||||||
const SUBSCRIBE_DELAY_SLOTS: u64 = 2;
|
const SUBSCRIBE_DELAY_SLOTS: u64 = 2;
|
||||||
/// Delay after a fork where we unsubscribe from pre-fork topics.
|
/// Delay after a fork where we unsubscribe from pre-fork topics.
|
||||||
const UNSUBSCRIBE_DELAY_EPOCHS: u64 = 2;
|
const UNSUBSCRIBE_DELAY_EPOCHS: u64 = 2;
|
||||||
|
/// Size of the queue for validator subnet subscriptions. The number is chosen so that we may be
|
||||||
|
/// able to run tens of thousands of validators on one BN.
|
||||||
|
const VALIDATOR_SUBSCRIPTION_MESSAGE_QUEUE_SIZE: usize = 65_536;
|
||||||
|
|
||||||
/// Application level requests sent to the network.
|
/// Application level requests sent to the network.
|
||||||
#[derive(Debug, Clone, Copy)]
|
#[derive(Debug, Clone, Copy)]
|
||||||
@ -51,15 +55,9 @@ pub enum RequestId {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Types of messages that the network service can receive.
|
/// Types of messages that the network service can receive.
|
||||||
#[derive(Debug)]
|
#[derive(Debug, IntoStaticStr)]
|
||||||
|
#[strum(serialize_all = "snake_case")]
|
||||||
pub enum NetworkMessage<T: EthSpec> {
|
pub enum NetworkMessage<T: EthSpec> {
|
||||||
/// Subscribes a list of validators to specific slots for attestation duties.
|
|
||||||
AttestationSubscribe {
|
|
||||||
subscriptions: Vec<ValidatorSubscription>,
|
|
||||||
},
|
|
||||||
SyncCommitteeSubscribe {
|
|
||||||
subscriptions: Vec<SyncCommitteeSubscription>,
|
|
||||||
},
|
|
||||||
/// Subscribes the beacon node to the core gossipsub topics. We do this when we are either
|
/// Subscribes the beacon node to the core gossipsub topics. We do this when we are either
|
||||||
/// synced or close to the head slot.
|
/// synced or close to the head slot.
|
||||||
SubscribeCoreTopics,
|
SubscribeCoreTopics,
|
||||||
@ -115,6 +113,59 @@ pub enum NetworkMessage<T: EthSpec> {
|
|||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Messages triggered by validators that may trigger a subscription to a subnet.
|
||||||
|
///
|
||||||
|
/// These messages can be very numerous with large validator counts (hundreds of thousands per
|
||||||
|
/// minute). Therefore we separate them from the separated from the `NetworkMessage` to provide
|
||||||
|
/// fairness regarding message processing.
|
||||||
|
#[derive(Debug, IntoStaticStr)]
|
||||||
|
#[strum(serialize_all = "snake_case")]
|
||||||
|
pub enum ValidatorSubscriptionMessage {
|
||||||
|
/// Subscribes a list of validators to specific slots for attestation duties.
|
||||||
|
AttestationSubscribe {
|
||||||
|
subscriptions: Vec<ValidatorSubscription>,
|
||||||
|
},
|
||||||
|
SyncCommitteeSubscribe {
|
||||||
|
subscriptions: Vec<SyncCommitteeSubscription>,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
#[derive(Clone)]
|
||||||
|
pub struct NetworkSenders<E: EthSpec> {
|
||||||
|
network_send: mpsc::UnboundedSender<NetworkMessage<E>>,
|
||||||
|
validator_subscription_send: mpsc::Sender<ValidatorSubscriptionMessage>,
|
||||||
|
}
|
||||||
|
|
||||||
|
pub struct NetworkReceivers<E: EthSpec> {
|
||||||
|
pub network_recv: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
||||||
|
pub validator_subscription_recv: mpsc::Receiver<ValidatorSubscriptionMessage>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<E: EthSpec> NetworkSenders<E> {
|
||||||
|
pub fn new() -> (Self, NetworkReceivers<E>) {
|
||||||
|
let (network_send, network_recv) = mpsc::unbounded_channel::<NetworkMessage<E>>();
|
||||||
|
let (validator_subscription_send, validator_subscription_recv) =
|
||||||
|
mpsc::channel(VALIDATOR_SUBSCRIPTION_MESSAGE_QUEUE_SIZE);
|
||||||
|
let senders = Self {
|
||||||
|
network_send,
|
||||||
|
validator_subscription_send,
|
||||||
|
};
|
||||||
|
let receivers = NetworkReceivers {
|
||||||
|
network_recv,
|
||||||
|
validator_subscription_recv,
|
||||||
|
};
|
||||||
|
(senders, receivers)
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn network_send(&self) -> mpsc::UnboundedSender<NetworkMessage<E>> {
|
||||||
|
self.network_send.clone()
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn validator_subscription_send(&self) -> mpsc::Sender<ValidatorSubscriptionMessage> {
|
||||||
|
self.validator_subscription_send.clone()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Service that handles communication between internal services and the `lighthouse_network` network service.
|
/// Service that handles communication between internal services and the `lighthouse_network` network service.
|
||||||
pub struct NetworkService<T: BeaconChainTypes> {
|
pub struct NetworkService<T: BeaconChainTypes> {
|
||||||
/// A reference to the underlying beacon chain.
|
/// A reference to the underlying beacon chain.
|
||||||
@ -127,6 +178,8 @@ pub struct NetworkService<T: BeaconChainTypes> {
|
|||||||
sync_committee_service: SyncCommitteeService<T>,
|
sync_committee_service: SyncCommitteeService<T>,
|
||||||
/// The receiver channel for lighthouse to communicate with the network service.
|
/// The receiver channel for lighthouse to communicate with the network service.
|
||||||
network_recv: mpsc::UnboundedReceiver<NetworkMessage<T::EthSpec>>,
|
network_recv: mpsc::UnboundedReceiver<NetworkMessage<T::EthSpec>>,
|
||||||
|
/// The receiver channel for lighthouse to send validator subscription requests.
|
||||||
|
validator_subscription_recv: mpsc::Receiver<ValidatorSubscriptionMessage>,
|
||||||
/// The sending channel for the network service to send messages to be routed throughout
|
/// The sending channel for the network service to send messages to be routed throughout
|
||||||
/// lighthouse.
|
/// lighthouse.
|
||||||
router_send: mpsc::UnboundedSender<RouterMessage<T::EthSpec>>,
|
router_send: mpsc::UnboundedSender<RouterMessage<T::EthSpec>>,
|
||||||
@ -168,18 +221,15 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
config: &NetworkConfig,
|
config: &NetworkConfig,
|
||||||
executor: task_executor::TaskExecutor,
|
executor: task_executor::TaskExecutor,
|
||||||
gossipsub_registry: Option<&'_ mut Registry>,
|
gossipsub_registry: Option<&'_ mut Registry>,
|
||||||
) -> error::Result<(
|
) -> error::Result<(Arc<NetworkGlobals<T::EthSpec>>, NetworkSenders<T::EthSpec>)> {
|
||||||
Arc<NetworkGlobals<T::EthSpec>>,
|
|
||||||
mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
|
||||||
)> {
|
|
||||||
let network_log = executor.log().clone();
|
let network_log = executor.log().clone();
|
||||||
// build the network channel
|
// build the channels for external comms
|
||||||
let (network_send, network_recv) = mpsc::unbounded_channel::<NetworkMessage<T::EthSpec>>();
|
let (network_senders, network_recievers) = NetworkSenders::new();
|
||||||
|
|
||||||
// try and construct UPnP port mappings if required.
|
// try and construct UPnP port mappings if required.
|
||||||
let upnp_config = crate::nat::UPnPConfig::from(config);
|
let upnp_config = crate::nat::UPnPConfig::from(config);
|
||||||
let upnp_log = network_log.new(o!("service" => "UPnP"));
|
let upnp_log = network_log.new(o!("service" => "UPnP"));
|
||||||
let upnp_network_send = network_send.clone();
|
let upnp_network_send = network_senders.network_send();
|
||||||
if config.upnp_enabled {
|
if config.upnp_enabled {
|
||||||
executor.spawn_blocking(
|
executor.spawn_blocking(
|
||||||
move || {
|
move || {
|
||||||
@ -244,7 +294,7 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
let router_send = Router::spawn(
|
let router_send = Router::spawn(
|
||||||
beacon_chain.clone(),
|
beacon_chain.clone(),
|
||||||
network_globals.clone(),
|
network_globals.clone(),
|
||||||
network_send.clone(),
|
network_senders.network_send(),
|
||||||
executor.clone(),
|
executor.clone(),
|
||||||
network_log.clone(),
|
network_log.clone(),
|
||||||
)?;
|
)?;
|
||||||
@ -263,6 +313,11 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
// create a timer for updating gossipsub parameters
|
// create a timer for updating gossipsub parameters
|
||||||
let gossipsub_parameter_update = tokio::time::interval(Duration::from_secs(60));
|
let gossipsub_parameter_update = tokio::time::interval(Duration::from_secs(60));
|
||||||
|
|
||||||
|
let NetworkReceivers {
|
||||||
|
network_recv,
|
||||||
|
validator_subscription_recv,
|
||||||
|
} = network_recievers;
|
||||||
|
|
||||||
// create the network service and spawn the task
|
// create the network service and spawn the task
|
||||||
let network_log = network_log.new(o!("service" => "network"));
|
let network_log = network_log.new(o!("service" => "network"));
|
||||||
let network_service = NetworkService {
|
let network_service = NetworkService {
|
||||||
@ -271,6 +326,7 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
attestation_service,
|
attestation_service,
|
||||||
sync_committee_service,
|
sync_committee_service,
|
||||||
network_recv,
|
network_recv,
|
||||||
|
validator_subscription_recv,
|
||||||
router_send,
|
router_send,
|
||||||
store,
|
store,
|
||||||
network_globals: network_globals.clone(),
|
network_globals: network_globals.clone(),
|
||||||
@ -290,7 +346,7 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
|
|
||||||
network_service.spawn_service(executor);
|
network_service.spawn_service(executor);
|
||||||
|
|
||||||
Ok((network_globals, network_send))
|
Ok((network_globals, network_senders))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Returns the required fork digests that gossipsub needs to subscribe to based on the current slot.
|
/// Returns the required fork digests that gossipsub needs to subscribe to based on the current slot.
|
||||||
@ -358,6 +414,9 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
// handle a message sent to the network
|
// handle a message sent to the network
|
||||||
Some(msg) = self.network_recv.recv() => self.on_network_msg(msg, &mut shutdown_sender).await,
|
Some(msg) = self.network_recv.recv() => self.on_network_msg(msg, &mut shutdown_sender).await,
|
||||||
|
|
||||||
|
// handle a message from a validator requesting a subscription to a subnet
|
||||||
|
Some(msg) = self.validator_subscription_recv.recv() => self.on_validator_subscription_msg(msg).await,
|
||||||
|
|
||||||
// process any attestation service events
|
// process any attestation service events
|
||||||
Some(msg) = self.attestation_service.next() => self.on_attestation_service_msg(msg),
|
Some(msg) = self.attestation_service.next() => self.on_attestation_service_msg(msg),
|
||||||
|
|
||||||
@ -505,6 +564,9 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
msg: NetworkMessage<T::EthSpec>,
|
msg: NetworkMessage<T::EthSpec>,
|
||||||
shutdown_sender: &mut Sender<ShutdownReason>,
|
shutdown_sender: &mut Sender<ShutdownReason>,
|
||||||
) {
|
) {
|
||||||
|
metrics::inc_counter_vec(&metrics::NETWORK_RECEIVE_EVENTS, &[(&msg).into()]);
|
||||||
|
let _timer = metrics::start_timer_vec(&metrics::NETWORK_RECEIVE_TIMES, &[(&msg).into()]);
|
||||||
|
|
||||||
match msg {
|
match msg {
|
||||||
NetworkMessage::SendRequest {
|
NetworkMessage::SendRequest {
|
||||||
peer_id,
|
peer_id,
|
||||||
@ -606,22 +668,6 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
reason,
|
reason,
|
||||||
source,
|
source,
|
||||||
} => self.libp2p.goodbye_peer(&peer_id, reason, source),
|
} => self.libp2p.goodbye_peer(&peer_id, reason, source),
|
||||||
NetworkMessage::AttestationSubscribe { subscriptions } => {
|
|
||||||
if let Err(e) = self
|
|
||||||
.attestation_service
|
|
||||||
.validator_subscriptions(subscriptions)
|
|
||||||
{
|
|
||||||
warn!(self.log, "Attestation validator subscription failed"; "error" => e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
NetworkMessage::SyncCommitteeSubscribe { subscriptions } => {
|
|
||||||
if let Err(e) = self
|
|
||||||
.sync_committee_service
|
|
||||||
.validator_subscriptions(subscriptions)
|
|
||||||
{
|
|
||||||
warn!(self.log, "Sync committee calidator subscription failed"; "error" => e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
NetworkMessage::SubscribeCoreTopics => {
|
NetworkMessage::SubscribeCoreTopics => {
|
||||||
if self.shutdown_after_sync {
|
if self.shutdown_after_sync {
|
||||||
if let Err(e) = shutdown_sender
|
if let Err(e) = shutdown_sender
|
||||||
@ -704,6 +750,28 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Handle a message sent to the network service.
|
||||||
|
async fn on_validator_subscription_msg(&mut self, msg: ValidatorSubscriptionMessage) {
|
||||||
|
match msg {
|
||||||
|
ValidatorSubscriptionMessage::AttestationSubscribe { subscriptions } => {
|
||||||
|
if let Err(e) = self
|
||||||
|
.attestation_service
|
||||||
|
.validator_subscriptions(subscriptions)
|
||||||
|
{
|
||||||
|
warn!(self.log, "Attestation validator subscription failed"; "error" => e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
ValidatorSubscriptionMessage::SyncCommitteeSubscribe { subscriptions } => {
|
||||||
|
if let Err(e) = self
|
||||||
|
.sync_committee_service
|
||||||
|
.validator_subscriptions(subscriptions)
|
||||||
|
{
|
||||||
|
warn!(self.log, "Sync committee calidator subscription failed"; "error" => e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
fn update_gossipsub_parameters(&mut self) {
|
fn update_gossipsub_parameters(&mut self) {
|
||||||
if let Ok(slot) = self.beacon_chain.slot() {
|
if let Ok(slot) = self.beacon_chain.slot() {
|
||||||
let active_validators_opt = self
|
let active_validators_opt = self
|
||||||
|
@ -682,7 +682,7 @@ pub struct ValidatorAggregateAttestationQuery {
|
|||||||
pub slot: Slot,
|
pub slot: Slot,
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
|
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Hash)]
|
||||||
pub struct BeaconCommitteeSubscription {
|
pub struct BeaconCommitteeSubscription {
|
||||||
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
#[serde(with = "eth2_serde_utils::quoted_u64")]
|
||||||
pub validator_index: u64,
|
pub validator_index: u64,
|
||||||
|
Loading…
Reference in New Issue
Block a user