use crate::eth1_chain::CachingEth1Backend; use crate::events::NullEventHandler; use crate::head_tracker::HeadTracker; use crate::persisted_beacon_chain::{PersistedBeaconChain, BEACON_CHAIN_DB_KEY}; use crate::timeout_rw_lock::TimeoutRwLock; use crate::{ BeaconChain, BeaconChainTypes, CheckPoint, Eth1Chain, Eth1ChainBackend, EventHandler, ForkChoice, }; use eth1::Config as Eth1Config; use operation_pool::OperationPool; use proto_array_fork_choice::ProtoArrayForkChoice; use slog::{info, Logger}; use slot_clock::{SlotClock, TestingSlotClock}; use std::marker::PhantomData; use std::sync::Arc; use std::time::Duration; use store::Store; use types::{ BeaconBlock, BeaconState, ChainSpec, EthSpec, Hash256, Signature, SignedBeaconBlock, Slot, }; /// An empty struct used to "witness" all the `BeaconChainTypes` traits. It has no user-facing /// functionality and only exists to satisfy the type system. pub struct Witness( PhantomData<( TStore, TStoreMigrator, TSlotClock, TEth1Backend, TEthSpec, TEventHandler, )>, ); impl BeaconChainTypes for Witness where TStore: Store + 'static, TStoreMigrator: store::Migrate + 'static, TSlotClock: SlotClock + 'static, TEth1Backend: Eth1ChainBackend + 'static, TEthSpec: EthSpec + 'static, TEventHandler: EventHandler + 'static, { type Store = TStore; type StoreMigrator = TStoreMigrator; type SlotClock = TSlotClock; type Eth1Chain = TEth1Backend; type EthSpec = TEthSpec; type EventHandler = TEventHandler; } /// Builds a `BeaconChain` by either creating anew from genesis, or, resuming from an existing chain /// persisted to `store`. /// /// Types may be elided and the compiler will infer them if all necessary builder methods have been /// called. If type inference errors are being raised, it is likely that not all required methods /// have been called. /// /// See the tests for an example of a complete working example. pub struct BeaconChainBuilder { store: Option>, store_migrator: Option, /// The finalized checkpoint to anchor the chain. May be genesis or a higher /// checkpoint. pub finalized_checkpoint: Option>, genesis_block_root: Option, op_pool: Option>, fork_choice: Option>, eth1_chain: Option>, event_handler: Option, slot_clock: Option, persisted_beacon_chain: Option>, head_tracker: Option, spec: ChainSpec, log: Option, } impl BeaconChainBuilder< Witness, > where TStore: Store + 'static, TStoreMigrator: store::Migrate + 'static, TSlotClock: SlotClock + 'static, TEth1Backend: Eth1ChainBackend + 'static, TEthSpec: EthSpec + 'static, TEventHandler: EventHandler + 'static, { /// Returns a new builder. /// /// The `_eth_spec_instance` parameter is only supplied to make concrete the `TEthSpec` trait. /// This should generally be either the `MinimalEthSpec` or `MainnetEthSpec` types. pub fn new(_eth_spec_instance: TEthSpec) -> Self { Self { store: None, store_migrator: None, finalized_checkpoint: None, genesis_block_root: None, op_pool: None, fork_choice: None, eth1_chain: None, event_handler: None, slot_clock: None, persisted_beacon_chain: None, head_tracker: None, spec: TEthSpec::default_spec(), log: None, } } /// Override the default spec (as defined by `TEthSpec`). /// /// This method should generally be called immediately after `Self::new` to ensure components /// are started with a consistent spec. pub fn custom_spec(mut self, spec: ChainSpec) -> Self { self.spec = spec; self } /// Sets the store (database). /// /// Should generally be called early in the build chain. pub fn store(mut self, store: Arc) -> Self { self.store = Some(store); self } /// Sets the store migrator. pub fn store_migrator(mut self, store_migrator: TStoreMigrator) -> Self { self.store_migrator = Some(store_migrator); self } /// Sets the logger. /// /// Should generally be called early in the build chain. pub fn logger(mut self, logger: Logger) -> Self { self.log = Some(logger); self } /// Attempt to load an existing chain from the builder's `Store`. /// /// May initialize several components; including the op_pool and finalized checkpoints. pub fn resume_from_db(mut self, config: Eth1Config) -> Result { let log = self .log .as_ref() .ok_or_else(|| "resume_from_db requires a log".to_string())?; info!( log, "Starting beacon chain"; "method" => "resume" ); let store = self .store .clone() .ok_or_else(|| "load_from_store requires a store.".to_string())?; let key = Hash256::from_slice(&BEACON_CHAIN_DB_KEY.as_bytes()); let p: PersistedBeaconChain< Witness, > = match store.get(&key) { Err(e) => { return Err(format!( "DB error when reading persisted beacon chain: {:?}", e )) } Ok(None) => return Err("No persisted beacon chain found in store".into()), Ok(Some(p)) => p, }; self.op_pool = Some( p.op_pool .clone() .into_operation_pool(&p.canonical_head.beacon_state, &self.spec), ); self.finalized_checkpoint = Some(p.finalized_checkpoint.clone()); self.genesis_block_root = Some(p.genesis_block_root); self.head_tracker = Some( HeadTracker::from_ssz_container(&p.ssz_head_tracker) .map_err(|e| format!("Failed to decode head tracker for database: {:?}", e))?, ); self.eth1_chain = match &p.eth1_cache { Some(cache) => Some(Eth1Chain::from_ssz_container(cache, config, store, log)?), None => None, }; self.persisted_beacon_chain = Some(p); Ok(self) } /// Starts a new chain from a genesis state. pub fn genesis_state( mut self, mut beacon_state: BeaconState, ) -> Result { let store = self .store .clone() .ok_or_else(|| "genesis_state requires a store")?; let beacon_block = genesis_block(&mut beacon_state, &self.spec)?; beacon_state .build_all_caches(&self.spec) .map_err(|e| format!("Failed to build genesis state caches: {:?}", e))?; let beacon_state_root = beacon_block.message.state_root; let beacon_block_root = beacon_block.canonical_root(); self.genesis_block_root = Some(beacon_block_root); store .put_state(&beacon_state_root, beacon_state.clone()) .map_err(|e| format!("Failed to store genesis state: {:?}", e))?; store .put(&beacon_block_root, &beacon_block) .map_err(|e| format!("Failed to store genesis block: {:?}", e))?; // Store the genesis block under the `ZERO_HASH` key. store.put(&Hash256::zero(), &beacon_block).map_err(|e| { format!( "Failed to store genesis block under 0x00..00 alias: {:?}", e ) })?; self.finalized_checkpoint = Some(CheckPoint { beacon_block_root, beacon_block, beacon_state_root, beacon_state, }); Ok(self.empty_op_pool()) } /// Sets the `BeaconChain` eth1 backend. pub fn eth1_backend(mut self, backend: Option) -> Self { self.eth1_chain = backend.map(Eth1Chain::new); self } /// Sets the `BeaconChain` event handler backend. /// /// For example, provide `WebSocketSender` as a `handler`. pub fn event_handler(mut self, handler: TEventHandler) -> Self { self.event_handler = Some(handler); self } /// Sets the `BeaconChain` slot clock. /// /// For example, provide `SystemTimeSlotClock` as a `clock`. pub fn slot_clock(mut self, clock: TSlotClock) -> Self { self.slot_clock = Some(clock); self } /// Creates a new, empty operation pool. fn empty_op_pool(mut self) -> Self { self.op_pool = Some(OperationPool::new()); self } /// Consumes `self`, returning a `BeaconChain` if all required parameters have been supplied. /// /// An error will be returned at runtime if all required parameters have not been configured. /// /// Will also raise ambiguous type errors at compile time if some parameters have not been /// configured. #[allow(clippy::type_complexity)] // I think there's nothing to be gained here from a type alias. pub fn build( self, ) -> Result< BeaconChain< Witness, >, String, > { let log = self .log .ok_or_else(|| "Cannot build without a logger".to_string())?; // If this beacon chain is being loaded from disk, use the stored head. Otherwise, just use // the finalized checkpoint (which is probably genesis). let mut canonical_head = if let Some(persisted_beacon_chain) = self.persisted_beacon_chain { persisted_beacon_chain.canonical_head } else { self.finalized_checkpoint .ok_or_else(|| "Cannot build without a state".to_string())? }; canonical_head .beacon_state .build_all_caches(&self.spec) .map_err(|e| format!("Failed to build state caches: {:?}", e))?; if canonical_head.beacon_block.state_root() != canonical_head.beacon_state_root { return Err("beacon_block.state_root != beacon_state".to_string()); } let beacon_chain = BeaconChain { spec: self.spec, store: self .store .ok_or_else(|| "Cannot build without store".to_string())?, store_migrator: self .store_migrator .ok_or_else(|| "Cannot build without store migrator".to_string())?, slot_clock: self .slot_clock .ok_or_else(|| "Cannot build without slot clock".to_string())?, op_pool: self .op_pool .ok_or_else(|| "Cannot build without op pool".to_string())?, eth1_chain: self.eth1_chain, canonical_head: TimeoutRwLock::new(canonical_head), genesis_block_root: self .genesis_block_root .ok_or_else(|| "Cannot build without a genesis block root".to_string())?, fork_choice: self .fork_choice .ok_or_else(|| "Cannot build without a fork choice".to_string())?, event_handler: self .event_handler .ok_or_else(|| "Cannot build without an event handler".to_string())?, head_tracker: self.head_tracker.unwrap_or_default(), log: log.clone(), }; let head = beacon_chain .head() .map_err(|e| format!("Failed to get head: {:?}", e))?; info!( log, "Beacon chain initialized"; "head_state" => format!("{}", head.beacon_state_root), "head_block" => format!("{}", head.beacon_block_root), "head_slot" => format!("{}", head.beacon_block.slot()), ); Ok(beacon_chain) } } impl BeaconChainBuilder< Witness, > where TStore: Store + 'static, TStoreMigrator: store::Migrate + 'static, TSlotClock: SlotClock + 'static, TEth1Backend: Eth1ChainBackend + 'static, TEthSpec: EthSpec + 'static, TEventHandler: EventHandler + 'static, { /// Initializes a fork choice with the `ThreadSafeReducedTree` backend. /// /// If this builder is being "resumed" from disk, then rebuild the last fork choice stored to /// the database. Otherwise, create a new, empty fork choice. pub fn reduced_tree_fork_choice(mut self) -> Result { let fork_choice = if let Some(persisted_beacon_chain) = &self.persisted_beacon_chain { ForkChoice::from_ssz_container(persisted_beacon_chain.fork_choice.clone()) .map_err(|e| format!("Unable to decode fork choice from db: {:?}", e))? } else { let finalized_checkpoint = &self .finalized_checkpoint .as_ref() .ok_or_else(|| "fork_choice_backend requires a finalized_checkpoint")?; let genesis_block_root = self .genesis_block_root .ok_or_else(|| "fork_choice_backend requires a genesis_block_root")?; let backend = ProtoArrayForkChoice::new( finalized_checkpoint.beacon_block.message.slot, // Note: here we set the `justified_epoch` to be the same as the epoch of the // finalized checkpoint. Whilst this finalized checkpoint may actually point to // a _later_ justified checkpoint, that checkpoint won't yet exist in the fork // choice. finalized_checkpoint.beacon_state.current_epoch(), finalized_checkpoint.beacon_state.current_epoch(), finalized_checkpoint.beacon_block_root, )?; ForkChoice::new( backend, genesis_block_root, &finalized_checkpoint.beacon_state, ) }; self.fork_choice = Some(fork_choice); Ok(self) } } impl BeaconChainBuilder< Witness< TStore, TStoreMigrator, TSlotClock, CachingEth1Backend, TEthSpec, TEventHandler, >, > where TStore: Store + 'static, TStoreMigrator: store::Migrate + 'static, TSlotClock: SlotClock + 'static, TEthSpec: EthSpec + 'static, TEventHandler: EventHandler + 'static, { /// Sets the `BeaconChain` eth1 back-end to `CachingEth1Backend`. pub fn caching_eth1_backend(self, backend: CachingEth1Backend) -> Self { self.eth1_backend(Some(backend)) } /// Do not use any eth1 backend. The client will not be able to produce beacon blocks. pub fn no_eth1_backend(self) -> Self { self.eth1_backend(None) } /// Sets the `BeaconChain` eth1 back-end to produce predictably junk data when producing blocks. pub fn dummy_eth1_backend(mut self) -> Result { let log = self .log .as_ref() .ok_or_else(|| "dummy_eth1_backend requires a log".to_string())?; let store = self .store .clone() .ok_or_else(|| "dummy_eth1_backend requires a store.".to_string())?; let backend = CachingEth1Backend::new(Eth1Config::default(), log.clone(), store); let mut eth1_chain = Eth1Chain::new(backend); eth1_chain.use_dummy_backend = true; self.eth1_chain = Some(eth1_chain); Ok(self) } } impl BeaconChainBuilder< Witness, > where TStore: Store + 'static, TStoreMigrator: store::Migrate + 'static, TEth1Backend: Eth1ChainBackend + 'static, TEthSpec: EthSpec + 'static, TEventHandler: EventHandler + 'static, { /// Sets the `BeaconChain` slot clock to `TestingSlotClock`. /// /// Requires the state to be initialized. pub fn testing_slot_clock(self, slot_duration: Duration) -> Result { let genesis_time = self .finalized_checkpoint .as_ref() .ok_or_else(|| "testing_slot_clock requires an initialized state")? .beacon_state .genesis_time; let slot_clock = TestingSlotClock::new( Slot::new(0), Duration::from_secs(genesis_time), slot_duration, ); Ok(self.slot_clock(slot_clock)) } } impl BeaconChainBuilder< Witness< TStore, TStoreMigrator, TSlotClock, TEth1Backend, TEthSpec, NullEventHandler, >, > where TStore: Store + 'static, TStoreMigrator: store::Migrate + 'static, TSlotClock: SlotClock + 'static, TEth1Backend: Eth1ChainBackend + 'static, TEthSpec: EthSpec + 'static, { /// Sets the `BeaconChain` event handler to `NullEventHandler`. pub fn null_event_handler(self) -> Self { let handler = NullEventHandler::default(); self.event_handler(handler) } } fn genesis_block( genesis_state: &mut BeaconState, spec: &ChainSpec, ) -> Result, String> { let mut genesis_block = SignedBeaconBlock { message: BeaconBlock::empty(&spec), // Empty signature, which should NEVER be read. This isn't to-spec, but makes the genesis // block consistent with every other block. signature: Signature::empty_signature(), }; genesis_block.message.state_root = genesis_state .update_tree_hash_cache() .map_err(|e| format!("Error hashing genesis state: {:?}", e))?; Ok(genesis_block) } #[cfg(not(debug_assertions))] #[cfg(test)] mod test { use super::*; use eth2_hashing::hash; use genesis::{generate_deterministic_keypairs, interop_genesis_state}; use sloggers::{null::NullLoggerBuilder, Build}; use ssz::Encode; use std::time::Duration; use store::{migrate::NullMigrator, MemoryStore}; use types::{EthSpec, MinimalEthSpec, Slot}; type TestEthSpec = MinimalEthSpec; fn get_logger() -> Logger { let builder = NullLoggerBuilder; builder.build().expect("should build logger") } #[test] fn recent_genesis() { let validator_count = 8; let genesis_time = 13_371_337; let log = get_logger(); let store = Arc::new(MemoryStore::open()); let spec = MinimalEthSpec::default_spec(); let genesis_state = interop_genesis_state( &generate_deterministic_keypairs(validator_count), genesis_time, &spec, ) .expect("should create interop genesis state"); let chain = BeaconChainBuilder::new(MinimalEthSpec) .logger(log.clone()) .store(store) .store_migrator(NullMigrator) .genesis_state(genesis_state) .expect("should build state using recent genesis") .dummy_eth1_backend() .expect("should build the dummy eth1 backend") .null_event_handler() .testing_slot_clock(Duration::from_secs(1)) .expect("should configure testing slot clock") .reduced_tree_fork_choice() .expect("should add fork choice to builder") .build() .expect("should build"); let head = chain.head().expect("should get head"); let state = head.beacon_state; let block = head.beacon_block; assert_eq!(state.slot, Slot::new(0), "should start from genesis"); assert_eq!( state.genesis_time, 13_371_337, "should have the correct genesis time" ); assert_eq!( block.state_root(), state.canonical_root(), "block should have correct state root" ); assert_eq!( chain .store .get_block(&Hash256::zero()) .expect("should read db") .expect("should find genesis block"), block, "should store genesis block under zero hash alias" ); assert_eq!( state.validators.len(), validator_count, "should have correct validator count" ); assert_eq!( chain.genesis_block_root, block.canonical_root(), "should have correct genesis block root" ); } #[test] fn interop_state() { let validator_count = 16; let genesis_time = 42; let spec = &TestEthSpec::default_spec(); let keypairs = generate_deterministic_keypairs(validator_count); let state = interop_genesis_state::(&keypairs, genesis_time, spec) .expect("should build state"); assert_eq!( state.eth1_data.block_hash, Hash256::from_slice(&[0x42; 32]), "eth1 block hash should be co-ordinated junk" ); assert_eq!( state.genesis_time, genesis_time, "genesis time should be as specified" ); for b in &state.balances { assert_eq!( *b, spec.max_effective_balance, "validator balances should be max effective balance" ); } for v in &state.validators { let creds = v.withdrawal_credentials.as_bytes(); assert_eq!( creds[0], spec.bls_withdrawal_prefix_byte, "first byte of withdrawal creds should be bls prefix" ); assert_eq!( &creds[1..], &hash(&v.pubkey.as_ssz_bytes())[1..], "rest of withdrawal creds should be pubkey hash" ) } assert_eq!( state.balances.len(), validator_count, "validator balances len should be correct" ); assert_eq!( state.validators.len(), validator_count, "validator count should be correct" ); } }