Improve tokio task execution (#1181)

* Add logging on shutdown

* Replace tokio::spawn with handle.spawn

* Upgrade tokio

* Add a task executor

* Beacon chain tasks use task executor

* Validator client tasks use task executor

* Rename runtime_handle to executor

* Add duration histograms; minor fixes

* Cleanup

* Fix logs

* Fix tests

* Remove random file

* Get enr dependency instead of libp2p

* Address some review comments

* Libp2p takes a TaskExecutor

* Ugly fix libp2p tests

* Move TaskExecutor to own file

* Upgrade Dockerfile rust version

* Minor fixes

* Revert "Ugly fix libp2p tests"

This reverts commit 58d4bb690f52de28d893943b7504d2d0c6621429.

* Pretty fix libp2p tests

* Add spawn_without_exit; change Counter to Gauge

* Tidy

* Move log from RuntimeContext to TaskExecutor

* Fix errors

* Replace histogram with int_gauge for async tasks

* Fix todo

* Fix memory leak in test by exiting all spawned tasks at the end
This commit is contained in:
Pawan Dhananjay 2020-06-04 17:18:05 +05:30 committed by GitHub
parent ce10db15da
commit 042e80570c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
53 changed files with 541 additions and 361 deletions

12
Cargo.lock generated
View File

@ -1161,7 +1161,10 @@ dependencies = [
"env_logger",
"eth2_config",
"eth2_testnet_config",
"exit-future",
"futures 0.3.5",
"lazy_static",
"lighthouse_metrics",
"logging",
"parking_lot 0.10.2",
"slog",
@ -1231,10 +1234,12 @@ dependencies = [
"base64 0.12.1",
"dirs",
"discv5",
"environment",
"error-chain",
"eth2_ssz",
"eth2_ssz_derive",
"eth2_ssz_types",
"exit-future",
"fnv",
"futures 0.3.5",
"hashset_delay",
@ -1365,7 +1370,7 @@ dependencies = [
name = "eth2_testnet_config"
version = "0.2.0"
dependencies = [
"eth2-libp2p",
"enr",
"eth2_ssz",
"reqwest",
"serde",
@ -2932,9 +2937,11 @@ name = "network"
version = "0.1.2"
dependencies = [
"beacon_chain",
"environment",
"error-chain",
"eth2-libp2p",
"eth2_ssz",
"exit-future",
"fnv",
"futures 0.3.5",
"genesis",
@ -3808,6 +3815,7 @@ dependencies = [
"assert_matches",
"beacon_chain",
"bls",
"environment",
"eth2-libp2p",
"eth2_config",
"eth2_ssz",
@ -4853,6 +4861,7 @@ name = "timer"
version = "0.1.2"
dependencies = [
"beacon_chain",
"environment",
"futures 0.3.5",
"parking_lot 0.10.2",
"slog",
@ -5823,6 +5832,7 @@ dependencies = [
name = "websocket_server"
version = "0.1.2"
dependencies = [
"environment",
"futures 0.3.5",
"serde",
"serde_derive",

View File

@ -31,5 +31,5 @@ eth2_wallet = { path = "../crypto/eth2_wallet" }
eth2_wallet_manager = { path = "../common/eth2_wallet_manager" }
rand = "0.7.2"
validator_dir = { path = "../common/validator_dir", features = ["unencrypted_keys"] }
tokio = {version = "0.2.20", features = ["full"]}
tokio = { version = "0.2.21", features = ["full"] }
eth2_keystore = { path = "../crypto/eth2_keystore" }

View File

@ -83,7 +83,7 @@ pub fn cli_run<T: EthSpec>(
matches: &ArgMatches<'_>,
mut env: Environment<T>,
) -> Result<(), String> {
let log = env.core_context().log;
let log = env.core_context().log().clone();
let data_dir = clap_utils::parse_path_with_default_in_home_dir(
matches,

View File

@ -27,7 +27,7 @@ slog = { version = "2.5.2", features = ["max_level_trace", "release_max_level_tr
slog-term = "2.5.0"
slog-async = "2.5.0"
ctrlc = { version = "3.1.4", features = ["termination"] }
tokio = {version = "0.2.20", features = ["time"] }
tokio = { version = "0.2.21", features = ["time"] }
exit-future = "0.2.0"
env_logger = "0.7.1"
dirs = "2.0.2"

View File

@ -33,7 +33,7 @@ eth2_ssz_derive = "0.1.0"
state_processing = { path = "../../consensus/state_processing" }
tree_hash = "0.1.0"
types = { path = "../../consensus/types" }
tokio = "0.2.20"
tokio = "0.2.21"
eth1 = { path = "../eth1" }
websocket_server = { path = "../websocket_server" }
futures = "0.3.5"
@ -46,7 +46,8 @@ tempfile = "3.1.0"
bitvec = "0.17.4"
bls = { path = "../../crypto/bls" }
safe_arith = { path = "../../consensus/safe_arith" }
environment = { path = "../../lighthouse/environment" }
[dev-dependencies]
lazy_static = "1.4.0"
environment = { path = "../../lighthouse/environment" }

View File

@ -1,4 +1,5 @@
use crate::metrics;
use environment::TaskExecutor;
use eth1::{Config as Eth1Config, Eth1Block, Service as HttpService};
use eth2_hashing::hash;
use slog::{debug, error, trace, Logger};
@ -285,10 +286,8 @@ impl<T: EthSpec, S: Store<T>> CachingEth1Backend<T, S> {
}
/// Starts the routine which connects to the external eth1 node and updates the caches.
pub fn start(&self, exit: tokio::sync::oneshot::Receiver<()>) {
// don't need to spawn as a task is being spawned in auto_update
// TODO: check if this is correct
HttpService::auto_update(self.core.clone(), exit);
pub fn start(&self, handle: TaskExecutor) {
HttpService::auto_update(self.core.clone(), handle);
}
/// Instantiates `self` from an existing service.

View File

@ -28,7 +28,7 @@ error-chain = "0.12.2"
serde_yaml = "0.8.11"
slog = { version = "2.5.2", features = ["max_level_trace"] }
slog-async = "2.5.0"
tokio = "0.2.20"
tokio = "0.2.21"
dirs = "2.0.2"
futures = "0.3.5"
reqwest = "0.10.4"

View File

@ -21,6 +21,7 @@ use std::net::SocketAddr;
use std::path::Path;
use std::sync::Arc;
use std::time::Duration;
use timer::spawn_timer;
use tokio::sync::mpsc::UnboundedSender;
use types::{test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec};
use websocket_server::{Config as WebSocketConfig, WebSocketSender};
@ -50,7 +51,6 @@ pub struct ClientBuilder<T: BeaconChainTypes> {
beacon_chain_builder: Option<BeaconChainBuilder<T>>,
beacon_chain: Option<Arc<BeaconChain<T>>>,
eth1_service: Option<Eth1Service>,
exit_channels: Vec<tokio::sync::oneshot::Sender<()>>,
event_handler: Option<T::EventHandler>,
network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
network_send: Option<UnboundedSender<NetworkMessage<T::EthSpec>>>,
@ -84,7 +84,6 @@ where
beacon_chain_builder: None,
beacon_chain: None,
eth1_service: None,
exit_channels: vec![],
event_handler: None,
network_globals: None,
network_send: None,
@ -132,7 +131,7 @@ where
.ok_or_else(|| "beacon_chain_start_method requires a chain spec".to_string())?;
let builder = BeaconChainBuilder::new(eth_spec_instance)
.logger(context.log.clone())
.logger(context.log().clone())
.store(store)
.store_migrator(store_migrator)
.data_dir(data_dir)
@ -150,7 +149,7 @@ where
// Alternatively, if there's a beacon chain in the database then always resume
// using it.
let client_genesis = if client_genesis == ClientGenesis::FromStore && !chain_exists {
info!(context.log, "Defaulting to deposit contract genesis");
info!(context.log(), "Defaulting to deposit contract genesis");
ClientGenesis::DepositContract
} else if chain_exists {
@ -172,7 +171,7 @@ where
genesis_state_bytes,
} => {
info!(
context.log,
context.log(),
"Starting from known genesis state";
);
@ -183,14 +182,14 @@ where
}
ClientGenesis::DepositContract => {
info!(
context.log,
context.log(),
"Waiting for eth2 genesis from eth1";
"eth1_endpoint" => &config.eth1.endpoint,
"contract_deploy_block" => config.eth1.deposit_contract_deploy_block,
"deposit_contract" => &config.eth1.deposit_contract_address
);
let genesis_service = Eth1GenesisService::new(config.eth1, context.log.clone());
let genesis_service = Eth1GenesisService::new(config.eth1, context.log().clone());
let genesis_state = genesis_service
.wait_for_genesis_state(
@ -223,19 +222,18 @@ where
.ok_or_else(|| "network requires a runtime_context")?
.clone();
let (network_globals, network_send, network_exit) =
NetworkService::start(beacon_chain, config, &context.runtime_handle, context.log)
let (network_globals, network_send) =
NetworkService::start(beacon_chain, config, context.executor)
.map_err(|e| format!("Failed to start network: {:?}", e))?;
self.network_globals = Some(network_globals);
self.network_send = Some(network_send);
self.exit_channels.push(network_exit);
Ok(self)
}
/// Immediately starts the timer service.
fn timer(mut self) -> Result<Self, String> {
fn timer(self) -> Result<Self, String> {
let context = self
.runtime_context
.as_ref()
@ -251,13 +249,9 @@ where
.ok_or_else(|| "node timer requires a chain spec".to_string())?
.milliseconds_per_slot;
let timer_exit = context
.runtime_handle
.enter(|| timer::spawn(beacon_chain, milliseconds_per_slot))
spawn_timer(context.executor, beacon_chain, milliseconds_per_slot)
.map_err(|e| format!("Unable to start node timer: {}", e))?;
self.exit_channels.push(timer_exit);
Ok(self)
}
@ -290,32 +284,28 @@ where
network_chan: network_send,
};
let log = context.log.clone();
let (exit_channel, listening_addr) = context.runtime_handle.enter(|| {
rest_api::start_server(
&client_config.rest_api,
beacon_chain,
network_info,
client_config
.create_db_path()
.map_err(|_| "unable to read data dir")?,
client_config
.create_freezer_db_path()
.map_err(|_| "unable to read freezer DB dir")?,
eth2_config.clone(),
log,
)
.map_err(|e| format!("Failed to start HTTP API: {:?}", e))
})?;
let listening_addr = rest_api::start_server(
context.executor,
&client_config.rest_api,
beacon_chain,
network_info,
client_config
.create_db_path()
.map_err(|_| "unable to read data dir")?,
client_config
.create_freezer_db_path()
.map_err(|_| "unable to read freezer DB dir")?,
eth2_config.clone(),
)
.map_err(|e| format!("Failed to start HTTP API: {:?}", e))?;
self.exit_channels.push(exit_channel);
self.http_listen_addr = Some(listening_addr);
Ok(self)
}
/// Immediately starts the service that periodically logs information each slot.
pub fn notifier(mut self) -> Result<Self, String> {
pub fn notifier(self) -> Result<Self, String> {
let context = self
.runtime_context
.as_ref()
@ -335,19 +325,13 @@ where
.ok_or_else(|| "slot_notifier requires a chain spec".to_string())?
.milliseconds_per_slot;
let exit_channel = context
.runtime_handle
.enter(|| {
spawn_notifier(
beacon_chain,
network_globals,
milliseconds_per_slot,
context.log.clone(),
)
})
.map_err(|e| format!("Unable to start slot notifier: {}", e))?;
self.exit_channels.push(exit_channel);
spawn_notifier(
context.executor,
beacon_chain,
network_globals,
milliseconds_per_slot,
)
.map_err(|e| format!("Unable to start slot notifier: {}", e))?;
Ok(self)
}
@ -365,7 +349,6 @@ where
network_globals: self.network_globals,
http_listen_addr: self.http_listen_addr,
websocket_listen_addr: self.websocket_listen_addr,
_exit_channels: self.exit_channels,
}
}
}
@ -436,22 +419,14 @@ where
.ok_or_else(|| "websocket_event_handler requires a runtime_context")?
.service_context("ws".into());
let (sender, exit_channel, listening_addr): (
WebSocketSender<TEthSpec>,
Option<_>,
Option<_>,
) = if config.enabled {
let (sender, exit, listening_addr) = context
.runtime_handle
.enter(|| websocket_server::start_server(&config, &context.log))?;
(sender, Some(exit), Some(listening_addr))
let (sender, listening_addr): (WebSocketSender<TEthSpec>, Option<_>) = if config.enabled {
let (sender, listening_addr) =
websocket_server::start_server(context.executor, &config)?;
(sender, Some(listening_addr))
} else {
(WebSocketSender::dummy(), None, None)
(WebSocketSender::dummy(), None)
};
if let Some(channel) = exit_channel {
self.exit_channels.push(channel);
}
self.event_handler = Some(sender);
self.websocket_listen_addr = listening_addr;
@ -494,7 +469,7 @@ where
.clone()
.ok_or_else(|| "disk_store requires a chain spec".to_string())?;
let store = HotColdDB::open(hot_path, cold_path, config, spec, context.log)
let store = HotColdDB::open(hot_path, cold_path, config, spec, context.log().clone())
.map_err(|e| format!("Unable to open database: {:?}", e))?;
self.store = Some(Arc::new(store));
Ok(self)
@ -555,7 +530,7 @@ where
let store = self.store.clone().ok_or_else(|| {
"background_migrator requires the store to be initialized".to_string()
})?;
self.store_migrator = Some(BackgroundMigrator::new(store, context.log.clone()));
self.store_migrator = Some(BackgroundMigrator::new(store, context.log().clone()));
Ok(self)
}
}
@ -617,25 +592,23 @@ where
&persisted,
config.clone(),
store.clone(),
&context.log,
&context.log().clone(),
)
.map(|chain| chain.into_backend())
})
.unwrap_or_else(|| {
Ok(CachingEth1Backend::new(config, context.log.clone(), store))
Ok(CachingEth1Backend::new(
config,
context.log().clone(),
store,
))
})?
};
self.eth1_service = None;
let exit = {
let (tx, rx) = tokio::sync::oneshot::channel();
self.exit_channels.push(tx);
rx
};
// Starts the service that connects to an eth1 node and periodically updates caches.
context.runtime_handle.enter(|| backend.start(exit));
backend.start(context.executor);
self.beacon_chain_builder = Some(beacon_chain_builder.eth1_backend(Some(backend)));

View File

@ -25,8 +25,6 @@ pub struct Client<T: BeaconChainTypes> {
network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
http_listen_addr: Option<SocketAddr>,
websocket_listen_addr: Option<SocketAddr>,
/// Exit channels will complete/error when dropped, causing each service to exit gracefully.
_exit_channels: Vec<tokio::sync::oneshot::Sender<()>>,
}
impl<T: BeaconChainTypes> Client<T> {

View File

@ -23,11 +23,11 @@ const SPEEDO_OBSERVATIONS: usize = 4;
/// Spawns a notifier service which periodically logs information about the node.
pub fn spawn_notifier<T: BeaconChainTypes>(
executor: environment::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
network: Arc<NetworkGlobals<T::EthSpec>>,
milliseconds_per_slot: u64,
log: slog::Logger,
) -> Result<tokio::sync::oneshot::Sender<()>, String> {
) -> Result<(), String> {
let slot_duration = Duration::from_millis(milliseconds_per_slot);
let duration_to_next_slot = beacon_chain
.slot_clock
@ -41,6 +41,7 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
let interval_duration = slot_duration;
let speedo = Mutex::new(Speedo::default());
let log = executor.log().clone();
let mut interval = tokio::time::interval_at(start_instant, interval_duration);
let interval_future = async move {
@ -163,12 +164,10 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
Ok::<(), ()>(())
};
let (exit_signal, exit) = tokio::sync::oneshot::channel();
// run the notifier on the current executor
tokio::spawn(futures::future::select(Box::pin(interval_future), exit));
executor.spawn(interval_future.unwrap_or_else(|_| ()), "notifier");
Ok(exit_signal)
Ok(())
}
/// Returns the peer count, returning something helpful if it's `usize::max_value` (effectively a

View File

@ -6,7 +6,6 @@ edition = "2018"
[dev-dependencies]
eth1_test_rig = { path = "../../testing/eth1_test_rig" }
environment = { path = "../../lighthouse/environment" }
toml = "0.5.6"
web3 = "0.11.0"
sloggers = "1.0.0"
@ -25,8 +24,9 @@ tree_hash = "0.1.0"
eth2_hashing = "0.1.0"
parking_lot = "0.10.2"
slog = "2.5.2"
tokio = { version = "0.2.20", features = ["full"] }
tokio = { version = "0.2.21", features = ["full"] }
state_processing = { path = "../../consensus/state_processing" }
libflate = "1.0.0"
lighthouse_metrics = { path = "../../common/lighthouse_metrics"}
lazy_static = "1.4.0"
environment = { path = "../../lighthouse/environment" }

View File

@ -290,7 +290,7 @@ impl Service {
/// - Err(_) if there is an error.
///
/// Emits logs for debugging and errors.
pub fn auto_update(service: Self, exit: tokio::sync::oneshot::Receiver<()>) {
pub fn auto_update(service: Self, handle: environment::TaskExecutor) {
let update_interval = Duration::from_millis(service.config().auto_update_interval_millis);
let mut interval = interval_at(Instant::now(), update_interval);
@ -303,9 +303,7 @@ impl Service {
}
};
let future = futures::future::select(Box::pin(update_future), exit);
tokio::task::spawn(future);
handle.spawn(update_future, "eth1");
}
async fn do_update(service: Self, update_interval: Duration) -> Result<(), ()> {

View File

@ -35,6 +35,7 @@ tokio-util = { version = "0.3.1", features = ["codec", "compat"] }
# Patched for quick updates
discv5 = { git = "https://github.com/sigp/discv5", rev = "7b3bd40591b62b8c002ffdb85de008aa9f82e2e5" }
tiny-keccak = "2.0.2"
environment = { path = "../../lighthouse/environment" }
libp2p-tcp = { version = "0.19.1", default-features = false, features = ["tokio"] }
[dependencies.libp2p]
@ -49,3 +50,4 @@ slog-stdlog = "4.0.0"
slog-term = "2.5.0"
slog-async = "2.5.0"
tempdir = "0.3.7"
exit-future = "0.2.0"

View File

@ -84,6 +84,7 @@ pub struct Service<TSpec: EthSpec> {
impl<TSpec: EthSpec> Service<TSpec> {
pub fn new(
executor: environment::TaskExecutor,
config: &NetworkConfig,
enr_fork_id: EnrForkId,
log: &slog::Logger,
@ -122,15 +123,15 @@ impl<TSpec: EthSpec> Service<TSpec> {
let behaviour = Behaviour::new(&local_keypair, config, network_globals.clone(), &log)?;
// use the executor for libp2p
struct Executor(tokio::runtime::Handle);
struct Executor(environment::TaskExecutor);
impl libp2p::core::Executor for Executor {
fn exec(&self, f: Pin<Box<dyn Future<Output = ()> + Send>>) {
self.0.spawn(f);
self.0.spawn(f, "libp2p");
}
}
SwarmBuilder::new(transport, behaviour, local_peer_id.clone())
.peer_connection_limit(MAX_CONNECTIONS_PER_PEER)
.executor(Box::new(Executor(tokio::runtime::Handle::current())))
.executor(Box::new(Executor(executor)))
.build()
};

View File

@ -12,6 +12,21 @@ use types::{EnrForkId, MinimalEthSpec};
type E = MinimalEthSpec;
use tempdir::TempDir;
pub struct Libp2pInstance(LibP2PService<E>, exit_future::Signal);
impl std::ops::Deref for Libp2pInstance {
type Target = LibP2PService<E>;
fn deref(&self) -> &Self::Target {
&self.0
}
}
impl std::ops::DerefMut for Libp2pInstance {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.0
}
}
pub fn build_log(level: slog::Level, enabled: bool) -> slog::Logger {
let decorator = slog_term::TermDecorator::new().build();
let drain = slog_term::FullFormat::new(decorator).build().fuse();
@ -82,13 +97,20 @@ pub fn build_libp2p_instance(
boot_nodes: Vec<Enr>,
secret_key: Option<String>,
log: slog::Logger,
) -> LibP2PService<E> {
) -> Libp2pInstance {
let port = unused_port("tcp").unwrap();
let config = build_config(port, boot_nodes, secret_key);
// launch libp2p service
LibP2PService::new(&config, EnrForkId::default(), &log)
.expect("should build libp2p instance")
.1
let (signal, exit) = exit_future::signal();
let executor =
environment::TaskExecutor::new(tokio::runtime::Handle::current(), exit, log.clone());
Libp2pInstance(
LibP2PService::new(executor, &config, EnrForkId::default(), &log)
.expect("should build libp2p instance")
.1,
signal,
)
}
#[allow(dead_code)]
@ -99,8 +121,8 @@ pub fn get_enr(node: &LibP2PService<E>) -> Enr {
// Returns `n` libp2p peers in fully connected topology.
#[allow(dead_code)]
pub fn build_full_mesh(log: slog::Logger, n: usize) -> Vec<LibP2PService<E>> {
let mut nodes: Vec<LibP2PService<E>> = (0..n)
pub fn build_full_mesh(log: slog::Logger, n: usize) -> Vec<Libp2pInstance> {
let mut nodes: Vec<_> = (0..n)
.map(|_| build_libp2p_instance(vec![], None, log.clone()))
.collect();
let multiaddrs: Vec<Multiaddr> = nodes
@ -124,7 +146,7 @@ pub fn build_full_mesh(log: slog::Logger, n: usize) -> Vec<LibP2PService<E>> {
// Constructs a pair of nodes with separate loggers. The sender dials the receiver.
// This returns a (sender, receiver) pair.
#[allow(dead_code)]
pub async fn build_node_pair(log: &slog::Logger) -> (LibP2PService<E>, LibP2PService<E>) {
pub async fn build_node_pair(log: &slog::Logger) -> (Libp2pInstance, Libp2pInstance) {
let sender_log = log.new(o!("who" => "sender"));
let receiver_log = log.new(o!("who" => "receiver"));
@ -168,8 +190,8 @@ pub async fn build_node_pair(log: &slog::Logger) -> (LibP2PService<E>, LibP2PSer
// Returns `n` peers in a linear topology
#[allow(dead_code)]
pub fn build_linear(log: slog::Logger, n: usize) -> Vec<LibP2PService<E>> {
let mut nodes: Vec<LibP2PService<E>> = (0..n)
pub fn build_linear(log: slog::Logger, n: usize) -> Vec<Libp2pInstance> {
let mut nodes: Vec<_> = (0..n)
.map(|_| build_libp2p_instance(vec![], None, log.clone()))
.collect();
let multiaddrs: Vec<Multiaddr> = nodes

View File

@ -136,7 +136,10 @@ async fn test_secio_noise_fallback() {
let port = common::unused_port("tcp").unwrap();
let noisy_config = common::build_config(port, vec![], None);
let mut noisy_node = Service::new(&noisy_config, EnrForkId::default(), &log)
let (_signal, exit) = exit_future::signal();
let executor =
environment::TaskExecutor::new(tokio::runtime::Handle::current(), exit, log.clone());
let mut noisy_node = Service::new(executor, &noisy_config, EnrForkId::default(), &log)
.expect("should build a libp2p instance")
.1;

View File

@ -18,7 +18,7 @@ merkle_proof = { path = "../../consensus/merkle_proof" }
eth2_ssz = "0.1.2"
eth2_hashing = "0.1.0"
tree_hash = "0.1.0"
tokio = { version = "0.2.20", features = ["full"] }
tokio = { version = "0.2.21", features = ["full"] }
parking_lot = "0.10.2"
slog = "2.5.2"
exit-future = "0.2.0"

View File

@ -24,7 +24,7 @@ pub fn new_env() -> Environment<MinimalEthSpec> {
#[test]
fn basic() {
let mut env = new_env();
let log = env.core_context().log.clone();
let log = env.core_context().log().clone();
let mut spec = env.eth2_config().spec.clone();
env.runtime().block_on(async {

View File

@ -10,6 +10,7 @@ genesis = { path = "../genesis" }
lazy_static = "1.4.0"
matches = "0.1.8"
tempfile = "3.1.0"
exit-future = "0.2.0"
[dependencies]
beacon_chain = { path = "../beacon_chain" }
@ -25,7 +26,7 @@ eth2_ssz = "0.1.2"
tree_hash = "0.1.0"
futures = "0.3.5"
error-chain = "0.12.2"
tokio = { version = "0.2.20", features = ["full"] }
tokio = { version = "0.2.21", features = ["full"] }
parking_lot = "0.10.2"
smallvec = "1.4.0"
# TODO: Remove rand crate for mainnet
@ -34,3 +35,4 @@ fnv = "1.0.6"
rlp = "0.4.5"
lazy_static = "1.4.0"
lighthouse_metrics = { path = "../../common/lighthouse_metrics" }
environment = { path = "../../lighthouse/environment" }

View File

@ -58,7 +58,7 @@ impl<T: BeaconChainTypes> Router<T> {
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
runtime_handle: &tokio::runtime::Handle,
executor: environment::TaskExecutor,
log: slog::Logger,
) -> error::Result<mpsc::UnboundedSender<RouterMessage<T::EthSpec>>> {
let message_handler_log = log.new(o!("service"=> "router"));
@ -68,7 +68,7 @@ impl<T: BeaconChainTypes> Router<T> {
// Initialise a message instance, which itself spawns the syncing thread.
let processor = Processor::new(
runtime_handle,
executor.clone(),
beacon_chain,
network_globals.clone(),
network_send.clone(),
@ -84,12 +84,15 @@ impl<T: BeaconChainTypes> Router<T> {
};
// spawn handler task and move the message handler instance into the spawned thread
runtime_handle.spawn(async move {
handler_recv
.for_each(move |msg| future::ready(handler.handle_message(msg)))
.await;
debug!(log, "Network message handler terminated.");
});
executor.spawn(
async move {
debug!(log, "Network message router started");
handler_recv
.for_each(move |msg| future::ready(handler.handle_message(msg)))
.await;
},
"router",
);
Ok(handler_send)
}

View File

@ -14,7 +14,7 @@ use slog::{debug, error, o, trace, warn};
use ssz::Encode;
use std::sync::Arc;
use store::Store;
use tokio::sync::{mpsc, oneshot};
use tokio::sync::mpsc;
use types::{
Attestation, ChainSpec, Epoch, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock,
Slot,
@ -33,8 +33,6 @@ pub struct Processor<T: BeaconChainTypes> {
chain: Arc<BeaconChain<T>>,
/// A channel to the syncing thread.
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
/// A oneshot channel for destroying the sync thread.
_sync_exit: oneshot::Sender<()>,
/// A network context to return and handle RPC requests.
network: HandlerNetworkContext<T::EthSpec>,
/// The `RPCHandler` logger.
@ -44,7 +42,7 @@ pub struct Processor<T: BeaconChainTypes> {
impl<T: BeaconChainTypes> Processor<T> {
/// Instantiate a `Processor` instance
pub fn new(
runtime_handle: &tokio::runtime::Handle,
executor: environment::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
@ -53,8 +51,8 @@ impl<T: BeaconChainTypes> Processor<T> {
let sync_logger = log.new(o!("service"=> "sync"));
// spawn the sync thread
let (sync_send, _sync_exit) = crate::sync::manager::spawn(
runtime_handle,
let sync_send = crate::sync::manager::spawn(
executor,
beacon_chain.clone(),
network_globals,
network_send.clone(),
@ -64,7 +62,6 @@ impl<T: BeaconChainTypes> Processor<T> {
Processor {
chain: beacon_chain,
sync_send,
_sync_exit,
network: HandlerNetworkContext::new(network_send, log.clone()),
log: log.clone(),
}

View File

@ -14,8 +14,7 @@ use rest_types::ValidatorSubscription;
use slog::{debug, error, info, o, trace};
use std::sync::Arc;
use std::time::Duration;
use tokio::runtime::Handle;
use tokio::sync::{mpsc, oneshot};
use tokio::sync::mpsc;
use tokio::time::Delay;
use types::EthSpec;
@ -53,13 +52,12 @@ impl<T: BeaconChainTypes> NetworkService<T> {
pub fn start(
beacon_chain: Arc<BeaconChain<T>>,
config: &NetworkConfig,
runtime_handle: &Handle,
network_log: slog::Logger,
executor: environment::TaskExecutor,
) -> error::Result<(
Arc<NetworkGlobals<T::EthSpec>>,
mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
oneshot::Sender<()>,
)> {
let network_log = executor.log().clone();
// build the network channel
let (network_send, network_recv) = mpsc::unbounded_channel::<NetworkMessage<T::EthSpec>>();
// get a reference to the beacon chain store
@ -75,7 +73,7 @@ impl<T: BeaconChainTypes> NetworkService<T> {
// launch libp2p service
let (network_globals, mut libp2p) =
runtime_handle.enter(|| LibP2PService::new(config, enr_fork_id, &network_log))?;
LibP2PService::new(executor.clone(), config, enr_fork_id, &network_log)?;
for enr in load_dht::<T::Store, T::EthSpec>(store.clone()) {
libp2p.swarm.add_enr(enr);
@ -88,7 +86,7 @@ impl<T: BeaconChainTypes> NetworkService<T> {
beacon_chain.clone(),
network_globals.clone(),
network_send.clone(),
runtime_handle,
executor.clone(),
network_log.clone(),
)?;
@ -111,19 +109,20 @@ impl<T: BeaconChainTypes> NetworkService<T> {
propagation_percentage,
};
let network_exit = runtime_handle.enter(|| spawn_service(network_service))?;
spawn_service(executor, network_service)?;
Ok((network_globals, network_send, network_exit))
Ok((network_globals, network_send))
}
}
fn spawn_service<T: BeaconChainTypes>(
executor: environment::TaskExecutor,
mut service: NetworkService<T>,
) -> error::Result<tokio::sync::oneshot::Sender<()>> {
let (network_exit, mut exit_rx) = tokio::sync::oneshot::channel();
) -> error::Result<()> {
let mut exit_rx = executor.exit();
// spawn on the current executor
tokio::spawn(async move {
executor.spawn_without_exit(async move {
loop {
// build the futures to check simultaneously
tokio::select! {
@ -361,9 +360,9 @@ fn spawn_service<T: BeaconChainTypes>(
}
}
}
});
}, "network");
Ok(network_exit)
Ok(())
}
/// Returns a `Delay` that triggers shortly after the next change in the beacon chain fork version.

View File

@ -32,7 +32,9 @@ mod tests {
let enrs = vec![enr1, enr2];
let runtime = Runtime::new().unwrap();
let handle = runtime.handle().clone();
let (signal, exit) = exit_future::signal();
let executor = environment::TaskExecutor::new(runtime.handle().clone(), exit, log.clone());
let mut config = NetworkConfig::default();
config.libp2p_port = 21212;
@ -42,8 +44,8 @@ mod tests {
// Create a new network service which implicitly gets dropped at the
// end of the block.
let _ =
NetworkService::start(beacon_chain.clone(), &config, &handle, log.clone()).unwrap();
let _ = NetworkService::start(beacon_chain.clone(), &config, executor).unwrap();
drop(signal);
});
runtime.shutdown_timeout(tokio::time::Duration::from_millis(300));

View File

@ -48,7 +48,7 @@ use smallvec::SmallVec;
use std::boxed::Box;
use std::ops::Sub;
use std::sync::Arc;
use tokio::sync::{mpsc, oneshot};
use tokio::sync::mpsc;
use types::{EthSpec, Hash256, SignedBeaconBlock, Slot};
/// The number of slots ahead of us that is allowed before requesting a long-range (batch) Sync
@ -181,17 +181,12 @@ impl SingleBlockRequest {
/// chain. This allows the chain to be
/// dropped during the syncing process which will gracefully end the `SyncManager`.
pub fn spawn<T: BeaconChainTypes>(
runtime_handle: &tokio::runtime::Handle,
executor: environment::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
log: slog::Logger,
) -> (
mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
oneshot::Sender<()>,
) {
// generate the exit channel
let (sync_exit, exit_rx) = tokio::sync::oneshot::channel();
) -> mpsc::UnboundedSender<SyncMessage<T::EthSpec>> {
// generate the message channel
let (sync_send, sync_recv) = mpsc::unbounded_channel::<SyncMessage<T::EthSpec>>();
@ -215,11 +210,8 @@ pub fn spawn<T: BeaconChainTypes>(
// spawn the sync manager thread
debug!(log, "Sync Manager started");
runtime_handle.spawn(async move {
futures::future::select(Box::pin(sync_manager.main()), exit_rx).await;
info!(log.clone(), "Sync Manager shutdown");
});
(sync_send, sync_exit)
executor.spawn(async move { Box::pin(sync_manager.main()).await }, "sync");
sync_send
}
impl<T: BeaconChainTypes> SyncManager<T> {

View File

@ -25,7 +25,7 @@ state_processing = { path = "../../consensus/state_processing" }
types = { path = "../../consensus/types" }
http = "0.2.1"
hyper = "0.13.5"
tokio = { version = "0.2", features = ["sync"] }
tokio = { version = "0.2.21", features = ["sync"] }
url = "2.1.1"
lazy_static = "1.4.0"
eth2_config = { path = "../../common/eth2_config" }
@ -36,6 +36,7 @@ parking_lot = "0.10.2"
futures = "0.3.5"
operation_pool = { path = "../operation_pool" }
rayon = "1.3.0"
environment = { path = "../../lighthouse/environment" }
[dev-dependencies]
assert_matches = "1.3.0"

View File

@ -35,7 +35,7 @@ use std::net::SocketAddr;
use std::ops::Deref;
use std::path::PathBuf;
use std::sync::Arc;
use tokio::sync::{mpsc, oneshot};
use tokio::sync::mpsc;
use url_query::UrlQuery;
pub use crate::helpers::parse_pubkey_bytes;
@ -51,14 +51,15 @@ pub struct NetworkInfo<T: BeaconChainTypes> {
// Allowing more than 7 arguments.
#[allow(clippy::too_many_arguments)]
pub fn start_server<T: BeaconChainTypes>(
executor: environment::TaskExecutor,
config: &Config,
beacon_chain: Arc<BeaconChain<T>>,
network_info: NetworkInfo<T>,
db_path: PathBuf,
freezer_db_path: PathBuf,
eth2_config: Eth2Config,
log: slog::Logger,
) -> Result<(oneshot::Sender<()>, SocketAddr), hyper::Error> {
) -> Result<SocketAddr, hyper::Error> {
let log = executor.log();
let inner_log = log.clone();
let eth2_config = Arc::new(eth2_config);
@ -98,7 +99,7 @@ pub fn start_server<T: BeaconChainTypes>(
let actual_listen_addr = server.local_addr();
// Build a channel to kill the HTTP server.
let (exit_signal, exit) = oneshot::channel::<()>();
let exit = executor.exit();
let inner_log = log.clone();
let server_exit = async move {
let _ = exit.await;
@ -116,7 +117,8 @@ pub fn start_server<T: BeaconChainTypes>(
inner_log,
"HTTP server failed to start, Unable to bind"; "address" => format!("{:?}", e)
)
});
})
.unwrap_or_else(|_| ());
info!(
log,
@ -125,9 +127,9 @@ pub fn start_server<T: BeaconChainTypes>(
"port" => actual_listen_addr.port(),
);
tokio::spawn(server_future);
executor.spawn_without_exit(server_future, "http");
Ok((exit_signal, actual_listen_addr))
Ok(actual_listen_addr)
}
#[derive(Clone)]

View File

@ -58,7 +58,7 @@ impl<E: EthSpec> ProductionBeaconNode<E> {
&matches,
&context.eth2_config.spec_constants,
&context.eth2_config().spec,
context.log.clone(),
context.log().clone(),
)?;
Self::new(context, client_config).await
}
@ -75,7 +75,7 @@ impl<E: EthSpec> ProductionBeaconNode<E> {
let client_config_1 = client_config.clone();
let client_genesis = client_config.genesis.clone();
let store_config = client_config.store.clone();
let log = context.log.clone();
let log = context.log().clone();
let db_path = client_config.create_db_path()?;
let freezer_db_path_res = client_config.create_freezer_db_path();

View File

@ -51,4 +51,5 @@ fn http_server_genesis_state() {
api_state, db_state,
"genesis state from api should match that from the DB"
);
env.fire_signal();
}

View File

@ -8,7 +8,8 @@ edition = "2018"
beacon_chain = { path = "../beacon_chain" }
types = { path = "../../consensus/types" }
slot_clock = { path = "../../common/slot_clock" }
tokio = { version = "0.2.20", features = ["full"] }
tokio = { version = "0.2.21", features = ["full"] }
slog = "2.5.2"
parking_lot = "0.10.2"
futures = "0.3.5"
environment = { path = "../../lighthouse/environment" }

View File

@ -3,23 +3,20 @@
//! This service allows task execution on the beacon node for various functionality.
use beacon_chain::{BeaconChain, BeaconChainTypes};
use futures::future;
use futures::stream::StreamExt;
use slog::info;
use slot_clock::SlotClock;
use std::sync::Arc;
use std::time::Duration;
use tokio::time::{interval_at, Instant};
/// Spawns a timer service which periodically executes tasks for the beacon chain
/// TODO: We might not need a `Handle` to the runtime since this function should be
/// called from the context of a runtime and we can simply spawn using task::spawn.
/// Check for issues without the Handle.
pub fn spawn<T: BeaconChainTypes>(
pub fn spawn_timer<T: BeaconChainTypes>(
executor: environment::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
milliseconds_per_slot: u64,
) -> Result<tokio::sync::oneshot::Sender<()>, &'static str> {
let (exit_signal, exit) = tokio::sync::oneshot::channel();
) -> Result<(), &'static str> {
let log = executor.log();
let start_instant = Instant::now()
+ beacon_chain
.slot_clock
@ -27,14 +24,15 @@ pub fn spawn<T: BeaconChainTypes>(
.ok_or_else(|| "slot_notifier unable to determine time to next slot")?;
// Warning: `interval_at` panics if `milliseconds_per_slot` = 0.
let timer_future = interval_at(start_instant, Duration::from_millis(milliseconds_per_slot))
.for_each(move |_| {
let mut interval = interval_at(start_instant, Duration::from_millis(milliseconds_per_slot));
let timer_future = async move {
while interval.next().await.is_some() {
beacon_chain.per_slot_task();
future::ready(())
});
}
};
let future = futures::future::select(timer_future, exit);
tokio::spawn(future);
executor.spawn(timer_future, "timer");
info!(log, "Timer service started");
Ok(exit_signal)
Ok(())
}

View File

@ -12,6 +12,7 @@ serde = "1.0.110"
serde_derive = "1.0.110"
serde_json = "1.0.52"
slog = "2.5.2"
tokio = { version = "0.2.20", features = ["full"] }
tokio = { version = "0.2.21", features = ["full"] }
types = { path = "../../consensus/types" }
ws = "0.9.1"
environment = { path = "../../lighthouse/environment" }

View File

@ -1,4 +1,4 @@
use slog::{debug, error, info, warn, Logger};
use slog::{debug, error, info, warn};
use std::marker::PhantomData;
use std::net::SocketAddr;
use types::EthSpec;
@ -34,16 +34,10 @@ impl<T: EthSpec> WebSocketSender<T> {
}
pub fn start_server<T: EthSpec>(
executor: environment::TaskExecutor,
config: &Config,
log: &Logger,
) -> Result<
(
WebSocketSender<T>,
tokio::sync::oneshot::Sender<()>,
SocketAddr,
),
String,
> {
) -> Result<(WebSocketSender<T>, SocketAddr), String> {
let log = executor.log();
let server_string = format!("{}:{}", config.listen_address, config.port);
// Create a server that simply ignores any incoming messages.
@ -67,31 +61,26 @@ pub fn start_server<T: EthSpec>(
let broadcaster = server.broadcaster();
// Produce a signal/channel that can gracefully shutdown the websocket server.
let exit_channel = {
let (exit_channel, exit) = tokio::sync::oneshot::channel();
let log_inner = log.clone();
let broadcaster_inner = server.broadcaster();
let exit_future = async move {
let _ = exit.await;
if let Err(e) = broadcaster_inner.shutdown() {
warn!(
log_inner,
"Websocket server errored on shutdown";
"error" => format!("{:?}", e)
);
} else {
info!(log_inner, "Websocket server shutdown");
}
};
// Place a future on the handle that will shutdown the websocket server when the
// application exits.
tokio::spawn(exit_future);
exit_channel
let exit = executor.exit();
let log_inner = log.clone();
let broadcaster_inner = server.broadcaster();
let exit_future = async move {
let _ = exit.await;
if let Err(e) = broadcaster_inner.shutdown() {
warn!(
log_inner,
"Websocket server errored on shutdown";
"error" => format!("{:?}", e)
);
} else {
info!(log_inner, "Websocket server shutdown");
}
};
// Place a future on the handle that will shutdown the websocket server when the
// application exits.
executor.runtime_handle().spawn(exit_future);
let log_inner = log.clone();
let _ = std::thread::spawn(move || match server.run() {
@ -122,7 +111,6 @@ pub fn start_server<T: EthSpec>(
sender: Some(broadcaster),
_phantom: PhantomData,
},
exit_channel,
actual_listen_addr,
))
}

View File

@ -16,5 +16,5 @@ tempdir = "0.3.7"
serde = "1.0.110"
serde_yaml = "0.8.11"
types = { path = "../../consensus/types"}
eth2-libp2p = { path = "../../beacon_node/eth2-libp2p"}
enr = { version = "0.1.0", features = ["libsecp256k1", "ed25519"] }
eth2_ssz = "0.1.2"

View File

@ -7,7 +7,7 @@
//!
//! https://github.com/sigp/lighthouse/pull/605
use eth2_libp2p::Enr;
use enr::{CombinedKey, Enr};
use ssz::{Decode, Encode};
use std::fs::{create_dir_all, File};
use std::io::{Read, Write};
@ -36,7 +36,7 @@ pub const HARDCODED_BOOT_ENR: &[u8] = include_bytes!("../witti-v0-11-3/boot_enr.
pub struct Eth2TestnetConfig<E: EthSpec> {
pub deposit_contract_address: String,
pub deposit_contract_deploy_block: u64,
pub boot_enr: Option<Vec<Enr>>,
pub boot_enr: Option<Vec<Enr<CombinedKey>>>,
pub genesis_state: Option<BeaconState<E>>,
pub yaml_config: Option<YamlConfig>,
}
@ -239,7 +239,7 @@ mod tests {
}
fn do_test<E: EthSpec>(
boot_enr: Option<Vec<Enr>>,
boot_enr: Option<Vec<Enr<CombinedKey>>>,
genesis_state: Option<BeaconState<E>>,
yaml_config: Option<YamlConfig>,
) {

View File

@ -6,7 +6,7 @@ edition = "2018"
[dependencies]
futures = "0.3.5"
tokio = { version = "0.2.20", features = ["time"] }
tokio = { version = "0.2.21", features = ["time"] }
[dev-dependencies]
tokio = { version = "0.2.20", features = ["time", "rt-threaded", "macros"] }
tokio = { version = "0.2.21", features = ["time", "rt-threaded", "macros"] }

View File

@ -56,7 +56,9 @@
use prometheus::{HistogramOpts, HistogramTimer, Opts};
pub use prometheus::{Encoder, Gauge, Histogram, IntCounter, IntGauge, Result, TextEncoder};
pub use prometheus::{
Encoder, Gauge, Histogram, HistogramVec, IntCounter, IntGauge, IntGaugeVec, Result, TextEncoder,
};
/// Collect all the metrics for reporting.
pub fn gather() -> Vec<prometheus::proto::MetricFamily> {
@ -99,6 +101,48 @@ pub fn try_create_histogram(name: &str, help: &str) -> Result<Histogram> {
Ok(histogram)
}
/// Attempts to crate a `HistogramVec`, returning `Err` if the registry does not accept the counter
/// (potentially due to naming conflict).
pub fn try_create_histogram_vec(
name: &str,
help: &str,
label_names: &[&str],
) -> Result<HistogramVec> {
let opts = HistogramOpts::new(name, help);
let histogram_vec = HistogramVec::new(opts, label_names)?;
prometheus::register(Box::new(histogram_vec.clone()))?;
Ok(histogram_vec)
}
/// Attempts to crate a `IntGaugeVec`, returning `Err` if the registry does not accept the gauge
/// (potentially due to naming conflict).
pub fn try_create_int_gauge_vec(
name: &str,
help: &str,
label_names: &[&str],
) -> Result<IntGaugeVec> {
let opts = Opts::new(name, help);
let counter_vec = IntGaugeVec::new(opts, label_names)?;
prometheus::register(Box::new(counter_vec.clone()))?;
Ok(counter_vec)
}
pub fn get_int_gauge(int_gauge_vec: &Result<IntGaugeVec>, name: &[&str]) -> Option<IntGauge> {
if let Ok(int_gauge_vec) = int_gauge_vec {
Some(int_gauge_vec.get_metric_with_label_values(name).ok()?)
} else {
None
}
}
pub fn get_histogram(histogram_vec: &Result<HistogramVec>, name: &[&str]) -> Option<Histogram> {
if let Ok(histogram_vec) = histogram_vec {
Some(histogram_vec.get_metric_with_label_values(name).ok()?)
} else {
None
}
}
/// Starts a timer for the given `Histogram`, stopping when it gets dropped or given to `stop_timer(..)`.
pub fn start_timer(histogram: &Result<Histogram>) -> Option<HistogramTimer> {
if let Ok(histogram) = histogram {
@ -133,6 +177,18 @@ pub fn set_gauge(gauge: &Result<IntGauge>, value: i64) {
}
}
pub fn inc_gauge(gauge: &Result<IntGauge>) {
if let Ok(gauge) = gauge {
gauge.inc();
}
}
pub fn dec_gauge(gauge: &Result<IntGauge>) {
if let Ok(gauge) = gauge {
gauge.dec();
}
}
pub fn maybe_set_gauge(gauge: &Result<IntGauge>, value_opt: Option<i64>) {
if let Some(value) = value_opt {
set_gauge(gauge, value)

View File

@ -27,7 +27,7 @@ dirs = "2.0.2"
genesis = { path = "../beacon_node/genesis" }
deposit_contract = { path = "../common/deposit_contract" }
tree_hash = "0.1.0"
tokio = { version = "0.2.20", features = ["full"] }
tokio = { version = "0.2.21", features = ["full"] }
clap_utils = { path = "../common/clap_utils" }
eth2-libp2p = { path = "../beacon_node/eth2-libp2p" }
validator_dir = { path = "../common/validator_dir", features = ["insecure_keys"] }

View File

@ -46,7 +46,7 @@ pub fn run<T: EthSpec>(mut env: Environment<T>, matches: &ArgMatches<'_>) -> Res
config.lowest_cached_block_number = eth2_testnet_config.deposit_contract_deploy_block;
config.follow_distance = spec.eth1_follow_distance / 2;
let genesis_service = Eth1GenesisService::new(config, env.core_context().log.clone());
let genesis_service = Eth1GenesisService::new(config, env.core_context().log().clone());
env.runtime().block_on(async {
let _ = genesis_service

View File

@ -9,7 +9,7 @@ write_ssz_files = ["beacon_node/write_ssz_files"] # Writes debugging .ssz files
[dependencies]
beacon_node = { "path" = "../beacon_node" }
tokio = "0.2.20"
tokio = "0.2.21"
slog = { version = "2.5.2", features = ["max_level_trace"] }
sloggers = "1.0.0"
types = { "path" = "../consensus/types" }

View File

@ -6,7 +6,7 @@ edition = "2018"
[dependencies]
clap = "2.33.0"
tokio = "0.2.20"
tokio = "0.2.21"
slog = { version = "2.5.2", features = ["max_level_trace"] }
sloggers = "1.0.0"
types = { "path" = "../../consensus/types" }
@ -20,6 +20,9 @@ ctrlc = { version = "3.1.4", features = ["termination"] }
futures = "0.3.5"
parking_lot = "0.10.2"
slog-json = "2.3.0"
exit-future = "0.2.0"
lazy_static = "1.4.0"
lighthouse_metrics = { path = "../../common/lighthouse_metrics" }
[dev-dependencies]
beacon_node = { path = "../../beacon_node" }

View File

@ -0,0 +1,128 @@
use crate::metrics;
use futures::prelude::*;
use slog::{debug, trace};
use tokio::runtime::Handle;
/// A wrapper over a runtime handle which can spawn async and blocking tasks.
#[derive(Clone)]
pub struct TaskExecutor {
/// The handle to the runtime on which tasks are spawned
pub(crate) handle: Handle,
/// The receiver exit future which on receiving shuts down the task
pub(crate) exit: exit_future::Exit,
pub(crate) log: slog::Logger,
}
impl TaskExecutor {
/// Create a new task executor.
///
/// Note: this function is mainly useful in tests. A `TaskExecutor` should be normally obtained from
/// a [`RuntimeContext`](struct.RuntimeContext.html)
pub fn new(handle: Handle, exit: exit_future::Exit, log: slog::Logger) -> Self {
Self { handle, exit, log }
}
/// Spawn a future on the tokio runtime wrapped in an `exit_future::Exit`. The task is canceled
/// when the corresponding exit_future `Signal` is fired/dropped.
///
/// This function generates prometheus metrics on number of tasks and task duration.
pub fn spawn(&self, task: impl Future<Output = ()> + Send + 'static, name: &'static str) {
let exit = self.exit.clone();
let log = self.log.clone();
if let Some(int_gauge) = metrics::get_int_gauge(&metrics::ASYNC_TASKS_COUNT, &[name]) {
// Task is shutdown before it completes if `exit` receives
let int_gauge_1 = int_gauge.clone();
let future = future::select(Box::pin(task), exit).then(move |either| {
match either {
future::Either::Left(_) => trace!(log, "Async task completed"; "task" => name),
future::Either::Right(_) => {
debug!(log, "Async task shutdown, exit received"; "task" => name)
}
}
int_gauge_1.dec();
futures::future::ready(())
});
int_gauge.inc();
self.handle.spawn(future);
}
}
/// Spawn a future on the tokio runtime. This function does not wrap the task in an `exit_future::Exit`
/// like [spawn](#method.spawn).
/// The caller of this function is responsible for wrapping up the task with an `exit_future::Exit` to
/// ensure that the task gets canceled appropriately.
/// This function generates prometheus metrics on number of tasks and task duration.
///
/// This is useful in cases where the future to be spawned needs to do additional cleanup work when
/// the task is completed/canceled (e.g. writing local variables to disk) or the task is created from
/// some framework which does its own cleanup (e.g. a hyper server).
pub fn spawn_without_exit(
&self,
task: impl Future<Output = ()> + Send + 'static,
name: &'static str,
) {
if let Some(int_gauge) = metrics::get_int_gauge(&metrics::ASYNC_TASKS_COUNT, &[name]) {
let int_gauge_1 = int_gauge.clone();
let future = task.then(move |_| {
int_gauge_1.dec();
futures::future::ready(())
});
int_gauge.inc();
self.handle.spawn(future);
}
}
/// Spawn a blocking task on a dedicated tokio thread pool wrapped in an exit future.
/// This function generates prometheus metrics on number of tasks and task duration.
pub fn spawn_blocking<F>(&self, task: F, name: &'static str)
where
F: FnOnce() -> () + Send + 'static,
{
let exit = self.exit.clone();
let log = self.log.clone();
if let Some(metric) = metrics::get_histogram(&metrics::BLOCKING_TASKS_HISTOGRAM, &[name]) {
if let Some(int_gauge) = metrics::get_int_gauge(&metrics::BLOCKING_TASKS_COUNT, &[name])
{
let int_gauge_1 = int_gauge.clone();
let timer = metric.start_timer();
let join_handle = self.handle.spawn_blocking(task);
let future = future::select(join_handle, exit).then(move |either| {
match either {
future::Either::Left(_) => {
trace!(log, "Blocking task completed"; "task" => name)
}
future::Either::Right(_) => {
debug!(log, "Blocking task shutdown, exit received"; "task" => name)
}
}
timer.observe_duration();
int_gauge_1.dec();
futures::future::ready(())
});
int_gauge.inc();
self.handle.spawn(future);
}
}
}
/// Returns the underlying runtime handle.
pub fn runtime_handle(&self) -> Handle {
self.handle.clone()
}
/// Returns a copy of the `exit_future::Exit`.
pub fn exit(&self) -> exit_future::Exit {
self.exit.clone()
}
/// Returns a reference to the logger.
pub fn log(&self) -> &slog::Logger {
&self.log
}
}

View File

@ -10,6 +10,8 @@
use eth2_config::Eth2Config;
use eth2_testnet_config::Eth2TestnetConfig;
use futures::channel::oneshot;
pub use executor::TaskExecutor;
use slog::{info, o, Drain, Level, Logger};
use sloggers::{null::NullLoggerBuilder, Build};
use std::cell::RefCell;
@ -17,8 +19,10 @@ use std::ffi::OsStr;
use std::fs::{rename as FsRename, OpenOptions};
use std::path::PathBuf;
use std::time::{SystemTime, UNIX_EPOCH};
use tokio::runtime::{Builder as RuntimeBuilder, Handle, Runtime};
use tokio::runtime::{Builder as RuntimeBuilder, Runtime};
use types::{EthSpec, InteropEthSpec, MainnetEthSpec, MinimalEthSpec};
mod executor;
mod metrics;
pub const ETH2_CONFIG_FILENAME: &str = "eth2-spec.toml";
@ -172,10 +176,13 @@ impl<E: EthSpec> EnvironmentBuilder<E> {
/// Consumes the builder, returning an `Environment`.
pub fn build(self) -> Result<Environment<E>, String> {
let (signal, exit) = exit_future::signal();
Ok(Environment {
runtime: self
.runtime
.ok_or_else(|| "Cannot build environment without runtime".to_string())?,
signal: Some(signal),
exit,
log: self
.log
.ok_or_else(|| "Cannot build environment without log".to_string())?,
@ -192,8 +199,7 @@ impl<E: EthSpec> EnvironmentBuilder<E> {
/// `Runtime`, instead it only has access to a `Runtime`.
#[derive(Clone)]
pub struct RuntimeContext<E: EthSpec> {
pub runtime_handle: Handle,
pub log: Logger,
pub executor: TaskExecutor,
pub eth_spec_instance: E,
pub eth2_config: Eth2Config,
}
@ -204,8 +210,11 @@ impl<E: EthSpec> RuntimeContext<E> {
/// The generated service will have the `service_name` in all it's logs.
pub fn service_context(&self, service_name: String) -> Self {
Self {
runtime_handle: self.runtime_handle.clone(),
log: self.log.new(o!("service" => service_name)),
executor: TaskExecutor {
handle: self.executor.handle.clone(),
exit: self.executor.exit.clone(),
log: self.executor.log.new(o!("service" => service_name)),
},
eth_spec_instance: self.eth_spec_instance.clone(),
eth2_config: self.eth2_config.clone(),
}
@ -215,12 +224,19 @@ impl<E: EthSpec> RuntimeContext<E> {
pub fn eth2_config(&self) -> &Eth2Config {
&self.eth2_config
}
/// Returns a reference to the logger for this service.
pub fn log(&self) -> &slog::Logger {
self.executor.log()
}
}
/// An environment where Lighthouse services can run. Used to start a production beacon node or
/// validator client, or to run tests that involve logging and async task execution.
pub struct Environment<E: EthSpec> {
runtime: Runtime,
signal: Option<exit_future::Signal>,
exit: exit_future::Exit,
log: Logger,
eth_spec_instance: E,
pub eth2_config: Eth2Config,
@ -239,8 +255,11 @@ impl<E: EthSpec> Environment<E> {
/// Returns a `Context` where no "service" has been added to the logger output.
pub fn core_context(&mut self) -> RuntimeContext<E> {
RuntimeContext {
runtime_handle: self.runtime.handle().clone(),
log: self.log.clone(),
executor: TaskExecutor {
exit: self.exit.clone(),
handle: self.runtime().handle().clone(),
log: self.log.clone(),
},
eth_spec_instance: self.eth_spec_instance.clone(),
eth2_config: self.eth2_config.clone(),
}
@ -249,8 +268,11 @@ impl<E: EthSpec> Environment<E> {
/// Returns a `Context` where the `service_name` is added to the logger output.
pub fn service_context(&mut self, service_name: String) -> RuntimeContext<E> {
RuntimeContext {
runtime_handle: self.runtime.handle().clone(),
log: self.log.new(o!("service" => service_name)),
executor: TaskExecutor {
exit: self.exit.clone(),
handle: self.runtime().handle().clone(),
log: self.log.new(o!("service" => service_name.clone())),
},
eth_spec_instance: self.eth_spec_instance.clone(),
eth2_config: self.eth2_config.clone(),
}
@ -279,6 +301,13 @@ impl<E: EthSpec> Environment<E> {
.shutdown_timeout(std::time::Duration::from_secs(2))
}
/// Fire exit signal which shuts down all spawned services
pub fn fire_signal(&mut self) {
if let Some(signal) = self.signal.take() {
let _ = signal.fire();
}
}
/// Sets the logger (and all child loggers) to log to a file.
pub fn log_to_json_file(
&mut self,

View File

@ -0,0 +1,21 @@
/// Handles async task metrics
use lazy_static::lazy_static;
pub use lighthouse_metrics::*;
lazy_static! {
pub static ref ASYNC_TASKS_COUNT: Result<IntGaugeVec> = try_create_int_gauge_vec(
"async_tasks_count",
"Total number of async tasks spawned using spawn",
&["async_task_count"]
);
pub static ref BLOCKING_TASKS_COUNT: Result<IntGaugeVec> = try_create_int_gauge_vec(
"blocking_tasks_count",
"Total number of async tasks spawned using spawn_blocking",
&["blocking_task_count"]
);
pub static ref BLOCKING_TASKS_HISTOGRAM: Result<HistogramVec> = try_create_histogram_vec(
"blocking_tasks_histogram",
"Time taken by blocking tasks",
&["blocking_task_hist"]
);
}

View File

@ -141,7 +141,7 @@ fn run<E: EthSpec>(
.eth2_testnet_config(eth2_testnet_config)?
.build()?;
let log = environment.core_context().log;
let log = environment.core_context().log().clone();
if let Some(log_path) = matches.value_of("logfile") {
let path = log_path
@ -216,11 +216,15 @@ fn run<E: EthSpec>(
))
.map_err(|e| format!("Failed to init validator client: {}", e))?;
environment.core_context().runtime_handle.enter(|| {
validator
.start_service()
.map_err(|e| format!("Failed to start validator client service: {}", e))
})?;
environment
.core_context()
.executor
.runtime_handle()
.enter(|| {
validator
.start_service()
.map_err(|e| format!("Failed to start validator client service: {}", e))
})?;
Some(validator)
} else {
@ -234,9 +238,9 @@ fn run<E: EthSpec>(
// Block this thread until Crtl+C is pressed.
environment.block_until_ctrl_c()?;
info!(log, "Shutting down..");
environment.fire_signal();
drop(beacon_node);
drop(validator_client);

View File

@ -5,8 +5,8 @@ authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018"
[dependencies]
tokio = { version = "0.2.21", features = ["time"] }
web3 = "0.11.0"
tokio = { version = "0.2.20", features = ["time"] }
futures = { version = "0.3.5", features = ["compat"] }
types = { path = "../../consensus/types"}
serde_json = "1.0.52"

View File

@ -12,7 +12,7 @@ types = { path = "../../consensus/types" }
validator_client = { path = "../../validator_client" }
parking_lot = "0.10.2"
futures = "0.3.5"
tokio = "0.2.20"
tokio = "0.2.21"
eth1_test_rig = { path = "../eth1_test_rig" }
env_logger = "0.7.1"
clap = "2.33.0"

View File

@ -9,7 +9,7 @@ name = "validator_client"
path = "src/lib.rs"
[dev-dependencies]
tokio = {version = "0.2.20", features = ["time", "rt-threaded", "macros"]}
tokio = { version = "0.2.21", features = ["time", "rt-threaded", "macros"] }
[dependencies]
eth2_ssz = "0.1.2"
@ -27,7 +27,7 @@ serde_json = "1.0.52"
slog = { version = "2.5.2", features = ["max_level_trace", "release_max_level_trace"] }
slog-async = "2.5.0"
slog-term = "2.5.0"
tokio = {version = "0.2.20", features = ["time"]}
tokio = { version = "0.2.21", features = ["time"] }
error-chain = "0.12.2"
bincode = "1.2.1"
futures = { version = "0.3.5", features = ["compat"] }

View File

@ -3,8 +3,7 @@ use crate::{
validator_store::ValidatorStore,
};
use environment::RuntimeContext;
use exit_future::Signal;
use futures::{FutureExt, StreamExt};
use futures::StreamExt;
use remote_beacon_node::{PublishStatus, RemoteBeaconNode};
use slog::{crit, debug, info, trace};
use slot_clock::SlotClock;
@ -118,8 +117,8 @@ impl<T, E: EthSpec> Deref for AttestationService<T, E> {
impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
/// Starts the service which periodically produces attestations.
pub fn start_update_service(self, spec: &ChainSpec) -> Result<Signal, String> {
let log = self.context.log.clone();
pub fn start_update_service(self, spec: &ChainSpec) -> Result<(), String> {
let log = self.context.log().clone();
let slot_duration = Duration::from_millis(spec.milliseconds_per_slot);
let duration_to_next_slot = self
@ -141,13 +140,11 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
)
};
let (exit_signal, exit_fut) = exit_future::signal();
let runtime_handle = self.context.runtime_handle.clone();
let executor = self.context.executor.clone();
let interval_fut = async move {
while interval.next().await.is_some() {
let log = &self.context.log;
let log = self.context.log();
if let Err(e) = self.spawn_attestation_tasks(slot_duration) {
crit!(
@ -164,13 +161,8 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
}
};
let future = futures::future::select(
Box::pin(interval_fut),
exit_fut.map(move |_| info!(log, "Shutdown complete")),
);
runtime_handle.spawn(future);
Ok(exit_signal)
executor.spawn(interval_fut, "attestation_service");
Ok(())
}
/// For each each required attestation, spawn a new task that downloads, signs and uploads the
@ -214,7 +206,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
.into_iter()
.for_each(|(committee_index, validator_duties)| {
// Spawn a separate task for each attestation.
self.inner.context.runtime_handle.spawn(
self.inner.context.executor.runtime_handle().spawn(
self.clone().publish_attestations_and_aggregates(
slot,
committee_index,
@ -243,7 +235,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
validator_duties: Vec<DutyAndProof>,
aggregate_production_instant: Instant,
) -> Result<(), ()> {
let log = &self.context.log;
let log = self.context.log();
// There's not need to produce `Attestation` or `SignedAggregateAndProof` if we do not have
// any validators for the given `slot` and `committee_index`.
@ -314,7 +306,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
committee_index: CommitteeIndex,
validator_duties: &[DutyAndProof],
) -> Result<Option<Attestation<E>>, String> {
let log = &self.context.log;
let log = self.context.log();
if validator_duties.is_empty() {
return Ok(None);
@ -448,7 +440,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
attestation: Attestation<E>,
validator_duties: &[DutyAndProof],
) -> Result<(), String> {
let log = &self.context.log;
let log = self.context.log();
let aggregated_attestation = self
.beacon_node
@ -548,6 +540,7 @@ impl<T: SlotClock + 'static, E: EthSpec> AttestationService<T, E> {
#[cfg(test)]
mod tests {
use super::*;
use futures::future::FutureExt;
use parking_lot::RwLock;
/// This test is to ensure that a `tokio_timer::Delay` with an instant in the past will still

View File

@ -1,7 +1,6 @@
use crate::{duties_service::DutiesService, validator_store::ValidatorStore};
use environment::RuntimeContext;
use exit_future::Signal;
use futures::{FutureExt, StreamExt, TryFutureExt};
use futures::{StreamExt, TryFutureExt};
use remote_beacon_node::{PublishStatus, RemoteBeaconNode};
use slog::{crit, error, info, trace};
use slot_clock::SlotClock;
@ -113,8 +112,8 @@ impl<T, E: EthSpec> Deref for BlockService<T, E> {
impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
/// Starts the service that periodically attempts to produce blocks.
pub fn start_update_service(self, spec: &ChainSpec) -> Result<Signal, String> {
let log = self.context.log.clone();
pub fn start_update_service(self, spec: &ChainSpec) -> Result<(), String> {
let log = self.context.log().clone();
let duration_to_next_slot = self
.slot_clock
@ -136,7 +135,7 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
)
};
let runtime_handle = self.inner.context.runtime_handle.clone();
let executor = self.inner.context.executor.clone();
let interval_fut = async move {
while interval.next().await.is_some() {
@ -144,20 +143,14 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
}
};
let (exit_signal, exit_fut) = exit_future::signal();
executor.spawn(interval_fut, "block_service");
let future = futures::future::select(
Box::pin(interval_fut),
exit_fut.map(move |_| info!(log, "Shutdown complete")),
);
runtime_handle.spawn(future);
Ok(exit_signal)
Ok(())
}
/// Attempt to produce a block for any block producers in the `ValidatorStore`.
async fn do_update(&self) -> Result<(), ()> {
let log = &self.context.log;
let log = self.context.log();
let slot = self.slot_clock.now().ok_or_else(move || {
crit!(log, "Duties manager failed to read slot clock");
@ -190,7 +183,7 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
iter.for_each(|validator_pubkey| {
let service = self.clone();
let log = log.clone();
self.inner.context.runtime_handle.spawn(
self.inner.context.executor.runtime_handle().spawn(
service
.publish_block(slot, validator_pubkey)
.map_err(move |e| {
@ -208,7 +201,7 @@ impl<T: SlotClock + 'static, E: EthSpec> BlockService<T, E> {
/// Produce a block at the given slot for validator_pubkey
async fn publish_block(self, slot: Slot, validator_pubkey: PublicKey) -> Result<(), String> {
let log = &self.context.log;
let log = self.context.log();
let current_slot = self
.slot_clock

View File

@ -1,11 +1,10 @@
use crate::{is_synced::is_synced, validator_store::ValidatorStore};
use environment::RuntimeContext;
use exit_future::Signal;
use futures::{FutureExt, StreamExt};
use futures::StreamExt;
use parking_lot::RwLock;
use remote_beacon_node::{PublishStatus, RemoteBeaconNode};
use rest_types::{ValidatorDuty, ValidatorDutyBytes, ValidatorSubscription};
use slog::{debug, error, info, trace, warn};
use slog::{debug, error, trace, warn};
use slot_clock::SlotClock;
use std::collections::HashMap;
use std::convert::TryInto;
@ -439,9 +438,7 @@ impl<T: SlotClock + 'static, E: EthSpec> DutiesService<T, E> {
}
/// Start the service that periodically polls the beacon node for validator duties.
pub fn start_update_service(self, spec: &ChainSpec) -> Result<Signal, String> {
let log = self.context.log.clone();
pub fn start_update_service(self, spec: &ChainSpec) -> Result<(), String> {
let duration_to_next_slot = self
.slot_clock
.duration_to_next_slot()
@ -456,15 +453,14 @@ impl<T: SlotClock + 'static, E: EthSpec> DutiesService<T, E> {
)
};
let (exit_signal, exit_fut) = exit_future::signal();
// Run an immediate update before starting the updater service.
self.inner
.context
.runtime_handle
.executor
.runtime_handle()
.spawn(self.clone().do_update());
let runtime_handle = self.inner.context.runtime_handle.clone();
let executor = self.inner.context.executor.clone();
let interval_fut = async move {
while interval.next().await.is_some() {
@ -472,18 +468,14 @@ impl<T: SlotClock + 'static, E: EthSpec> DutiesService<T, E> {
}
};
let future = futures::future::select(
Box::pin(interval_fut),
exit_fut.map(move |_| info!(log, "Shutdown complete")),
);
runtime_handle.spawn(future);
executor.spawn(interval_fut, "duties_service");
Ok(exit_signal)
Ok(())
}
/// Attempt to download the duties of all managed validators for this epoch and the next.
async fn do_update(self) -> Result<(), ()> {
let log = &self.context.log;
let log = self.context.log();
if !is_synced(&self.beacon_node, &self.slot_clock, None).await
&& !self.allow_unsynced_beacon_node
@ -550,7 +542,7 @@ impl<T: SlotClock + 'static, E: EthSpec> DutiesService<T, E> {
.await
.map_err(move |e| format!("Failed to get duties for epoch {}: {:?}", epoch, e))?;
let log = self.context.log.clone();
let log = self.context.log().clone();
let mut new_validator = 0;
let mut new_epoch = 0;
@ -652,7 +644,7 @@ impl<T: SlotClock + 'static, E: EthSpec> DutiesService<T, E> {
)
}
let log = self.context.log.clone();
let log = self.context.log().clone();
let count = validator_subscriptions.len();
if count == 0 {

View File

@ -1,9 +1,8 @@
use environment::RuntimeContext;
use exit_future::Signal;
use futures::{FutureExt, StreamExt};
use futures::StreamExt;
use parking_lot::RwLock;
use remote_beacon_node::RemoteBeaconNode;
use slog::{debug, info, trace};
use slog::{debug, trace};
use slot_clock::SlotClock;
use std::ops::Deref;
use std::sync::Arc;
@ -100,9 +99,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ForkService<T, E> {
}
/// Starts the service that periodically polls for the `Fork`.
pub fn start_update_service(self, spec: &ChainSpec) -> Result<Signal, String> {
let log = self.context.log.clone();
pub fn start_update_service(self, spec: &ChainSpec) -> Result<(), String> {
let duration_to_next_epoch = self
.slot_clock
.duration_to_next_epoch(E::slots_per_epoch())
@ -117,15 +114,14 @@ impl<T: SlotClock + 'static, E: EthSpec> ForkService<T, E> {
)
};
let (exit_signal, exit_fut) = exit_future::signal();
// Run an immediate update before starting the updater service.
self.inner
.context
.runtime_handle
.executor
.runtime_handle()
.spawn(self.clone().do_update());
let runtime_handle = self.inner.context.runtime_handle.clone();
let executor = self.inner.context.executor.clone();
let interval_fut = async move {
while interval.next().await.is_some() {
@ -133,18 +129,14 @@ impl<T: SlotClock + 'static, E: EthSpec> ForkService<T, E> {
}
};
let future = futures::future::select(
Box::pin(interval_fut),
exit_fut.map(move |_| info!(log, "Shutdown complete")),
);
runtime_handle.spawn(future);
executor.spawn(interval_fut, "fork_service");
Ok(exit_signal)
Ok(())
}
/// Attempts to download the `Fork` from the server.
async fn do_update(self) -> Result<(), ()> {
let log = &self.context.log;
let log = self.context.log();
let fork = self
.inner

View File

@ -17,7 +17,6 @@ use clap::ArgMatches;
use config::SLASHING_PROTECTION_FILENAME;
use duties_service::{DutiesService, DutiesServiceBuilder};
use environment::RuntimeContext;
use exit_future::Signal;
use fork_service::{ForkService, ForkServiceBuilder};
use notifier::spawn_notifier;
use remote_beacon_node::RemoteBeaconNode;
@ -41,7 +40,6 @@ pub struct ProductionValidatorClient<T: EthSpec> {
fork_service: ForkService<SystemTimeSlotClock, T>,
block_service: BlockService<SystemTimeSlotClock, T>,
attestation_service: AttestationService<SystemTimeSlotClock, T>,
exit_signals: Vec<Signal>,
config: Config,
}
@ -60,10 +58,10 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
/// Instantiates the validator client, _without_ starting the timers to trigger block
/// and attestation production.
pub async fn new(mut context: RuntimeContext<T>, config: Config) -> Result<Self, String> {
let log_1 = context.log.clone();
let log_2 = context.log.clone();
let log_3 = context.log.clone();
let log_4 = context.log.clone();
let log_1 = context.log().clone();
let log_2 = context.log().clone();
let log_3 = context.log().clone();
let log_4 = context.log().clone();
info!(
log_1,
@ -217,46 +215,32 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
fork_service,
block_service,
attestation_service,
exit_signals: vec![],
config,
})
}
pub fn start_service(&mut self) -> Result<(), String> {
let duties_exit = self
.duties_service
self.duties_service
.clone()
.start_update_service(&self.context.eth2_config.spec)
.map_err(|e| format!("Unable to start duties service: {}", e))?;
let fork_exit = self
.fork_service
self.fork_service
.clone()
.start_update_service(&self.context.eth2_config.spec)
.map_err(|e| format!("Unable to start fork service: {}", e))?;
let block_exit = self
.block_service
self.block_service
.clone()
.start_update_service(&self.context.eth2_config.spec)
.map_err(|e| format!("Unable to start block service: {}", e))?;
let attestation_exit = self
.attestation_service
self.attestation_service
.clone()
.start_update_service(&self.context.eth2_config.spec)
.map_err(|e| format!("Unable to start attestation service: {}", e))?;
let notifier_exit =
spawn_notifier(self).map_err(|e| format!("Failed to start notifier: {}", e))?;
self.exit_signals = vec![
duties_exit,
fork_exit,
block_exit,
attestation_exit,
notifier_exit,
];
spawn_notifier(self).map_err(|e| format!("Failed to start notifier: {}", e))?;
Ok(())
}

View File

@ -1,16 +1,14 @@
use crate::{is_synced::is_synced, ProductionValidatorClient};
use exit_future::Signal;
use futures::{FutureExt, StreamExt};
use futures::StreamExt;
use slog::{error, info};
use slot_clock::SlotClock;
use tokio::time::{interval_at, Duration, Instant};
use types::EthSpec;
/// Spawns a notifier service which periodically logs information about the node.
pub fn spawn_notifier<T: EthSpec>(client: &ProductionValidatorClient<T>) -> Result<Signal, String> {
pub fn spawn_notifier<T: EthSpec>(client: &ProductionValidatorClient<T>) -> Result<(), String> {
let context = client.context.service_context("notifier".into());
let runtime_handle = context.runtime_handle.clone();
let log = context.log.clone();
let executor = context.executor.clone();
let duties_service = client.duties_service.clone();
let allow_unsynced_beacon_node = client.config.allow_unsynced_beacon_node;
@ -25,7 +23,7 @@ pub fn spawn_notifier<T: EthSpec>(client: &ProductionValidatorClient<T>) -> Resu
let mut interval = interval_at(start_instant, slot_duration);
let interval_fut = async move {
let log = &context.log;
let log = context.log();
while interval.next().await.is_some() {
if !is_synced(
@ -83,12 +81,6 @@ pub fn spawn_notifier<T: EthSpec>(client: &ProductionValidatorClient<T>) -> Resu
}
};
let (exit_signal, exit) = exit_future::signal();
let future = futures::future::select(
Box::pin(interval_fut),
exit.map(move |_| info!(log, "Shutdown complete")),
);
runtime_handle.spawn(future);
Ok(exit_signal)
executor.spawn(interval_fut, "validator_notifier");
Ok(())
}