Merge RPC branch
This commit is contained in:
commit
214fb5b8ff
@ -29,4 +29,5 @@ members = [
|
||||
"beacon_node/beacon_chain/test_harness",
|
||||
"protos",
|
||||
"validator_client",
|
||||
"account_manager",
|
||||
]
|
||||
|
13
account_manager/Cargo.toml
Normal file
13
account_manager/Cargo.toml
Normal file
@ -0,0 +1,13 @@
|
||||
[package]
|
||||
name = "account_manager"
|
||||
version = "0.0.1"
|
||||
authors = ["Luke Anderson <luke@sigmaprime.io>"]
|
||||
edition = "2018"
|
||||
|
||||
[dependencies]
|
||||
bls = { path = "../eth2/utils/bls" }
|
||||
clap = "2.32.0"
|
||||
slog = "^2.2.3"
|
||||
slog-term = "^2.4.0"
|
||||
slog-async = "^2.3.0"
|
||||
validator_client = { path = "../validator_client" }
|
24
account_manager/README.md
Normal file
24
account_manager/README.md
Normal file
@ -0,0 +1,24 @@
|
||||
# Lighthouse Accounts Manager
|
||||
|
||||
The accounts manager (AM) is a stand-alone binary which allows
|
||||
users to generate and manage the cryptographic keys necessary to
|
||||
interact with Ethereum Serenity.
|
||||
|
||||
## Roles
|
||||
|
||||
The AM is responsible for the following tasks:
|
||||
- Generation of cryptographic key pairs
|
||||
- Must acquire sufficient entropy to ensure keys are generated securely (TBD)
|
||||
- Secure storage of private keys
|
||||
- Keys must be encrypted while at rest on the disk (TBD)
|
||||
- The format is compatible with the validator client
|
||||
- Produces messages and transactions necessary to initiate
|
||||
staking on Ethereum 1.x (TPD)
|
||||
|
||||
|
||||
## Implementation
|
||||
|
||||
The AM is not a service, and does not run continuously, nor does it
|
||||
interact with any running services.
|
||||
It is intended to be executed separately from other Lighthouse binaries
|
||||
and produce files which can be consumed by them.
|
58
account_manager/src/main.rs
Normal file
58
account_manager/src/main.rs
Normal file
@ -0,0 +1,58 @@
|
||||
use bls::Keypair;
|
||||
use clap::{App, Arg, SubCommand};
|
||||
use slog::{debug, info, o, Drain};
|
||||
use std::path::PathBuf;
|
||||
use validator_client::Config as ValidatorClientConfig;
|
||||
|
||||
fn main() {
|
||||
// Logging
|
||||
let decorator = slog_term::TermDecorator::new().build();
|
||||
let drain = slog_term::CompactFormat::new(decorator).build().fuse();
|
||||
let drain = slog_async::Async::new(drain).build().fuse();
|
||||
let log = slog::Logger::root(drain, o!());
|
||||
|
||||
// CLI
|
||||
let matches = App::new("Lighthouse Accounts Manager")
|
||||
.version("0.0.1")
|
||||
.author("Sigma Prime <contact@sigmaprime.io>")
|
||||
.about("Eth 2.0 Accounts Manager")
|
||||
.arg(
|
||||
Arg::with_name("datadir")
|
||||
.long("datadir")
|
||||
.value_name("DIR")
|
||||
.help("Data directory for keys and databases.")
|
||||
.takes_value(true),
|
||||
)
|
||||
.subcommand(
|
||||
SubCommand::with_name("generate")
|
||||
.about("Generates a new validator private key")
|
||||
.version("0.0.1")
|
||||
.author("Sigma Prime <contact@sigmaprime.io>"),
|
||||
)
|
||||
.get_matches();
|
||||
|
||||
let config = ValidatorClientConfig::parse_args(&matches, &log)
|
||||
.expect("Unable to build a configuration for the account manager.");
|
||||
|
||||
// Log configuration
|
||||
info!(log, "";
|
||||
"data_dir" => &config.data_dir.to_str());
|
||||
|
||||
match matches.subcommand() {
|
||||
("generate", Some(_gen_m)) => {
|
||||
let keypair = Keypair::random();
|
||||
let key_path: PathBuf = config
|
||||
.save_key(&keypair)
|
||||
.expect("Unable to save newly generated private key.");
|
||||
debug!(
|
||||
log,
|
||||
"Keypair generated {:?}, saved to: {:?}",
|
||||
keypair.identifier(),
|
||||
key_path.to_string_lossy()
|
||||
);
|
||||
}
|
||||
_ => panic!(
|
||||
"The account manager must be run with a subcommand. See help for more information."
|
||||
),
|
||||
}
|
||||
}
|
@ -456,8 +456,8 @@ where
|
||||
}
|
||||
|
||||
/// Produce an `AttestationData` that is valid for the present `slot` and given `shard`.
|
||||
pub fn produce_attestation_data(&self, shard: u64) -> Result<AttestationData, Error> {
|
||||
trace!("BeaconChain::produce_attestation_data: shard: {}", shard);
|
||||
pub fn produce_attestation(&self, shard: u64) -> Result<AttestationData, Error> {
|
||||
trace!("BeaconChain::produce_attestation: shard: {}", shard);
|
||||
let source_epoch = self.state.read().current_justified_epoch;
|
||||
let source_root = *self.state.read().get_block_root(
|
||||
source_epoch.start_slot(self.spec.slots_per_epoch),
|
||||
|
@ -50,18 +50,18 @@ impl<T: ClientDB, U: SlotClock, F: ForkChoice> DirectBeaconNode<T, U, F> {
|
||||
}
|
||||
|
||||
impl<T: ClientDB, U: SlotClock, F: ForkChoice> AttesterBeaconNode for DirectBeaconNode<T, U, F> {
|
||||
fn produce_attestation_data(
|
||||
fn produce_attestation(
|
||||
&self,
|
||||
_slot: Slot,
|
||||
shard: u64,
|
||||
) -> Result<Option<AttestationData>, NodeError> {
|
||||
match self.beacon_chain.produce_attestation_data(shard) {
|
||||
match self.beacon_chain.produce_attestation(shard) {
|
||||
Ok(attestation_data) => Ok(Some(attestation_data)),
|
||||
Err(e) => Err(NodeError::RemoteFailure(format!("{:?}", e))),
|
||||
}
|
||||
}
|
||||
|
||||
fn publish_attestation_data(
|
||||
fn publish_attestation(
|
||||
&self,
|
||||
free_attestation: FreeAttestation,
|
||||
) -> Result<AttestationPublishOutcome, NodeError> {
|
||||
|
@ -24,12 +24,8 @@ pub struct Client<T: ClientTypes> {
|
||||
beacon_chain: Arc<BeaconChain<T::DB, T::SlotClock, T::ForkChoice>>,
|
||||
/// Reference to the network service.
|
||||
pub network: Arc<NetworkService>,
|
||||
/// Future to stop and begin shutdown of the Client.
|
||||
//TODO: Decide best way to handle shutdown
|
||||
pub exit: exit_future::Exit,
|
||||
/// The sending future to call to terminate the Client.
|
||||
//TODO: Decide best way to handle shutdown
|
||||
pub exit_signal: Signal,
|
||||
/// Signal to terminate the RPC server.
|
||||
pub rpc_exit_signal: Option<Signal>,
|
||||
/// The clients logger.
|
||||
log: slog::Logger,
|
||||
/// Marker to pin the beacon chain generics.
|
||||
@ -43,8 +39,6 @@ impl<TClientType: ClientTypes> Client<TClientType> {
|
||||
log: slog::Logger,
|
||||
executor: &TaskExecutor,
|
||||
) -> error::Result<Self> {
|
||||
let (exit_signal, exit) = exit_future::signal();
|
||||
|
||||
// generate a beacon chain
|
||||
let beacon_chain = TClientType::initialise_beacon_chain(&config);
|
||||
|
||||
@ -59,16 +53,21 @@ impl<TClientType: ClientTypes> Client<TClientType> {
|
||||
network_logger,
|
||||
)?;
|
||||
|
||||
let mut rpc_exit_signal = None;
|
||||
// spawn the RPC server
|
||||
if config.rpc_conf.enabled {
|
||||
rpc::start_server(&config.rpc_conf, &log);
|
||||
rpc_exit_signal = Some(rpc::start_server(
|
||||
&config.rpc_conf,
|
||||
executor,
|
||||
beacon_chain.clone(),
|
||||
&log,
|
||||
));
|
||||
}
|
||||
|
||||
Ok(Client {
|
||||
config,
|
||||
beacon_chain,
|
||||
exit,
|
||||
exit_signal,
|
||||
rpc_exit_signal,
|
||||
log,
|
||||
network,
|
||||
phantom: PhantomData,
|
||||
|
@ -7,7 +7,10 @@ edition = "2018"
|
||||
[dependencies]
|
||||
bls = { path = "../../eth2/utils/bls" }
|
||||
beacon_chain = { path = "../beacon_chain" }
|
||||
|
||||
version = { path = "../version" }
|
||||
types = { path = "../../eth2/types" }
|
||||
ssz = { path = "../../eth2/utils/ssz" }
|
||||
slot_clock = { path = "../../eth2/utils/slot_clock" }
|
||||
protos = { path = "../../protos" }
|
||||
grpcio = { version = "0.4", default-features = false, features = ["protobuf-codec"] }
|
||||
protobuf = "2.0.2"
|
||||
@ -16,8 +19,7 @@ db = { path = "../db" }
|
||||
dirs = "1.0.3"
|
||||
futures = "0.1.23"
|
||||
slog = "^2.2.3"
|
||||
slot_clock = { path = "../../eth2/utils/slot_clock" }
|
||||
slog-term = "^2.4.0"
|
||||
slog-async = "^2.3.0"
|
||||
types = { path = "../../eth2/types" }
|
||||
ssz = { path = "../../eth2/utils/ssz" }
|
||||
tokio = "0.1.17"
|
||||
exit-future = "0.1.4"
|
||||
|
61
beacon_node/rpc/src/beacon_attester.rs
Normal file
61
beacon_node/rpc/src/beacon_attester.rs
Normal file
@ -0,0 +1,61 @@
|
||||
use futures::Future;
|
||||
use grpcio::{RpcContext, UnarySink};
|
||||
use protos::services::{
|
||||
Attestation as AttestationProto, ProduceAttestation, ProduceAttestationResponse,
|
||||
ProduceAttestationRequest, PublishAttestationResponse, PublishAttestationRequest,
|
||||
PublishAttestation
|
||||
};
|
||||
use protos::services_grpc::BeaconBlockService;
|
||||
use slog::Logger;
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct AttestationServiceInstance {
|
||||
pub log: Logger,
|
||||
}
|
||||
|
||||
impl AttestationService for AttestationServiceInstance {
|
||||
/// Produce a `BeaconBlock` for signing by a validator.
|
||||
fn produce_attestation(
|
||||
&mut self,
|
||||
ctx: RpcContext,
|
||||
req: ProduceAttestationRequest,
|
||||
sink: UnarySink<ProduceAttestationResponse>,
|
||||
) {
|
||||
println!("producing attestation at slot {}", req.get_slot());
|
||||
|
||||
// TODO: build a legit block.
|
||||
let mut attestation = AttestationProto::new();
|
||||
attestation.set_slot(req.get_slot());
|
||||
// TODO Set the shard to something legit.
|
||||
attestation.set_shard(0);
|
||||
attestation.set_block_root(b"cats".to_vec());
|
||||
|
||||
let mut resp = ProduceAttestationResponse::new();
|
||||
resp.set_attestation_data(attestation);
|
||||
|
||||
let f = sink
|
||||
.success(resp)
|
||||
.map_err(move |e| println!("failed to reply {:?}: {:?}", req, e));
|
||||
ctx.spawn(f)
|
||||
}
|
||||
|
||||
/// Accept some fully-formed `BeaconBlock`, process and publish it.
|
||||
fn publish_attestation(
|
||||
&mut self,
|
||||
ctx: RpcContext,
|
||||
req: PublishAttestationRequest,
|
||||
sink: UnarySink<PublishAttestationResponse>,
|
||||
) {
|
||||
println!("publishing attestation {:?}", req.get_block());
|
||||
|
||||
// TODO: actually process the block.
|
||||
let mut resp = PublishAttestationResponse::new();
|
||||
|
||||
resp.set_success(true);
|
||||
|
||||
let f = sink
|
||||
.success(resp)
|
||||
.map_err(move |e| println!("failed to reply {:?}: {:?}", req, e));
|
||||
ctx.spawn(f)
|
||||
}
|
||||
}
|
31
beacon_node/rpc/src/beacon_chain.rs
Normal file
31
beacon_node/rpc/src/beacon_chain.rs
Normal file
@ -0,0 +1,31 @@
|
||||
use beacon_chain::BeaconChain as RawBeaconChain;
|
||||
use beacon_chain::{
|
||||
db::ClientDB,
|
||||
fork_choice::ForkChoice,
|
||||
parking_lot::RwLockReadGuard,
|
||||
slot_clock::SlotClock,
|
||||
types::{BeaconState, ChainSpec},
|
||||
CheckPoint,
|
||||
};
|
||||
|
||||
/// The RPC's API to the beacon chain.
|
||||
pub trait BeaconChain: Send + Sync {
|
||||
fn get_spec(&self) -> &ChainSpec;
|
||||
|
||||
fn get_state(&self) -> RwLockReadGuard<BeaconState>;
|
||||
}
|
||||
|
||||
impl<T, U, F> BeaconChain for RawBeaconChain<T, U, F>
|
||||
where
|
||||
T: ClientDB + Sized,
|
||||
U: SlotClock,
|
||||
F: ForkChoice,
|
||||
{
|
||||
fn get_spec(&self) -> &ChainSpec {
|
||||
&self.spec
|
||||
}
|
||||
|
||||
fn get_state(&self) -> RwLockReadGuard<BeaconState> {
|
||||
self.state.read()
|
||||
}
|
||||
}
|
46
beacon_node/rpc/src/beacon_node.rs
Normal file
46
beacon_node/rpc/src/beacon_node.rs
Normal file
@ -0,0 +1,46 @@
|
||||
use crate::beacon_chain::BeaconChain;
|
||||
use futures::Future;
|
||||
use grpcio::{RpcContext, UnarySink};
|
||||
use protos::services::{Empty, Fork, NodeInfo};
|
||||
use protos::services_grpc::BeaconNodeService;
|
||||
use slog::{trace, warn};
|
||||
use std::sync::Arc;
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct BeaconNodeServiceInstance {
|
||||
pub chain: Arc<BeaconChain>,
|
||||
pub log: slog::Logger,
|
||||
}
|
||||
|
||||
impl BeaconNodeService for BeaconNodeServiceInstance {
|
||||
/// Provides basic node information.
|
||||
fn info(&mut self, ctx: RpcContext, _req: Empty, sink: UnarySink<NodeInfo>) {
|
||||
trace!(self.log, "Node info requested via RPC");
|
||||
|
||||
// build the response
|
||||
let mut node_info = NodeInfo::new();
|
||||
node_info.set_version(version::version());
|
||||
|
||||
// get the chain state
|
||||
let state = self.chain.get_state();
|
||||
let state_fork = state.fork.clone();
|
||||
let genesis_time = state.genesis_time.clone();
|
||||
|
||||
// build the rpc fork struct
|
||||
let mut fork = Fork::new();
|
||||
fork.set_previous_version(state_fork.previous_version.to_vec());
|
||||
fork.set_current_version(state_fork.current_version.to_vec());
|
||||
fork.set_epoch(state_fork.epoch.into());
|
||||
|
||||
node_info.set_fork(fork);
|
||||
node_info.set_genesis_time(genesis_time);
|
||||
node_info.set_chain_id(self.chain.get_spec().chain_id as u32);
|
||||
|
||||
// send the node_info the requester
|
||||
let error_log = self.log.clone();
|
||||
let f = sink
|
||||
.success(node_info)
|
||||
.map_err(move |e| warn!(error_log, "failed to reply {:?}", e));
|
||||
ctx.spawn(f)
|
||||
}
|
||||
}
|
@ -1,20 +1,44 @@
|
||||
mod beacon_block;
|
||||
pub mod beacon_chain;
|
||||
mod beacon_node;
|
||||
pub mod config;
|
||||
mod validator;
|
||||
|
||||
use self::beacon_block::BeaconBlockServiceInstance;
|
||||
use self::beacon_chain::BeaconChain;
|
||||
use self::beacon_node::BeaconNodeServiceInstance;
|
||||
use self::validator::ValidatorServiceInstance;
|
||||
pub use config::Config as RPCConfig;
|
||||
use futures::{future, Future};
|
||||
use grpcio::{Environment, Server, ServerBuilder};
|
||||
use protos::services_grpc::{create_beacon_block_service, create_validator_service};
|
||||
use protos::services_grpc::{
|
||||
create_beacon_block_service, create_beacon_node_service, create_validator_service,
|
||||
};
|
||||
use slog::{info, o, warn};
|
||||
use std::sync::Arc;
|
||||
use tokio::runtime::TaskExecutor;
|
||||
|
||||
use slog::{info, o};
|
||||
|
||||
pub fn start_server(config: &RPCConfig, log: &slog::Logger) -> Server {
|
||||
pub fn start_server(
|
||||
config: &RPCConfig,
|
||||
executor: &TaskExecutor,
|
||||
beacon_chain: Arc<BeaconChain>,
|
||||
log: &slog::Logger,
|
||||
) -> exit_future::Signal {
|
||||
let log = log.new(o!("Service"=>"RPC"));
|
||||
let env = Arc::new(Environment::new(1));
|
||||
|
||||
// build a channel to kill the rpc server
|
||||
let (rpc_exit_signal, rpc_exit) = exit_future::signal();
|
||||
|
||||
// build the individual rpc services
|
||||
let beacon_node_service = {
|
||||
let instance = BeaconNodeServiceInstance {
|
||||
chain: beacon_chain.clone(),
|
||||
log: log.clone(),
|
||||
};
|
||||
create_beacon_node_service(instance)
|
||||
};
|
||||
|
||||
let beacon_block_service = {
|
||||
let instance = BeaconBlockServiceInstance { log: log.clone() };
|
||||
create_beacon_block_service(instance)
|
||||
@ -27,12 +51,26 @@ pub fn start_server(config: &RPCConfig, log: &slog::Logger) -> Server {
|
||||
let mut server = ServerBuilder::new(env)
|
||||
.register_service(beacon_block_service)
|
||||
.register_service(validator_service)
|
||||
.register_service(beacon_node_service)
|
||||
.bind(config.listen_address.to_string(), config.port)
|
||||
.build()
|
||||
.unwrap();
|
||||
|
||||
let spawn_rpc = {
|
||||
server.start();
|
||||
for &(ref host, port) in server.bind_addrs() {
|
||||
info!(log, "gRPC listening on {}:{}", host, port);
|
||||
}
|
||||
rpc_exit.and_then(move |_| {
|
||||
info!(log, "RPC Server shutting down");
|
||||
server
|
||||
.shutdown()
|
||||
.wait()
|
||||
.map(|_| ())
|
||||
.map_err(|e| warn!(log, "RPC server failed to shutdown: {:?}", e))?;
|
||||
Ok(())
|
||||
})
|
||||
};
|
||||
executor.spawn(spawn_rpc);
|
||||
rpc_exit_signal
|
||||
}
|
||||
|
@ -94,7 +94,7 @@ impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> Attester<T, U, V,
|
||||
}
|
||||
|
||||
fn produce_attestation(&mut self, slot: Slot, shard: u64) -> Result<PollOutcome, Error> {
|
||||
let attestation_data = match self.beacon_node.produce_attestation_data(slot, shard)? {
|
||||
let attestation_data = match self.beacon_node.produce_attestation(slot, shard)? {
|
||||
Some(attestation_data) => attestation_data,
|
||||
None => return Ok(PollOutcome::BeaconNodeUnableToProduceAttestation(slot)),
|
||||
};
|
||||
@ -120,7 +120,7 @@ impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> Attester<T, U, V,
|
||||
};
|
||||
|
||||
self.beacon_node
|
||||
.publish_attestation_data(free_attestation)?;
|
||||
.publish_attestation(free_attestation)?;
|
||||
Ok(PollOutcome::AttestationProduced(slot))
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ impl SimulatedBeaconNode {
|
||||
}
|
||||
|
||||
impl BeaconNode for SimulatedBeaconNode {
|
||||
fn produce_attestation_data(&self, slot: Slot, shard: u64) -> ProduceResult {
|
||||
fn produce_attestation(&self, slot: Slot, shard: u64) -> ProduceResult {
|
||||
*self.produce_input.write().unwrap() = Some((slot, shard));
|
||||
match *self.produce_result.read().unwrap() {
|
||||
Some(ref r) => r.clone(),
|
||||
@ -34,7 +34,7 @@ impl BeaconNode for SimulatedBeaconNode {
|
||||
}
|
||||
}
|
||||
|
||||
fn publish_attestation_data(&self, free_attestation: FreeAttestation) -> PublishResult {
|
||||
fn publish_attestation(&self, free_attestation: FreeAttestation) -> PublishResult {
|
||||
*self.publish_input.write().unwrap() = Some(free_attestation.clone());
|
||||
match *self.publish_result.read().unwrap() {
|
||||
Some(ref r) => r.clone(),
|
||||
|
@ -14,13 +14,13 @@ pub enum PublishOutcome {
|
||||
|
||||
/// Defines the methods required to produce and publish blocks on a Beacon Node.
|
||||
pub trait BeaconNode: Send + Sync {
|
||||
fn produce_attestation_data(
|
||||
fn produce_attestation(
|
||||
&self,
|
||||
slot: Slot,
|
||||
shard: u64,
|
||||
) -> Result<Option<AttestationData>, BeaconNodeError>;
|
||||
|
||||
fn publish_attestation_data(
|
||||
fn publish_attestation(
|
||||
&self,
|
||||
free_attestation: FreeAttestation,
|
||||
) -> Result<PublishOutcome, BeaconNodeError>;
|
||||
|
@ -120,7 +120,7 @@ pub struct ChainSpec {
|
||||
*
|
||||
*/
|
||||
pub boot_nodes: Vec<Multiaddr>,
|
||||
pub network_id: u8,
|
||||
pub chain_id: u8,
|
||||
}
|
||||
|
||||
impl ChainSpec {
|
||||
@ -257,7 +257,7 @@ impl ChainSpec {
|
||||
* Boot nodes
|
||||
*/
|
||||
boot_nodes: vec![],
|
||||
network_id: 1, // foundation network id
|
||||
chain_id: 1, // foundation chain id
|
||||
}
|
||||
}
|
||||
|
||||
@ -274,7 +274,7 @@ impl ChainSpec {
|
||||
|
||||
Self {
|
||||
boot_nodes,
|
||||
network_id: 2, // lighthouse testnet network id
|
||||
chain_id: 2, // lighthouse testnet chain id
|
||||
..ChainSpec::few_validators()
|
||||
}
|
||||
}
|
||||
|
@ -14,4 +14,8 @@ impl Keypair {
|
||||
let pk = PublicKey::from_secret_key(&sk);
|
||||
Keypair { sk, pk }
|
||||
}
|
||||
|
||||
pub fn identifier(&self) -> String {
|
||||
self.pk.concatenated_hex_id()
|
||||
}
|
||||
}
|
||||
|
@ -12,24 +12,55 @@ syntax = "proto3";
|
||||
|
||||
package ethereum.beacon.rpc.v1;
|
||||
|
||||
// Service that currently identifies a beacon node
|
||||
service BeaconNodeService {
|
||||
rpc Info(Empty) returns (NodeInfo);
|
||||
}
|
||||
|
||||
/// Service that handles block production
|
||||
service BeaconBlockService {
|
||||
rpc ProduceBeaconBlock(ProduceBeaconBlockRequest) returns (ProduceBeaconBlockResponse);
|
||||
rpc PublishBeaconBlock(PublishBeaconBlockRequest) returns (PublishBeaconBlockResponse);
|
||||
}
|
||||
|
||||
/// Service that provides the validator client with requisite knowledge about
|
||||
//its public keys
|
||||
service ValidatorService {
|
||||
// rpc ValidatorAssignment(ValidatorAssignmentRequest) returns (ValidatorAssignmentResponse);
|
||||
rpc ProposeBlockSlot(ProposeBlockSlotRequest) returns (ProposeBlockSlotResponse);
|
||||
rpc ValidatorIndex(PublicKey) returns (IndexResponse);
|
||||
// rpc ValidatorAssignment(ValidatorAssignmentRequest) returns (ValidatorAssignmentResponse);
|
||||
}
|
||||
|
||||
message BeaconBlock {
|
||||
uint64 slot = 1;
|
||||
bytes block_root = 2;
|
||||
bytes randao_reveal = 3;
|
||||
bytes signature = 4;
|
||||
/// Service that handles validator attestations
|
||||
service AttestationService {
|
||||
rpc ProduceAttestation(ProduceAttestationRequest) returns (ProduceAttestationResponse);
|
||||
rpc PublishAttestation(PublishAttestationRequest) returns (PublishAttestationResponse);
|
||||
}
|
||||
|
||||
/*
|
||||
* Beacon Node Service Message
|
||||
*/
|
||||
message NodeInfo {
|
||||
string version = 1;
|
||||
Fork fork = 2;
|
||||
uint32 chain_id = 3;
|
||||
uint64 genesis_time = 4;
|
||||
}
|
||||
|
||||
message Fork {
|
||||
bytes previous_version = 1;
|
||||
bytes current_version = 2;
|
||||
uint64 epoch = 3;
|
||||
}
|
||||
|
||||
message Empty {
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Block Production Service Messages
|
||||
*/
|
||||
|
||||
// Validator requests an unsigned proposal.
|
||||
message ProduceBeaconBlockRequest {
|
||||
uint64 slot = 1;
|
||||
@ -51,6 +82,22 @@ message PublishBeaconBlockResponse {
|
||||
bytes msg = 2;
|
||||
}
|
||||
|
||||
message BeaconBlock {
|
||||
uint64 slot = 1;
|
||||
bytes block_root = 2;
|
||||
bytes randao_reveal = 3;
|
||||
bytes signature = 4;
|
||||
}
|
||||
|
||||
/*
|
||||
* Validator Service Messages
|
||||
*/
|
||||
/*
|
||||
message ValidatorAssignmentRequest {
|
||||
uint64 epoch = 1;
|
||||
bytes validator_index = 2;
|
||||
}
|
||||
|
||||
// A validators duties for some epoch.
|
||||
// TODO: add shard duties.
|
||||
message ValidatorAssignment {
|
||||
@ -59,15 +106,20 @@ message ValidatorAssignment {
|
||||
uint64 block_production_slot = 2;
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
message ValidatorAssignmentRequest {
|
||||
uint64 epoch = 1;
|
||||
bytes validator_index = 2;
|
||||
// Validator Assignment
|
||||
|
||||
message PublicKey {
|
||||
bytes public_key = 1;
|
||||
}
|
||||
|
||||
/*
|
||||
* Propose slot
|
||||
*/
|
||||
message IndexResponse {
|
||||
uint64 index = 1;
|
||||
}
|
||||
|
||||
|
||||
// Propose slot
|
||||
|
||||
message ProposeBlockSlotRequest {
|
||||
uint64 epoch = 1;
|
||||
@ -81,14 +133,49 @@ message ProposeBlockSlotResponse {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Validator Assignment
|
||||
* Attestation Service Messages
|
||||
*/
|
||||
|
||||
message PublicKey {
|
||||
bytes public_key = 1;
|
||||
message ProduceAttestationRequest {
|
||||
uint64 slot = 1;
|
||||
uint64 shard = 2;
|
||||
}
|
||||
|
||||
message IndexResponse {
|
||||
uint64 index = 1;
|
||||
message ProduceAttestationResponse {
|
||||
Attestation attestation_data = 1;
|
||||
}
|
||||
|
||||
message PublishAttestationRequest {
|
||||
FreeAttestation free_attestation = 1;
|
||||
}
|
||||
|
||||
message PublishAttestationResponse {
|
||||
bool success = 1;
|
||||
bytes msg = 2;
|
||||
}
|
||||
|
||||
message Crosslink {
|
||||
uint64 epoch = 1;
|
||||
bytes crosslink_data_root = 2;
|
||||
|
||||
}
|
||||
|
||||
message Attestation {
|
||||
uint64 slot = 1;
|
||||
uint64 shard = 2;
|
||||
bytes beacon_block_root = 3;
|
||||
bytes epoch_boundary_root = 4;
|
||||
bytes crosslink_data_root = 5;
|
||||
Crosslink latest_crosslink = 6;
|
||||
uint64 justified_epoch = 7;
|
||||
bytes justified_block_root = 8;
|
||||
|
||||
}
|
||||
|
||||
message FreeAttestation {
|
||||
Attestation attestation_data = 1;
|
||||
bytes signature = 2;
|
||||
uint64 validator_index = 3;
|
||||
}
|
||||
|
@ -1,12 +1,23 @@
|
||||
[package]
|
||||
name = "validator_client"
|
||||
version = "0.1.0"
|
||||
authors = ["Paul Hauner <paul@paulhauner.com>"]
|
||||
authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com>", "Luke Anderson <luke@lukeanderson.com.au>"]
|
||||
edition = "2018"
|
||||
|
||||
[[bin]]
|
||||
name = "validator_client"
|
||||
path = "src/main.rs"
|
||||
|
||||
[lib]
|
||||
name = "validator_client"
|
||||
path = "src/lib.rs"
|
||||
|
||||
|
||||
[dependencies]
|
||||
block_proposer = { path = "../eth2/block_proposer" }
|
||||
attester = { path = "../eth2/attester" }
|
||||
bls = { path = "../eth2/utils/bls" }
|
||||
ssz = { path = "../eth2/utils/ssz" }
|
||||
clap = "2.32.0"
|
||||
dirs = "1.0.3"
|
||||
grpcio = { version = "0.4", default-features = false, features = ["protobuf-codec"] }
|
||||
@ -17,4 +28,7 @@ types = { path = "../eth2/types" }
|
||||
slog = "^2.2.3"
|
||||
slog-term = "^2.4.0"
|
||||
slog-async = "^2.3.0"
|
||||
ssz = { path = "../eth2/utils/ssz" }
|
||||
tokio = "0.1.18"
|
||||
tokio-timer = "0.2.10"
|
||||
error-chain = "0.12.0"
|
||||
bincode = "^1.1.2"
|
||||
|
@ -57,10 +57,30 @@ complete and return a block from the BN.
|
||||
|
||||
### Configuration
|
||||
|
||||
Presently the validator specifics (pubkey, etc.) are randomly generated and the
|
||||
chain specification (slot length, BLS domain, etc.) are fixed to foundation
|
||||
parameters. This is temporary and will be upgrade so these parameters can be
|
||||
read from file (or initialized on first-boot).
|
||||
Validator configurations are stored in a separate data directory from the main Beacon Node
|
||||
binary. The validator data directory defaults to:
|
||||
`$HOME/.lighthouse-validator`, however an alternative can be specified on the command line
|
||||
with `--datadir`.
|
||||
|
||||
The configuration directory structure looks like:
|
||||
```
|
||||
~/.lighthouse-validator
|
||||
├── 3cf4210d58ec
|
||||
│ └── private.key
|
||||
├── 9b5d8b5be4e7
|
||||
│ └── private.key
|
||||
└── cf6e07188f48
|
||||
└── private.key
|
||||
```
|
||||
|
||||
Where the hex value of the directory is a portion of the validator public key.
|
||||
|
||||
Validator keys must be generated using the separate `accounts_manager` binary, which will
|
||||
place the keys into this directory structure in a format compatible with the validator client.
|
||||
|
||||
The chain specification (slot length, BLS domain, etc.) defaults to foundation
|
||||
parameters, however is temporary and an upgrade will allow these parameters to be
|
||||
read from a file (or initialized on first-boot).
|
||||
|
||||
## BN Communication
|
||||
|
||||
|
@ -0,0 +1,44 @@
|
||||
use protos::services_grpc::AttestationServiceClient;
|
||||
use std::sync::Arc;
|
||||
|
||||
use attester::{BeaconNode, BeaconNodeError, PublishOutcome};
|
||||
use protos::services::ProduceAttestationRequest;
|
||||
use types::{AttestationData, FreeAttestation, Slot};
|
||||
|
||||
pub struct AttestationGrpcClient {
|
||||
client: Arc<AttestationServiceClient>,
|
||||
}
|
||||
|
||||
impl AttestationGrpcClient {
|
||||
pub fn new(client: Arc<AttestationServiceClient>) -> Self {
|
||||
Self { client }
|
||||
}
|
||||
}
|
||||
|
||||
impl BeaconNode for AttestationGrpcClient {
|
||||
fn produce_attestation(
|
||||
&self,
|
||||
slot: Slot,
|
||||
shard: u64,
|
||||
) -> Result<Option<AttestationData>, BeaconNodeError> {
|
||||
let mut req = ProduceAttestationRequest::new();
|
||||
req.set_slot(slot.as_u64());
|
||||
req.set_shard(shard);
|
||||
|
||||
let reply = self
|
||||
.client
|
||||
.produce_attestation(&req)
|
||||
.map_err(|err| BeaconNodeError::RemoteFailure(format!("{:?}", err)))?;
|
||||
|
||||
// TODO: return correct Attestation
|
||||
Err(BeaconNodeError::DecodeFailure)
|
||||
}
|
||||
|
||||
fn publish_attestation(
|
||||
&self,
|
||||
free_attestation: FreeAttestation,
|
||||
) -> Result<PublishOutcome, BeaconNodeError> {
|
||||
// TODO: return correct PublishOutcome
|
||||
Err(BeaconNodeError::DecodeFailure)
|
||||
}
|
||||
}
|
54
validator_client/src/attester_service/mod.rs
Normal file
54
validator_client/src/attester_service/mod.rs
Normal file
@ -0,0 +1,54 @@
|
||||
mod attestation_grpc_client;
|
||||
use attester::{Attester, BeaconNode, DutiesReader, PollOutcome as AttesterPollOutcome, Signer};
|
||||
use slog::{error, info, warn, Logger};
|
||||
use slot_clock::SlotClock;
|
||||
use std::time::Duration;
|
||||
|
||||
pub use self::attestation_grpc_client::AttestationGrpcClient;
|
||||
|
||||
pub struct AttesterService<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> {
|
||||
pub attester: Attester<T, U, V, W>,
|
||||
pub poll_interval_millis: u64,
|
||||
pub log: Logger,
|
||||
}
|
||||
|
||||
impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> AttesterService<T, U, V, W> {
|
||||
/// Run a loop which polls the Attester each `poll_interval_millis` millseconds.
|
||||
///
|
||||
/// Logs the results of the polls.
|
||||
pub fn run(&mut self) {
|
||||
loop {
|
||||
match self.attester.poll() {
|
||||
Err(error) => {
|
||||
error!(self.log, "Attester poll error"; "error" => format!("{:?}", error))
|
||||
}
|
||||
Ok(AttesterPollOutcome::AttestationProduced(slot)) => {
|
||||
info!(self.log, "Produced Attestation"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::SlashableAttestationNotProduced(slot)) => {
|
||||
warn!(self.log, "Slashable attestation was not produced"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::AttestationNotRequired(slot)) => {
|
||||
info!(self.log, "Attestation not required"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::ProducerDutiesUnknown(slot)) => {
|
||||
error!(self.log, "Attestation duties unknown"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::SlotAlreadyProcessed(slot)) => {
|
||||
warn!(self.log, "Attempted to re-process slot"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::BeaconNodeUnableToProduceAttestation(slot)) => {
|
||||
error!(self.log, "Beacon node unable to produce attestation"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::SignerRejection(slot)) => {
|
||||
error!(self.log, "The cryptographic signer refused to sign the attestation"; "slot" => slot)
|
||||
}
|
||||
Ok(AttesterPollOutcome::ValidatorIsUnknown(slot)) => {
|
||||
error!(self.log, "The Beacon Node does not recognise the validator"; "slot" => slot)
|
||||
}
|
||||
};
|
||||
|
||||
std::thread::sleep(Duration::from_millis(self.poll_interval_millis));
|
||||
}
|
||||
}
|
||||
}
|
@ -1,28 +1,38 @@
|
||||
use clap::ArgMatches;
|
||||
use bincode;
|
||||
use bls::Keypair;
|
||||
use slog::{debug, error, info};
|
||||
use std::fs;
|
||||
use std::fs::File;
|
||||
use std::io::{Error, ErrorKind};
|
||||
use std::path::PathBuf;
|
||||
use types::ChainSpec;
|
||||
|
||||
/// Stores the core configuration for this validator instance.
|
||||
#[derive(Clone)]
|
||||
pub struct ClientConfig {
|
||||
pub struct Config {
|
||||
/// The data directory, which stores all validator databases
|
||||
pub data_dir: PathBuf,
|
||||
/// The server at which the Beacon Node can be contacted
|
||||
pub server: String,
|
||||
/// The chain specification that we are connecting to
|
||||
pub spec: ChainSpec,
|
||||
}
|
||||
|
||||
const DEFAULT_LIGHTHOUSE_DIR: &str = ".lighthouse-validators";
|
||||
const DEFAULT_PRIVATE_KEY_FILENAME: &str = "private.key";
|
||||
|
||||
impl ClientConfig {
|
||||
impl Default for Config {
|
||||
/// Build a new configuration from defaults.
|
||||
pub fn default() -> Self {
|
||||
fn default() -> Self {
|
||||
let data_dir = {
|
||||
let home = dirs::home_dir().expect("Unable to determine home dir.");
|
||||
home.join(DEFAULT_LIGHTHOUSE_DIR)
|
||||
let home = dirs::home_dir().expect("Unable to determine home directory.");
|
||||
home.join(".lighthouse-validator")
|
||||
};
|
||||
fs::create_dir_all(&data_dir)
|
||||
.unwrap_or_else(|_| panic!("Unable to create {:?}", &data_dir));
|
||||
let server = "localhost:50051".to_string();
|
||||
|
||||
let server = "localhost:5051".to_string();
|
||||
|
||||
let spec = ChainSpec::foundation();
|
||||
|
||||
Self {
|
||||
data_dir,
|
||||
server,
|
||||
@ -30,3 +40,119 @@ impl ClientConfig {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Config {
|
||||
/// Build a new configuration from defaults, which are overrided by arguments provided.
|
||||
pub fn parse_args(args: &ArgMatches, log: &slog::Logger) -> Result<Self, Error> {
|
||||
let mut config = Config::default();
|
||||
|
||||
// Use the specified datadir, or default in the home directory
|
||||
if let Some(datadir) = args.value_of("datadir") {
|
||||
config.data_dir = PathBuf::from(datadir);
|
||||
info!(log, "Using custom data dir: {:?}", &config.data_dir);
|
||||
};
|
||||
|
||||
fs::create_dir_all(&config.data_dir)
|
||||
.unwrap_or_else(|_| panic!("Unable to create {:?}", &config.data_dir));
|
||||
|
||||
if let Some(srv) = args.value_of("server") {
|
||||
//TODO: Validate the server value, to ensure it makes sense.
|
||||
config.server = srv.to_string();
|
||||
info!(log, "Using custom server: {:?}", &config.server);
|
||||
};
|
||||
|
||||
// TODO: Permit loading a custom spec from file.
|
||||
if let Some(spec_str) = args.value_of("spec") {
|
||||
info!(log, "Using custom spec: {:?}", spec_str);
|
||||
config.spec = match spec_str {
|
||||
"foundation" => ChainSpec::foundation(),
|
||||
"few_validators" => ChainSpec::few_validators(),
|
||||
// Should be impossible due to clap's `possible_values(..)` function.
|
||||
_ => unreachable!(),
|
||||
};
|
||||
};
|
||||
// Log configuration
|
||||
info!(log, "";
|
||||
"data_dir" => &config.data_dir.to_str(),
|
||||
"server" => &config.server);
|
||||
|
||||
Ok(config)
|
||||
}
|
||||
|
||||
/// Try to load keys from validator_dir, returning None if none are found or an error.
|
||||
pub fn fetch_keys(&self, log: &slog::Logger) -> Option<Vec<Keypair>> {
|
||||
let key_pairs: Vec<Keypair> = fs::read_dir(&self.data_dir)
|
||||
.unwrap()
|
||||
.filter_map(|validator_dir| {
|
||||
let validator_dir = validator_dir.ok()?;
|
||||
|
||||
if !(validator_dir.file_type().ok()?.is_dir()) {
|
||||
// Skip non-directories (i.e. no files/symlinks)
|
||||
return None;
|
||||
}
|
||||
|
||||
let key_filename = validator_dir.path().join(DEFAULT_PRIVATE_KEY_FILENAME);
|
||||
|
||||
if !(key_filename.is_file()) {
|
||||
info!(
|
||||
log,
|
||||
"Private key is not a file: {:?}",
|
||||
key_filename.to_str()
|
||||
);
|
||||
return None;
|
||||
}
|
||||
|
||||
debug!(
|
||||
log,
|
||||
"Deserializing private key from file: {:?}",
|
||||
key_filename.to_str()
|
||||
);
|
||||
|
||||
let mut key_file = File::open(key_filename.clone()).ok()?;
|
||||
|
||||
let key: Keypair = if let Ok(key_ok) = bincode::deserialize_from(&mut key_file) {
|
||||
key_ok
|
||||
} else {
|
||||
error!(
|
||||
log,
|
||||
"Unable to deserialize the private key file: {:?}", key_filename
|
||||
);
|
||||
return None;
|
||||
};
|
||||
|
||||
let ki = key.identifier();
|
||||
if ki != validator_dir.file_name().into_string().ok()? {
|
||||
error!(
|
||||
log,
|
||||
"The validator key ({:?}) did not match the directory filename {:?}.",
|
||||
ki,
|
||||
&validator_dir.path().to_string_lossy()
|
||||
);
|
||||
return None;
|
||||
}
|
||||
Some(key)
|
||||
})
|
||||
.collect();
|
||||
|
||||
// Check if it's an empty vector, and return none.
|
||||
if key_pairs.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(key_pairs)
|
||||
}
|
||||
}
|
||||
|
||||
/// Saves a keypair to a file inside the appropriate validator directory. Returns the saved path filename.
|
||||
pub fn save_key(&self, key: &Keypair) -> Result<PathBuf, Error> {
|
||||
let validator_config_path = self.data_dir.join(key.identifier());
|
||||
let key_path = validator_config_path.join(DEFAULT_PRIVATE_KEY_FILENAME);
|
||||
|
||||
fs::create_dir_all(&validator_config_path)?;
|
||||
|
||||
let mut key_file = File::create(&key_path)?;
|
||||
|
||||
bincode::serialize_into(&mut key_file, &key)
|
||||
.map_err(|e| Error::new(ErrorKind::InvalidData, e))?;
|
||||
Ok(key_path)
|
||||
}
|
||||
}
|
||||
|
@ -1,21 +1,19 @@
|
||||
mod epoch_duties;
|
||||
mod grpc;
|
||||
mod service;
|
||||
#[cfg(test)]
|
||||
mod test_node;
|
||||
mod traits;
|
||||
|
||||
pub use self::epoch_duties::EpochDutiesMap;
|
||||
use self::epoch_duties::{EpochDuties, EpochDutiesMapError};
|
||||
pub use self::service::DutiesManagerService;
|
||||
use self::traits::{BeaconNode, BeaconNodeError};
|
||||
use bls::PublicKey;
|
||||
use slot_clock::SlotClock;
|
||||
use std::sync::Arc;
|
||||
use types::{ChainSpec, Epoch};
|
||||
use types::{ChainSpec, Epoch, Slot};
|
||||
|
||||
#[derive(Debug, PartialEq, Clone, Copy)]
|
||||
pub enum PollOutcome {
|
||||
pub enum UpdateOutcome {
|
||||
/// The `EpochDuties` were not updated during this poll.
|
||||
NoChange(Epoch),
|
||||
/// The `EpochDuties` for the `epoch` were previously unknown, but obtained in the poll.
|
||||
@ -39,45 +37,42 @@ pub enum Error {
|
||||
/// A polling state machine which ensures the latest `EpochDuties` are obtained from the Beacon
|
||||
/// Node.
|
||||
///
|
||||
/// There is a single `DutiesManager` per validator instance.
|
||||
/// This keeps track of all validator keys and required voting slots.
|
||||
pub struct DutiesManager<T: SlotClock, U: BeaconNode> {
|
||||
pub duties_map: Arc<EpochDutiesMap>,
|
||||
/// The validator's public key.
|
||||
pub pubkey: PublicKey,
|
||||
/// A list of all public keys known to the validator service.
|
||||
pub pubkeys: Vec<PublicKey>,
|
||||
pub spec: Arc<ChainSpec>,
|
||||
pub slot_clock: Arc<T>,
|
||||
pub beacon_node: Arc<U>,
|
||||
}
|
||||
|
||||
impl<T: SlotClock, U: BeaconNode> DutiesManager<T, U> {
|
||||
/// Poll the Beacon Node for `EpochDuties`.
|
||||
/// Check the Beacon Node for `EpochDuties`.
|
||||
///
|
||||
/// The present `epoch` will be learned from the supplied `SlotClock`. In production this will
|
||||
/// be a wall-clock (e.g., system time, remote server time, etc.).
|
||||
pub fn poll(&self) -> Result<PollOutcome, Error> {
|
||||
let slot = self
|
||||
.slot_clock
|
||||
.present_slot()
|
||||
.map_err(|_| Error::SlotClockError)?
|
||||
.ok_or(Error::SlotUnknowable)?;
|
||||
|
||||
pub fn update(&self, slot: Slot) -> Result<UpdateOutcome, Error> {
|
||||
let epoch = slot.epoch(self.spec.slots_per_epoch);
|
||||
|
||||
if let Some(duties) = self.beacon_node.request_shuffling(epoch, &self.pubkey)? {
|
||||
if let Some(duties) = self
|
||||
.beacon_node
|
||||
.request_shuffling(epoch, &self.pubkeys[0])?
|
||||
{
|
||||
// If these duties were known, check to see if they're updates or identical.
|
||||
let result = if let Some(known_duties) = self.duties_map.get(epoch)? {
|
||||
if known_duties == duties {
|
||||
Ok(PollOutcome::NoChange(epoch))
|
||||
Ok(UpdateOutcome::NoChange(epoch))
|
||||
} else {
|
||||
Ok(PollOutcome::DutiesChanged(epoch, duties))
|
||||
Ok(UpdateOutcome::DutiesChanged(epoch, duties))
|
||||
}
|
||||
} else {
|
||||
Ok(PollOutcome::NewDuties(epoch, duties))
|
||||
Ok(UpdateOutcome::NewDuties(epoch, duties))
|
||||
};
|
||||
self.duties_map.insert(epoch, duties)?;
|
||||
result
|
||||
} else {
|
||||
Ok(PollOutcome::UnknownValidatorOrEpoch(epoch))
|
||||
Ok(UpdateOutcome::UnknownValidatorOrEpoch(epoch))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,40 +0,0 @@
|
||||
use super::traits::BeaconNode;
|
||||
use super::{DutiesManager, PollOutcome};
|
||||
use slog::{debug, error, info, Logger};
|
||||
use slot_clock::SlotClock;
|
||||
use std::time::Duration;
|
||||
|
||||
pub struct DutiesManagerService<T: SlotClock, U: BeaconNode> {
|
||||
pub manager: DutiesManager<T, U>,
|
||||
pub poll_interval_millis: u64,
|
||||
pub log: Logger,
|
||||
}
|
||||
|
||||
impl<T: SlotClock, U: BeaconNode> DutiesManagerService<T, U> {
|
||||
/// Run a loop which polls the manager each `poll_interval_millis` milliseconds.
|
||||
///
|
||||
/// Logs the results of the polls.
|
||||
pub fn run(&mut self) {
|
||||
loop {
|
||||
match self.manager.poll() {
|
||||
Err(error) => {
|
||||
error!(self.log, "Epoch duties poll error"; "error" => format!("{:?}", error))
|
||||
}
|
||||
Ok(PollOutcome::NoChange(epoch)) => {
|
||||
debug!(self.log, "No change in duties"; "epoch" => epoch)
|
||||
}
|
||||
Ok(PollOutcome::DutiesChanged(epoch, duties)) => {
|
||||
info!(self.log, "Duties changed (potential re-org)"; "epoch" => epoch, "duties" => format!("{:?}", duties))
|
||||
}
|
||||
Ok(PollOutcome::NewDuties(epoch, duties)) => {
|
||||
info!(self.log, "New duties obtained"; "epoch" => epoch, "duties" => format!("{:?}", duties))
|
||||
}
|
||||
Ok(PollOutcome::UnknownValidatorOrEpoch(epoch)) => {
|
||||
error!(self.log, "Epoch or validator unknown"; "epoch" => epoch)
|
||||
}
|
||||
};
|
||||
|
||||
std::thread::sleep(Duration::from_millis(self.poll_interval_millis));
|
||||
}
|
||||
}
|
||||
}
|
22
validator_client/src/error.rs
Normal file
22
validator_client/src/error.rs
Normal file
@ -0,0 +1,22 @@
|
||||
use slot_clock;
|
||||
|
||||
use error_chain::{
|
||||
error_chain, error_chain_processing, impl_error_chain_kind, impl_error_chain_processed,
|
||||
impl_extract_backtrace,
|
||||
};
|
||||
|
||||
error_chain! {
|
||||
links { }
|
||||
|
||||
errors {
|
||||
SlotClockError(e: slot_clock::SystemTimeSlotClockError) {
|
||||
description("Error reading system time"),
|
||||
display("SlotClockError: '{:?}'", e)
|
||||
}
|
||||
|
||||
SystemTimeError(t: String ) {
|
||||
description("Error reading system time"),
|
||||
display("SystemTimeError: '{}'", t)
|
||||
}
|
||||
}
|
||||
}
|
3
validator_client/src/lib.rs
Normal file
3
validator_client/src/lib.rs
Normal file
@ -0,0 +1,3 @@
|
||||
pub mod config;
|
||||
|
||||
pub use crate::config::Config;
|
@ -1,21 +1,14 @@
|
||||
use self::block_producer_service::{BeaconBlockGrpcClient, BlockProducerService};
|
||||
use self::duties::{DutiesManager, DutiesManagerService, EpochDutiesMap};
|
||||
use crate::config::ClientConfig;
|
||||
use block_proposer::{test_utils::LocalSigner, BlockProducer};
|
||||
use bls::Keypair;
|
||||
use clap::{App, Arg};
|
||||
use grpcio::{ChannelBuilder, EnvBuilder};
|
||||
use protos::services_grpc::{BeaconBlockServiceClient, ValidatorServiceClient};
|
||||
use slog::{error, info, o, Drain};
|
||||
use slot_clock::SystemTimeSlotClock;
|
||||
use std::path::PathBuf;
|
||||
use std::sync::Arc;
|
||||
use std::thread;
|
||||
use types::ChainSpec;
|
||||
|
||||
mod attester_service;
|
||||
mod block_producer_service;
|
||||
mod config;
|
||||
mod duties;
|
||||
pub mod error;
|
||||
mod service;
|
||||
|
||||
use crate::config::Config as ValidatorClientConfig;
|
||||
use clap::{App, Arg};
|
||||
use service::Service as ValidatorService;
|
||||
use slog::{error, info, o, Drain};
|
||||
|
||||
fn main() {
|
||||
// Logging
|
||||
@ -55,136 +48,12 @@ fn main() {
|
||||
)
|
||||
.get_matches();
|
||||
|
||||
let mut config = ClientConfig::default();
|
||||
let config = ValidatorClientConfig::parse_args(&matches, &log)
|
||||
.expect("Unable to build a configuration for the validator client.");
|
||||
|
||||
// Custom datadir
|
||||
if let Some(dir) = matches.value_of("datadir") {
|
||||
config.data_dir = PathBuf::from(dir.to_string());
|
||||
}
|
||||
|
||||
// Custom server port
|
||||
if let Some(server_str) = matches.value_of("server") {
|
||||
if let Ok(addr) = server_str.parse::<u16>() {
|
||||
config.server = addr.to_string();
|
||||
} else {
|
||||
error!(log, "Invalid address"; "server" => server_str);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: Permit loading a custom spec from file.
|
||||
// Custom spec
|
||||
if let Some(spec_str) = matches.value_of("spec") {
|
||||
match spec_str {
|
||||
"foundation" => config.spec = ChainSpec::foundation(),
|
||||
"few_validators" => config.spec = ChainSpec::few_validators(),
|
||||
// Should be impossible due to clap's `possible_values(..)` function.
|
||||
_ => unreachable!(),
|
||||
};
|
||||
}
|
||||
|
||||
// Log configuration
|
||||
info!(log, "";
|
||||
"data_dir" => &config.data_dir.to_str(),
|
||||
"server" => &config.server);
|
||||
|
||||
// Beacon node gRPC beacon block endpoints.
|
||||
let beacon_block_grpc_client = {
|
||||
let env = Arc::new(EnvBuilder::new().build());
|
||||
let ch = ChannelBuilder::new(env).connect(&config.server);
|
||||
Arc::new(BeaconBlockServiceClient::new(ch))
|
||||
};
|
||||
|
||||
// Beacon node gRPC validator endpoints.
|
||||
let validator_grpc_client = {
|
||||
let env = Arc::new(EnvBuilder::new().build());
|
||||
let ch = ChannelBuilder::new(env).connect(&config.server);
|
||||
Arc::new(ValidatorServiceClient::new(ch))
|
||||
};
|
||||
|
||||
// Spec
|
||||
let spec = Arc::new(config.spec.clone());
|
||||
|
||||
// Clock for determining the present slot.
|
||||
// TODO: this shouldn't be a static time, instead it should be pulled from the beacon node.
|
||||
// https://github.com/sigp/lighthouse/issues/160
|
||||
let genesis_time = 1_549_935_547;
|
||||
let slot_clock = {
|
||||
info!(log, "Genesis time"; "unix_epoch_seconds" => genesis_time);
|
||||
let clock = SystemTimeSlotClock::new(genesis_time, spec.seconds_per_slot)
|
||||
.expect("Unable to instantiate SystemTimeSlotClock.");
|
||||
Arc::new(clock)
|
||||
};
|
||||
|
||||
let poll_interval_millis = spec.seconds_per_slot * 1000 / 10; // 10% epoch time precision.
|
||||
info!(log, "Starting block producer service"; "polls_per_epoch" => spec.seconds_per_slot * 1000 / poll_interval_millis);
|
||||
|
||||
/*
|
||||
* Start threads.
|
||||
*/
|
||||
let mut threads = vec![];
|
||||
// TODO: keypairs are randomly generated; they should be loaded from a file or generated.
|
||||
// https://github.com/sigp/lighthouse/issues/160
|
||||
let keypairs = vec![Keypair::random()];
|
||||
|
||||
for keypair in keypairs {
|
||||
info!(log, "Starting validator services"; "validator" => keypair.pk.concatenated_hex_id());
|
||||
let duties_map = Arc::new(EpochDutiesMap::new(spec.slots_per_epoch));
|
||||
|
||||
// Spawn a new thread to maintain the validator's `EpochDuties`.
|
||||
let duties_manager_thread = {
|
||||
let spec = spec.clone();
|
||||
let duties_map = duties_map.clone();
|
||||
let slot_clock = slot_clock.clone();
|
||||
let log = log.clone();
|
||||
let beacon_node = validator_grpc_client.clone();
|
||||
let pubkey = keypair.pk.clone();
|
||||
thread::spawn(move || {
|
||||
let manager = DutiesManager {
|
||||
duties_map,
|
||||
pubkey,
|
||||
spec,
|
||||
slot_clock,
|
||||
beacon_node,
|
||||
};
|
||||
let mut duties_manager_service = DutiesManagerService {
|
||||
manager,
|
||||
poll_interval_millis,
|
||||
log,
|
||||
};
|
||||
|
||||
duties_manager_service.run();
|
||||
})
|
||||
};
|
||||
|
||||
// Spawn a new thread to perform block production for the validator.
|
||||
let producer_thread = {
|
||||
let spec = spec.clone();
|
||||
let signer = Arc::new(LocalSigner::new(keypair.clone()));
|
||||
let duties_map = duties_map.clone();
|
||||
let slot_clock = slot_clock.clone();
|
||||
let log = log.clone();
|
||||
let client = Arc::new(BeaconBlockGrpcClient::new(beacon_block_grpc_client.clone()));
|
||||
thread::spawn(move || {
|
||||
let block_producer =
|
||||
BlockProducer::new(spec, duties_map, slot_clock, client, signer);
|
||||
let mut block_producer_service = BlockProducerService {
|
||||
block_producer,
|
||||
poll_interval_millis,
|
||||
log,
|
||||
};
|
||||
|
||||
block_producer_service.run();
|
||||
})
|
||||
};
|
||||
|
||||
threads.push((duties_manager_thread, producer_thread));
|
||||
}
|
||||
|
||||
// Naively wait for all the threads to complete.
|
||||
for tuple in threads {
|
||||
let (manager, producer) = tuple;
|
||||
let _ = producer.join();
|
||||
let _ = manager.join();
|
||||
// start the validator service.
|
||||
match ValidatorService::start(config, log.clone()) {
|
||||
Ok(_) => info!(log, "Validator client shutdown successfully."),
|
||||
Err(e) => error!(log, "Validator exited due to: {}", e.to_string()),
|
||||
}
|
||||
}
|
||||
|
364
validator_client/src/service.rs
Normal file
364
validator_client/src/service.rs
Normal file
@ -0,0 +1,364 @@
|
||||
/// The validator service. Connects to a beacon node and signs blocks when required.
|
||||
use crate::attester_service::{AttestationGrpcClient, AttesterService};
|
||||
use crate::block_producer_service::{BeaconBlockGrpcClient, BlockProducerService};
|
||||
use crate::config::Config as ValidatorConfig;
|
||||
use crate::duties::UpdateOutcome;
|
||||
use crate::duties::{DutiesManager, EpochDutiesMap};
|
||||
use crate::error as error_chain;
|
||||
use crate::error::ErrorKind;
|
||||
use attester::test_utils::EpochMap;
|
||||
use attester::{test_utils::LocalSigner as AttesterLocalSigner, Attester};
|
||||
use block_proposer::{test_utils::LocalSigner as BlockProposerLocalSigner, BlockProducer};
|
||||
use bls::Keypair;
|
||||
use grpcio::{ChannelBuilder, EnvBuilder};
|
||||
use protos::services::Empty;
|
||||
use protos::services_grpc::{
|
||||
AttestationServiceClient, BeaconBlockServiceClient, BeaconNodeServiceClient,
|
||||
ValidatorServiceClient,
|
||||
};
|
||||
use slog::{debug, error, info, warn};
|
||||
use slot_clock::{SlotClock, SystemTimeSlotClock};
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant, SystemTime};
|
||||
use tokio::prelude::*;
|
||||
use tokio::runtime::Builder;
|
||||
use tokio::timer::Interval;
|
||||
use tokio_timer::clock::Clock;
|
||||
use types::{Epoch, Fork, Slot};
|
||||
|
||||
//TODO: This service should be simplified in the future. Can be made more steamlined.
|
||||
|
||||
/// The validator service. This is the main thread that executes and maintains validator
|
||||
/// duties.
|
||||
pub struct Service {
|
||||
/// The node we currently connected to.
|
||||
connected_node_version: String,
|
||||
/// The chain id we are processing on.
|
||||
chain_id: u16,
|
||||
/// The fork state we processing on.
|
||||
fork: Fork,
|
||||
/// The slot clock keeping track of time.
|
||||
slot_clock: Arc<SystemTimeSlotClock>,
|
||||
/// The current slot we are processing.
|
||||
current_slot: Slot,
|
||||
/// Duration until the next slot. This is used for initializing the tokio timer interval.
|
||||
duration_to_next_slot: Duration,
|
||||
// GRPC Clients
|
||||
/// The beacon block GRPC client.
|
||||
beacon_block_client: Arc<BeaconBlockServiceClient>,
|
||||
/// The validator GRPC client.
|
||||
validator_client: Arc<ValidatorServiceClient>,
|
||||
/// The attester GRPC client.
|
||||
attester_client: Arc<AttestationServiceClient>,
|
||||
/// The validator client logger.
|
||||
log: slog::Logger,
|
||||
}
|
||||
|
||||
impl Service {
|
||||
/// Initial connection to the beacon node to determine its properties.
|
||||
///
|
||||
/// This tries to connect to a beacon node. Once connected, it initialised the gRPC clients
|
||||
/// and returns an instance of the service.
|
||||
fn initialize_service(
|
||||
config: &ValidatorConfig,
|
||||
log: slog::Logger,
|
||||
) -> error_chain::Result<Self> {
|
||||
// initialise the beacon node client to check for a connection
|
||||
|
||||
let env = Arc::new(EnvBuilder::new().build());
|
||||
// Beacon node gRPC beacon node endpoints.
|
||||
let beacon_node_client = {
|
||||
let ch = ChannelBuilder::new(env.clone()).connect(&config.server);
|
||||
Arc::new(BeaconNodeServiceClient::new(ch))
|
||||
};
|
||||
|
||||
// retrieve node information
|
||||
let node_info = loop {
|
||||
let info = match beacon_node_client.info(&Empty::new()) {
|
||||
Err(e) => {
|
||||
warn!(log, "Could not connect to node. Error: {}", e);
|
||||
info!(log, "Retrying in 5 seconds...");
|
||||
std::thread::sleep(Duration::from_secs(5));
|
||||
continue;
|
||||
}
|
||||
Ok(info) => {
|
||||
if SystemTime::now()
|
||||
.duration_since(SystemTime::UNIX_EPOCH)
|
||||
.unwrap()
|
||||
.as_secs()
|
||||
< info.genesis_time
|
||||
{
|
||||
warn!(
|
||||
log,
|
||||
"Beacon Node's genesis time is in the future. No work to do.\n Exiting"
|
||||
);
|
||||
return Err("Genesis time in the future".into());
|
||||
}
|
||||
break info;
|
||||
}
|
||||
};
|
||||
};
|
||||
|
||||
// build requisite objects to form Self
|
||||
let genesis_time = node_info.get_genesis_time();
|
||||
|
||||
info!(log,"Beacon node connected"; "Node Version" => node_info.version.clone(), "Chain ID" => node_info.chain_id, "Genesis time" => genesis_time);
|
||||
|
||||
let proto_fork = node_info.get_fork();
|
||||
let mut previous_version: [u8; 4] = [0; 4];
|
||||
let mut current_version: [u8; 4] = [0; 4];
|
||||
previous_version.copy_from_slice(&proto_fork.get_previous_version()[..4]);
|
||||
current_version.copy_from_slice(&proto_fork.get_current_version()[..4]);
|
||||
let fork = Fork {
|
||||
previous_version,
|
||||
current_version,
|
||||
epoch: Epoch::from(proto_fork.get_epoch()),
|
||||
};
|
||||
|
||||
// build the validator slot clock
|
||||
let slot_clock = {
|
||||
let clock = SystemTimeSlotClock::new(genesis_time, config.spec.seconds_per_slot)
|
||||
.expect("Unable to instantiate SystemTimeSlotClock.");
|
||||
Arc::new(clock)
|
||||
};
|
||||
|
||||
// initialize the RPC clients
|
||||
|
||||
// Beacon node gRPC beacon block endpoints.
|
||||
let beacon_block_client = {
|
||||
let ch = ChannelBuilder::new(env.clone()).connect(&config.server);
|
||||
Arc::new(BeaconBlockServiceClient::new(ch))
|
||||
};
|
||||
|
||||
// Beacon node gRPC validator endpoints.
|
||||
let validator_client = {
|
||||
let ch = ChannelBuilder::new(env.clone()).connect(&config.server);
|
||||
Arc::new(ValidatorServiceClient::new(ch))
|
||||
};
|
||||
|
||||
//Beacon node gRPC attester endpoints.
|
||||
let attester_client = {
|
||||
let ch = ChannelBuilder::new(env.clone()).connect(&config.server);
|
||||
Arc::new(AttestationServiceClient::new(ch))
|
||||
};
|
||||
|
||||
let current_slot = slot_clock
|
||||
.present_slot()
|
||||
.map_err(|e| ErrorKind::SlotClockError(e))?
|
||||
.expect("Genesis must be in the future");
|
||||
|
||||
// calculate the duration to the next slot
|
||||
let duration_to_next_slot = {
|
||||
let seconds_per_slot = config.spec.seconds_per_slot;
|
||||
let syslot_time = SystemTime::now();
|
||||
let duration_since_epoch = syslot_time
|
||||
.duration_since(SystemTime::UNIX_EPOCH)
|
||||
.map_err(|e| ErrorKind::SystemTimeError(e.to_string()))?;
|
||||
let duration_since_genesis = duration_since_epoch
|
||||
.checked_sub(Duration::from_secs(genesis_time))
|
||||
.expect("Genesis must be in the future. Checked on connection");
|
||||
let elapsed_slots = duration_since_epoch
|
||||
.as_secs()
|
||||
.checked_div(seconds_per_slot as u64)
|
||||
.expect("Seconds per slot should not be 0");
|
||||
|
||||
// the duration to the next slot
|
||||
Duration::from_secs(
|
||||
(elapsed_slots + 1)
|
||||
.checked_mul(seconds_per_slot)
|
||||
.expect("Next slot time should not overflow u64"),
|
||||
)
|
||||
.checked_sub(duration_since_genesis)
|
||||
.expect("This should never saturate")
|
||||
};
|
||||
|
||||
Ok(Self {
|
||||
connected_node_version: node_info.version,
|
||||
chain_id: node_info.chain_id as u16,
|
||||
fork,
|
||||
slot_clock,
|
||||
current_slot,
|
||||
duration_to_next_slot,
|
||||
beacon_block_client,
|
||||
validator_client,
|
||||
attester_client,
|
||||
log,
|
||||
})
|
||||
}
|
||||
|
||||
/// Initialise the service then run the core thread.
|
||||
pub fn start(config: ValidatorConfig, log: slog::Logger) -> error_chain::Result<()> {
|
||||
// connect to the node and retrieve its properties and initialize the gRPC clients
|
||||
let service = Service::initialize_service(&config, log)?;
|
||||
|
||||
// we have connected to a node and established its parameters. Spin up the core service
|
||||
|
||||
// set up the validator service runtime
|
||||
let mut runtime = Builder::new()
|
||||
.clock(Clock::system())
|
||||
.name_prefix("validator-client-")
|
||||
.build()
|
||||
.map_err(|e| format!("Tokio runtime failed: {}", e))?;
|
||||
|
||||
// set up the validator work interval - start at next slot and proceed every slot
|
||||
let interval = {
|
||||
// Set the interval to start at the next slot, and every slot after
|
||||
let slot_duration = Duration::from_secs(config.spec.seconds_per_slot);
|
||||
//TODO: Handle checked add correctly
|
||||
Interval::new(
|
||||
Instant::now() + service.duration_to_next_slot,
|
||||
slot_duration,
|
||||
)
|
||||
};
|
||||
|
||||
// kick off core service
|
||||
|
||||
// generate keypairs
|
||||
|
||||
// TODO: keypairs are randomly generated; they should be loaded from a file or generated.
|
||||
// https://github.com/sigp/lighthouse/issues/160
|
||||
let keypairs = Arc::new(vec![Keypair::random()]);
|
||||
|
||||
// build requisite objects to pass to core thread.
|
||||
let duties_map = Arc::new(EpochDutiesMap::new(config.spec.slots_per_epoch));
|
||||
let epoch_map_for_attester = Arc::new(EpochMap::new(config.spec.slots_per_epoch));
|
||||
let manager = DutiesManager {
|
||||
duties_map,
|
||||
pubkeys: keypairs.iter().map(|keypair| keypair.pk.clone()).collect(),
|
||||
spec: Arc::new(config.spec),
|
||||
slot_clock: service.slot_clock.clone(),
|
||||
beacon_node: service.validator_client.clone(),
|
||||
};
|
||||
|
||||
// run the core thread
|
||||
runtime
|
||||
.block_on(interval.for_each(move |_| {
|
||||
// get the current slot
|
||||
let current_slot = match service.slot_clock.present_slot() {
|
||||
Err(e) => {
|
||||
error!(service.log, "SystemTimeError {:?}", e);
|
||||
return Ok(());
|
||||
}
|
||||
Ok(slot) => slot.expect("Genesis is in the future"),
|
||||
};
|
||||
|
||||
debug_assert!(
|
||||
current_slot > service.current_slot,
|
||||
"The Timer should poll a new slot"
|
||||
);
|
||||
|
||||
info!(service.log, "Processing slot: {}", current_slot.as_u64());
|
||||
|
||||
// check for new duties
|
||||
// TODO: Convert to its own thread
|
||||
match manager.update(current_slot) {
|
||||
Err(error) => {
|
||||
error!(service.log, "Epoch duties poll error"; "error" => format!("{:?}", error))
|
||||
}
|
||||
Ok(UpdateOutcome::NoChange(epoch)) => {
|
||||
debug!(service.log, "No change in duties"; "epoch" => epoch)
|
||||
}
|
||||
Ok(UpdateOutcome::DutiesChanged(epoch, duties)) => {
|
||||
info!(service.log, "Duties changed (potential re-org)"; "epoch" => epoch, "duties" => format!("{:?}", duties))
|
||||
}
|
||||
Ok(UpdateOutcome::NewDuties(epoch, duties)) => {
|
||||
info!(service.log, "New duties obtained"; "epoch" => epoch, "duties" => format!("{:?}", duties))
|
||||
}
|
||||
Ok(UpdateOutcome::UnknownValidatorOrEpoch(epoch)) => {
|
||||
error!(service.log, "Epoch or validator unknown"; "epoch" => epoch)
|
||||
}
|
||||
};
|
||||
|
||||
Ok(())
|
||||
}))
|
||||
.map_err(|e| format!("Service thread failed: {:?}", e))?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/*
|
||||
|
||||
let duties_map = Arc::new(EpochDutiesMap::new(spec.slots_per_epoch));
|
||||
let epoch_map_for_attester = Arc::new(EpochMap::new(spec.slots_per_epoch));
|
||||
|
||||
|
||||
for keypair in keypairs {
|
||||
info!(self.log, "Starting validator services"; "validator" => keypair.pk.concatenated_hex_id());
|
||||
|
||||
// Spawn a new thread to maintain the validator's `EpochDuties`.
|
||||
let duties_manager_thread = {
|
||||
let spec = spec.clone();
|
||||
let duties_map = duties_map.clone();
|
||||
let slot_clock = self.slot_clock.clone();
|
||||
let log = self.log.clone();
|
||||
let beacon_node = self.validator_client.clone();
|
||||
let pubkey = keypair.pk.clone();
|
||||
thread::spawn(move || {
|
||||
let manager = DutiesManager {
|
||||
duties_map,
|
||||
pubkey,
|
||||
spec,
|
||||
slot_clock,
|
||||
beacon_node,
|
||||
};
|
||||
let mut duties_manager_service = DutiesManagerService {
|
||||
manager,
|
||||
poll_interval_millis,
|
||||
log,
|
||||
};
|
||||
|
||||
duties_manager_service.run();
|
||||
})
|
||||
};
|
||||
|
||||
// Spawn a new thread to perform block production for the validator.
|
||||
let producer_thread = {
|
||||
let spec = spec.clone();
|
||||
let signer = Arc::new(BlockProposerLocalSigner::new(keypair.clone()));
|
||||
let duties_map = duties_map.clone();
|
||||
let slot_clock = slot_clock.clone();
|
||||
let log = log.clone();
|
||||
let client = Arc::new(BeaconBlockGrpcClient::new(beacon_block_grpc_client.clone()));
|
||||
thread::spawn(move || {
|
||||
let block_producer =
|
||||
BlockProducer::new(spec, duties_map, slot_clock, client, signer);
|
||||
let mut block_producer_service = BlockProducerService {
|
||||
block_producer,
|
||||
poll_interval_millis,
|
||||
log,
|
||||
};
|
||||
|
||||
block_producer_service.run();
|
||||
})
|
||||
};
|
||||
|
||||
// Spawn a new thread for attestation for the validator.
|
||||
let attester_thread = {
|
||||
let signer = Arc::new(AttesterLocalSigner::new(keypair.clone()));
|
||||
let epoch_map = epoch_map_for_attester.clone();
|
||||
let slot_clock = slot_clock.clone();
|
||||
let log = log.clone();
|
||||
let client = Arc::new(AttestationGrpcClient::new(attester_grpc_client.clone()));
|
||||
thread::spawn(move || {
|
||||
let attester = Attester::new(epoch_map, slot_clock, client, signer);
|
||||
let mut attester_service = AttesterService {
|
||||
attester,
|
||||
poll_interval_millis,
|
||||
log,
|
||||
};
|
||||
|
||||
attester_service.run();
|
||||
})
|
||||
};
|
||||
|
||||
threads.push((duties_manager_thread, producer_thread, attester_thread));
|
||||
}
|
||||
|
||||
// Naively wait for all the threads to complete.
|
||||
for tuple in threads {
|
||||
let (manager, producer, attester) = tuple;
|
||||
let _ = producer.join();
|
||||
let _ = manager.join();
|
||||
let _ = attester.join();
|
||||
}
|
||||
*/
|
||||
}
|
Loading…
Reference in New Issue
Block a user