Implement produce beacon block on gRPC beacon node server
This commit is contained in:
parent
ba90901730
commit
6e254551af
@ -7,7 +7,7 @@ pub mod test_utils;
|
|||||||
|
|
||||||
pub use self::beacon_chain::{BeaconChain, BlockProcessingOutcome, InvalidBlock, ValidBlock};
|
pub use self::beacon_chain::{BeaconChain, BlockProcessingOutcome, InvalidBlock, ValidBlock};
|
||||||
pub use self::checkpoint::CheckPoint;
|
pub use self::checkpoint::CheckPoint;
|
||||||
pub use self::errors::BeaconChainError;
|
pub use self::errors::{BeaconChainError, BlockProductionError};
|
||||||
pub use attestation_aggregator::Outcome as AggregationOutcome;
|
pub use attestation_aggregator::Outcome as AggregationOutcome;
|
||||||
pub use db;
|
pub use db;
|
||||||
pub use fork_choice;
|
pub use fork_choice;
|
||||||
|
@ -3,7 +3,7 @@ use crossbeam_channel;
|
|||||||
use eth2_libp2p::rpc::methods::BlockRootSlot;
|
use eth2_libp2p::rpc::methods::BlockRootSlot;
|
||||||
use eth2_libp2p::PubsubMessage;
|
use eth2_libp2p::PubsubMessage;
|
||||||
use futures::Future;
|
use futures::Future;
|
||||||
use grpcio::{RpcContext, UnarySink};
|
use grpcio::{RpcContext, RpcStatus, RpcStatusCode, UnarySink};
|
||||||
use network::NetworkMessage;
|
use network::NetworkMessage;
|
||||||
use protos::services::{
|
use protos::services::{
|
||||||
BeaconBlock as BeaconBlockProto, ProduceBeaconBlockRequest, ProduceBeaconBlockResponse,
|
BeaconBlock as BeaconBlockProto, ProduceBeaconBlockRequest, ProduceBeaconBlockResponse,
|
||||||
@ -11,10 +11,10 @@ use protos::services::{
|
|||||||
};
|
};
|
||||||
use protos::services_grpc::BeaconBlockService;
|
use protos::services_grpc::BeaconBlockService;
|
||||||
use slog::Logger;
|
use slog::Logger;
|
||||||
use slog::{debug, error, info, warn};
|
use slog::{error, info, trace, warn};
|
||||||
use ssz::{Decodable, TreeHash};
|
use ssz::{ssz_encode, Decodable, TreeHash};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use types::{BeaconBlock, Hash256, Slot};
|
use types::{BeaconBlock, Hash256, Signature, Slot};
|
||||||
|
|
||||||
#[derive(Clone)]
|
#[derive(Clone)]
|
||||||
pub struct BeaconBlockServiceInstance {
|
pub struct BeaconBlockServiceInstance {
|
||||||
@ -31,11 +31,44 @@ impl BeaconBlockService for BeaconBlockServiceInstance {
|
|||||||
req: ProduceBeaconBlockRequest,
|
req: ProduceBeaconBlockRequest,
|
||||||
sink: UnarySink<ProduceBeaconBlockResponse>,
|
sink: UnarySink<ProduceBeaconBlockResponse>,
|
||||||
) {
|
) {
|
||||||
println!("producing at slot {}", req.get_slot());
|
trace!(self.log, "Generating a beacon block"; "req" => format!("{:?}", req));
|
||||||
|
|
||||||
|
// decode the request
|
||||||
|
// TODO: requested slot currently unused, see: https://github.com/sigp/lighthouse/issues/336
|
||||||
|
let _requested_slot = Slot::from(req.get_slot());
|
||||||
|
let (randao_reveal, _index) = match Signature::ssz_decode(req.get_randao_reveal(), 0) {
|
||||||
|
Ok(v) => v,
|
||||||
|
Err(_) => {
|
||||||
|
// decode error, incorrect signature
|
||||||
|
let log_clone = self.log.clone();
|
||||||
|
let f = sink
|
||||||
|
.fail(RpcStatus::new(
|
||||||
|
RpcStatusCode::InvalidArgument,
|
||||||
|
Some(format!("Invalid randao reveal signature")),
|
||||||
|
))
|
||||||
|
.map_err(move |e| warn!(log_clone, "failed to reply {:?}: {:?}", req, e));
|
||||||
|
return ctx.spawn(f);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
let produced_block = match self.chain.produce_block(randao_reveal) {
|
||||||
|
Ok((block, _state)) => block,
|
||||||
|
Err(e) => {
|
||||||
|
// could not produce a block
|
||||||
|
let log_clone = self.log.clone();
|
||||||
|
warn!(self.log, "RPC Error"; "Error" => format!("Could not produce a block:{:?}",e));
|
||||||
|
let f = sink
|
||||||
|
.fail(RpcStatus::new(
|
||||||
|
RpcStatusCode::Unknown,
|
||||||
|
Some(format!("Could not produce a block: {:?}", e)),
|
||||||
|
))
|
||||||
|
.map_err(move |e| warn!(log_clone, "failed to reply {:?}: {:?}", req, e));
|
||||||
|
return ctx.spawn(f);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
// TODO: build a legit block.
|
|
||||||
let mut block = BeaconBlockProto::new();
|
let mut block = BeaconBlockProto::new();
|
||||||
block.set_ssz(b"cats".to_vec());
|
block.set_ssz(ssz_encode(&produced_block));
|
||||||
|
|
||||||
let mut resp = ProduceBeaconBlockResponse::new();
|
let mut resp = ProduceBeaconBlockResponse::new();
|
||||||
resp.set_block(block);
|
resp.set_block(block);
|
||||||
@ -81,11 +114,16 @@ impl BeaconBlockService for BeaconBlockServiceInstance {
|
|||||||
slot: block.slot,
|
slot: block.slot,
|
||||||
});
|
});
|
||||||
|
|
||||||
println!("Sending beacon block to gossipsub");
|
match self.network_chan.send(NetworkMessage::Publish {
|
||||||
self.network_chan.send(NetworkMessage::Publish {
|
|
||||||
topics: vec![topic],
|
topics: vec![topic],
|
||||||
message,
|
message,
|
||||||
});
|
}) {
|
||||||
|
Ok(_) => {}
|
||||||
|
Err(_) => warn!(
|
||||||
|
self.log,
|
||||||
|
"Could not send published block to the network service"
|
||||||
|
),
|
||||||
|
}
|
||||||
|
|
||||||
resp.set_success(true);
|
resp.set_success(true);
|
||||||
} else if outcome.is_invalid() {
|
} else if outcome.is_invalid() {
|
||||||
|
@ -4,7 +4,8 @@ use beacon_chain::{
|
|||||||
fork_choice::ForkChoice,
|
fork_choice::ForkChoice,
|
||||||
parking_lot::RwLockReadGuard,
|
parking_lot::RwLockReadGuard,
|
||||||
slot_clock::SlotClock,
|
slot_clock::SlotClock,
|
||||||
types::{BeaconState, ChainSpec},
|
types::{BeaconState, ChainSpec, Signature},
|
||||||
|
BlockProductionError,
|
||||||
};
|
};
|
||||||
pub use beacon_chain::{BeaconChainError, BlockProcessingOutcome};
|
pub use beacon_chain::{BeaconChainError, BlockProcessingOutcome};
|
||||||
use types::BeaconBlock;
|
use types::BeaconBlock;
|
||||||
@ -17,6 +18,11 @@ pub trait BeaconChain: Send + Sync {
|
|||||||
|
|
||||||
fn process_block(&self, block: BeaconBlock)
|
fn process_block(&self, block: BeaconBlock)
|
||||||
-> Result<BlockProcessingOutcome, BeaconChainError>;
|
-> Result<BlockProcessingOutcome, BeaconChainError>;
|
||||||
|
|
||||||
|
fn produce_block(
|
||||||
|
&self,
|
||||||
|
randao_reveal: Signature,
|
||||||
|
) -> Result<(BeaconBlock, BeaconState), BlockProductionError>;
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T, U, F> BeaconChain for RawBeaconChain<T, U, F>
|
impl<T, U, F> BeaconChain for RawBeaconChain<T, U, F>
|
||||||
@ -39,4 +45,11 @@ where
|
|||||||
) -> Result<BlockProcessingOutcome, BeaconChainError> {
|
) -> Result<BlockProcessingOutcome, BeaconChainError> {
|
||||||
self.process_block(block)
|
self.process_block(block)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
fn produce_block(
|
||||||
|
&self,
|
||||||
|
randao_reveal: Signature,
|
||||||
|
) -> Result<(BeaconBlock, BeaconState), BlockProductionError> {
|
||||||
|
self.produce_block(randao_reveal)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -120,7 +120,7 @@ impl TestingBeaconStateBuilder {
|
|||||||
})
|
})
|
||||||
.collect();
|
.collect();
|
||||||
|
|
||||||
let genesis_time = 1553776331; // arbitrary
|
let genesis_time = 1553918534; // arbitrary
|
||||||
|
|
||||||
let mut state = BeaconState::genesis(
|
let mut state = BeaconState::genesis(
|
||||||
genesis_time,
|
genesis_time,
|
||||||
|
@ -4,6 +4,7 @@ mod grpc;
|
|||||||
use self::beacon_block_node::{BeaconBlockNode, BeaconBlockNodeError};
|
use self::beacon_block_node::{BeaconBlockNode, BeaconBlockNodeError};
|
||||||
pub use self::grpc::BeaconBlockGrpcClient;
|
pub use self::grpc::BeaconBlockGrpcClient;
|
||||||
use crate::signer::Signer;
|
use crate::signer::Signer;
|
||||||
|
use slog::{error, info};
|
||||||
use ssz::{SignedRoot, TreeHash};
|
use ssz::{SignedRoot, TreeHash};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use types::{BeaconBlock, ChainSpec, Domain, Fork, Slot};
|
use types::{BeaconBlock, ChainSpec, Domain, Fork, Slot};
|
||||||
@ -23,8 +24,6 @@ pub enum ValidatorEvent {
|
|||||||
BeaconNodeUnableToProduceBlock(Slot),
|
BeaconNodeUnableToProduceBlock(Slot),
|
||||||
/// The signer failed to sign the message.
|
/// The signer failed to sign the message.
|
||||||
SignerRejection(Slot),
|
SignerRejection(Slot),
|
||||||
/// The public key for this validator is not an active validator.
|
|
||||||
ValidatorIsUnknown(Slot),
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// This struct contains the logic for requesting and signing beacon blocks for a validator. The
|
/// This struct contains the logic for requesting and signing beacon blocks for a validator. The
|
||||||
@ -43,6 +42,25 @@ pub struct BlockProducer<'a, B: BeaconBlockNode, S: Signer> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
impl<'a, B: BeaconBlockNode, S: Signer> BlockProducer<'a, B, S> {
|
impl<'a, B: BeaconBlockNode, S: Signer> BlockProducer<'a, B, S> {
|
||||||
|
/// Handle outputs and results from block production.
|
||||||
|
pub fn handle_produce_block(&mut self, log: slog::Logger) {
|
||||||
|
match self.produce_block() {
|
||||||
|
Ok(ValidatorEvent::BlockProduced(_slot)) => {
|
||||||
|
info!(log, "Block produced"; "Validator" => format!("{}", self.signer))
|
||||||
|
}
|
||||||
|
Err(e) => error!(log, "Block production error"; "Error" => format!("{:?}", e)),
|
||||||
|
Ok(ValidatorEvent::SignerRejection(_slot)) => {
|
||||||
|
error!(log, "Block production error"; "Error" => format!("Signer Could not sign the block"))
|
||||||
|
}
|
||||||
|
Ok(ValidatorEvent::SlashableBlockNotProduced(_slot)) => {
|
||||||
|
error!(log, "Block production error"; "Error" => format!("Rejected the block as it could have been slashed"))
|
||||||
|
}
|
||||||
|
Ok(ValidatorEvent::BeaconNodeUnableToProduceBlock(_slot)) => {
|
||||||
|
error!(log, "Block production error"; "Error" => format!("Beacon node was unable to produce a block"))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Produce a block at some slot.
|
/// Produce a block at some slot.
|
||||||
///
|
///
|
||||||
/// Assumes that a block is required at this slot (does not check the duties).
|
/// Assumes that a block is required at this slot (does not check the duties).
|
||||||
@ -53,7 +71,7 @@ impl<'a, B: BeaconBlockNode, S: Signer> BlockProducer<'a, B, S> {
|
|||||||
///
|
///
|
||||||
/// The slash-protection code is not yet implemented. There is zero protection against
|
/// The slash-protection code is not yet implemented. There is zero protection against
|
||||||
/// slashing.
|
/// slashing.
|
||||||
fn produce_block(&mut self) -> Result<ValidatorEvent, Error> {
|
pub fn produce_block(&mut self) -> Result<ValidatorEvent, Error> {
|
||||||
let epoch = self.slot.epoch(self.spec.slots_per_epoch);
|
let epoch = self.slot.epoch(self.spec.slots_per_epoch);
|
||||||
|
|
||||||
let randao_reveal = {
|
let randao_reveal = {
|
||||||
|
@ -302,20 +302,23 @@ impl<B: BeaconNodeDuties + 'static, S: Signer + 'static> Service<B, S> {
|
|||||||
for (signer_index, work_type) in work {
|
for (signer_index, work_type) in work {
|
||||||
if work_type.produce_block {
|
if work_type.produce_block {
|
||||||
// spawns a thread to produce a beacon block
|
// spawns a thread to produce a beacon block
|
||||||
let signers = self.duties_manager.signers.clone();
|
let signers = self.duties_manager.signers.clone(); // this is an arc
|
||||||
let fork = self.fork.clone();
|
let fork = self.fork.clone();
|
||||||
let slot = self.current_slot.clone();
|
let slot = self.current_slot.clone();
|
||||||
let spec = self.spec.clone();
|
let spec = self.spec.clone();
|
||||||
let beacon_node = self.beacon_block_client.clone();
|
let beacon_node = self.beacon_block_client.clone();
|
||||||
|
let log = self.log.clone();
|
||||||
std::thread::spawn(move || {
|
std::thread::spawn(move || {
|
||||||
|
info!(log, "Producing a block"; "Validator"=> format!("{}", signers[signer_index]));
|
||||||
let signer = &signers[signer_index];
|
let signer = &signers[signer_index];
|
||||||
let block_producer = BlockProducer {
|
let mut block_producer = BlockProducer {
|
||||||
fork,
|
fork,
|
||||||
slot,
|
slot,
|
||||||
spec,
|
spec,
|
||||||
beacon_node,
|
beacon_node,
|
||||||
signer,
|
signer,
|
||||||
};
|
};
|
||||||
|
block_producer.handle_produce_block(log);
|
||||||
});
|
});
|
||||||
|
|
||||||
// TODO: Produce a beacon block in a new thread
|
// TODO: Produce a beacon block in a new thread
|
||||||
|
Loading…
Reference in New Issue
Block a user