Merge pull request #516 from sigp/interop

Interop
This commit is contained in:
Paul Hauner 2019-09-26 14:02:00 +10:00 committed by GitHub
commit 2399b9bbe0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
198 changed files with 10117 additions and 4598 deletions

3
.gitmodules vendored
View File

@ -1,3 +0,0 @@
[submodule "tests/ef_tests/eth2.0-spec-tests"]
path = tests/ef_tests/eth2.0-spec-tests
url = https://github.com/ethereum/eth2.0-spec-tests

View File

@ -12,6 +12,7 @@ members = [
"eth2/utils/logging",
"eth2/utils/eth2_hashing",
"eth2/utils/lighthouse_metrics",
"eth2/utils/lighthouse_bootstrap",
"eth2/utils/merkle_proof",
"eth2/utils/int_to_bytes",
"eth2/utils/serde_hex",
@ -32,8 +33,9 @@ members = [
"beacon_node/rpc",
"beacon_node/version",
"beacon_node/beacon_chain",
"beacon_node/websocket_server",
"tests/ef_tests",
"tests/cli_util",
"lcli",
"protos",
"validator_client",
"account_manager",

31
Makefile Normal file
View File

@ -0,0 +1,31 @@
TESTS_TAG := v0.8.3
TESTS = general minimal mainnet
TESTS_BASE_DIR := ./tests/ef_tests
REPO_NAME := eth2.0-spec-tests
OUTPUT_DIR := $(TESTS_BASE_DIR)/$(REPO_NAME)
BASE_URL := https://github.com/ethereum/$(REPO_NAME)/releases/download/$(SPEC_VERSION)
release:
cargo build --all --release
clean_ef_tests:
rm -r $(OUTPUT_DIR)
ef_tests: download_tests extract_tests
mkdir $(OUTPUT_DIR)
for test in $(TESTS); do \
tar -C $(OUTPUT_DIR) -xvf $(TESTS_BASE_DIR)/$$test.tar ;\
rm $(TESTS_BASE_DIR)/$$test.tar ;\
done
extract_tests:
for test in $(TESTS); do \
gzip -df $(TESTS_BASE_DIR)/$$test.tar.gz ;\
done
download_tests:
for test in $(TESTS); do \
wget -P $(TESTS_BASE_DIR) $(BASE_URL)/$$test.tar.gz; \
done

View File

@ -125,9 +125,13 @@ fn main() {
}
}
}
_ => panic!(
"The account manager must be run with a subcommand. See help for more information."
),
_ => {
crit!(
log,
"The account manager must be run with a subcommand. See help for more information."
);
return;
}
}
}

View File

@ -6,11 +6,14 @@ edition = "2018"
[dependencies]
eth2_config = { path = "../eth2/utils/eth2_config" }
lighthouse_bootstrap = { path = "../eth2/utils/lighthouse_bootstrap" }
beacon_chain = { path = "beacon_chain" }
types = { path = "../eth2/types" }
store = { path = "./store" }
client = { path = "client" }
version = { path = "version" }
clap = "2.32.0"
rand = "0.7"
slog = { version = "^2.2.3" , features = ["max_level_trace", "release_max_level_trace"] }
slog-term = "^2.4.0"
slog-async = "^2.3.0"

View File

@ -4,19 +4,29 @@ version = "0.1.0"
authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com>"]
edition = "2018"
[features]
write_ssz_files = [] # Writes debugging .ssz files to /tmp during block processing.
[dependencies]
eth2_config = { path = "../../eth2/utils/eth2_config" }
merkle_proof = { path = "../../eth2/utils/merkle_proof" }
store = { path = "../store" }
parking_lot = "0.7"
lazy_static = "1.3.0"
lighthouse_metrics = { path = "../../eth2/utils/lighthouse_metrics" }
lighthouse_bootstrap = { path = "../../eth2/utils/lighthouse_bootstrap" }
log = "0.4"
operation_pool = { path = "../../eth2/operation_pool" }
rayon = "1.0"
serde = "1.0"
serde_derive = "1.0"
serde_yaml = "0.8"
serde_json = "^1.0"
slog = { version = "^2.2.3" , features = ["max_level_trace"] }
sloggers = { version = "^0.3" }
slot_clock = { path = "../../eth2/utils/slot_clock" }
eth2_hashing = { path = "../../eth2/utils/eth2_hashing" }
eth2_ssz = "0.1"
eth2_ssz_derive = "0.1"
state_processing = { path = "../../eth2/state_processing" }

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,334 @@
use crate::{BeaconChain, BeaconChainTypes};
use eth2_hashing::hash;
use lighthouse_bootstrap::Bootstrapper;
use merkle_proof::MerkleTree;
use rayon::prelude::*;
use slog::Logger;
use ssz::{Decode, Encode};
use state_processing::initialize_beacon_state_from_eth1;
use std::fs::File;
use std::io::prelude::*;
use std::path::PathBuf;
use std::sync::Arc;
use std::time::SystemTime;
use tree_hash::{SignedRoot, TreeHash};
use types::{
BeaconBlock, BeaconState, ChainSpec, Deposit, DepositData, Domain, EthSpec, Fork, Hash256,
Keypair, PublicKey, Signature,
};
enum BuildStrategy<T: BeaconChainTypes> {
FromGenesis {
genesis_state: Box<BeaconState<T::EthSpec>>,
genesis_block: Box<BeaconBlock<T::EthSpec>>,
},
LoadFromStore,
}
pub struct BeaconChainBuilder<T: BeaconChainTypes> {
build_strategy: BuildStrategy<T>,
spec: ChainSpec,
log: Logger,
}
impl<T: BeaconChainTypes> BeaconChainBuilder<T> {
pub fn recent_genesis(
keypairs: &[Keypair],
minutes: u64,
spec: ChainSpec,
log: Logger,
) -> Result<Self, String> {
Self::quick_start(recent_genesis_time(minutes), keypairs, spec, log)
}
pub fn quick_start(
genesis_time: u64,
keypairs: &[Keypair],
spec: ChainSpec,
log: Logger,
) -> Result<Self, String> {
let genesis_state = interop_genesis_state(keypairs, genesis_time, &spec)?;
Ok(Self::from_genesis_state(genesis_state, spec, log))
}
pub fn yaml_state(file: &PathBuf, spec: ChainSpec, log: Logger) -> Result<Self, String> {
let file = File::open(file.clone())
.map_err(|e| format!("Unable to open YAML genesis state file {:?}: {:?}", file, e))?;
let genesis_state = serde_yaml::from_reader(file)
.map_err(|e| format!("Unable to parse YAML genesis state file: {:?}", e))?;
Ok(Self::from_genesis_state(genesis_state, spec, log))
}
pub fn ssz_state(file: &PathBuf, spec: ChainSpec, log: Logger) -> Result<Self, String> {
let mut file = File::open(file.clone())
.map_err(|e| format!("Unable to open SSZ genesis state file {:?}: {:?}", file, e))?;
let mut bytes = vec![];
file.read_to_end(&mut bytes)
.map_err(|e| format!("Failed to read SSZ file: {:?}", e))?;
let genesis_state = BeaconState::from_ssz_bytes(&bytes)
.map_err(|e| format!("Unable to parse SSZ genesis state file: {:?}", e))?;
Ok(Self::from_genesis_state(genesis_state, spec, log))
}
pub fn json_state(file: &PathBuf, spec: ChainSpec, log: Logger) -> Result<Self, String> {
let file = File::open(file.clone())
.map_err(|e| format!("Unable to open JSON genesis state file {:?}: {:?}", file, e))?;
let genesis_state = serde_json::from_reader(file)
.map_err(|e| format!("Unable to parse JSON genesis state file: {:?}", e))?;
Ok(Self::from_genesis_state(genesis_state, spec, log))
}
pub fn http_bootstrap(server: &str, spec: ChainSpec, log: Logger) -> Result<Self, String> {
let bootstrapper = Bootstrapper::connect(server.to_string(), &log)
.map_err(|e| format!("Failed to initialize bootstrap client: {}", e))?;
let (genesis_state, genesis_block) = bootstrapper
.genesis()
.map_err(|e| format!("Failed to bootstrap genesis state: {}", e))?;
Ok(Self {
build_strategy: BuildStrategy::FromGenesis {
genesis_block: Box::new(genesis_block),
genesis_state: Box::new(genesis_state),
},
spec,
log,
})
}
fn from_genesis_state(
genesis_state: BeaconState<T::EthSpec>,
spec: ChainSpec,
log: Logger,
) -> Self {
Self {
build_strategy: BuildStrategy::FromGenesis {
genesis_block: Box::new(genesis_block(&genesis_state, &spec)),
genesis_state: Box::new(genesis_state),
},
spec,
log,
}
}
pub fn from_store(spec: ChainSpec, log: Logger) -> Self {
Self {
build_strategy: BuildStrategy::LoadFromStore,
spec,
log,
}
}
pub fn build(
self,
store: Arc<T::Store>,
eth1_backend: T::Eth1Chain,
event_handler: T::EventHandler,
) -> Result<BeaconChain<T>, String> {
Ok(match self.build_strategy {
BuildStrategy::LoadFromStore => {
BeaconChain::from_store(store, eth1_backend, event_handler, self.spec, self.log)
.map_err(|e| format!("Error loading BeaconChain from database: {:?}", e))?
.ok_or_else(|| format!("Unable to find exising BeaconChain in database."))?
}
BuildStrategy::FromGenesis {
genesis_block,
genesis_state,
} => BeaconChain::from_genesis(
store,
eth1_backend,
event_handler,
genesis_state.as_ref().clone(),
genesis_block.as_ref().clone(),
self.spec,
self.log,
)
.map_err(|e| format!("Failed to initialize new beacon chain: {:?}", e))?,
})
}
}
fn genesis_block<T: EthSpec>(genesis_state: &BeaconState<T>, spec: &ChainSpec) -> BeaconBlock<T> {
let mut genesis_block = BeaconBlock::empty(&spec);
genesis_block.state_root = genesis_state.canonical_root();
genesis_block
}
/// Builds a genesis state as defined by the Eth2 interop procedure (see below).
///
/// Reference:
/// https://github.com/ethereum/eth2.0-pm/tree/6e41fcf383ebeb5125938850d8e9b4e9888389b4/interop/mocked_start
fn interop_genesis_state<T: EthSpec>(
keypairs: &[Keypair],
genesis_time: u64,
spec: &ChainSpec,
) -> Result<BeaconState<T>, String> {
let eth1_block_hash = Hash256::from_slice(&[0x42; 32]);
let eth1_timestamp = 2_u64.pow(40);
let amount = spec.max_effective_balance;
let withdrawal_credentials = |pubkey: &PublicKey| {
let mut credentials = hash(&pubkey.as_ssz_bytes());
credentials[0] = spec.bls_withdrawal_prefix_byte;
Hash256::from_slice(&credentials)
};
let datas = keypairs
.into_par_iter()
.map(|keypair| {
let mut data = DepositData {
withdrawal_credentials: withdrawal_credentials(&keypair.pk),
pubkey: keypair.pk.clone().into(),
amount,
signature: Signature::empty_signature().into(),
};
let domain = spec.get_domain(
spec.genesis_slot.epoch(T::slots_per_epoch()),
Domain::Deposit,
&Fork::default(),
);
data.signature = Signature::new(&data.signed_root()[..], domain, &keypair.sk).into();
data
})
.collect::<Vec<_>>();
let deposit_root_leaves = datas
.par_iter()
.map(|data| Hash256::from_slice(&data.tree_hash_root()))
.collect::<Vec<_>>();
let mut proofs = vec![];
for i in 1..=deposit_root_leaves.len() {
// Note: this implementation is not so efficient.
//
// If `MerkleTree` had a push method, we could just build one tree and sample it instead of
// rebuilding the tree for each deposit.
let tree = MerkleTree::create(
&deposit_root_leaves[0..i],
spec.deposit_contract_tree_depth as usize,
);
let (_, mut proof) = tree.generate_proof(i - 1, spec.deposit_contract_tree_depth as usize);
proof.push(Hash256::from_slice(&int_to_bytes32(i)));
assert_eq!(
proof.len(),
spec.deposit_contract_tree_depth as usize + 1,
"Deposit proof should be correct len"
);
proofs.push(proof);
}
let deposits = datas
.into_par_iter()
.zip(proofs.into_par_iter())
.map(|(data, proof)| (data, proof.into()))
.map(|(data, proof)| Deposit { proof, data })
.collect::<Vec<_>>();
let mut state =
initialize_beacon_state_from_eth1(eth1_block_hash, eth1_timestamp, deposits, spec)
.map_err(|e| format!("Unable to initialize genesis state: {:?}", e))?;
state.genesis_time = genesis_time;
// Invalid all the caches after all the manual state surgery.
state.drop_all_caches();
Ok(state)
}
/// Returns `int` as little-endian bytes with a length of 32.
fn int_to_bytes32(int: usize) -> Vec<u8> {
let mut vec = int.to_le_bytes().to_vec();
vec.resize(32, 0);
vec
}
/// Returns the system time, mod 30 minutes.
///
/// Used for easily creating testnets.
fn recent_genesis_time(minutes: u64) -> u64 {
let now = SystemTime::now()
.duration_since(SystemTime::UNIX_EPOCH)
.unwrap()
.as_secs();
let secs_after_last_period = now.checked_rem(minutes * 60).unwrap_or(0);
now - secs_after_last_period
}
#[cfg(test)]
mod test {
use super::*;
use types::{test_utils::generate_deterministic_keypairs, EthSpec, MinimalEthSpec};
type TestEthSpec = MinimalEthSpec;
#[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::<TestEthSpec>(&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"
);
}
}

View File

@ -1,3 +1,4 @@
use crate::eth1_chain::Error as Eth1ChainError;
use crate::fork_choice::Error as ForkChoiceError;
use state_processing::per_block_processing::errors::AttestationValidationError;
use state_processing::BlockProcessingError;
@ -23,6 +24,8 @@ pub enum BeaconChainError {
previous_epoch: Epoch,
new_epoch: Epoch,
},
SlotClockDidNotStart,
NoStateForSlot(Slot),
UnableToFindTargetRoot(Slot),
BeaconStateError(BeaconStateError),
DBInconsistent(String),
@ -31,24 +34,30 @@ pub enum BeaconChainError {
MissingBeaconBlock(Hash256),
MissingBeaconState(Hash256),
SlotProcessingError(SlotProcessingError),
UnableToAdvanceState(String),
NoStateForAttestation {
beacon_block_root: Hash256,
},
AttestationValidationError(AttestationValidationError),
/// Returned when an internal check fails, indicating corrupt data.
InvariantViolated(String),
}
easy_from_to!(SlotProcessingError, BeaconChainError);
easy_from_to!(AttestationValidationError, BeaconChainError);
#[derive(Debug, PartialEq)]
pub enum BlockProductionError {
UnableToGetBlockRootFromState,
UnableToReadSlot,
UnableToProduceAtSlot(Slot),
SlotProcessingError(SlotProcessingError),
BlockProcessingError(BlockProcessingError),
Eth1ChainError(Eth1ChainError),
BeaconStateError(BeaconStateError),
}
easy_from_to!(BlockProcessingError, BlockProductionError);
easy_from_to!(BeaconStateError, BlockProductionError);
easy_from_to!(SlotProcessingError, BlockProductionError);
easy_from_to!(AttestationValidationError, BeaconChainError);
easy_from_to!(Eth1ChainError, BlockProductionError);

View File

@ -0,0 +1,110 @@
use crate::BeaconChainTypes;
use eth2_hashing::hash;
use std::marker::PhantomData;
use types::{BeaconState, Deposit, Eth1Data, EthSpec, Hash256};
type Result<T> = std::result::Result<T, Error>;
/// Holds an `Eth1ChainBackend` and serves requests from the `BeaconChain`.
pub struct Eth1Chain<T: BeaconChainTypes> {
backend: T::Eth1Chain,
}
impl<T: BeaconChainTypes> Eth1Chain<T> {
pub fn new(backend: T::Eth1Chain) -> Self {
Self { backend }
}
/// Returns the `Eth1Data` that should be included in a block being produced for the given
/// `state`.
pub fn eth1_data_for_block_production(
&self,
state: &BeaconState<T::EthSpec>,
) -> Result<Eth1Data> {
self.backend.eth1_data(state)
}
/// Returns a list of `Deposits` that may be included in a block.
///
/// Including all of the returned `Deposits` in a block should _not_ cause it to become
/// invalid.
pub fn deposits_for_block_inclusion(
&self,
state: &BeaconState<T::EthSpec>,
) -> Result<Vec<Deposit>> {
let deposits = self.backend.queued_deposits(state)?;
// TODO: truncate deposits if required.
Ok(deposits)
}
}
#[derive(Debug, PartialEq)]
pub enum Error {
/// Unable to return an Eth1Data for the given epoch.
EpochUnavailable,
/// An error from the backend service (e.g., the web3 data fetcher).
BackendError(String),
}
pub trait Eth1ChainBackend<T: EthSpec>: Sized + Send + Sync {
fn new(server: String) -> Result<Self>;
/// Returns the `Eth1Data` that should be included in a block being produced for the given
/// `state`.
fn eth1_data(&self, beacon_state: &BeaconState<T>) -> Result<Eth1Data>;
/// Returns all `Deposits` between `state.eth1_deposit_index` and
/// `state.eth1_data.deposit_count`.
///
/// # Note:
///
/// It is possible that not all returned `Deposits` can be included in a block. E.g., there may
/// be more than `MAX_DEPOSIT_COUNT` or the churn may be too high.
fn queued_deposits(&self, beacon_state: &BeaconState<T>) -> Result<Vec<Deposit>>;
}
pub struct InteropEth1ChainBackend<T: EthSpec> {
_phantom: PhantomData<T>,
}
impl<T: EthSpec> Eth1ChainBackend<T> for InteropEth1ChainBackend<T> {
fn new(_server: String) -> Result<Self> {
Ok(Self::default())
}
fn eth1_data(&self, state: &BeaconState<T>) -> Result<Eth1Data> {
let current_epoch = state.current_epoch();
let slots_per_voting_period = T::slots_per_eth1_voting_period() as u64;
let current_voting_period: u64 = current_epoch.as_u64() / slots_per_voting_period;
let deposit_root = hash(&int_to_bytes32(current_voting_period));
let block_hash = hash(&deposit_root);
Ok(Eth1Data {
deposit_root: Hash256::from_slice(&deposit_root),
deposit_count: state.eth1_deposit_index,
block_hash: Hash256::from_slice(&block_hash),
})
}
fn queued_deposits(&self, _: &BeaconState<T>) -> Result<Vec<Deposit>> {
Ok(vec![])
}
}
impl<T: EthSpec> Default for InteropEth1ChainBackend<T> {
fn default() -> Self {
Self {
_phantom: PhantomData,
}
}
}
/// Returns `int` as little-endian bytes with a length of 32.
fn int_to_bytes32(int: u64) -> Vec<u8> {
let mut vec = int.to_le_bytes().to_vec();
vec.resize(32, 0);
vec
}

View File

@ -0,0 +1,55 @@
use serde_derive::{Deserialize, Serialize};
use std::marker::PhantomData;
use types::{Attestation, BeaconBlock, Epoch, EthSpec, Hash256};
pub trait EventHandler<T: EthSpec>: Sized + Send + Sync {
fn register(&self, kind: EventKind<T>) -> Result<(), String>;
}
pub struct NullEventHandler<T: EthSpec>(PhantomData<T>);
impl<T: EthSpec> EventHandler<T> for NullEventHandler<T> {
fn register(&self, _kind: EventKind<T>) -> Result<(), String> {
Ok(())
}
}
impl<T: EthSpec> Default for NullEventHandler<T> {
fn default() -> Self {
NullEventHandler(PhantomData)
}
}
#[derive(Debug, Serialize, Deserialize)]
#[serde(
bound = "T: EthSpec",
rename_all = "snake_case",
tag = "event",
content = "data"
)]
pub enum EventKind<T: EthSpec> {
BeaconHeadChanged {
reorg: bool,
current_head_beacon_block_root: Hash256,
previous_head_beacon_block_root: Hash256,
},
BeaconFinalization {
epoch: Epoch,
root: Hash256,
},
BeaconBlockImported {
block_root: Hash256,
block: Box<BeaconBlock<T>>,
},
BeaconBlockRejected {
reason: String,
block: Box<BeaconBlock<T>>,
},
BeaconAttestationImported {
attestation: Box<Attestation<T>>,
},
BeaconAttestationRejected {
reason: String,
attestation: Box<Attestation<T>>,
},
}

View File

@ -199,6 +199,14 @@ impl<T: BeaconChainTypes> ForkChoice<T> {
self.backend.latest_message(validator_index)
}
/// Runs an integrity verification function on the underlying fork choice algorithm.
///
/// Returns `Ok(())` if the underlying fork choice has maintained it's integrity,
/// `Err(description)` otherwise.
pub fn verify_integrity(&self) -> core::result::Result<(), String> {
self.backend.verify_integrity()
}
/// Inform the fork choice that the given block (and corresponding root) have been finalized so
/// it may prune it's storage.
///

View File

@ -3,8 +3,11 @@
extern crate lazy_static;
mod beacon_chain;
mod beacon_chain_builder;
mod checkpoint;
mod errors;
mod eth1_chain;
pub mod events;
mod fork_choice;
mod iter;
mod metrics;
@ -16,6 +19,8 @@ pub use self::beacon_chain::{
};
pub use self::checkpoint::CheckPoint;
pub use self::errors::{BeaconChainError, BlockProductionError};
pub use beacon_chain_builder::BeaconChainBuilder;
pub use eth1_chain::{Eth1ChainBackend, InteropEth1ChainBackend};
pub use lmd_ghost;
pub use metrics::scrape_for_metrics;
pub use parking_lot;

View File

@ -3,7 +3,7 @@ use operation_pool::PersistedOperationPool;
use ssz::{Decode, Encode};
use ssz_derive::{Decode, Encode};
use store::{DBColumn, Error as StoreError, StoreItem};
use types::{BeaconState, Hash256};
use types::Hash256;
/// 32-byte key for accessing the `PersistedBeaconChain`.
pub const BEACON_CHAIN_DB_KEY: &str = "PERSISTEDBEACONCHAINPERSISTEDBEA";
@ -13,7 +13,6 @@ pub struct PersistedBeaconChain<T: BeaconChainTypes> {
pub canonical_head: CheckPoint<T::EthSpec>,
pub op_pool: PersistedOperationPool<T::EthSpec>,
pub genesis_block_root: Hash256,
pub state: BeaconState<T::EthSpec>,
}
impl<T: BeaconChainTypes> StoreItem for PersistedBeaconChain<T> {

View File

@ -1,23 +1,28 @@
use crate::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
use crate::{
events::NullEventHandler, AttestationProcessingOutcome, BeaconChain, BeaconChainBuilder,
BeaconChainTypes, BlockProcessingOutcome, InteropEth1ChainBackend,
};
use lmd_ghost::LmdGhost;
use rayon::prelude::*;
use sloggers::{null::NullLoggerBuilder, Build};
use slot_clock::SlotClock;
use sloggers::{terminal::TerminalLoggerBuilder, types::Severity, Build};
use slot_clock::TestingSlotClock;
use state_processing::per_slot_processing;
use std::marker::PhantomData;
use std::sync::Arc;
use store::MemoryStore;
use store::Store;
use tree_hash::{SignedRoot, TreeHash};
use types::{
test_utils::TestingBeaconStateBuilder, AggregateSignature, Attestation,
AttestationDataAndCustodyBit, BeaconBlock, BeaconState, BitList, ChainSpec, Domain, EthSpec,
Hash256, Keypair, RelativeEpoch, SecretKey, Signature, Slot,
AggregateSignature, Attestation, AttestationDataAndCustodyBit, BeaconBlock, BeaconState,
BitList, ChainSpec, Domain, EthSpec, Hash256, Keypair, RelativeEpoch, SecretKey, Signature,
Slot,
};
pub use types::test_utils::generate_deterministic_keypairs;
pub use crate::persisted_beacon_chain::{PersistedBeaconChain, BEACON_CHAIN_DB_KEY};
pub const HARNESS_GENESIS_TIME: u64 = 1567552690; // 4th September 2019
/// Indicates how the `BeaconChainHarness` should produce blocks.
#[derive(Clone, Copy, Debug)]
pub enum BlockStrategy {
@ -61,7 +66,9 @@ where
type Store = MemoryStore;
type SlotClock = TestingSlotClock;
type LmdGhost = L;
type Eth1Chain = InteropEth1ChainBackend<E>;
type EthSpec = E;
type EventHandler = NullEventHandler<E>;
}
/// A testing harness which can instantiate a `BeaconChain` and populate it with blocks and
@ -84,53 +91,25 @@ where
E: EthSpec,
{
/// Instantiate a new harness with `validator_count` initial validators.
pub fn new(validator_count: usize) -> Self {
let state_builder = TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(
validator_count,
&E::default_spec(),
);
let (genesis_state, keypairs) = state_builder.build();
Self::from_state_and_keypairs(genesis_state, keypairs)
}
/// Instantiate a new harness with an initial validator for each key supplied.
pub fn from_keypairs(keypairs: Vec<Keypair>) -> Self {
let state_builder = TestingBeaconStateBuilder::from_keypairs(keypairs, &E::default_spec());
let (genesis_state, keypairs) = state_builder.build();
Self::from_state_and_keypairs(genesis_state, keypairs)
}
/// Instantiate a new harness with the given genesis state and a keypair for each of the
/// initial validators in the given state.
pub fn from_state_and_keypairs(genesis_state: BeaconState<E>, keypairs: Vec<Keypair>) -> Self {
pub fn new(keypairs: Vec<Keypair>) -> Self {
let spec = E::default_spec();
let log = TerminalLoggerBuilder::new()
.level(Severity::Warning)
.build()
.expect("logger should build");
let store = Arc::new(MemoryStore::open());
let mut genesis_block = BeaconBlock::empty(&spec);
genesis_block.state_root = Hash256::from_slice(&genesis_state.tree_hash_root());
let builder = NullLoggerBuilder;
let log = builder.build().expect("logger should build");
// Slot clock
let slot_clock = TestingSlotClock::new(
spec.genesis_slot,
genesis_state.genesis_time,
spec.seconds_per_slot,
);
let chain = BeaconChain::from_genesis(
store,
slot_clock,
genesis_state,
genesis_block,
spec.clone(),
log,
)
.expect("Terminate if beacon chain generation fails");
let chain =
BeaconChainBuilder::quick_start(HARNESS_GENESIS_TIME, &keypairs, spec.clone(), log)
.unwrap_or_else(|e| panic!("Failed to create beacon chain builder: {}", e))
.build(
store.clone(),
InteropEth1ChainBackend::default(),
NullEventHandler::default(),
)
.unwrap_or_else(|e| panic!("Failed to build beacon chain: {}", e));
Self {
chain,
@ -144,7 +123,6 @@ where
/// Does not produce blocks or attestations.
pub fn advance_slot(&self) {
self.chain.slot_clock.advance_slot();
self.chain.catchup_state().expect("should catchup state");
}
/// Extend the `BeaconChain` with some blocks and attestations. Returns the root of the
@ -166,26 +144,27 @@ where
// Determine the slot for the first block (or skipped block).
let state_slot = match block_strategy {
BlockStrategy::OnCanonicalHead => {
self.chain.read_slot_clock().expect("should know slot") - 1
self.chain.slot().expect("should have a slot") - 1
}
BlockStrategy::ForkCanonicalChainAt { previous_slot, .. } => previous_slot,
};
self.get_state_at_slot(state_slot)
self.chain
.state_at_slot(state_slot)
.expect("should find state for slot")
.clone()
};
// Determine the first slot where a block should be built.
let mut slot = match block_strategy {
BlockStrategy::OnCanonicalHead => {
self.chain.read_slot_clock().expect("should know slot")
}
BlockStrategy::OnCanonicalHead => self.chain.slot().expect("should have a slot"),
BlockStrategy::ForkCanonicalChainAt { first_slot, .. } => first_slot,
};
let mut head_block_root = None;
for _ in 0..num_blocks {
while self.chain.read_slot_clock().expect("should have a slot") < slot {
while self.chain.slot().expect("should have a slot") < slot {
self.advance_slot();
}
@ -211,21 +190,6 @@ where
head_block_root.expect("did not produce any blocks")
}
fn get_state_at_slot(&self, state_slot: Slot) -> BeaconState<E> {
let state_root = self
.chain
.rev_iter_state_roots()
.find(|(_hash, slot)| *slot == state_slot)
.map(|(hash, _slot)| hash)
.expect("could not find state root");
self.chain
.store
.get(&state_root)
.expect("should read db")
.expect("should find state root")
}
/// Returns a newly created block, signed by the proposer for the given slot.
fn build_block(
&self,
@ -299,9 +263,14 @@ where
)
.into_iter()
.for_each(|attestation| {
self.chain
match self
.chain
.process_attestation(attestation)
.expect("should process attestation");
.expect("should not error during attestation processing")
{
AttestationProcessingOutcome::Processed => (),
other => panic!("did not successfully process attestation: {:?}", other),
}
});
}

View File

@ -3,11 +3,14 @@
#[macro_use]
extern crate lazy_static;
use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, CommonTypes, PersistedBeaconChain,
BEACON_CHAIN_DB_KEY,
};
use beacon_chain::AttestationProcessingOutcome;
use beacon_chain::{
test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, CommonTypes, PersistedBeaconChain,
BEACON_CHAIN_DB_KEY,
},
BlockProcessingOutcome,
};
use lmd_ghost::ThreadSafeReducedTree;
use rand::Rng;
use store::{MemoryStore, Store};
@ -25,7 +28,7 @@ lazy_static! {
type TestForkChoice = ThreadSafeReducedTree<MemoryStore, MinimalEthSpec>;
fn get_harness(validator_count: usize) -> BeaconChainHarness<TestForkChoice, MinimalEthSpec> {
let harness = BeaconChainHarness::from_keypairs(KEYPAIRS[0..validator_count].to_vec());
let harness = BeaconChainHarness::new(KEYPAIRS[0..validator_count].to_vec());
harness.advance_slot();
@ -322,7 +325,9 @@ fn roundtrip_operation_pool() {
let p: PersistedBeaconChain<CommonTypes<TestForkChoice, MinimalEthSpec>> =
harness.chain.store.get(&key).unwrap().unwrap();
let restored_op_pool = p.op_pool.into_operation_pool(&p.state, &harness.spec);
let restored_op_pool = p
.op_pool
.into_operation_pool(&p.canonical_head.beacon_state, &harness.spec);
assert_eq!(harness.chain.op_pool, restored_op_pool);
}
@ -459,3 +464,48 @@ fn free_attestations_added_to_fork_choice_all_updated() {
}
}
}
fn run_skip_slot_test(skip_slots: u64) {
let num_validators = 8;
let harness_a = get_harness(num_validators);
let harness_b = get_harness(num_validators);
for _ in 0..skip_slots {
harness_a.advance_slot();
harness_b.advance_slot();
}
harness_a.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
// No attestation required for test.
AttestationStrategy::SomeValidators(vec![]),
);
assert_eq!(
harness_a.chain.head().beacon_block.slot,
Slot::new(skip_slots + 1)
);
assert_eq!(harness_b.chain.head().beacon_block.slot, Slot::new(0));
assert_eq!(
harness_b
.chain
.process_block(harness_a.chain.head().beacon_block.clone()),
Ok(BlockProcessingOutcome::Processed {
block_root: harness_a.chain.head().beacon_block_root
})
);
assert_eq!(
harness_b.chain.head().beacon_block.slot,
Slot::new(skip_slots + 1)
);
}
#[test]
fn produces_and_processes_with_genesis_skip_slots() {
for i in 0..MinimalEthSpec::slots_per_epoch() * 4 {
run_skip_slot_test(i)
}
}

View File

@ -6,10 +6,12 @@ edition = "2018"
[dependencies]
beacon_chain = { path = "../beacon_chain" }
store = { path = "../store" }
network = { path = "../network" }
eth2-libp2p = { path = "../eth2-libp2p" }
rpc = { path = "../rpc" }
rest_api = { path = "../rest_api" }
websocket_server = { path = "../websocket_server" }
prometheus = "^0.6"
types = { path = "../../eth2/types" }
tree_hash = "0.1"

View File

@ -1,169 +0,0 @@
use crate::bootstrapper::Bootstrapper;
use crate::error::Result;
use crate::{config::GenesisState, ClientConfig};
use beacon_chain::{
lmd_ghost::{LmdGhost, ThreadSafeReducedTree},
slot_clock::SystemTimeSlotClock,
store::Store,
BeaconChain, BeaconChainTypes,
};
use slog::{crit, info, Logger};
use slot_clock::SlotClock;
use std::fs::File;
use std::marker::PhantomData;
use std::sync::Arc;
use std::time::SystemTime;
use tree_hash::TreeHash;
use types::{
test_utils::TestingBeaconStateBuilder, BeaconBlock, BeaconState, ChainSpec, EthSpec, Hash256,
};
/// Provides a new, initialized `BeaconChain`
pub trait InitialiseBeaconChain<T: BeaconChainTypes> {
fn initialise_beacon_chain(
store: Arc<T::Store>,
config: &ClientConfig,
spec: ChainSpec,
log: Logger,
) -> Result<BeaconChain<T>> {
maybe_load_from_store_for_testnet::<_, T::Store, T::EthSpec>(store, config, spec, log)
}
}
#[derive(Clone)]
pub struct ClientType<S: Store, E: EthSpec> {
_phantom_t: PhantomData<S>,
_phantom_u: PhantomData<E>,
}
impl<S, E> BeaconChainTypes for ClientType<S, E>
where
S: Store + 'static,
E: EthSpec,
{
type Store = S;
type SlotClock = SystemTimeSlotClock;
type LmdGhost = ThreadSafeReducedTree<S, E>;
type EthSpec = E;
}
impl<T: Store, E: EthSpec, X: BeaconChainTypes> InitialiseBeaconChain<X> for ClientType<T, E> {}
/// Loads a `BeaconChain` from `store`, if it exists. Otherwise, create a new chain from genesis.
fn maybe_load_from_store_for_testnet<T, U: Store, V: EthSpec>(
store: Arc<U>,
config: &ClientConfig,
spec: ChainSpec,
log: Logger,
) -> Result<BeaconChain<T>>
where
T: BeaconChainTypes<Store = U, EthSpec = V>,
T::LmdGhost: LmdGhost<U, V>,
{
let genesis_state = match &config.genesis_state {
GenesisState::Mainnet => {
crit!(log, "This release does not support mainnet genesis state.");
return Err("Mainnet is unsupported".into());
}
GenesisState::RecentGenesis { validator_count } => {
generate_testnet_genesis_state(*validator_count, recent_genesis_time(), &spec)
}
GenesisState::Generated {
validator_count,
genesis_time,
} => generate_testnet_genesis_state(*validator_count, *genesis_time, &spec),
GenesisState::Yaml { file } => {
let file = File::open(file).map_err(|e| {
format!("Unable to open YAML genesis state file {:?}: {:?}", file, e)
})?;
serde_yaml::from_reader(file)
.map_err(|e| format!("Unable to parse YAML genesis state file: {:?}", e))?
}
GenesisState::HttpBootstrap { server } => {
let bootstrapper = Bootstrapper::from_server_string(server.to_string())
.map_err(|e| format!("Failed to initialize bootstrap client: {}", e))?;
let (state, _block) = bootstrapper
.genesis()
.map_err(|e| format!("Failed to bootstrap genesis state: {}", e))?;
state
}
};
let mut genesis_block = BeaconBlock::empty(&spec);
genesis_block.state_root = Hash256::from_slice(&genesis_state.tree_hash_root());
let genesis_block_root = genesis_block.canonical_root();
// Slot clock
let slot_clock = T::SlotClock::new(
spec.genesis_slot,
genesis_state.genesis_time,
spec.seconds_per_slot,
);
// Try load an existing `BeaconChain` from the store. If unable, create a new one.
if let Ok(Some(beacon_chain)) =
BeaconChain::from_store(store.clone(), spec.clone(), log.clone())
{
// Here we check to ensure that the `BeaconChain` loaded from store has the expected
// genesis block.
//
// Without this check, it's possible that there will be an existing DB with a `BeaconChain`
// that has different parameters than provided to this executable.
if beacon_chain.genesis_block_root == genesis_block_root {
info!(
log,
"Loaded BeaconChain from store";
"slot" => beacon_chain.head().beacon_state.slot,
"best_slot" => beacon_chain.best_slot(),
);
Ok(beacon_chain)
} else {
crit!(
log,
"The BeaconChain loaded from disk has an incorrect genesis root. \
This may be caused by an old database in located in datadir."
);
Err("Incorrect genesis root".into())
}
} else {
BeaconChain::from_genesis(
store,
slot_clock,
genesis_state,
genesis_block,
spec,
log.clone(),
)
.map_err(|e| format!("Failed to initialize new beacon chain: {:?}", e).into())
}
}
fn generate_testnet_genesis_state<E: EthSpec>(
validator_count: usize,
genesis_time: u64,
spec: &ChainSpec,
) -> BeaconState<E> {
let (mut genesis_state, _keypairs) =
TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(validator_count, spec)
.build();
genesis_state.genesis_time = genesis_time;
genesis_state
}
/// Returns the system time, mod 30 minutes.
///
/// Used for easily creating testnets.
fn recent_genesis_time() -> u64 {
let now = SystemTime::now()
.duration_since(SystemTime::UNIX_EPOCH)
.unwrap()
.as_secs();
let secs_after_last_period = now.checked_rem(30 * 60).unwrap_or(0);
// genesis is now the last 30 minute block.
now - secs_after_last_period
}

View File

@ -1,15 +1,11 @@
use crate::{Bootstrapper, Eth2Config};
use clap::ArgMatches;
use network::NetworkConfig;
use serde_derive::{Deserialize, Serialize};
use slog::{info, o, warn, Drain};
use slog::{info, o, Drain};
use std::fs::{self, OpenOptions};
use std::path::PathBuf;
use std::sync::Mutex;
/// The number initial validators when starting the `Minimal`.
const TESTNET_VALIDATOR_COUNT: usize = 16;
/// The number initial validators when starting the `Minimal`.
const TESTNET_SPEC_CONSTANTS: &str = "minimal";
@ -21,33 +17,74 @@ pub struct Config {
db_name: String,
pub log_file: PathBuf,
pub spec_constants: String,
pub genesis_state: GenesisState,
/// Defines how we should initialize a BeaconChain instances.
///
/// This field is not serialized, there for it will not be written to (or loaded from) config
/// files. It can only be configured via the CLI.
#[serde(skip)]
pub beacon_chain_start_method: BeaconChainStartMethod,
pub eth1_backend_method: Eth1BackendMethod,
pub network: network::NetworkConfig,
pub rpc: rpc::RPCConfig,
pub rest_api: rest_api::ApiConfig,
pub websocket_server: websocket_server::Config,
}
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(tag = "type")]
pub enum GenesisState {
/// Use the mainnet genesis state.
///
/// Mainnet genesis state is not presently known, so this is a place-holder.
/// Defines how the client should initialize a BeaconChain.
///
/// In general, there are two methods:
/// - resuming a new chain, or
/// - initializing a new one.
#[derive(Debug, Clone)]
pub enum BeaconChainStartMethod {
/// Resume from an existing BeaconChain, loaded from the existing local database.
Resume,
/// Resume from an existing BeaconChain, loaded from the existing local database.
Mainnet,
/// Generate a state with `validator_count` validators, all with well-known secret keys.
/// Create a new beacon chain that can connect to mainnet.
///
/// Set the genesis time to be the start of the previous 30-minute window.
RecentGenesis { validator_count: usize },
/// Generate a state with `genesis_time` and `validator_count` validators, all with well-known
RecentGenesis {
validator_count: usize,
minutes: u64,
},
/// Create a new beacon chain with `genesis_time` and `validator_count` validators, all with well-known
/// secret keys.
Generated {
validator_count: usize,
genesis_time: u64,
},
/// Load a YAML-encoded genesis state from a file.
/// Create a new beacon chain by loading a YAML-encoded genesis state from a file.
Yaml { file: PathBuf },
/// Use a HTTP server (running our REST-API) to load genesis and finalized states and blocks.
HttpBootstrap { server: String },
/// Create a new beacon chain by loading a SSZ-encoded genesis state from a file.
Ssz { file: PathBuf },
/// Create a new beacon chain by loading a JSON-encoded genesis state from a file.
Json { file: PathBuf },
/// Create a new beacon chain by using a HTTP server (running our REST-API) to load genesis and
/// finalized states and blocks.
HttpBootstrap { server: String, port: Option<u16> },
}
impl Default for BeaconChainStartMethod {
fn default() -> Self {
BeaconChainStartMethod::Resume
}
}
/// Defines which Eth1 backend the client should use.
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(tag = "type")]
pub enum Eth1BackendMethod {
/// Use the mocked eth1 backend used in interop testing
Interop,
/// Use a web3 connection to a running Eth1 node.
Web3 { server: String },
}
impl Default for Eth1BackendMethod {
fn default() -> Self {
Eth1BackendMethod::Interop
}
}
impl Default for Config {
@ -58,12 +95,12 @@ impl Default for Config {
db_type: "disk".to_string(),
db_name: "chain_db".to_string(),
network: NetworkConfig::new(),
rpc: rpc::RPCConfig::default(),
rest_api: rest_api::ApiConfig::default(),
rpc: <_>::default(),
rest_api: <_>::default(),
websocket_server: <_>::default(),
spec_constants: TESTNET_SPEC_CONSTANTS.into(),
genesis_state: GenesisState::RecentGenesis {
validator_count: TESTNET_VALIDATOR_COUNT,
},
beacon_chain_start_method: <_>::default(),
eth1_backend_method: <_>::default(),
}
}
}
@ -76,6 +113,8 @@ impl Config {
}
/// Returns the core path for the client.
///
/// Creates the directory if it does not exist.
pub fn data_dir(&self) -> Option<PathBuf> {
let path = dirs::home_dir()?.join(&self.data_dir);
fs::create_dir_all(&path).ok()?;
@ -127,15 +166,6 @@ impl Config {
self.data_dir = PathBuf::from(dir);
};
if let Some(default_spec) = args.value_of("default-spec") {
match default_spec {
"mainnet" => self.spec_constants = Eth2Config::mainnet().spec_constants,
"minimal" => self.spec_constants = Eth2Config::minimal().spec_constants,
"interop" => self.spec_constants = Eth2Config::interop().spec_constants,
_ => {} // not supported
}
}
if let Some(dir) = args.value_of("db") {
self.db_type = dir.to_string();
};
@ -143,46 +173,13 @@ impl Config {
self.network.apply_cli_args(args)?;
self.rpc.apply_cli_args(args)?;
self.rest_api.apply_cli_args(args)?;
self.websocket_server.apply_cli_args(args)?;
if let Some(log_file) = args.value_of("logfile") {
self.log_file = PathBuf::from(log_file);
self.update_logger(log)?;
};
// If the `--bootstrap` flag is provided, overwrite the default configuration.
if let Some(server) = args.value_of("bootstrap") {
do_bootstrapping(self, server.to_string(), &log)?;
}
Ok(())
}
}
/// Perform the HTTP bootstrapping procedure, reading an ENR and multiaddr from the HTTP server and
/// adding them to the `config`.
fn do_bootstrapping(config: &mut Config, server: String, log: &slog::Logger) -> Result<(), String> {
// Set the genesis state source.
config.genesis_state = GenesisState::HttpBootstrap {
server: server.to_string(),
};
let bootstrapper = Bootstrapper::from_server_string(server.to_string())?;
config.network.boot_nodes.push(bootstrapper.enr()?);
if let Some(server_multiaddr) = bootstrapper.best_effort_multiaddr() {
info!(
log,
"Estimated bootstrapper libp2p address";
"multiaddr" => format!("{:?}", server_multiaddr)
);
config.network.libp2p_nodes.push(server_multiaddr);
} else {
warn!(
log,
"Unable to estimate a bootstrapper libp2p address, this node may not find any peers."
);
}
Ok(())
}

View File

@ -1,97 +1,216 @@
extern crate slog;
mod beacon_chain_types;
mod bootstrapper;
mod config;
pub mod error;
pub mod notifier;
use beacon_chain::BeaconChain;
use beacon_chain::{
lmd_ghost::ThreadSafeReducedTree, slot_clock::SystemTimeSlotClock, store::Store,
test_utils::generate_deterministic_keypairs, BeaconChain, BeaconChainBuilder,
};
use exit_future::Signal;
use futures::{future::Future, Stream};
use network::Service as NetworkService;
use slog::{error, info, o};
use slog::{crit, error, info, o};
use slot_clock::SlotClock;
use std::marker::PhantomData;
use std::sync::Arc;
use std::time::{Duration, Instant};
use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH};
use tokio::runtime::TaskExecutor;
use tokio::timer::Interval;
use types::EthSpec;
use websocket_server::WebSocketSender;
pub use beacon_chain::BeaconChainTypes;
pub use beacon_chain_types::ClientType;
pub use beacon_chain_types::InitialiseBeaconChain;
pub use bootstrapper::Bootstrapper;
pub use config::{Config as ClientConfig, GenesisState};
pub use beacon_chain::{BeaconChainTypes, Eth1ChainBackend, InteropEth1ChainBackend};
pub use config::{BeaconChainStartMethod, Config as ClientConfig, Eth1BackendMethod};
pub use eth2_config::Eth2Config;
#[derive(Clone)]
pub struct RuntimeBeaconChainTypes<S: Store, E: EthSpec> {
_phantom_s: PhantomData<S>,
_phantom_e: PhantomData<E>,
}
impl<S, E> BeaconChainTypes for RuntimeBeaconChainTypes<S, E>
where
S: Store + 'static,
E: EthSpec,
{
type Store = S;
type SlotClock = SystemTimeSlotClock;
type LmdGhost = ThreadSafeReducedTree<S, E>;
type Eth1Chain = InteropEth1ChainBackend<E>;
type EthSpec = E;
type EventHandler = WebSocketSender<E>;
}
/// Main beacon node client service. This provides the connection and initialisation of the clients
/// sub-services in multiple threads.
pub struct Client<T: BeaconChainTypes> {
pub struct Client<S, E>
where
S: Store + Clone + 'static,
E: EthSpec,
{
/// Configuration for the lighthouse client.
_client_config: ClientConfig,
/// The beacon chain for the running client.
beacon_chain: Arc<BeaconChain<T>>,
beacon_chain: Arc<BeaconChain<RuntimeBeaconChainTypes<S, E>>>,
/// Reference to the network service.
pub network: Arc<NetworkService<T>>,
pub network: Arc<NetworkService<RuntimeBeaconChainTypes<S, E>>>,
/// Signal to terminate the RPC server.
pub rpc_exit_signal: Option<Signal>,
/// Signal to terminate the slot timer.
pub slot_timer_exit_signal: Option<Signal>,
/// Signal to terminate the API
pub api_exit_signal: Option<Signal>,
/// Signal to terminate the websocket server
pub websocket_exit_signal: Option<Signal>,
/// The clients logger.
log: slog::Logger,
/// Marker to pin the beacon chain generics.
phantom: PhantomData<T>,
}
impl<T> Client<T>
impl<S, E> Client<S, E>
where
T: BeaconChainTypes + InitialiseBeaconChain<T> + Clone,
S: Store + Clone + 'static,
E: EthSpec,
{
/// Generate an instance of the client. Spawn and link all internal sub-processes.
pub fn new(
client_config: ClientConfig,
eth2_config: Eth2Config,
store: T::Store,
store: S,
log: slog::Logger,
executor: &TaskExecutor,
) -> error::Result<Self> {
let store = Arc::new(store);
let seconds_per_slot = eth2_config.spec.seconds_per_slot;
let milliseconds_per_slot = eth2_config.spec.milliseconds_per_slot;
// Load a `BeaconChain` from the store, or create a new one if it does not exist.
let beacon_chain = Arc::new(T::initialise_beacon_chain(
store,
&client_config,
eth2_config.spec.clone(),
log.clone(),
)?);
let spec = &eth2_config.spec.clone();
if beacon_chain.read_slot_clock().is_none() {
panic!("Cannot start client before genesis!")
}
let beacon_chain_builder = match &client_config.beacon_chain_start_method {
BeaconChainStartMethod::Resume => {
info!(
log,
"Starting beacon chain";
"method" => "resume"
);
BeaconChainBuilder::from_store(spec.clone(), log.clone())
}
BeaconChainStartMethod::Mainnet => {
crit!(log, "No mainnet beacon chain startup specification.");
return Err("Mainnet launch is not yet announced.".into());
}
BeaconChainStartMethod::RecentGenesis {
validator_count,
minutes,
} => {
info!(
log,
"Starting beacon chain";
"validator_count" => validator_count,
"minutes" => minutes,
"method" => "recent"
);
BeaconChainBuilder::recent_genesis(
&generate_deterministic_keypairs(*validator_count),
*minutes,
spec.clone(),
log.clone(),
)?
}
BeaconChainStartMethod::Generated {
validator_count,
genesis_time,
} => {
info!(
log,
"Starting beacon chain";
"validator_count" => validator_count,
"genesis_time" => genesis_time,
"method" => "quick"
);
BeaconChainBuilder::quick_start(
*genesis_time,
&generate_deterministic_keypairs(*validator_count),
spec.clone(),
log.clone(),
)?
}
BeaconChainStartMethod::Yaml { file } => {
info!(
log,
"Starting beacon chain";
"file" => format!("{:?}", file),
"method" => "yaml"
);
BeaconChainBuilder::yaml_state(file, spec.clone(), log.clone())?
}
BeaconChainStartMethod::Ssz { file } => {
info!(
log,
"Starting beacon chain";
"file" => format!("{:?}", file),
"method" => "ssz"
);
BeaconChainBuilder::ssz_state(file, spec.clone(), log.clone())?
}
BeaconChainStartMethod::Json { file } => {
info!(
log,
"Starting beacon chain";
"file" => format!("{:?}", file),
"method" => "json"
);
BeaconChainBuilder::json_state(file, spec.clone(), log.clone())?
}
BeaconChainStartMethod::HttpBootstrap { server, port } => {
info!(
log,
"Starting beacon chain";
"port" => port,
"server" => server,
"method" => "bootstrap"
);
BeaconChainBuilder::http_bootstrap(server, spec.clone(), log.clone())?
}
};
// Block starting the client until we have caught the state up to the current slot.
//
// If we don't block here we create an initial scenario where we're unable to process any
// blocks and we're basically useless.
{
let state_slot = beacon_chain.head().beacon_state.slot;
let wall_clock_slot = beacon_chain.read_slot_clock().unwrap();
let slots_since_genesis = beacon_chain.slots_since_genesis().unwrap();
let eth1_backend =
InteropEth1ChainBackend::new(String::new()).map_err(|e| format!("{:?}", e))?;
// Start the websocket server.
let (websocket_sender, websocket_exit_signal): (WebSocketSender<E>, Option<_>) =
if client_config.websocket_server.enabled {
let (sender, exit) = websocket_server::start_server(
&client_config.websocket_server,
executor,
&log,
)?;
(sender, Some(exit))
} else {
(WebSocketSender::dummy(), None)
};
let beacon_chain: Arc<BeaconChain<RuntimeBeaconChainTypes<S, E>>> = Arc::new(
beacon_chain_builder
.build(store, eth1_backend, websocket_sender)
.map_err(error::Error::from)?,
);
let since_epoch = SystemTime::now()
.duration_since(UNIX_EPOCH)
.map_err(|e| format!("Unable to read system time: {}", e))?;
let since_genesis = Duration::from_secs(beacon_chain.head().beacon_state.genesis_time);
if since_genesis > since_epoch {
info!(
log,
"BeaconState cache init";
"state_slot" => state_slot,
"wall_clock_slot" => wall_clock_slot,
"slots_since_genesis" => slots_since_genesis,
"catchup_distance" => wall_clock_slot - state_slot,
"Starting node prior to genesis";
"now" => since_epoch.as_secs(),
"genesis_seconds" => since_genesis.as_secs(),
);
}
do_state_catchup(&beacon_chain, &log);
let network_config = &client_config.network;
let (network, network_send) =
@ -117,7 +236,9 @@ where
executor,
beacon_chain.clone(),
network.clone(),
network_send.clone(),
client_config.db_path().expect("unable to read datadir"),
eth2_config.clone(),
&log,
) {
Ok(s) => Some(s),
@ -131,11 +252,11 @@ where
};
let (slot_timer_exit_signal, exit) = exit_future::signal();
if let Ok(Some(duration_to_next_slot)) = beacon_chain.slot_clock.duration_to_next_slot() {
if let Some(duration_to_next_slot) = beacon_chain.slot_clock.duration_to_next_slot() {
// 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(seconds_per_slot);
let slot_duration = Duration::from_millis(milliseconds_per_slot);
//TODO: Handle checked add correctly
Interval::new(Instant::now() + duration_to_next_slot, slot_duration)
};
@ -146,7 +267,7 @@ where
exit.until(
interval
.for_each(move |_| {
do_state_catchup(&chain, &log);
log_new_slot(&chain, &log);
Ok(())
})
@ -162,49 +283,33 @@ where
rpc_exit_signal,
slot_timer_exit_signal: Some(slot_timer_exit_signal),
api_exit_signal,
websocket_exit_signal,
log,
network,
phantom: PhantomData,
})
}
}
impl<T: BeaconChainTypes> Drop for Client<T> {
impl<S: Store + Clone, E: EthSpec> Drop for Client<S, E> {
fn drop(&mut self) {
// Save the beacon chain to it's store before dropping.
let _result = self.beacon_chain.persist();
}
}
fn do_state_catchup<T: BeaconChainTypes>(chain: &Arc<BeaconChain<T>>, log: &slog::Logger) {
// Only attempt to `catchup_state` if we can read the slot clock.
if let Some(current_slot) = chain.read_slot_clock() {
let state_catchup_result = chain.catchup_state();
fn log_new_slot<T: BeaconChainTypes>(chain: &Arc<BeaconChain<T>>, log: &slog::Logger) {
let best_slot = chain.head().beacon_block.slot;
let latest_block_root = chain.head().beacon_block_root;
let best_slot = chain.head().beacon_block.slot;
let latest_block_root = chain.head().beacon_block_root;
let common = o!(
if let Ok(current_slot) = chain.slot() {
info!(
log,
"Slot start";
"skip_slots" => current_slot.saturating_sub(best_slot),
"best_block_root" => format!("{}", latest_block_root),
"best_block_slot" => best_slot,
"slot" => current_slot,
);
if let Err(e) = state_catchup_result {
error!(
log,
"State catchup failed";
"error" => format!("{:?}", e),
common
)
} else {
info!(
log,
"Slot start";
common
)
}
)
} else {
error!(
log,

View File

@ -1,11 +1,12 @@
use crate::Client;
use beacon_chain::BeaconChainTypes;
use exit_future::Exit;
use futures::{Future, Stream};
use slog::{debug, o, warn};
use std::time::{Duration, Instant};
use store::Store;
use tokio::runtime::TaskExecutor;
use tokio::timer::Interval;
use types::EthSpec;
/// The interval between heartbeat events.
pub const HEARTBEAT_INTERVAL_SECONDS: u64 = 15;
@ -17,7 +18,11 @@ pub const WARN_PEER_COUNT: usize = 1;
/// durations.
///
/// Presently unused, but remains for future use.
pub fn run<T: BeaconChainTypes>(client: &Client<T>, executor: TaskExecutor, exit: Exit) {
pub fn run<S, E>(client: &Client<S, E>, executor: TaskExecutor, exit: Exit)
where
S: Store + Clone + 'static,
E: EthSpec,
{
// notification heartbeat
let interval = Interval::new(
Instant::now(),
@ -34,10 +39,10 @@ pub fn run<T: BeaconChainTypes>(client: &Client<T>, executor: TaskExecutor, exit
// Panics if libp2p is poisoned.
let connected_peer_count = libp2p.lock().swarm.connected_peers();
debug!(log, "Libp2p connected peer status"; "peer_count" => connected_peer_count);
debug!(log, "Connected peer status"; "peer_count" => connected_peer_count);
if connected_peer_count <= WARN_PEER_COUNT {
warn!(log, "Low libp2p peer count"; "peer_count" => connected_peer_count);
warn!(log, "Low peer count"; "peer_count" => connected_peer_count);
}
Ok(())

View File

@ -6,9 +6,10 @@ edition = "2018"
[dependencies]
clap = "2.32.0"
hex = "0.3"
#SigP repository
libp2p = { git = "https://github.com/SigP/rust-libp2p", rev = "61036890d574f5b46573952b20def2baafd6a6e9" }
enr = { git = "https://github.com/SigP/rust-libp2p/", rev = "61036890d574f5b46573952b20def2baafd6a6e9", features = ["serde"] }
libp2p = { git = "https://github.com/SigP/rust-libp2p", rev = "8ac9c744197faaadc0e2b64fed7470ac4e2a41ca" }
enr = { git = "https://github.com/SigP/rust-libp2p/", rev = "8ac9c744197faaadc0e2b64fed7470ac4e2a41ca", features = ["serde"] }
types = { path = "../../eth2/types" }
serde = "1.0"
serde_derive = "1.0"

View File

@ -15,7 +15,7 @@ use libp2p::{
tokio_io::{AsyncRead, AsyncWrite},
NetworkBehaviour, PeerId,
};
use slog::{debug, o, trace};
use slog::{debug, o};
use std::num::NonZeroU32;
use std::time::Duration;
@ -90,13 +90,15 @@ impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<GossipsubE
{
fn inject_event(&mut self, event: GossipsubEvent) {
match event {
GossipsubEvent::Message(gs_msg) => {
trace!(self.log, "Received GossipEvent");
GossipsubEvent::Message(propagation_source, gs_msg) => {
let id = gs_msg.id();
let msg = PubsubMessage::from_topics(&gs_msg.topics, gs_msg.data);
// Note: We are keeping track here of the peer that sent us the message, not the
// peer that originally published the message.
self.events.push(BehaviourEvent::GossipMessage {
source: gs_msg.source,
id,
source: propagation_source,
topics: gs_msg.topics,
message: msg,
});
@ -199,6 +201,13 @@ impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
}
}
/// Forwards a message that is waiting in gossipsub's mcache. Messages are only propagated
/// once validated by the beacon chain.
pub fn propagate_message(&mut self, propagation_source: &PeerId, message_id: String) {
self.gossipsub
.propagate_message(&message_id, propagation_source);
}
/* Eth2 RPC behaviour functions */
/// Sends an RPC Request/Response via the RPC protocol.
@ -214,12 +223,21 @@ impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
/// The types of events than can be obtained from polling the behaviour.
pub enum BehaviourEvent {
/// A received RPC event and the peer that it was received from.
RPC(PeerId, RPCEvent),
/// We have completed an initial connection to a new peer.
PeerDialed(PeerId),
/// A peer has disconnected.
PeerDisconnected(PeerId),
/// A gossipsub message has been received.
GossipMessage {
/// The gossipsub message id. Used when propagating blocks after validation.
id: String,
/// The peer from which we received this message, not the peer that published it.
source: PeerId,
/// The topics that this message was sent on.
topics: Vec<TopicHash>,
/// The message itself.
message: PubsubMessage,
},
}

View File

@ -40,6 +40,12 @@ pub struct Config {
/// Target number of connected peers.
pub max_peers: usize,
/// A secp256k1 secret key, as bytes in ASCII-encoded hex.
///
/// With or without `0x` prefix.
#[serde(skip)]
pub secret_key_hex: Option<String>,
/// Gossipsub configuration parameters.
#[serde(skip)]
pub gs_config: GossipsubConfig,
@ -70,11 +76,13 @@ impl Default for Config {
discovery_address: "127.0.0.1".parse().expect("valid ip address"),
discovery_port: 9000,
max_peers: 10,
secret_key_hex: None,
// Note: The topics by default are sent as plain strings. Hashes are an optional
// parameter.
gs_config: GossipsubConfigBuilder::new()
.max_transmit_size(1_048_576)
.heartbeat_interval(Duration::from_secs(20))
.heartbeat_interval(Duration::from_secs(20)) // TODO: Reduce for mainnet
.propagate_messages(false) // require validation before propagation
.build(),
boot_nodes: vec![],
libp2p_nodes: vec![],
@ -157,6 +165,10 @@ impl Config {
.map_err(|_| format!("Invalid discovery port: {}", disc_port_str))?;
}
if let Some(p2p_priv_key) = args.value_of("p2p-priv-key") {
self.secret_key_hex = Some(p2p_priv_key.to_string());
}
Ok(())
}
}

View File

@ -114,7 +114,7 @@ impl<TSubstream> Discovery<TSubstream> {
self.find_peers();
}
/// Add an Enr to the routing table of the discovery mechanism.
/// Add an ENR to the routing table of the discovery mechanism.
pub fn add_enr(&mut self, enr: Enr) {
self.discovery.add_enr(enr);
}
@ -169,6 +169,7 @@ where
fn inject_connected(&mut self, peer_id: PeerId, _endpoint: ConnectedPoint) {
self.connected_peers.insert(peer_id);
// TODO: Drop peers if over max_peer limit
metrics::inc_counter(&metrics::PEER_CONNECT_EVENT_COUNT);
metrics::set_gauge(&metrics::PEERS_CONNECTED, self.connected_peers() as i64);

View File

@ -101,13 +101,15 @@ where
type Error = <TCodec as Decoder>::Error;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
// if we have only received the response code, wait for more bytes
if src.len() == 1 {
return Ok(None);
}
// using the response code determine which kind of payload needs to be decoded.
let response_code = {
if let Some(resp_code) = self.response_code {
resp_code
} else {
// buffer should not be empty
debug_assert!(!src.is_empty());
let resp_byte = src.split_to(1);
let mut resp_code_byte = [0; 1];
resp_code_byte.copy_from_slice(&resp_byte);

View File

@ -4,7 +4,7 @@ use crate::rpc::{
protocol::{ProtocolId, RPCError},
};
use crate::rpc::{ErrorMessage, RPCErrorResponse, RPCRequest, RPCResponse};
use bytes::{Bytes, BytesMut};
use bytes::{BufMut, Bytes, BytesMut};
use ssz::{Decode, Encode};
use tokio::codec::{Decoder, Encoder};
use unsigned_varint::codec::UviBytes;
@ -56,6 +56,10 @@ impl Encoder for SSZInboundCodec {
.inner
.encode(Bytes::from(bytes), dst)
.map_err(RPCError::from);
} else {
// payload is empty, add a 0-byte length prefix
dst.reserve(1);
dst.put_u8(0);
}
Ok(())
}
@ -152,45 +156,49 @@ impl Decoder for SSZOutboundCodec {
type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
match self.inner.decode(src).map_err(RPCError::from) {
Ok(Some(packet)) => match self.protocol.message_name.as_str() {
if src.len() == 1 && src[0] == 0_u8 {
// the object is empty. We return the empty object if this is the case
match self.protocol.message_name.as_str() {
"hello" => match self.protocol.version.as_str() {
"1" => Ok(Some(RPCResponse::Hello(HelloMessage::from_ssz_bytes(
&packet,
)?))),
"1" => Err(RPCError::Custom(
"Hello stream terminated unexpectedly".into(),
)), // cannot have an empty HELLO message. The stream has terminated unexpectedly
_ => unreachable!("Cannot negotiate an unknown version"),
},
"goodbye" => Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response")),
"beacon_blocks" => match self.protocol.version.as_str() {
"1" => Ok(Some(RPCResponse::BeaconBlocks(packet.to_vec()))),
"1" => Ok(Some(RPCResponse::BeaconBlocks(Vec::new()))),
_ => unreachable!("Cannot negotiate an unknown version"),
},
"recent_beacon_blocks" => match self.protocol.version.as_str() {
"1" => Ok(Some(RPCResponse::RecentBeaconBlocks(packet.to_vec()))),
"1" => Ok(Some(RPCResponse::RecentBeaconBlocks(Vec::new()))),
_ => unreachable!("Cannot negotiate an unknown version"),
},
_ => unreachable!("Cannot negotiate an unknown protocol"),
},
Ok(None) => {
// the object sent could be a empty. We return the empty object if this is the case
match self.protocol.message_name.as_str() {
}
} else {
match self.inner.decode(src).map_err(RPCError::from) {
Ok(Some(packet)) => match self.protocol.message_name.as_str() {
"hello" => match self.protocol.version.as_str() {
"1" => Ok(None), // cannot have an empty HELLO message. The stream has terminated unexpectedly
"1" => Ok(Some(RPCResponse::Hello(HelloMessage::from_ssz_bytes(
&packet,
)?))),
_ => unreachable!("Cannot negotiate an unknown version"),
},
"goodbye" => Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response")),
"beacon_blocks" => match self.protocol.version.as_str() {
"1" => Ok(Some(RPCResponse::BeaconBlocks(Vec::new()))),
"1" => Ok(Some(RPCResponse::BeaconBlocks(packet.to_vec()))),
_ => unreachable!("Cannot negotiate an unknown version"),
},
"recent_beacon_blocks" => match self.protocol.version.as_str() {
"1" => Ok(Some(RPCResponse::RecentBeaconBlocks(Vec::new()))),
"1" => Ok(Some(RPCResponse::RecentBeaconBlocks(packet.to_vec()))),
_ => unreachable!("Cannot negotiate an unknown version"),
},
_ => unreachable!("Cannot negotiate an unknown protocol"),
}
},
Ok(None) => Ok(None), // waiting for more bytes
Err(e) => Err(e),
}
Err(e) => Err(e),
}
}
}

View File

@ -1,6 +1,5 @@
//!Available RPC methods types and ids.
use ssz::{impl_decode_via_from, impl_encode_via_from};
use ssz_derive::{Decode, Encode};
use types::{Epoch, Hash256, Slot};
@ -66,8 +65,38 @@ impl Into<u64> for GoodbyeReason {
}
}
impl_encode_via_from!(GoodbyeReason, u64);
impl_decode_via_from!(GoodbyeReason, u64);
impl ssz::Encode for GoodbyeReason {
fn is_ssz_fixed_len() -> bool {
<u64 as ssz::Encode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<u64 as ssz::Encode>::ssz_fixed_len()
}
fn ssz_bytes_len(&self) -> usize {
0_u64.ssz_bytes_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
let conv: u64 = self.clone().into();
conv.ssz_append(buf)
}
}
impl ssz::Decode for GoodbyeReason {
fn is_ssz_fixed_len() -> bool {
<u64 as ssz::Decode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<u64 as ssz::Decode>::ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, ssz::DecodeError> {
u64::from_ssz_bytes(bytes).and_then(|n| Ok(n.into()))
}
}
/// Request a number of beacon block roots from a peer.
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
@ -157,3 +186,53 @@ impl ErrorMessage {
String::from_utf8(self.error_message.clone()).unwrap_or_else(|_| "".into())
}
}
impl std::fmt::Display for HelloMessage {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "Hello Message: Fork Version: {:?}, Finalized Root: {}, Finalized Epoch: {}, Head Root: {}, Head Slot: {}", self.fork_version, self.finalized_root, self.finalized_epoch, self.head_root, self.head_slot)
}
}
impl std::fmt::Display for RPCResponse {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
RPCResponse::Hello(hello) => write!(f, "{}", hello),
RPCResponse::BeaconBlocks(data) => write!(f, "<BeaconBlocks>, len: {}", data.len()),
RPCResponse::RecentBeaconBlocks(data) => {
write!(f, "<RecentBeaconBlocks>, len: {}", data.len())
}
}
}
}
impl std::fmt::Display for RPCErrorResponse {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
RPCErrorResponse::Success(res) => write!(f, "{}", res),
RPCErrorResponse::InvalidRequest(err) => write!(f, "Invalid Request: {:?}", err),
RPCErrorResponse::ServerError(err) => write!(f, "Server Error: {:?}", err),
RPCErrorResponse::Unknown(err) => write!(f, "Unknown Error: {:?}", err),
}
}
}
impl std::fmt::Display for GoodbyeReason {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
GoodbyeReason::ClientShutdown => write!(f, "Client Shutdown"),
GoodbyeReason::IrrelevantNetwork => write!(f, "Irrelevant Network"),
GoodbyeReason::Fault => write!(f, "Fault"),
GoodbyeReason::Unknown => write!(f, "Unknown Reason"),
}
}
}
impl std::fmt::Display for BeaconBlocksRequest {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"Head Block Root: {}, Start Slot: {}, Count: {}, Step: {}",
self.head_block_root, self.start_slot, self.count, self.step
)
}
}

View File

@ -47,6 +47,16 @@ impl RPCEvent {
}
}
impl std::fmt::Display for RPCEvent {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
RPCEvent::Request(id, req) => write!(f, "RPC Request(Id: {}, {})", id, req),
RPCEvent::Response(id, res) => write!(f, "RPC Response(Id: {}, {})", id, res),
RPCEvent::Error(id, err) => write!(f, "RPC Request(Id: {}, Error: {:?})", id, err),
}
}
}
/// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level
/// logic.
pub struct RPC<TSubstream> {

View File

@ -288,3 +288,14 @@ impl std::error::Error for RPCError {
}
}
}
impl std::fmt::Display for RPCRequest {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
RPCRequest::Hello(hello) => write!(f, "Hello Message: {}", hello),
RPCRequest::Goodbye(reason) => write!(f, "Goodbye: {}", reason),
RPCRequest::BeaconBlocks(req) => write!(f, "Beacon Blocks: {}", req),
RPCRequest::RecentBeaconBlocks(req) => write!(f, "Recent Beacon Blocks: {:?}", req),
}
}
}

View File

@ -42,16 +42,21 @@ impl Service {
pub fn new(config: NetworkConfig, log: slog::Logger) -> error::Result<Self> {
trace!(log, "Libp2p Service starting");
let local_keypair = if let Some(hex_bytes) = &config.secret_key_hex {
keypair_from_hex(hex_bytes)?
} else {
load_private_key(&config, &log)
};
// load the private key from CLI flag, disk or generate a new one
let local_private_key = load_private_key(&config, &log);
let local_peer_id = PeerId::from(local_private_key.public());
let local_peer_id = PeerId::from(local_keypair.public());
info!(log, "Libp2p Service"; "peer_id" => format!("{:?}", local_peer_id));
let mut swarm = {
// Set up the transport - tcp/ws with secio and mplex/yamux
let transport = build_transport(local_private_key.clone());
let transport = build_transport(local_keypair.clone());
// Lighthouse network behaviour
let behaviour = Behaviour::new(&local_private_key, &config, &log)?;
let behaviour = Behaviour::new(&local_keypair, &config, &log)?;
Swarm::new(transport, behaviour, local_peer_id.clone())
};
@ -79,15 +84,32 @@ impl Service {
}
};
// attempt to connect to user-input libp2p nodes
for multiaddr in config.libp2p_nodes {
// helper closure for dialing peers
let mut dial_addr = |multiaddr: Multiaddr| {
match Swarm::dial_addr(&mut swarm, multiaddr.clone()) {
Ok(()) => debug!(log, "Dialing libp2p peer"; "address" => format!("{}", multiaddr)),
Err(err) => debug!(
log,
"Could not connect to peer"; "address" => format!("{}", multiaddr), "Error" => format!("{:?}", err)
"Could not connect to peer"; "address" => format!("{}", multiaddr), "error" => format!("{:?}", err)
),
};
};
// attempt to connect to user-input libp2p nodes
for multiaddr in config.libp2p_nodes {
dial_addr(multiaddr);
}
// attempt to connect to any specified boot-nodes
for bootnode_enr in config.boot_nodes {
for multiaddr in bootnode_enr.multiaddr() {
// ignore udp multiaddr if it exists
let components = multiaddr.iter().collect::<Vec<_>>();
if let Protocol::Udp(_) = components[1] {
continue;
}
dial_addr(multiaddr);
}
}
// subscribe to default gossipsub topics
@ -145,16 +167,16 @@ impl Stream for Service {
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
loop {
match self.swarm.poll() {
//Behaviour events
Ok(Async::Ready(Some(event))) => match event {
// TODO: Stub here for debugging
BehaviourEvent::GossipMessage {
id,
source,
topics,
message,
} => {
trace!(self.log, "Gossipsub message received"; "service" => "Swarm");
return Ok(Async::Ready(Some(Libp2pEvent::PubsubMessage {
id,
source,
topics,
message,
@ -222,12 +244,34 @@ pub enum Libp2pEvent {
PeerDisconnected(PeerId),
/// Received pubsub message.
PubsubMessage {
id: String,
source: PeerId,
topics: Vec<TopicHash>,
message: PubsubMessage,
},
}
fn keypair_from_hex(hex_bytes: &str) -> error::Result<Keypair> {
let hex_bytes = if hex_bytes.starts_with("0x") {
hex_bytes[2..].to_string()
} else {
hex_bytes.to_string()
};
hex::decode(&hex_bytes)
.map_err(|e| format!("Failed to parse p2p secret key bytes: {:?}", e).into())
.and_then(keypair_from_bytes)
}
fn keypair_from_bytes(mut bytes: Vec<u8>) -> error::Result<Keypair> {
libp2p::core::identity::secp256k1::SecretKey::from_bytes(&mut bytes)
.map(|secret| {
let keypair: libp2p::core::identity::secp256k1::Keypair = secret.into();
Keypair::Secp256k1(keypair)
})
.map_err(|e| format!("Unable to parse p2p secret key: {:?}", e).into())
}
/// Loads a private key from disk. If this fails, a new key is
/// generated and is then saved to disk.
///

View File

@ -13,9 +13,11 @@ store = { path = "../store" }
eth2-libp2p = { path = "../eth2-libp2p" }
types = { path = "../../eth2/types" }
slog = { version = "^2.2.3" , features = ["max_level_trace"] }
hex = "0.3"
eth2_ssz = "0.1"
tree_hash = "0.1"
futures = "0.1.25"
error-chain = "0.12.0"
tokio = "0.1.16"
parking_lot = "0.9.0"
smallvec = "0.6.10"

View File

@ -1,6 +1,6 @@
use crate::error;
use crate::service::NetworkMessage;
use crate::sync::SimpleSync;
use crate::sync::MessageProcessor;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::{
behaviour::PubsubMessage,
@ -9,18 +9,22 @@ use eth2_libp2p::{
};
use futures::future::Future;
use futures::stream::Stream;
use slog::{debug, trace, warn};
use slog::{debug, o, trace, warn};
use ssz::{Decode, DecodeError};
use std::sync::Arc;
use tokio::sync::mpsc;
use types::{Attestation, AttesterSlashing, BeaconBlock, ProposerSlashing, VoluntaryExit};
/// Handles messages received from the network and client and organises syncing.
/// Handles messages received from the network and client and organises syncing. This
/// functionality of this struct is to validate an decode messages from the network before
/// passing them to the internal message processor. The message processor spawns a syncing thread
/// which manages which blocks need to be requested and processed.
pub struct MessageHandler<T: BeaconChainTypes> {
/// Currently loaded and initialised beacon chain.
_chain: Arc<BeaconChain<T>>,
/// The syncing framework.
sync: SimpleSync<T>,
/// A channel to the network service to allow for gossip propagation.
network_send: mpsc::UnboundedSender<NetworkMessage>,
/// Processes validated and decoded messages from the network. Has direct access to the
/// sync manager.
message_processor: MessageProcessor<T>,
/// The `MessageHandler` logger.
log: slog::Logger,
}
@ -34,8 +38,9 @@ pub enum HandlerMessage {
PeerDisconnected(PeerId),
/// An RPC response/request has been received.
RPC(PeerId, RPCEvent),
/// A gossip message has been received.
PubsubMessage(PeerId, PubsubMessage),
/// A gossip message has been received. The fields are: message id, the peer that sent us this
/// message and the message itself.
PubsubMessage(String, PeerId, PubsubMessage),
}
impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
@ -46,17 +51,20 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
executor: &tokio::runtime::TaskExecutor,
log: slog::Logger,
) -> error::Result<mpsc::UnboundedSender<HandlerMessage>> {
trace!(log, "Service starting");
let message_handler_log = log.new(o!("Service"=> "Message Handler"));
trace!(message_handler_log, "Service starting");
let (handler_send, handler_recv) = mpsc::unbounded_channel();
// Initialise sync and begin processing in thread
let sync = SimpleSync::new(beacon_chain.clone(), network_send, &log);
// Initialise a message instance, which itself spawns the syncing thread.
let message_processor =
MessageProcessor::new(executor, beacon_chain, network_send.clone(), &log);
// generate the Message handler
let mut handler = MessageHandler {
_chain: beacon_chain.clone(),
sync,
log: log.clone(),
network_send,
message_processor,
log: message_handler_log,
};
// spawn handler task and move the message handler instance into the spawned thread
@ -76,19 +84,19 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
match message {
// we have initiated a connection to a peer
HandlerMessage::PeerDialed(peer_id) => {
self.sync.on_connect(peer_id);
self.message_processor.on_connect(peer_id);
}
// A peer has disconnected
HandlerMessage::PeerDisconnected(peer_id) => {
self.sync.on_disconnect(peer_id);
self.message_processor.on_disconnect(peer_id);
}
// An RPC message request/response has been received
HandlerMessage::RPC(peer_id, rpc_event) => {
self.handle_rpc_message(peer_id, rpc_event);
}
// An RPC message request/response has been received
HandlerMessage::PubsubMessage(peer_id, gossip) => {
self.handle_gossip(peer_id, gossip);
HandlerMessage::PubsubMessage(id, peer_id, gossip) => {
self.handle_gossip(id, peer_id, gossip);
}
}
}
@ -108,7 +116,7 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
fn handle_rpc_request(&mut self, peer_id: PeerId, request_id: RequestId, request: RPCRequest) {
match request {
RPCRequest::Hello(hello_message) => {
self.sync
self.message_processor
.on_hello_request(peer_id, request_id, hello_message)
}
RPCRequest::Goodbye(goodbye_reason) => {
@ -117,13 +125,13 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
"peer" => format!("{:?}", peer_id),
"reason" => format!("{:?}", goodbye_reason),
);
self.sync.on_disconnect(peer_id);
self.message_processor.on_disconnect(peer_id);
}
RPCRequest::BeaconBlocks(request) => self
.sync
.message_processor
.on_beacon_blocks_request(peer_id, request_id, request),
RPCRequest::RecentBeaconBlocks(request) => self
.sync
.message_processor
.on_recent_beacon_blocks_request(peer_id, request_id, request),
}
}
@ -150,12 +158,13 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
RPCErrorResponse::Success(response) => {
match response {
RPCResponse::Hello(hello_message) => {
self.sync.on_hello_response(peer_id, hello_message);
self.message_processor
.on_hello_response(peer_id, hello_message);
}
RPCResponse::BeaconBlocks(response) => {
match self.decode_beacon_blocks(&response) {
Ok(beacon_blocks) => {
self.sync.on_beacon_blocks_response(
self.message_processor.on_beacon_blocks_response(
peer_id,
request_id,
beacon_blocks,
@ -170,7 +179,7 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
RPCResponse::RecentBeaconBlocks(response) => {
match self.decode_beacon_blocks(&response) {
Ok(beacon_blocks) => {
self.sync.on_recent_beacon_blocks_response(
self.message_processor.on_recent_beacon_blocks_response(
peer_id,
request_id,
beacon_blocks,
@ -194,24 +203,37 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
}
/// Handle RPC messages
fn handle_gossip(&mut self, peer_id: PeerId, gossip_message: PubsubMessage) {
fn handle_gossip(&mut self, id: String, peer_id: PeerId, gossip_message: PubsubMessage) {
match gossip_message {
PubsubMessage::Block(message) => match self.decode_gossip_block(message) {
Ok(block) => {
let _should_forward_on = self.sync.on_block_gossip(peer_id, block);
let should_forward_on = self
.message_processor
.on_block_gossip(peer_id.clone(), block);
// TODO: Apply more sophisticated validation and decoding logic
if should_forward_on {
self.propagate_message(id, peer_id.clone());
}
}
Err(e) => {
debug!(self.log, "Invalid gossiped beacon block"; "peer_id" => format!("{}", peer_id), "Error" => format!("{:?}", e));
}
},
PubsubMessage::Attestation(message) => match self.decode_gossip_attestation(message) {
Ok(attestation) => self.sync.on_attestation_gossip(peer_id, attestation),
Ok(attestation) => {
// TODO: Apply more sophisticated validation and decoding logic
self.propagate_message(id, peer_id.clone());
self.message_processor
.on_attestation_gossip(peer_id, attestation);
}
Err(e) => {
debug!(self.log, "Invalid gossiped attestation"; "peer_id" => format!("{}", peer_id), "Error" => format!("{:?}", e));
}
},
PubsubMessage::VoluntaryExit(message) => match self.decode_gossip_exit(message) {
Ok(_exit) => {
// TODO: Apply more sophisticated validation and decoding logic
self.propagate_message(id, peer_id.clone());
// TODO: Handle exits
debug!(self.log, "Received a voluntary exit"; "peer_id" => format!("{}", peer_id) );
}
@ -222,6 +244,8 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
PubsubMessage::ProposerSlashing(message) => {
match self.decode_gossip_proposer_slashing(message) {
Ok(_slashing) => {
// TODO: Apply more sophisticated validation and decoding logic
self.propagate_message(id, peer_id.clone());
// TODO: Handle proposer slashings
debug!(self.log, "Received a proposer slashing"; "peer_id" => format!("{}", peer_id) );
}
@ -233,6 +257,8 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
PubsubMessage::AttesterSlashing(message) => {
match self.decode_gossip_attestation_slashing(message) {
Ok(_slashing) => {
// TODO: Apply more sophisticated validation and decoding logic
self.propagate_message(id, peer_id.clone());
// TODO: Handle attester slashings
debug!(self.log, "Received an attester slashing"; "peer_id" => format!("{}", peer_id) );
}
@ -248,6 +274,21 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
}
}
/// Informs the network service that the message should be forwarded to other peers.
fn propagate_message(&mut self, message_id: String, propagation_source: PeerId) {
self.network_send
.try_send(NetworkMessage::Propagate {
propagation_source,
message_id,
})
.unwrap_or_else(|_| {
warn!(
self.log,
"Could not send propagation request to the network service"
)
});
}
/* Decoding of gossipsub objects from the network.
*
* The decoding is done in the message handler as it has access to to a `BeaconChain` and can

View File

@ -34,13 +34,8 @@ impl<T: BeaconChainTypes + 'static> Service<T> {
// build the network channel
let (network_send, network_recv) = mpsc::unbounded_channel::<NetworkMessage>();
// launch message handler thread
let message_handler_log = log.new(o!("Service" => "MessageHandler"));
let message_handler_send = MessageHandler::spawn(
beacon_chain,
network_send.clone(),
executor,
message_handler_log,
)?;
let message_handler_send =
MessageHandler::spawn(beacon_chain, network_send.clone(), executor, log.clone())?;
let network_log = log.new(o!("Service" => "Network"));
// launch libp2p service
@ -159,12 +154,23 @@ fn network_service(
// poll the network channel
match network_recv.poll() {
Ok(Async::Ready(Some(message))) => match message {
NetworkMessage::Send(peer_id, outgoing_message) => match outgoing_message {
OutgoingMessage::RPC(rpc_event) => {
trace!(log, "Sending RPC Event: {:?}", rpc_event);
libp2p_service.lock().swarm.send_rpc(peer_id, rpc_event);
}
},
NetworkMessage::RPC(peer_id, rpc_event) => {
trace!(log, "{}", rpc_event);
libp2p_service.lock().swarm.send_rpc(peer_id, rpc_event);
}
NetworkMessage::Propagate {
propagation_source,
message_id,
} => {
trace!(log, "Propagating gossipsub message";
"propagation_peer" => format!("{:?}", propagation_source),
"message_id" => format!("{}", message_id),
);
libp2p_service
.lock()
.swarm
.propagate_message(&propagation_source, message_id);
}
NetworkMessage::Publish { topics, message } => {
debug!(log, "Sending pubsub message"; "topics" => format!("{:?}",topics));
libp2p_service.lock().swarm.publish(&topics, message);
@ -185,7 +191,7 @@ fn network_service(
match libp2p_service.lock().poll() {
Ok(Async::Ready(Some(event))) => match event {
Libp2pEvent::RPC(peer_id, rpc_event) => {
trace!(log, "RPC Event: RPC message received: {:?}", rpc_event);
trace!(log, "{}", rpc_event);
message_handler_send
.try_send(HandlerMessage::RPC(peer_id, rpc_event))
.map_err(|_| "Failed to send RPC to handler")?;
@ -203,13 +209,14 @@ fn network_service(
.map_err(|_| "Failed to send PeerDisconnected to handler")?;
}
Libp2pEvent::PubsubMessage {
source, message, ..
id,
source,
message,
..
} => {
//TODO: Decide if we need to propagate the topic upwards. (Potentially for
//attestations)
message_handler_send
.try_send(HandlerMessage::PubsubMessage(source, message))
.map_err(|_| " failed to send pubsub message to handler")?;
.try_send(HandlerMessage::PubsubMessage(id, source, message))
.map_err(|_| "Failed to send pubsub message to handler")?;
}
},
Ok(Async::Ready(None)) => unreachable!("Stream never ends"),
@ -225,19 +232,16 @@ fn network_service(
/// Types of messages that the network service can receive.
#[derive(Debug)]
pub enum NetworkMessage {
/// Send a message to libp2p service.
//TODO: Define typing for messages across the wire
Send(PeerId, OutgoingMessage),
/// Publish a message to pubsub mechanism.
/// Send an RPC message to the libp2p service.
RPC(PeerId, RPCEvent),
/// Publish a message to gossipsub.
Publish {
topics: Vec<Topic>,
message: PubsubMessage,
},
}
/// Type of outgoing messages that can be sent through the network service.
#[derive(Debug)]
pub enum OutgoingMessage {
/// Send an RPC request/response.
RPC(RPCEvent),
/// Propagate a received gossipsub message
Propagate {
propagation_source: PeerId,
message_id: String,
},
}

File diff suppressed because it is too large Load Diff

View File

@ -4,7 +4,7 @@ mod manager;
/// Stores the various syncing methods for the beacon chain.
mod simple_sync;
pub use simple_sync::SimpleSync;
pub use simple_sync::MessageProcessor;
/// Currently implemented sync methods.
pub enum SyncMethod {

View File

@ -1,23 +1,26 @@
use super::manager::{ImportManager, ImportManagerOutcome};
use crate::service::{NetworkMessage, OutgoingMessage};
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
use super::manager::SyncMessage;
use crate::service::NetworkMessage;
use beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BlockProcessingOutcome,
};
use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::{RPCEvent, RPCRequest, RPCResponse, RequestId};
use eth2_libp2p::PeerId;
use slog::{debug, info, o, trace, warn};
use slog::{debug, error, info, o, trace, warn};
use ssz::Encode;
use std::ops::Sub;
use std::sync::Arc;
use store::Store;
use tokio::sync::mpsc;
use tokio::sync::{mpsc, oneshot};
use tree_hash::SignedRoot;
use types::{Attestation, BeaconBlock, Epoch, EthSpec, Hash256, Slot};
//TODO: Put a maximum limit on the number of block that can be requested.
//TODO: Rate limit requests
/// If a block is more than `FUTURE_SLOT_TOLERANCE` slots ahead of our slot clock, we drop it.
/// Otherwise we queue it.
pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1;
/// The number of slots behind our head that we still treat a peer as a fully synced peer.
const FULL_PEER_TOLERANCE: u64 = 10;
const SHOULD_FORWARD_GOSSIP_BLOCK: bool = true;
const SHOULD_NOT_FORWARD_GOSSIP_BLOCK: bool = false;
@ -49,45 +52,63 @@ impl<T: BeaconChainTypes> From<&Arc<BeaconChain<T>>> for PeerSyncInfo {
}
}
/// The current syncing state.
#[derive(PartialEq)]
pub enum SyncState {
_Idle,
_Downloading,
_Stopped,
}
/// Simple Syncing protocol.
pub struct SimpleSync<T: BeaconChainTypes> {
/// Processes validated messages from the network. It relays necessary data to the syncing thread
/// and processes blocks from the pubsub network.
pub struct MessageProcessor<T: BeaconChainTypes> {
/// A reference to the underlying beacon chain.
chain: Arc<BeaconChain<T>>,
manager: ImportManager<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 nextwork context to return and handle RPC requests.
network: NetworkContext,
/// The `RPCHandler` logger.
log: slog::Logger,
}
impl<T: BeaconChainTypes> SimpleSync<T> {
/// Instantiate a `SimpleSync` instance, with no peers and an empty queue.
impl<T: BeaconChainTypes> MessageProcessor<T> {
/// Instantiate a `MessageProcessor` instance
pub fn new(
executor: &tokio::runtime::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
network_send: mpsc::UnboundedSender<NetworkMessage>,
log: &slog::Logger,
) -> Self {
let sync_logger = log.new(o!("Service"=> "Sync"));
let sync_network_context = NetworkContext::new(network_send.clone(), sync_logger.clone());
SimpleSync {
chain: beacon_chain.clone(),
manager: ImportManager::new(beacon_chain, log),
// spawn the sync thread
let (sync_send, _sync_exit) = super::manager::spawn(
executor,
Arc::downgrade(&beacon_chain),
sync_network_context,
sync_logger,
);
MessageProcessor {
chain: beacon_chain,
sync_send,
_sync_exit,
network: NetworkContext::new(network_send, log.clone()),
log: sync_logger,
log: log.clone(),
}
}
fn send_to_sync(&mut self, message: SyncMessage<T::EthSpec>) {
self.sync_send.try_send(message).unwrap_or_else(|_| {
warn!(
self.log,
"Could not send message to the sync service";
)
});
}
/// Handle a peer disconnect.
///
/// Removes the peer from the manager.
pub fn on_disconnect(&mut self, peer_id: PeerId) {
self.manager.peer_disconnect(&peer_id);
self.send_to_sync(SyncMessage::Disconnect(peer_id));
}
/// Handle the connection of a new peer.
@ -107,6 +128,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
request_id: RequestId,
hello: HelloMessage,
) {
// ignore hello responses if we are shutting down
trace!(self.log, "HelloRequest"; "peer" => format!("{:?}", peer_id));
// Say hello back.
@ -149,7 +171,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
} else if remote.finalized_epoch <= local.finalized_epoch
&& remote.finalized_root != Hash256::zero()
&& local.finalized_root != Hash256::zero()
&& (self.root_at_slot(start_slot(remote.finalized_epoch))
&& (self.chain.root_at_slot(start_slot(remote.finalized_epoch))
!= Some(remote.finalized_root))
{
// The remotes finalized epoch is less than or greater than ours, but the block root is
@ -189,18 +211,16 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
.exists::<BeaconBlock<T::EthSpec>>(&remote.head_root)
.unwrap_or_else(|_| false)
{
trace!(
self.log, "Peer with known chain found";
"peer" => format!("{:?}", peer_id),
"remote_head_slot" => remote.head_slot,
"remote_latest_finalized_epoch" => remote.finalized_epoch,
);
// If the node's best-block is already known to us and they are close to our current
// head, treat them as a fully sync'd peer.
if self.chain.best_slot().sub(remote.head_slot).as_u64() < FULL_PEER_TOLERANCE {
self.manager.add_full_peer(peer_id);
self.process_sync();
} else {
debug!(
self.log,
"Out of sync peer connected";
"peer" => format!("{:?}", peer_id),
);
}
self.send_to_sync(SyncMessage::AddPeer(peer_id, remote));
} else {
// The remote node has an equal or great finalized epoch and we don't know it's head.
//
@ -212,87 +232,10 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
"local_finalized_epoch" => local.finalized_epoch,
"remote_latest_finalized_epoch" => remote.finalized_epoch,
);
self.manager.add_peer(peer_id, remote);
self.process_sync();
self.send_to_sync(SyncMessage::AddPeer(peer_id, remote));
}
}
fn process_sync(&mut self) {
loop {
match self.manager.poll() {
ImportManagerOutcome::Hello(peer_id) => {
trace!(
self.log,
"RPC Request";
"method" => "HELLO",
"peer" => format!("{:?}", peer_id)
);
self.network.send_rpc_request(
None,
peer_id,
RPCRequest::Hello(hello_message(&self.chain)),
);
}
ImportManagerOutcome::RequestBlocks {
peer_id,
request_id,
request,
} => {
trace!(
self.log,
"RPC Request";
"method" => "BeaconBlocks",
"id" => request_id,
"count" => request.count,
"peer" => format!("{:?}", peer_id)
);
self.network.send_rpc_request(
Some(request_id),
peer_id.clone(),
RPCRequest::BeaconBlocks(request),
);
}
ImportManagerOutcome::RecentRequest(peer_id, req) => {
trace!(
self.log,
"RPC Request";
"method" => "RecentBeaconBlocks",
"count" => req.block_roots.len(),
"peer" => format!("{:?}", peer_id)
);
self.network.send_rpc_request(
None,
peer_id.clone(),
RPCRequest::RecentBeaconBlocks(req),
);
}
ImportManagerOutcome::DownvotePeer(peer_id) => {
trace!(
self.log,
"Peer downvoted";
"peer" => format!("{:?}", peer_id)
);
// TODO: Implement reputation
self.network
.disconnect(peer_id.clone(), GoodbyeReason::Fault);
}
ImportManagerOutcome::Idle => {
// nothing to do
return;
}
}
}
}
//TODO: Move to beacon chain
fn root_at_slot(&self, target_slot: Slot) -> Option<Hash256> {
self.chain
.rev_iter_block_roots()
.find(|(_root, slot)| *slot == target_slot)
.map(|(root, _slot)| root)
}
/// Handle a `RecentBeaconBlocks` request from the peer.
pub fn on_recent_beacon_blocks_request(
&mut self,
@ -321,7 +264,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
debug!(
self.log,
"BlockBodiesRequest";
"RecentBeaconBlocksRequest";
"peer" => format!("{:?}", peer_id),
"requested" => request.block_roots.len(),
"returned" => blocks.len(),
@ -380,18 +323,16 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
blocks.reverse();
blocks.dedup_by_key(|brs| brs.slot);
if blocks.len() as u64 != req.count {
debug!(
self.log,
"BeaconBlocksRequest response";
"peer" => format!("{:?}", peer_id),
"msg" => "Failed to return all requested hashes",
"start_slot" => req.start_slot,
"current_slot" => self.chain.present_slot(),
"requested" => req.count,
"returned" => blocks.len(),
);
}
debug!(
self.log,
"BeaconBlocksRequest response";
"peer" => format!("{:?}", peer_id),
"msg" => "Failed to return all requested hashes",
"start_slot" => req.start_slot,
"current_slot" => self.chain.slot().unwrap_or_else(|_| Slot::from(0_u64)).as_u64(),
"requested" => req.count,
"returned" => blocks.len(),
);
self.network.send_rpc_response(
peer_id,
@ -414,10 +355,11 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
"count" => beacon_blocks.len(),
);
self.manager
.beacon_blocks_response(peer_id, request_id, beacon_blocks);
self.process_sync();
self.send_to_sync(SyncMessage::BeaconBlocksResponse {
peer_id,
request_id,
beacon_blocks,
});
}
/// Handle a `RecentBeaconBlocks` response from the peer.
@ -429,15 +371,16 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
) {
debug!(
self.log,
"BeaconBlocksResponse";
"RecentBeaconBlocksResponse";
"peer" => format!("{:?}", peer_id),
"count" => beacon_blocks.len(),
);
self.manager
.recent_blocks_response(peer_id, request_id, beacon_blocks);
self.process_sync();
self.send_to_sync(SyncMessage::RecentBeaconBlocksResponse {
peer_id,
request_id,
beacon_blocks,
});
}
/// Process a gossip message declaring a new block.
@ -446,8 +389,8 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
///
/// Returns a `bool` which, if `true`, indicates we should forward the block to our peers.
pub fn on_block_gossip(&mut self, peer_id: PeerId, block: BeaconBlock<T::EthSpec>) -> bool {
if let Ok(outcome) = self.chain.process_block(block.clone()) {
match outcome {
match self.chain.process_block(block.clone()) {
Ok(outcome) => match outcome {
BlockProcessingOutcome::Processed { .. } => {
trace!(self.log, "Gossipsub block processed";
"peer_id" => format!("{:?}",peer_id));
@ -455,9 +398,9 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
}
BlockProcessingOutcome::ParentUnknown { parent: _ } => {
// Inform the sync manager to find parents for this block
trace!(self.log, "Unknown parent gossip";
trace!(self.log, "Block with unknown parent received";
"peer_id" => format!("{:?}",peer_id));
self.manager.add_unknown_block(block.clone(), peer_id);
self.send_to_sync(SyncMessage::UnknownBlock(peer_id, block.clone()));
SHOULD_FORWARD_GOSSIP_BLOCK
}
BlockProcessingOutcome::FutureSlot {
@ -468,10 +411,36 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
SHOULD_FORWARD_GOSSIP_BLOCK
}
BlockProcessingOutcome::BlockIsAlreadyKnown => SHOULD_FORWARD_GOSSIP_BLOCK,
_ => SHOULD_NOT_FORWARD_GOSSIP_BLOCK,
other => {
warn!(
self.log,
"Invalid gossip beacon block";
"outcome" => format!("{:?}", other),
"block root" => format!("{}", Hash256::from_slice(&block.signed_root()[..])),
"block slot" => block.slot
);
trace!(
self.log,
"Invalid gossip beacon block ssz";
"ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())),
);
SHOULD_NOT_FORWARD_GOSSIP_BLOCK //TODO: Decide if we want to forward these
}
},
Err(e) => {
error!(
self.log,
"Error processing gossip beacon block";
"error" => format!("{:?}", e),
"block slot" => block.slot
);
trace!(
self.log,
"Erroneous gossip beacon block ssz";
"ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())),
);
SHOULD_NOT_FORWARD_GOSSIP_BLOCK
}
} else {
SHOULD_NOT_FORWARD_GOSSIP_BLOCK
}
}
@ -479,27 +448,37 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
///
/// Not currently implemented.
pub fn on_attestation_gossip(&mut self, _peer_id: PeerId, msg: Attestation<T::EthSpec>) {
match self.chain.process_attestation(msg) {
Ok(outcome) => info!(
self.log,
"Processed attestation";
"source" => "gossip",
"outcome" => format!("{:?}", outcome)
),
match self.chain.process_attestation(msg.clone()) {
Ok(outcome) => {
info!(
self.log,
"Processed attestation";
"source" => "gossip",
"outcome" => format!("{:?}", outcome)
);
if outcome != AttestationProcessingOutcome::Processed {
trace!(
self.log,
"Invalid gossip attestation ssz";
"ssz" => format!("0x{}", hex::encode(msg.as_ssz_bytes())),
);
}
}
Err(e) => {
warn!(self.log, "InvalidAttestation"; "source" => "gossip", "error" => format!("{:?}", e))
trace!(
self.log,
"Erroneous gossip attestation ssz";
"ssz" => format!("0x{}", hex::encode(msg.as_ssz_bytes())),
);
error!(self.log, "Invalid gossip attestation"; "error" => format!("{:?}", e));
}
}
}
/// Generates our current state in the form of a HELLO RPC message.
pub fn generate_hello(&self) -> HelloMessage {
hello_message(&self.chain)
}
}
/// Build a `HelloMessage` representing the state of the given `beacon_chain`.
fn hello_message<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) -> HelloMessage {
pub(crate) fn hello_message<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) -> HelloMessage {
let state = &beacon_chain.head().beacon_state;
HelloMessage {
@ -525,6 +504,12 @@ impl NetworkContext {
}
pub fn disconnect(&mut self, peer_id: PeerId, reason: GoodbyeReason) {
warn!(
&self.log,
"Disconnecting peer (RPC)";
"reason" => format!("{:?}", reason),
"peer_id" => format!("{:?}", peer_id),
);
self.send_rpc_request(None, peer_id, RPCRequest::Goodbye(reason))
// TODO: disconnect peers.
}
@ -554,12 +539,8 @@ impl NetworkContext {
}
fn send_rpc_event(&mut self, peer_id: PeerId, rpc_event: RPCEvent) {
self.send(peer_id, OutgoingMessage::RPC(rpc_event))
}
fn send(&mut self, peer_id: PeerId, outgoing_message: OutgoingMessage) {
self.network_send
.try_send(NetworkMessage::Send(peer_id, outgoing_message))
.try_send(NetworkMessage::RPC(peer_id, rpc_event))
.unwrap_or_else(|_| {
warn!(
self.log,

View File

@ -14,20 +14,26 @@ store = { path = "../store" }
version = { path = "../version" }
serde = { version = "1.0", features = ["derive"] }
serde_json = "^1.0"
serde_yaml = "0.8"
slog = "^2.2.3"
slog-term = "^2.4.0"
slog-async = "^2.3.0"
eth2_ssz = { path = "../../eth2/utils/ssz" }
eth2_ssz_derive = { path = "../../eth2/utils/ssz_derive" }
state_processing = { path = "../../eth2/state_processing" }
types = { path = "../../eth2/types" }
clap = "2.32.0"
http = "^0.1.17"
prometheus = { version = "^0.6", features = ["process"] }
hyper = "0.12.32"
futures = "0.1"
hyper = "0.12.34"
exit-future = "0.1.3"
tokio = "0.1.17"
url = "2.0"
lazy_static = "1.3.0"
eth2_config = { path = "../../eth2/utils/eth2_config" }
lighthouse_metrics = { path = "../../eth2/utils/lighthouse_metrics" }
slot_clock = { path = "../../eth2/utils/slot_clock" }
hex = "0.3.2"
parking_lot = "0.9"
futures = "0.1.25"

View File

@ -1,17 +1,25 @@
use super::{success_response, ApiResult};
use crate::{helpers::*, ApiError, UrlQuery};
use crate::helpers::*;
use crate::response_builder::ResponseBuilder;
use crate::{ApiError, ApiResult, UrlQuery};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use hyper::{Body, Request};
use serde::Serialize;
use ssz_derive::Encode;
use std::sync::Arc;
use store::Store;
use types::{BeaconBlock, BeaconState, EthSpec, Hash256, Slot};
use types::{BeaconBlock, BeaconState, Epoch, EthSpec, Hash256, Slot, Validator};
#[derive(Serialize)]
#[derive(Serialize, Encode)]
pub struct HeadResponse {
pub slot: Slot,
pub block_root: Hash256,
pub state_root: Hash256,
pub finalized_slot: Slot,
pub finalized_block_root: Hash256,
pub justified_slot: Slot,
pub justified_block_root: Hash256,
pub previous_justified_slot: Slot,
pub previous_justified_block_root: Hash256,
}
/// HTTP handler to return a `BeaconBlock` at a given `root` or `slot`.
@ -21,19 +29,36 @@ pub fn get_head<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let chain_head = beacon_chain.head();
let head = HeadResponse {
slot: beacon_chain.head().beacon_state.slot,
block_root: beacon_chain.head().beacon_block_root,
state_root: beacon_chain.head().beacon_state_root,
slot: chain_head.beacon_state.slot,
block_root: chain_head.beacon_block_root,
state_root: chain_head.beacon_state_root,
finalized_slot: chain_head
.beacon_state
.finalized_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
finalized_block_root: chain_head.beacon_state.finalized_checkpoint.root,
justified_slot: chain_head
.beacon_state
.current_justified_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
justified_block_root: chain_head.beacon_state.current_justified_checkpoint.root,
previous_justified_slot: chain_head
.beacon_state
.previous_justified_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
previous_justified_block_root: chain_head.beacon_state.previous_justified_checkpoint.root,
};
let json: String = serde_json::to_string(&head)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize HeadResponse: {:?}", e)))?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body(&head)
}
#[derive(Serialize)]
#[derive(Serialize, Encode)]
#[serde(bound = "T: EthSpec")]
pub struct BlockResponse<T: EthSpec> {
pub root: Hash256,
@ -55,7 +80,7 @@ pub fn get_block<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
let target = parse_slot(&value)?;
block_root_at_slot(&beacon_chain, target).ok_or_else(|| {
ApiError::NotFound(format!("Unable to find BeaconBlock for slot {}", target))
ApiError::NotFound(format!("Unable to find BeaconBlock for slot {:?}", target))
})?
}
("root", value) => parse_root(&value)?,
@ -67,7 +92,7 @@ pub fn get_block<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
.get::<BeaconBlock<T::EthSpec>>(&block_root)?
.ok_or_else(|| {
ApiError::NotFound(format!(
"Unable to find BeaconBlock for root {}",
"Unable to find BeaconBlock for root {:?}",
block_root
))
})?;
@ -77,34 +102,62 @@ pub fn get_block<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
beacon_block: block,
};
let json: String = serde_json::to_string(&response).map_err(|e| {
ApiError::ServerError(format!("Unable to serialize BlockResponse: {:?}", e))
})?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body(&response)
}
/// HTTP handler to return a `BeaconBlock` root at a given `slot`.
pub fn get_block_root<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let slot_string = UrlQuery::from_request(&req)?.only_one("slot")?;
let target = parse_slot(&slot_string)?;
let root = block_root_at_slot(&beacon_chain, target).ok_or_else(|| {
ApiError::NotFound(format!("Unable to find BeaconBlock for slot {}", target))
ApiError::NotFound(format!("Unable to find BeaconBlock for slot {:?}", target))
})?;
let json: String = serde_json::to_string(&root)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize root: {:?}", e)))?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body(&root)
}
#[derive(Serialize)]
/// HTTP handler to return the `Fork` of the current head.
pub fn get_fork<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
ResponseBuilder::new(&req)?.body(&beacon_chain.head().beacon_state.fork)
}
/// HTTP handler to return the set of validators for an `Epoch`
///
/// The `Epoch` parameter can be any epoch number. If it is not specified,
/// the current epoch is assumed.
pub fn get_validators<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let epoch = match UrlQuery::from_request(&req) {
// We have some parameters, so make sure it's the epoch one and parse it
Ok(query) => query
.only_one("epoch")?
.parse::<u64>()
.map(Epoch::from)
.map_err(|e| {
ApiError::BadRequest(format!("Invalid epoch parameter, must be a u64. {:?}", e))
})?,
// In this case, our url query did not contain any parameters, so we take the default
Err(_) => beacon_chain.epoch().map_err(|e| {
ApiError::ServerError(format!("Unable to determine current epoch: {:?}", e))
})?,
};
let all_validators = &beacon_chain.head().beacon_state.validators;
let active_vals: Vec<Validator> = all_validators
.iter()
.filter(|v| v.is_active_at(epoch))
.cloned()
.collect();
ResponseBuilder::new(&req)?.body(&active_vals)
}
#[derive(Serialize, Encode)]
#[serde(bound = "T: EthSpec")]
pub struct StateResponse<T: EthSpec> {
pub root: Hash256,
@ -116,13 +169,23 @@ pub struct StateResponse<T: EthSpec> {
/// Will not return a state if the request slot is in the future. Will return states higher than
/// the current head by skipping slots.
pub fn get_state<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let head_state = beacon_chain.head().beacon_state;
let query_params = ["root", "slot"];
let (key, value) = UrlQuery::from_request(&req)?.first_of(&query_params)?;
let (key, value) = match UrlQuery::from_request(&req) {
Ok(query) => {
// We have *some* parameters, just check them.
let query_params = ["root", "slot"];
query.first_of(&query_params)?
}
Err(ApiError::BadRequest(_)) => {
// No parameters provided at all, use current slot.
(String::from("slot"), head_state.slot.to_string())
}
Err(e) => {
return Err(e);
}
};
let (root, state): (Hash256, BeaconState<T::EthSpec>) = match (key.as_ref(), value) {
("slot", value) => state_at_slot(&beacon_chain, parse_slot(&value)?)?,
@ -132,7 +195,7 @@ pub fn get_state<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
let state = beacon_chain
.store
.get(root)?
.ok_or_else(|| ApiError::NotFound(format!("No state for root: {}", root)))?;
.ok_or_else(|| ApiError::NotFound(format!("No state for root: {:?}", root)))?;
(*root, state)
}
@ -144,11 +207,7 @@ pub fn get_state<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
beacon_state: state,
};
let json: String = serde_json::to_string(&response).map_err(|e| {
ApiError::ServerError(format!("Unable to serialize StateResponse: {:?}", e))
})?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body(&response)
}
/// HTTP handler to return a `BeaconState` root at a given `slot`.
@ -156,39 +215,33 @@ pub fn get_state<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult
/// Will not return a state if the request slot is in the future. Will return states higher than
/// the current head by skipping slots.
pub fn get_state_root<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let slot_string = UrlQuery::from_request(&req)?.only_one("slot")?;
let slot = parse_slot(&slot_string)?;
let root = state_root_at_slot(&beacon_chain, slot)?;
let json: String = serde_json::to_string(&root)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize root: {:?}", e)))?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body(&root)
}
/// HTTP handler to return the highest finalized slot.
pub fn get_latest_finalized_checkpoint<T: BeaconChainTypes + 'static>(
pub fn get_current_finalized_checkpoint<T: BeaconChainTypes + 'static>(
req: Request<Body>,
) -> ApiResult {
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let head_state = beacon_chain.head().beacon_state;
let checkpoint = beacon_chain
.head()
.beacon_state
.finalized_checkpoint
.clone();
let checkpoint = head_state.finalized_checkpoint.clone();
let json: String = serde_json::to_string(&checkpoint)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize checkpoint: {:?}", e)))?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body(&checkpoint)
}
/// HTTP handler to return a `BeaconState` at the genesis block.
pub fn get_genesis_state<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let (_root, state) = state_at_slot(&beacon_chain, Slot::new(0))?;
ResponseBuilder::new(&req)?.body(&state)
}

View File

@ -16,7 +16,7 @@ pub struct Config {
impl Default for Config {
fn default() -> Self {
Config {
enabled: true, // rest_api enabled by default
enabled: true,
listen_address: Ipv4Addr::new(127, 0, 0, 1),
port: 5052,
}
@ -25,8 +25,8 @@ impl Default for Config {
impl Config {
pub fn apply_cli_args(&mut self, args: &ArgMatches) -> Result<(), &'static str> {
if args.is_present("api") {
self.enabled = true;
if args.is_present("no-api") {
self.enabled = false;
}
if let Some(rpc_address) = args.value_of("api-address") {

View File

@ -0,0 +1,86 @@
use crate::BoxFut;
use hyper::{Body, Response, StatusCode};
use std::error::Error as StdError;
#[derive(PartialEq, Debug, Clone)]
pub enum ApiError {
MethodNotAllowed(String),
ServerError(String),
NotImplemented(String),
BadRequest(String),
NotFound(String),
UnsupportedType(String),
ImATeapot(String), // Just in case.
ProcessingError(String), // A 202 error, for when a block/attestation cannot be processed, but still transmitted.
}
pub type ApiResult = Result<Response<Body>, ApiError>;
impl ApiError {
pub fn status_code(self) -> (StatusCode, String) {
match self {
ApiError::MethodNotAllowed(desc) => (StatusCode::METHOD_NOT_ALLOWED, desc),
ApiError::ServerError(desc) => (StatusCode::INTERNAL_SERVER_ERROR, desc),
ApiError::NotImplemented(desc) => (StatusCode::NOT_IMPLEMENTED, desc),
ApiError::BadRequest(desc) => (StatusCode::BAD_REQUEST, desc),
ApiError::NotFound(desc) => (StatusCode::NOT_FOUND, desc),
ApiError::UnsupportedType(desc) => (StatusCode::UNSUPPORTED_MEDIA_TYPE, desc),
ApiError::ImATeapot(desc) => (StatusCode::IM_A_TEAPOT, desc),
ApiError::ProcessingError(desc) => (StatusCode::ACCEPTED, desc),
}
}
}
impl Into<Response<Body>> for ApiError {
fn into(self) -> Response<Body> {
let status_code = self.status_code();
Response::builder()
.status(status_code.0)
.header("content-type", "text/plain; charset=utf-8")
.body(Body::from(status_code.1))
.expect("Response should always be created.")
}
}
impl Into<BoxFut> for ApiError {
fn into(self) -> BoxFut {
Box::new(futures::future::err(self))
}
}
impl From<store::Error> for ApiError {
fn from(e: store::Error) -> ApiError {
ApiError::ServerError(format!("Database error: {:?}", e))
}
}
impl From<types::BeaconStateError> for ApiError {
fn from(e: types::BeaconStateError) -> ApiError {
ApiError::ServerError(format!("BeaconState error: {:?}", e))
}
}
impl From<state_processing::per_slot_processing::Error> for ApiError {
fn from(e: state_processing::per_slot_processing::Error) -> ApiError {
ApiError::ServerError(format!("PerSlotProcessing error: {:?}", e))
}
}
impl From<hyper::error::Error> for ApiError {
fn from(e: hyper::error::Error) -> ApiError {
ApiError::ServerError(format!("Networking error: {:?}", e))
}
}
impl StdError for ApiError {
fn cause(&self) -> Option<&dyn StdError> {
None
}
}
impl std::fmt::Display for ApiError {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
let status = self.clone().status_code();
write!(f, "{:?}: {:?}", status.0, status.1)
}
}

View File

@ -1,12 +1,20 @@
use crate::{ApiError, ApiResult};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use bls::PublicKey;
use eth2_libp2p::{PubsubMessage, Topic};
use eth2_libp2p::{
BEACON_ATTESTATION_TOPIC, BEACON_BLOCK_TOPIC, TOPIC_ENCODING_POSTFIX, TOPIC_PREFIX,
};
use hex;
use hyper::{Body, Request, StatusCode};
use serde::de::value::StringDeserializer;
use serde_json::Deserializer;
use http::header;
use hyper::{Body, Request};
use network::NetworkMessage;
use parking_lot::RwLock;
use ssz::Encode;
use std::sync::Arc;
use store::{iter::AncestorIter, Store};
use types::{BeaconState, EthSpec, Hash256, RelativeEpoch, Slot};
use tokio::sync::mpsc;
use types::{Attestation, BeaconBlock, BeaconState, EthSpec, Hash256, RelativeEpoch, Slot};
/// Parse a slot from a `0x` preixed string.
///
@ -15,7 +23,22 @@ pub fn parse_slot(string: &str) -> Result<Slot, ApiError> {
string
.parse::<u64>()
.map(Slot::from)
.map_err(|e| ApiError::InvalidQueryParams(format!("Unable to parse slot: {:?}", e)))
.map_err(|e| ApiError::BadRequest(format!("Unable to parse slot: {:?}", e)))
}
/// Checks the provided request to ensure that the `content-type` header.
///
/// The content-type header should either be omitted, in which case JSON is assumed, or it should
/// explicity specify `application/json`. If anything else is provided, an error is returned.
pub fn check_content_type_for_json(req: &Request<Body>) -> Result<(), ApiError> {
match req.headers().get(header::CONTENT_TYPE) {
Some(h) if h == "application/json" => Ok(()),
Some(h) => Err(ApiError::BadRequest(format!(
"The provided content-type {:?} is not available, this endpoint only supports json.",
h
))),
_ => Ok(()),
}
}
/// Parse a root from a `0x` preixed string.
@ -28,9 +51,9 @@ pub fn parse_root(string: &str) -> Result<Hash256, ApiError> {
let trimmed = string.trim_start_matches(PREFIX);
trimmed
.parse()
.map_err(|e| ApiError::InvalidQueryParams(format!("Unable to parse root: {:?}", e)))
.map_err(|e| ApiError::BadRequest(format!("Unable to parse root: {:?}", e)))
} else {
Err(ApiError::InvalidQueryParams(
Err(ApiError::BadRequest(
"Root must have a '0x' prefix".to_string(),
))
}
@ -41,13 +64,13 @@ pub fn parse_pubkey(string: &str) -> Result<PublicKey, ApiError> {
const PREFIX: &str = "0x";
if string.starts_with(PREFIX) {
let pubkey_bytes = hex::decode(string.trim_start_matches(PREFIX))
.map_err(|e| ApiError::InvalidQueryParams(format!("Invalid hex string: {:?}", e)))?;
.map_err(|e| ApiError::BadRequest(format!("Invalid hex string: {:?}", e)))?;
let pubkey = PublicKey::from_bytes(pubkey_bytes.as_slice()).map_err(|e| {
ApiError::InvalidQueryParams(format!("Unable to deserialize public key: {:?}.", e))
ApiError::BadRequest(format!("Unable to deserialize public key: {:?}.", e))
})?;
return Ok(pubkey);
} else {
return Err(ApiError::InvalidQueryParams(
return Err(ApiError::BadRequest(
"Public key must have a '0x' prefix".to_string(),
));
}
@ -110,8 +133,8 @@ pub fn state_root_at_slot<T: BeaconChainTypes>(
) -> Result<Hash256, ApiError> {
let head_state = &beacon_chain.head().beacon_state;
let current_slot = beacon_chain
.read_slot_clock()
.ok_or_else(|| ApiError::ServerError("Unable to read slot clock".to_string()))?;
.slot()
.map_err(|_| ApiError::ServerError("Unable to read slot clock".to_string()))?;
// There are four scenarios when obtaining a state for a given slot:
//
@ -124,7 +147,7 @@ pub fn state_root_at_slot<T: BeaconChainTypes>(
//
// We could actually speculate about future state roots by skipping slots, however that's
// likely to cause confusion for API users.
Err(ApiError::InvalidQueryParams(format!(
Err(ApiError::BadRequest(format!(
"Requested slot {} is past the current slot {}",
slot, current_slot
)))
@ -171,6 +194,78 @@ pub fn implementation_pending_response(_req: Request<Body>) -> ApiResult {
))
}
pub fn get_beacon_chain_from_request<T: BeaconChainTypes + 'static>(
req: &Request<Body>,
) -> Result<(Arc<BeaconChain<T>>), ApiError> {
// Get beacon state
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".into()))?;
Ok(beacon_chain.clone())
}
pub fn get_logger_from_request(req: &Request<Body>) -> slog::Logger {
let log = req
.extensions()
.get::<slog::Logger>()
.expect("Should always get the logger from the request, since we put it in there.");
log.to_owned()
}
pub fn publish_beacon_block_to_network<T: BeaconChainTypes + 'static>(
chan: Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>,
block: BeaconBlock<T::EthSpec>,
) -> Result<(), ApiError> {
// create the network topic to send on
let topic_string = format!(
"/{}/{}/{}",
TOPIC_PREFIX, BEACON_BLOCK_TOPIC, TOPIC_ENCODING_POSTFIX
);
let topic = Topic::new(topic_string);
let message = PubsubMessage::Block(block.as_ssz_bytes());
// Publish the block to the p2p network via gossipsub.
if let Err(e) = chan.write().try_send(NetworkMessage::Publish {
topics: vec![topic],
message,
}) {
return Err(ApiError::ServerError(format!(
"Unable to send new block to network: {:?}",
e
)));
}
Ok(())
}
pub fn publish_attestation_to_network<T: BeaconChainTypes + 'static>(
chan: Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>,
attestation: Attestation<T::EthSpec>,
) -> Result<(), ApiError> {
// create the network topic to send on
let topic_string = format!(
"/{}/{}/{}",
TOPIC_PREFIX, BEACON_ATTESTATION_TOPIC, TOPIC_ENCODING_POSTFIX
);
let topic = Topic::new(topic_string);
let message = PubsubMessage::Attestation(attestation.as_ssz_bytes());
// Publish the attestation to the p2p network via gossipsub.
if let Err(e) = chan.write().try_send(NetworkMessage::Publish {
topics: vec![topic],
message,
}) {
return Err(ApiError::ServerError(format!(
"Unable to send new attestation to network: {:?}",
e
)));
}
Ok(())
}
#[cfg(test)]
mod test {
use super::*;

View File

@ -1,76 +1,192 @@
#[macro_use]
mod macros;
#[macro_use]
extern crate lazy_static;
extern crate network as client_network;
mod beacon;
mod config;
mod error;
mod helpers;
mod metrics;
mod network;
mod node;
mod response_builder;
mod spec;
mod url_query;
mod validator;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use client_network::NetworkMessage;
use client_network::Service as NetworkService;
use error::{ApiError, ApiResult};
use eth2_config::Eth2Config;
use futures::future::IntoFuture;
use hyper::rt::Future;
use hyper::service::service_fn_ok;
use hyper::{Body, Method, Response, Server, StatusCode};
use hyper::service::Service;
use hyper::{Body, Method, Request, Response, Server};
use parking_lot::RwLock;
use slog::{info, o, warn};
use std::ops::Deref;
use std::path::PathBuf;
use std::sync::Arc;
use tokio::runtime::TaskExecutor;
use tokio::sync::mpsc;
use url_query::UrlQuery;
pub use beacon::{BlockResponse, HeadResponse, StateResponse};
pub use config::Config as ApiConfig;
#[derive(PartialEq, Debug)]
pub enum ApiError {
MethodNotAllowed(String),
ServerError(String),
NotImplemented(String),
InvalidQueryParams(String),
NotFound(String),
ImATeapot(String), // Just in case.
type BoxFut = Box<dyn Future<Item = Response<Body>, Error = ApiError> + Send>;
pub struct ApiService<T: BeaconChainTypes + 'static> {
log: slog::Logger,
beacon_chain: Arc<BeaconChain<T>>,
db_path: DBPath,
network_service: Arc<NetworkService<T>>,
network_channel: Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>,
eth2_config: Arc<Eth2Config>,
}
pub type ApiResult = Result<Response<Body>, ApiError>;
fn into_boxfut<F: IntoFuture + 'static>(item: F) -> BoxFut
where
F: IntoFuture<Item = Response<Body>, Error = ApiError>,
F::Future: Send,
{
Box::new(item.into_future())
}
impl Into<Response<Body>> for ApiError {
fn into(self) -> Response<Body> {
let status_code: (StatusCode, String) = match self {
ApiError::MethodNotAllowed(desc) => (StatusCode::METHOD_NOT_ALLOWED, desc),
ApiError::ServerError(desc) => (StatusCode::INTERNAL_SERVER_ERROR, desc),
ApiError::NotImplemented(desc) => (StatusCode::NOT_IMPLEMENTED, desc),
ApiError::InvalidQueryParams(desc) => (StatusCode::BAD_REQUEST, desc),
ApiError::NotFound(desc) => (StatusCode::NOT_FOUND, desc),
ApiError::ImATeapot(desc) => (StatusCode::IM_A_TEAPOT, desc),
impl<T: BeaconChainTypes> Service for ApiService<T> {
type ReqBody = Body;
type ResBody = Body;
type Error = ApiError;
type Future = BoxFut;
fn call(&mut self, mut req: Request<Body>) -> Self::Future {
metrics::inc_counter(&metrics::REQUEST_COUNT);
let timer = metrics::start_timer(&metrics::REQUEST_RESPONSE_TIME);
// Add all the useful bits into the request, so that we can pull them out in the individual
// functions.
req.extensions_mut()
.insert::<slog::Logger>(self.log.clone());
req.extensions_mut()
.insert::<Arc<BeaconChain<T>>>(self.beacon_chain.clone());
req.extensions_mut().insert::<DBPath>(self.db_path.clone());
req.extensions_mut()
.insert::<Arc<NetworkService<T>>>(self.network_service.clone());
req.extensions_mut()
.insert::<Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>>(
self.network_channel.clone(),
);
req.extensions_mut()
.insert::<Arc<Eth2Config>>(self.eth2_config.clone());
let path = req.uri().path().to_string();
// Route the request to the correct handler.
let result = match (req.method(), path.as_ref()) {
// Methods for Client
(&Method::GET, "/node/version") => into_boxfut(node::get_version(req)),
(&Method::GET, "/node/genesis_time") => into_boxfut(node::get_genesis_time::<T>(req)),
(&Method::GET, "/node/syncing") => {
into_boxfut(helpers::implementation_pending_response(req))
}
// Methods for Network
(&Method::GET, "/network/enr") => into_boxfut(network::get_enr::<T>(req)),
(&Method::GET, "/network/peer_count") => into_boxfut(network::get_peer_count::<T>(req)),
(&Method::GET, "/network/peer_id") => into_boxfut(network::get_peer_id::<T>(req)),
(&Method::GET, "/network/peers") => into_boxfut(network::get_peer_list::<T>(req)),
(&Method::GET, "/network/listen_port") => {
into_boxfut(network::get_listen_port::<T>(req))
}
(&Method::GET, "/network/listen_addresses") => {
into_boxfut(network::get_listen_addresses::<T>(req))
}
// Methods for Beacon Node
(&Method::GET, "/beacon/head") => into_boxfut(beacon::get_head::<T>(req)),
(&Method::GET, "/beacon/block") => into_boxfut(beacon::get_block::<T>(req)),
(&Method::GET, "/beacon/block_root") => into_boxfut(beacon::get_block_root::<T>(req)),
(&Method::GET, "/beacon/blocks") => {
into_boxfut(helpers::implementation_pending_response(req))
}
(&Method::GET, "/beacon/fork") => into_boxfut(beacon::get_fork::<T>(req)),
(&Method::GET, "/beacon/attestations") => {
into_boxfut(helpers::implementation_pending_response(req))
}
(&Method::GET, "/beacon/attestations/pending") => {
into_boxfut(helpers::implementation_pending_response(req))
}
(&Method::GET, "/beacon/validators") => into_boxfut(beacon::get_validators::<T>(req)),
(&Method::GET, "/beacon/validators/indicies") => {
into_boxfut(helpers::implementation_pending_response(req))
}
(&Method::GET, "/beacon/validators/pubkeys") => {
into_boxfut(helpers::implementation_pending_response(req))
}
// Methods for Validator
(&Method::GET, "/beacon/validator/duties") => {
into_boxfut(validator::get_validator_duties::<T>(req))
}
(&Method::GET, "/beacon/validator/block") => {
into_boxfut(validator::get_new_beacon_block::<T>(req))
}
(&Method::POST, "/beacon/validator/block") => validator::publish_beacon_block::<T>(req),
(&Method::GET, "/beacon/validator/attestation") => {
into_boxfut(validator::get_new_attestation::<T>(req))
}
(&Method::POST, "/beacon/validator/attestation") => {
validator::publish_attestation::<T>(req)
}
(&Method::GET, "/beacon/state") => into_boxfut(beacon::get_state::<T>(req)),
(&Method::GET, "/beacon/state_root") => into_boxfut(beacon::get_state_root::<T>(req)),
(&Method::GET, "/beacon/state/current_finalized_checkpoint") => {
into_boxfut(beacon::get_current_finalized_checkpoint::<T>(req))
}
(&Method::GET, "/beacon/state/genesis") => {
into_boxfut(beacon::get_genesis_state::<T>(req))
}
//TODO: Add aggreggate/filtered state lookups here, e.g. /beacon/validators/balances
// Methods for bootstrap and checking configuration
(&Method::GET, "/spec") => into_boxfut(spec::get_spec::<T>(req)),
(&Method::GET, "/spec/slots_per_epoch") => {
into_boxfut(spec::get_slots_per_epoch::<T>(req))
}
(&Method::GET, "/spec/deposit_contract") => {
into_boxfut(helpers::implementation_pending_response(req))
}
(&Method::GET, "/spec/eth2_config") => into_boxfut(spec::get_eth2_config::<T>(req)),
(&Method::GET, "/metrics") => into_boxfut(metrics::get_prometheus::<T>(req)),
_ => Box::new(futures::future::err(ApiError::NotFound(
"Request path and/or method not found.".to_owned(),
))),
};
Response::builder()
.status(status_code.0)
.body(Body::from(status_code.1))
.expect("Response should always be created.")
}
}
impl From<store::Error> for ApiError {
fn from(e: store::Error) -> ApiError {
ApiError::ServerError(format!("Database error: {:?}", e))
}
}
let response = match result.wait() {
// Return the `hyper::Response`.
Ok(response) => {
metrics::inc_counter(&metrics::SUCCESS_COUNT);
slog::debug!(self.log, "Request successful: {:?}", path);
response
}
// Map the `ApiError` into `hyper::Response`.
Err(e) => {
slog::debug!(self.log, "Request failure: {:?}", path);
e.into()
}
};
impl From<types::BeaconStateError> for ApiError {
fn from(e: types::BeaconStateError) -> ApiError {
ApiError::ServerError(format!("BeaconState error: {:?}", e))
}
}
metrics::stop_timer(timer);
impl From<state_processing::per_slot_processing::Error> for ApiError {
fn from(e: state_processing::per_slot_processing::Error) -> ApiError {
ApiError::ServerError(format!("PerSlotProcessing error: {:?}", e))
Box::new(futures::future::ok(response))
}
}
@ -79,7 +195,9 @@ pub fn start_server<T: BeaconChainTypes>(
executor: &TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>,
network_service: Arc<NetworkService<T>>,
network_chan: mpsc::UnboundedSender<NetworkMessage>,
db_path: PathBuf,
eth2_config: Eth2Config,
log: &slog::Logger,
) -> Result<exit_future::Signal, hyper::Error> {
let log = log.new(o!("Service" => "Api"));
@ -101,113 +219,16 @@ pub fn start_server<T: BeaconChainTypes>(
// Clone our stateful objects, for use in service closure.
let server_log = log.clone();
let server_bc = beacon_chain.clone();
let eth2_config = Arc::new(eth2_config);
let service = move || {
let log = server_log.clone();
let beacon_chain = server_bc.clone();
let db_path = db_path.clone();
let network_service = network_service.clone();
// Create a simple handler for the router, inject our stateful objects into the request.
service_fn_ok(move |mut req| {
metrics::inc_counter(&metrics::REQUEST_COUNT);
let timer = metrics::start_timer(&metrics::REQUEST_RESPONSE_TIME);
req.extensions_mut().insert::<slog::Logger>(log.clone());
req.extensions_mut()
.insert::<Arc<BeaconChain<T>>>(beacon_chain.clone());
req.extensions_mut().insert::<DBPath>(db_path.clone());
req.extensions_mut()
.insert::<Arc<NetworkService<T>>>(network_service.clone());
let path = req.uri().path().to_string();
// Route the request to the correct handler.
let result = match (req.method(), path.as_ref()) {
// Methods for Beacon Node
//TODO: Remove?
//(&Method::GET, "/beacon/best_slot") => beacon::get_best_slot::<T>(req),
(&Method::GET, "/beacon/head") => beacon::get_head::<T>(req),
(&Method::GET, "/beacon/block") => beacon::get_block::<T>(req),
(&Method::GET, "/beacon/blocks") => helpers::implementation_pending_response(req),
//TODO Is the below replaced by finalized_checkpoint?
(&Method::GET, "/beacon/chainhead") => {
helpers::implementation_pending_response(req)
}
(&Method::GET, "/beacon/block_root") => beacon::get_block_root::<T>(req),
(&Method::GET, "/beacon/latest_finalized_checkpoint") => {
beacon::get_latest_finalized_checkpoint::<T>(req)
}
(&Method::GET, "/beacon/state") => beacon::get_state::<T>(req),
(&Method::GET, "/beacon/state_root") => beacon::get_state_root::<T>(req),
//TODO: Add aggreggate/filtered state lookups here, e.g. /beacon/validators/balances
// Methods for Client
(&Method::GET, "/metrics") => metrics::get_prometheus::<T>(req),
(&Method::GET, "/network/enr") => network::get_enr::<T>(req),
(&Method::GET, "/network/peer_count") => network::get_peer_count::<T>(req),
(&Method::GET, "/network/peer_id") => network::get_peer_id::<T>(req),
(&Method::GET, "/network/peers") => network::get_peer_list::<T>(req),
(&Method::GET, "/network/listen_port") => network::get_listen_port::<T>(req),
(&Method::GET, "/network/listen_addresses") => {
network::get_listen_addresses::<T>(req)
}
(&Method::GET, "/node/version") => node::get_version(req),
(&Method::GET, "/node/genesis_time") => node::get_genesis_time::<T>(req),
(&Method::GET, "/node/deposit_contract") => {
helpers::implementation_pending_response(req)
}
(&Method::GET, "/node/syncing") => helpers::implementation_pending_response(req),
(&Method::GET, "/node/fork") => helpers::implementation_pending_response(req),
// Methods for Network
(&Method::GET, "/network/enr") => network::get_enr::<T>(req),
(&Method::GET, "/network/peer_count") => network::get_peer_count::<T>(req),
(&Method::GET, "/network/peer_id") => network::get_peer_id::<T>(req),
(&Method::GET, "/network/peers") => network::get_peer_list::<T>(req),
(&Method::GET, "/network/listen_addresses") => {
network::get_listen_addresses::<T>(req)
}
// Methods for Validator
(&Method::GET, "/validator/duties") => validator::get_validator_duties::<T>(req),
(&Method::GET, "/validator/block") => helpers::implementation_pending_response(req),
(&Method::POST, "/validator/block") => {
helpers::implementation_pending_response(req)
}
(&Method::GET, "/validator/attestation") => {
helpers::implementation_pending_response(req)
}
(&Method::POST, "/validator/attestation") => {
helpers::implementation_pending_response(req)
}
(&Method::GET, "/spec") => spec::get_spec::<T>(req),
(&Method::GET, "/spec/slots_per_epoch") => spec::get_slots_per_epoch::<T>(req),
_ => Err(ApiError::NotFound(
"Request path and/or method not found.".to_owned(),
)),
};
let response = match result {
// Return the `hyper::Response`.
Ok(response) => {
metrics::inc_counter(&metrics::SUCCESS_COUNT);
slog::debug!(log, "Request successful: {:?}", path);
response
}
// Map the `ApiError` into `hyper::Response`.
Err(e) => {
slog::debug!(log, "Request failure: {:?}", path);
e.into()
}
};
metrics::stop_timer(timer);
response
let service = move || -> futures::future::FutureResult<ApiService<T>, String> {
futures::future::ok(ApiService {
log: server_log.clone(),
beacon_chain: server_bc.clone(),
db_path: db_path.clone(),
network_service: network_service.clone(),
network_channel: Arc::new(RwLock::new(network_chan.clone())),
eth2_config: eth2_config.clone(),
})
};
@ -217,16 +238,16 @@ pub fn start_server<T: BeaconChainTypes>(
.with_graceful_shutdown(server_exit)
.map_err(move |e| {
warn!(
log_clone,
"API failed to start, Unable to bind"; "address" => format!("{:?}", e)
log_clone,
"API failed to start, Unable to bind"; "address" => format!("{:?}", e)
)
});
info!(
log,
"REST API started";
"address" => format!("{}", config.listen_address),
"port" => config.port,
log,
"REST API started";
"address" => format!("{}", config.listen_address),
"port" => config.port,
);
executor.spawn(server);
@ -234,13 +255,6 @@ pub fn start_server<T: BeaconChainTypes>(
Ok(exit_signal)
}
fn success_response(body: Body) -> Response<Body> {
Response::builder()
.status(StatusCode::OK)
.body(body)
.expect("We should always be able to make response from the success body.")
}
#[derive(Clone)]
pub struct DBPath(PathBuf);

View File

@ -0,0 +1,13 @@
macro_rules! try_future {
($expr:expr) => {
match $expr {
core::result::Result::Ok(val) => val,
core::result::Result::Err(err) => {
return Box::new(futures::future::err(std::convert::From::from(err)))
}
}
};
($expr:expr,) => {
$crate::try_future!($expr)
};
}

View File

@ -1,8 +1,9 @@
use crate::{success_response, ApiError, ApiResult, DBPath};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use crate::helpers::get_beacon_chain_from_request;
use crate::response_builder::ResponseBuilder;
use crate::{ApiError, ApiResult, DBPath};
use beacon_chain::BeaconChainTypes;
use hyper::{Body, Request};
use prometheus::{Encoder, TextEncoder};
use std::sync::Arc;
pub use lighthouse_metrics::*;
@ -30,10 +31,7 @@ pub fn get_prometheus<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiR
let mut buffer = vec![];
let encoder = TextEncoder::new();
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let db_path = req
.extensions()
.get::<DBPath>()
@ -64,6 +62,6 @@ pub fn get_prometheus<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiR
.unwrap();
String::from_utf8(buffer)
.map(|string| success_response(Body::from(string)))
.map_err(|e| ApiError::ServerError(format!("Failed to encode prometheus info: {:?}", e)))
.map(|string| ResponseBuilder::new(&req)?.body_text(string))
.map_err(|e| ApiError::ServerError(format!("Failed to encode prometheus info: {:?}", e)))?
}

View File

@ -1,108 +1,78 @@
use crate::{success_response, ApiError, ApiResult, NetworkService};
use crate::error::ApiResult;
use crate::response_builder::ResponseBuilder;
use crate::NetworkService;
use beacon_chain::BeaconChainTypes;
use eth2_libp2p::{Enr, Multiaddr, PeerId};
use eth2_libp2p::{Multiaddr, PeerId};
use hyper::{Body, Request};
use std::sync::Arc;
/// HTTP handle to return the list of libp2p multiaddr the client is listening on.
/// HTTP handler to return the list of libp2p multiaddr the client is listening on.
///
/// Returns a list of `Multiaddr`, serialized according to their `serde` impl.
pub fn get_listen_addresses<T: BeaconChainTypes>(req: Request<Body>) -> ApiResult {
let network = req
.extensions()
.get::<Arc<NetworkService<T>>>()
.ok_or_else(|| ApiError::ServerError("NetworkService extension missing".to_string()))?;
.expect("The network service should always be there, we put it there");
let multiaddresses: Vec<Multiaddr> = network.listen_multiaddrs();
Ok(success_response(Body::from(
serde_json::to_string(&multiaddresses)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize Enr: {:?}", e)))?,
)))
ResponseBuilder::new(&req)?.body_no_ssz(&multiaddresses)
}
/// HTTP handle to return the list of libp2p multiaddr the client is listening on.
/// HTTP handler to return the network port the client is listening on.
///
/// Returns a list of `Multiaddr`, serialized according to their `serde` impl.
/// Returns the TCP port number in its plain form (which is also valid JSON serialization)
pub fn get_listen_port<T: BeaconChainTypes>(req: Request<Body>) -> ApiResult {
let network = req
.extensions()
.get::<Arc<NetworkService<T>>>()
.ok_or_else(|| ApiError::ServerError("NetworkService extension missing".to_string()))?;
Ok(success_response(Body::from(
serde_json::to_string(&network.listen_port())
.map_err(|e| ApiError::ServerError(format!("Unable to serialize port: {:?}", e)))?,
)))
.expect("The network service should always be there, we put it there")
.clone();
ResponseBuilder::new(&req)?.body(&network.listen_port())
}
/// HTTP handle to return the Discv5 ENR from the client's libp2p service.
/// HTTP handler to return the Discv5 ENR from the client's libp2p service.
///
/// ENR is encoded as base64 string.
pub fn get_enr<T: BeaconChainTypes>(req: Request<Body>) -> ApiResult {
let network = req
.extensions()
.get::<Arc<NetworkService<T>>>()
.ok_or_else(|| ApiError::ServerError("NetworkService extension missing".to_string()))?;
let enr: Enr = network.local_enr();
Ok(success_response(Body::from(
serde_json::to_string(&enr.to_base64())
.map_err(|e| ApiError::ServerError(format!("Unable to serialize Enr: {:?}", e)))?,
)))
.expect("The network service should always be there, we put it there");
ResponseBuilder::new(&req)?.body_no_ssz(&network.local_enr().to_base64())
}
/// HTTP handle to return the `PeerId` from the client's libp2p service.
/// HTTP handler to return the `PeerId` from the client's libp2p service.
///
/// PeerId is encoded as base58 string.
pub fn get_peer_id<T: BeaconChainTypes>(req: Request<Body>) -> ApiResult {
let network = req
.extensions()
.get::<Arc<NetworkService<T>>>()
.ok_or_else(|| ApiError::ServerError("NetworkService extension missing".to_string()))?;
let peer_id: PeerId = network.local_peer_id();
Ok(success_response(Body::from(
serde_json::to_string(&peer_id.to_base58())
.map_err(|e| ApiError::ServerError(format!("Unable to serialize Enr: {:?}", e)))?,
)))
.expect("The network service should always be there, we put it there");
ResponseBuilder::new(&req)?.body_no_ssz(&network.local_peer_id().to_base58())
}
/// HTTP handle to return the number of peers connected in the client's libp2p service.
/// HTTP handler to return the number of peers connected in the client's libp2p service.
pub fn get_peer_count<T: BeaconChainTypes>(req: Request<Body>) -> ApiResult {
let network = req
.extensions()
.get::<Arc<NetworkService<T>>>()
.ok_or_else(|| ApiError::ServerError("NetworkService extension missing".to_string()))?;
let connected_peers: usize = network.connected_peers();
Ok(success_response(Body::from(
serde_json::to_string(&connected_peers)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize Enr: {:?}", e)))?,
)))
.expect("The network service should always be there, we put it there");
ResponseBuilder::new(&req)?.body(&network.connected_peers())
}
/// HTTP handle to return the list of peers connected to the client's libp2p service.
/// HTTP handler to return the list of peers connected to the client's libp2p service.
///
/// Peers are presented as a list of `PeerId::to_string()`.
pub fn get_peer_list<T: BeaconChainTypes>(req: Request<Body>) -> ApiResult {
let network = req
.extensions()
.get::<Arc<NetworkService<T>>>()
.ok_or_else(|| ApiError::ServerError("NetworkService extension missing".to_string()))?;
.expect("The network service should always be there, we put it there");
let connected_peers: Vec<String> = network
.connected_peer_set()
.iter()
.map(PeerId::to_string)
.collect();
Ok(success_response(Body::from(
serde_json::to_string(&connected_peers).map_err(|e| {
ApiError::ServerError(format!("Unable to serialize Vec<PeerId>: {:?}", e))
})?,
)))
ResponseBuilder::new(&req)?.body_no_ssz(&connected_peers)
}

View File

@ -1,25 +1,17 @@
use crate::{success_response, ApiResult};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use crate::helpers::get_beacon_chain_from_request;
use crate::response_builder::ResponseBuilder;
use crate::ApiResult;
use beacon_chain::BeaconChainTypes;
use hyper::{Body, Request};
use std::sync::Arc;
use version;
/// Read the version string from the current Lighthouse build.
pub fn get_version(_req: Request<Body>) -> ApiResult {
let body = Body::from(
serde_json::to_string(&version::version())
.expect("Version should always be serialializable as JSON."),
);
Ok(success_response(body))
pub fn get_version(req: Request<Body>) -> ApiResult {
ResponseBuilder::new(&req)?.body_no_ssz(&version::version())
}
/// Read the genesis time from the current beacon chain state.
pub fn get_genesis_time<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = req.extensions().get::<Arc<BeaconChain<T>>>().unwrap();
let gen_time: u64 = beacon_chain.head().beacon_state.genesis_time;
let body = Body::from(
serde_json::to_string(&gen_time)
.expect("Genesis should time always have a valid JSON serialization."),
);
Ok(success_response(body))
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
ResponseBuilder::new(&req)?.body(&beacon_chain.head().beacon_state.genesis_time)
}

View File

@ -0,0 +1,99 @@
use super::{ApiError, ApiResult};
use http::header;
use hyper::{Body, Request, Response, StatusCode};
use serde::Serialize;
use ssz::Encode;
pub enum Encoding {
JSON,
SSZ,
YAML,
TEXT,
}
pub struct ResponseBuilder {
encoding: Encoding,
}
impl ResponseBuilder {
pub fn new(req: &Request<Body>) -> Result<Self, ApiError> {
let content_header: String = req
.headers()
.get(header::CONTENT_TYPE)
.map_or(Ok(""), |h| h.to_str())
.map_err(|e| {
ApiError::BadRequest(format!(
"The content-type header contains invalid characters: {:?}",
e
))
})
.map(|h| String::from(h))?;
// JSON is our default encoding, unless something else is requested.
let encoding = match content_header {
ref h if h.starts_with("application/ssz") => Encoding::SSZ,
ref h if h.starts_with("application/yaml") => Encoding::YAML,
ref h if h.starts_with("text/") => Encoding::TEXT,
_ => Encoding::JSON,
};
Ok(Self { encoding })
}
pub fn body<T: Serialize + Encode>(self, item: &T) -> ApiResult {
match self.encoding {
Encoding::SSZ => Response::builder()
.status(StatusCode::OK)
.header("content-type", "application/ssz")
.body(Body::from(item.as_ssz_bytes()))
.map_err(|e| ApiError::ServerError(format!("Failed to build response: {:?}", e))),
_ => self.body_no_ssz(item),
}
}
pub fn body_no_ssz<T: Serialize>(self, item: &T) -> ApiResult {
let (body, content_type) = match self.encoding {
Encoding::JSON => (
Body::from(serde_json::to_string(&item).map_err(|e| {
ApiError::ServerError(format!(
"Unable to serialize response body as JSON: {:?}",
e
))
})?),
"application/json",
),
Encoding::SSZ => {
return Err(ApiError::UnsupportedType(
"Response cannot be encoded as SSZ.".into(),
));
}
Encoding::YAML => (
Body::from(serde_yaml::to_string(&item).map_err(|e| {
ApiError::ServerError(format!(
"Unable to serialize response body as YAML: {:?}",
e
))
})?),
"application/yaml",
),
Encoding::TEXT => {
return Err(ApiError::UnsupportedType(
"Response cannot be encoded as plain text.".into(),
));
}
};
Response::builder()
.status(StatusCode::OK)
.header("content-type", content_type)
.body(Body::from(body))
.map_err(|e| ApiError::ServerError(format!("Failed to build response: {:?}", e)))
}
pub fn body_text(self, text: String) -> ApiResult {
Response::builder()
.status(StatusCode::OK)
.header("content-type", "text/plain; charset=utf-8")
.body(Body::from(text))
.map_err(|e| ApiError::ServerError(format!("Failed to build response: {:?}", e)))
}
}

View File

@ -1,27 +1,30 @@
use super::{success_response, ApiResult};
use super::ApiResult;
use crate::helpers::get_beacon_chain_from_request;
use crate::response_builder::ResponseBuilder;
use crate::ApiError;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use beacon_chain::BeaconChainTypes;
use eth2_config::Eth2Config;
use hyper::{Body, Request};
use std::sync::Arc;
use types::EthSpec;
/// HTTP handler to return the full spec object.
pub fn get_spec<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = req
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
ResponseBuilder::new(&req)?.body_no_ssz(&beacon_chain.spec)
}
/// HTTP handler to return the full Eth2Config object.
pub fn get_eth2_config<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let eth2_config = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
.get::<Arc<Eth2Config>>()
.ok_or_else(|| ApiError::ServerError("Eth2Config extension missing".to_string()))?;
let json: String = serde_json::to_string(&beacon_chain.spec)
.map_err(|e| ApiError::ServerError(format!("Unable to serialize spec: {:?}", e)))?;
Ok(success_response(Body::from(json)))
ResponseBuilder::new(&req)?.body_no_ssz(eth2_config.as_ref())
}
/// HTTP handler to return the full spec object.
pub fn get_slots_per_epoch<T: BeaconChainTypes + 'static>(_req: Request<Body>) -> ApiResult {
let json: String = serde_json::to_string(&T::EthSpec::slots_per_epoch())
.map_err(|e| ApiError::ServerError(format!("Unable to serialize epoch: {:?}", e)))?;
Ok(success_response(Body::from(json)))
pub fn get_slots_per_epoch<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
ResponseBuilder::new(&req)?.body(&T::EthSpec::slots_per_epoch())
}

View File

@ -12,7 +12,7 @@ impl<'a> UrlQuery<'a> {
/// Returns `Err` if `req` does not contain any query parameters.
pub fn from_request<T>(req: &'a Request<T>) -> Result<Self, ApiError> {
let query_str = req.uri().query().ok_or_else(|| {
ApiError::InvalidQueryParams(
ApiError::BadRequest(
"URL query must be valid and contain at least one key.".to_string(),
)
})?;
@ -28,7 +28,7 @@ impl<'a> UrlQuery<'a> {
.find(|(key, _value)| keys.contains(&&**key))
.map(|(key, value)| (key.into_owned(), value.into_owned()))
.ok_or_else(|| {
ApiError::InvalidQueryParams(format!(
ApiError::BadRequest(format!(
"URL query must contain at least one of the following keys: {:?}",
keys
))
@ -48,13 +48,13 @@ impl<'a> UrlQuery<'a> {
if first_key == key {
Ok(first_value.to_string())
} else {
Err(ApiError::InvalidQueryParams(format!(
Err(ApiError::BadRequest(format!(
"Only the {} query parameter is supported",
key
)))
}
} else {
Err(ApiError::InvalidQueryParams(format!(
Err(ApiError::BadRequest(format!(
"Only one query parameter is allowed, {} supplied",
queries.len()
)))
@ -64,7 +64,7 @@ impl<'a> UrlQuery<'a> {
/// Returns a vector of all values present where `key` is in `keys
///
/// If no match is found, an `InvalidQueryParams` error is returned.
pub fn all_of(mut self, key: &str) -> Result<Vec<String>, ApiError> {
pub fn all_of(self, key: &str) -> Result<Vec<String>, ApiError> {
let queries: Vec<_> = self
.0
.filter_map(|(k, v)| {

View File

@ -1,13 +1,23 @@
use super::{success_response, ApiResult};
use crate::{helpers::*, ApiError, UrlQuery};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use bls::PublicKey;
use crate::helpers::{
check_content_type_for_json, get_beacon_chain_from_request, get_logger_from_request,
parse_pubkey, publish_attestation_to_network, publish_beacon_block_to_network,
};
use crate::response_builder::ResponseBuilder;
use crate::{ApiError, ApiResult, BoxFut, UrlQuery};
use beacon_chain::{AttestationProcessingOutcome, BeaconChainTypes, BlockProcessingOutcome};
use bls::{AggregateSignature, PublicKey, Signature};
use futures::future::Future;
use futures::stream::Stream;
use hyper::{Body, Request};
use network::NetworkMessage;
use parking_lot::RwLock;
use serde::{Deserialize, Serialize};
use slog::{info, trace, warn};
use std::sync::Arc;
use store::Store;
use tokio;
use tokio::sync::mpsc;
use types::beacon_state::EthSpec;
use types::{BeaconBlock, BeaconState, Epoch, RelativeEpoch, Shard, Slot};
use types::{Attestation, BeaconBlock, BitList, Epoch, RelativeEpoch, Shard, Slot};
#[derive(Debug, Serialize, Deserialize)]
pub struct ValidatorDuty {
@ -34,49 +44,47 @@ impl ValidatorDuty {
/// HTTP Handler to retrieve a the duties for a set of validators during a particular epoch
pub fn get_validator_duties<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
// Get beacon state
let beacon_chain = req
.extensions()
.get::<Arc<BeaconChain<T>>>()
.ok_or_else(|| ApiError::ServerError("Beacon chain extension missing".to_string()))?;
let _ = beacon_chain
.ensure_state_caches_are_built()
.map_err(|e| ApiError::ServerError(format!("Unable to build state caches: {:?}", e)))?;
let head_state = beacon_chain
.speculative_state()
.expect("This is legacy code and should be removed.");
let log = get_logger_from_request(&req);
slog::trace!(log, "Validator duties requested of API: {:?}", &req);
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let mut head_state = beacon_chain.head().beacon_state;
slog::trace!(log, "Got head state from request.");
// Parse and check query parameters
let query = UrlQuery::from_request(&req)?;
let current_epoch = head_state.current_epoch();
let epoch = match query.first_of(&["epoch"]) {
Ok((_, v)) => Epoch::new(v.parse::<u64>().map_err(|e| {
ApiError::InvalidQueryParams(format!("Invalid epoch parameter, must be a u64. {:?}", e))
})?),
Ok((_, v)) => {
slog::trace!(log, "Requested epoch {:?}", v);
Epoch::new(v.parse::<u64>().map_err(|e| {
slog::info!(log, "Invalid epoch {:?}", e);
ApiError::BadRequest(format!("Invalid epoch parameter, must be a u64. {:?}", e))
})?)
}
Err(_) => {
// epoch not supplied, use the current epoch
slog::info!(log, "Using default epoch {:?}", current_epoch);
current_epoch
}
};
let relative_epoch = RelativeEpoch::from_epoch(current_epoch, epoch).map_err(|e| {
ApiError::InvalidQueryParams(format!(
slog::info!(log, "Requested epoch out of range.");
ApiError::BadRequest(format!(
"Cannot get RelativeEpoch, epoch out of range: {:?}",
e
))
})?;
//TODO: Handle an array of validators, currently only takes one
let mut validators: Vec<PublicKey> = match query.all_of("validator_pubkeys") {
Ok(v) => v
.iter()
.map(|pk| parse_pubkey(pk))
.collect::<Result<Vec<_>, _>>()?,
Err(e) => {
return Err(e);
}
};
let validators: Vec<PublicKey> = query
.all_of("validator_pubkeys")?
.iter()
.map(|pk| parse_pubkey(pk))
.collect::<Result<Vec<_>, _>>()?;
let mut duties: Vec<ValidatorDuty> = Vec::new();
// Build cache for the requested epoch
head_state
.build_committee_cache(relative_epoch, &beacon_chain.spec)
.map_err(|e| ApiError::ServerError(format!("Unable to build committee cache: {:?}", e)))?;
// Get a list of all validators for this epoch
let validator_proposers: Vec<usize> = epoch
.slot_iter(T::EthSpec::slots_per_epoch())
@ -141,9 +149,263 @@ pub fn get_validator_duties<T: BeaconChainTypes + 'static>(req: Request<Body>) -
duties.append(&mut vec![duty]);
}
let body = Body::from(
serde_json::to_string(&duties)
.expect("We should always be able to serialize the duties we created."),
);
Ok(success_response(body))
ResponseBuilder::new(&req)?.body_no_ssz(&duties)
}
/// HTTP Handler to produce a new BeaconBlock from the current state, ready to be signed by a validator.
pub fn get_new_beacon_block<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let query = UrlQuery::from_request(&req)?;
let slot = query
.first_of(&["slot"])
.map(|(_key, value)| value)?
.parse::<u64>()
.map(Slot::from)
.map_err(|e| {
ApiError::BadRequest(format!("Invalid slot parameter, must be a u64. {:?}", e))
})?;
let randao_bytes = query
.first_of(&["randao_reveal"])
.map(|(_key, value)| value)
.map(hex::decode)?
.map_err(|e| {
ApiError::BadRequest(format!("Invalid hex string for randao_reveal: {:?}", e))
})?;
let randao_reveal = Signature::from_bytes(randao_bytes.as_slice()).map_err(|e| {
ApiError::BadRequest(format!("randao_reveal is not a valid signature: {:?}", e))
})?;
let (new_block, _state) = beacon_chain
.produce_block(randao_reveal, slot)
.map_err(|e| {
ApiError::ServerError(format!(
"Beacon node is not able to produce a block: {:?}",
e
))
})?;
ResponseBuilder::new(&req)?.body(&new_block)
}
/// HTTP Handler to publish a BeaconBlock, which has been signed by a validator.
pub fn publish_beacon_block<T: BeaconChainTypes + 'static>(req: Request<Body>) -> BoxFut {
let _ = try_future!(check_content_type_for_json(&req));
let log = get_logger_from_request(&req);
let beacon_chain = try_future!(get_beacon_chain_from_request::<T>(&req));
// Get the network sending channel from the request, for later transmission
let network_chan = req
.extensions()
.get::<Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>>()
.expect("Should always get the network channel from the request, since we put it in there.")
.clone();
let response_builder = ResponseBuilder::new(&req);
let body = req.into_body();
trace!(
log,
"Got the request body, now going to parse it into a block."
);
Box::new(body
.concat2()
.map_err(|e| ApiError::ServerError(format!("Unable to get request body: {:?}",e)))
.map(|chunk| chunk.iter().cloned().collect::<Vec<u8>>())
.and_then(|chunks| {
serde_json::from_slice(&chunks.as_slice()).map_err(|e| {
ApiError::BadRequest(format!(
"Unable to deserialize JSON into a BeaconBlock: {:?}",
e
))
})
})
.and_then(move |block: BeaconBlock<T::EthSpec>| {
let slot = block.slot;
match beacon_chain.process_block(block.clone()) {
Ok(BlockProcessingOutcome::Processed { block_root }) => {
// Block was processed, publish via gossipsub
info!(log, "Processed valid block from API, transmitting to network."; "block_slot" => slot, "block_root" => format!("{}", block_root));
publish_beacon_block_to_network::<T>(network_chan, block)
}
Ok(outcome) => {
warn!(log, "BeaconBlock could not be processed, but is being sent to the network anyway."; "outcome" => format!("{:?}", outcome));
publish_beacon_block_to_network::<T>(network_chan, block)?;
Err(ApiError::ProcessingError(format!(
"The BeaconBlock could not be processed, but has still been published: {:?}",
outcome
)))
}
Err(e) => {
Err(ApiError::ServerError(format!(
"Error while processing block: {:?}",
e
)))
}
}
}).and_then(|_| {
response_builder?.body_no_ssz(&())
}))
}
/// HTTP Handler to produce a new Attestation from the current state, ready to be signed by a validator.
pub fn get_new_attestation<T: BeaconChainTypes + 'static>(req: Request<Body>) -> ApiResult {
let beacon_chain = get_beacon_chain_from_request::<T>(&req)?;
let mut head_state = beacon_chain.head().beacon_state;
let query = UrlQuery::from_request(&req)?;
let val_pk_str = query
.first_of(&["validator_pubkey"])
.map(|(_key, value)| value)?;
let val_pk = parse_pubkey(val_pk_str.as_str())?;
head_state
.update_pubkey_cache()
.map_err(|e| ApiError::ServerError(format!("Unable to build pubkey cache: {:?}", e)))?;
// Get the validator index from the supplied public key
// If it does not exist in the index, we cannot continue.
let val_index = head_state
.get_validator_index(&val_pk)
.map_err(|e| {
ApiError::ServerError(format!("Unable to read validator index cache. {:?}", e))
})?
.ok_or(ApiError::BadRequest(
"The provided validator public key does not correspond to a validator index.".into(),
))?;
// Build cache for the requested epoch
head_state
.build_committee_cache(RelativeEpoch::Current, &beacon_chain.spec)
.map_err(|e| ApiError::ServerError(format!("Unable to build committee cache: {:?}", e)))?;
// Get the duties of the validator, to make sure they match up.
// If they don't have duties this epoch, then return an error
let val_duty = head_state
.get_attestation_duties(val_index, RelativeEpoch::Current)
.map_err(|e| {
ApiError::ServerError(format!(
"unable to read cache for attestation duties: {:?}",
e
))
})?
.ok_or(ApiError::BadRequest("No validator duties could be found for the requested validator. Cannot provide valid attestation.".into()))?;
// Check that we are requesting an attestation during the slot where it is relevant.
let present_slot = beacon_chain.slot().map_err(|e| ApiError::ServerError(
format!("Beacon node is unable to determine present slot, either the state isn't generated or the chain hasn't begun. {:?}", e)
))?;
if val_duty.slot != present_slot {
return Err(ApiError::BadRequest(format!("Validator is only able to request an attestation during the slot they are allocated. Current slot: {:?}, allocated slot: {:?}", head_state.slot, val_duty.slot)));
}
// Parse the POC bit and insert it into the aggregation bits
let poc_bit = query
.first_of(&["poc_bit"])
.map(|(_key, value)| value)?
.parse::<bool>()
.map_err(|e| {
ApiError::BadRequest(format!("Invalid slot parameter, must be a u64. {:?}", e))
})?;
let mut aggregation_bits = BitList::with_capacity(val_duty.committee_len)
.expect("An empty BitList should always be created, or we have bigger problems.");
aggregation_bits
.set(val_duty.committee_index, poc_bit)
.map_err(|e| {
ApiError::ServerError(format!(
"Unable to set aggregation bits for the attestation: {:?}",
e
))
})?;
// Allow a provided slot parameter to check against the expected slot as a sanity check only.
// Presently, we don't support attestations at future or past slots.
let requested_slot = query
.first_of(&["slot"])
.map(|(_key, value)| value)?
.parse::<u64>()
.map(Slot::from)
.map_err(|e| {
ApiError::BadRequest(format!("Invalid slot parameter, must be a u64. {:?}", e))
})?;
let current_slot = beacon_chain.head().beacon_state.slot.as_u64();
if requested_slot != current_slot {
return Err(ApiError::BadRequest(format!("Attestation data can only be requested for the current slot ({:?}), not your requested slot ({:?})", current_slot, requested_slot)));
}
let shard = query
.first_of(&["shard"])
.map(|(_key, value)| value)?
.parse::<u64>()
.map_err(|e| ApiError::BadRequest(format!("Shard is not a valid u64 value: {:?}", e)))?;
let attestation_data = beacon_chain
.produce_attestation_data(shard, current_slot.into())
.map_err(|e| ApiError::ServerError(format!("Could not produce an attestation: {:?}", e)))?;
let attestation: Attestation<T::EthSpec> = Attestation {
aggregation_bits,
data: attestation_data,
custody_bits: BitList::with_capacity(val_duty.committee_len)
.expect("Should be able to create an empty BitList for the custody bits."),
signature: AggregateSignature::new(),
};
ResponseBuilder::new(&req)?.body(&attestation)
}
/// HTTP Handler to publish an Attestation, which has been signed by a validator.
pub fn publish_attestation<T: BeaconChainTypes + 'static>(req: Request<Body>) -> BoxFut {
let _ = try_future!(check_content_type_for_json(&req));
let log = get_logger_from_request(&req);
let beacon_chain = try_future!(get_beacon_chain_from_request::<T>(&req));
// Get the network sending channel from the request, for later transmission
let network_chan = req
.extensions()
.get::<Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>>()
.expect("Should always get the network channel from the request, since we put it in there.")
.clone();
let response_builder = ResponseBuilder::new(&req);
let body = req.into_body();
trace!(
log,
"Got the request body, now going to parse it into an attesation."
);
Box::new(body
.concat2()
.map_err(|e| ApiError::ServerError(format!("Unable to get request body: {:?}",e)))
.map(|chunk| chunk.iter().cloned().collect::<Vec<u8>>())
.and_then(|chunks| {
serde_json::from_slice(&chunks.as_slice()).map_err(|e| {
ApiError::BadRequest(format!(
"Unable to deserialize JSON into a BeaconBlock: {:?}",
e
))
})
})
.and_then(move |attestation: Attestation<T::EthSpec>| {
match beacon_chain.process_attestation(attestation.clone()) {
Ok(AttestationProcessingOutcome::Processed) => {
// Block was processed, publish via gossipsub
info!(log, "Processed valid attestation from API, transmitting to network.");
publish_attestation_to_network::<T>(network_chan, attestation)
}
Ok(outcome) => {
warn!(log, "Attestation could not be processed, but is being sent to the network anyway."; "outcome" => format!("{:?}", outcome));
publish_attestation_to_network::<T>(network_chan, attestation)?;
Err(ApiError::ProcessingError(format!(
"The Attestation could not be processed, but has still been published: {:?}",
outcome
)))
}
Err(e) => {
Err(ApiError::ServerError(format!(
"Error while processing attestation: {:?}",
e
)))
}
}
}).and_then(|_| {
response_builder?.body_no_ssz(&())
}))
}

View File

@ -14,7 +14,7 @@ use slog::{error, info, trace, warn};
use ssz::{ssz_encode, Decode, Encode};
use std::sync::Arc;
use tokio::sync::mpsc;
use types::Attestation;
use types::{Attestation, Slot};
#[derive(Clone)]
pub struct AttestationServiceInstance<T: BeaconChainTypes> {
@ -37,49 +37,13 @@ impl<T: BeaconChainTypes> AttestationService for AttestationServiceInstance<T> {
req.get_slot()
);
// verify the slot, drop lock on state afterwards
{
let slot_requested = req.get_slot();
// TODO: this whole module is legacy and not maintained well.
let state = &self
.chain
.speculative_state()
.expect("This is legacy code and should be removed");
// Start by performing some checks
// Check that the AttestationData is for the current slot (otherwise it will not be valid)
if slot_requested > state.slot.as_u64() {
let log_clone = self.log.clone();
let f = sink
.fail(RpcStatus::new(
RpcStatusCode::OutOfRange,
Some(
"AttestationData request for a slot that is in the future.".to_string(),
),
))
.map_err(move |e| {
error!(log_clone, "Failed to reply with failure {:?}: {:?}", req, e)
});
return ctx.spawn(f);
}
// currently cannot handle past slots. TODO: Handle this case
else if slot_requested < state.slot.as_u64() {
let log_clone = self.log.clone();
let f = sink
.fail(RpcStatus::new(
RpcStatusCode::InvalidArgument,
Some("AttestationData request for a slot that is in the past.".to_string()),
))
.map_err(move |e| {
error!(log_clone, "Failed to reply with failure {:?}: {:?}", req, e)
});
return ctx.spawn(f);
}
}
// Then get the AttestationData from the beacon chain
let shard = req.get_shard();
let attestation_data = match self.chain.produce_attestation_data(shard) {
let slot_requested = req.get_slot();
let attestation_data = match self
.chain
.produce_attestation_data(shard, Slot::from(slot_requested))
{
Ok(v) => v,
Err(e) => {
// Could not produce an attestation

View File

@ -34,8 +34,7 @@ impl<T: BeaconChainTypes> BeaconBlockService for BeaconBlockServiceInstance<T> {
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 requested_slot = Slot::from(req.get_slot());
let randao_reveal = match Signature::from_ssz_bytes(req.get_randao_reveal()) {
Ok(reveal) => reveal,
Err(_) => {
@ -51,7 +50,7 @@ impl<T: BeaconChainTypes> BeaconBlockService for BeaconBlockServiceInstance<T> {
}
};
let produced_block = match self.chain.produce_block(randao_reveal) {
let produced_block = match self.chain.produce_block(randao_reveal, requested_slot) {
Ok((block, _state)) => block,
Err(e) => {
// could not produce a block
@ -67,6 +66,11 @@ impl<T: BeaconChainTypes> BeaconBlockService for BeaconBlockServiceInstance<T> {
}
};
assert_eq!(
produced_block.slot, requested_slot,
"should produce at the requested slot"
);
let mut block = BeaconBlockProto::new();
block.set_ssz(ssz_encode(&produced_block));

View File

@ -16,7 +16,7 @@ pub struct Config {
impl Default for Config {
fn default() -> Self {
Config {
enabled: false, // rpc disabled by default
enabled: true,
listen_address: Ipv4Addr::new(127, 0, 0, 1),
port: 5051,
}
@ -25,8 +25,8 @@ impl Default for Config {
impl Config {
pub fn apply_cli_args(&mut self, args: &ArgMatches) -> Result<(), &'static str> {
if args.is_present("rpc") {
self.enabled = true;
if args.is_present("no-grpc") {
self.enabled = false;
}
if let Some(rpc_address) = args.value_of("rpc-address") {

View File

@ -80,7 +80,12 @@ pub fn start_server<T: BeaconChainTypes + Clone + 'static>(
let spawn_rpc = {
server.start();
for &(ref host, port) in server.bind_addrs() {
info!(log, "gRPC listening on {}:{}", host, port);
info!(
log,
"gRPC API started";
"port" => port,
"host" => host,
);
}
rpc_exit.and_then(move |_| {
info!(log, "RPC Server shutting down");

View File

@ -25,39 +25,41 @@ impl<T: BeaconChainTypes> ValidatorService for ValidatorServiceInstance<T> {
req: GetDutiesRequest,
sink: UnarySink<GetDutiesResponse>,
) {
let validators = req.get_validators();
trace!(self.log, "RPC request"; "endpoint" => "GetValidatorDuties", "epoch" => req.get_epoch());
let validators = req.get_validators();
let spec = &self.chain.spec;
// TODO: this whole module is legacy and not maintained well.
let state = &self
.chain
.speculative_state()
.expect("This is legacy code and should be removed");
let epoch = Epoch::from(req.get_epoch());
let slot = epoch.start_slot(T::EthSpec::slots_per_epoch());
let mut state = if let Ok(state) = self.chain.state_at_slot(slot) {
state.clone()
} else {
let log_clone = self.log.clone();
let f = sink
.fail(RpcStatus::new(
RpcStatusCode::FailedPrecondition,
Some("No state".to_string()),
))
.map_err(move |e| warn!(log_clone, "failed to reply {:?}: {:?}", req, e));
return ctx.spawn(f);
};
let _ = state.build_all_caches(&self.chain.spec);
assert_eq!(
state.current_epoch(),
epoch,
"Retrieved state should be from the same epoch"
);
let mut resp = GetDutiesResponse::new();
let resp_validators = resp.mut_active_validators();
let relative_epoch =
match RelativeEpoch::from_epoch(state.slot.epoch(T::EthSpec::slots_per_epoch()), epoch)
{
Ok(v) => v,
Err(e) => {
// incorrect epoch
let log_clone = self.log.clone();
let f = sink
.fail(RpcStatus::new(
RpcStatusCode::FailedPrecondition,
Some(format!("Invalid epoch: {:?}", e)),
))
.map_err(move |e| warn!(log_clone, "failed to reply {:?}: {:?}", req, e));
return ctx.spawn(f);
}
};
let validator_proposers: Result<Vec<usize>, _> = epoch
.slot_iter(T::EthSpec::slots_per_epoch())
.map(|slot| state.get_beacon_proposer_index(slot, relative_epoch, &spec))
.map(|slot| {
state.get_beacon_proposer_index(slot, RelativeEpoch::Current, &self.chain.spec)
})
.collect();
let validator_proposers = match validator_proposers {
Ok(v) => v,

575
beacon_node/src/config.rs Normal file
View File

@ -0,0 +1,575 @@
use clap::ArgMatches;
use client::{BeaconChainStartMethod, ClientConfig, Eth1BackendMethod, Eth2Config};
use eth2_config::{read_from_file, write_to_file};
use lighthouse_bootstrap::Bootstrapper;
use rand::{distributions::Alphanumeric, Rng};
use slog::{crit, info, warn, Logger};
use std::fs;
use std::net::Ipv4Addr;
use std::path::{Path, PathBuf};
pub const DEFAULT_DATA_DIR: &str = ".lighthouse";
pub const CLIENT_CONFIG_FILENAME: &str = "beacon-node.toml";
pub const ETH2_CONFIG_FILENAME: &str = "eth2-spec.toml";
type Result<T> = std::result::Result<T, String>;
type Config = (ClientConfig, Eth2Config, Logger);
/// Gets the fully-initialized global client and eth2 configuration objects.
///
/// The top-level `clap` arguments should be provied as `cli_args`.
///
/// The output of this function depends primarily upon the given `cli_args`, however it's behaviour
/// may be influenced by other external services like the contents of the file system or the
/// response of some remote server.
pub fn get_configs(cli_args: &ArgMatches, core_log: Logger) -> Result<Config> {
let log = core_log.clone();
let mut builder = ConfigBuilder::new(cli_args, core_log)?;
if let Some(server) = cli_args.value_of("eth1-server") {
builder.set_eth1_backend_method(Eth1BackendMethod::Web3 {
server: server.into(),
})
} else {
builder.set_eth1_backend_method(Eth1BackendMethod::Interop)
}
match cli_args.subcommand() {
("testnet", Some(sub_cmd_args)) => {
process_testnet_subcommand(&mut builder, sub_cmd_args, &log)?
}
// No sub-command assumes a resume operation.
_ => {
info!(
log,
"Resuming from existing datadir";
"path" => format!("{:?}", builder.client_config.data_dir)
);
// If no primary subcommand was given, start the beacon chain from an existing
// database.
builder.set_beacon_chain_start_method(BeaconChainStartMethod::Resume);
// Whilst there is no large testnet or mainnet force the user to specify how they want
// to start a new chain (e.g., from a genesis YAML file, another node, etc).
if !builder.client_config.data_dir.exists() {
return Err(
"No datadir found. To start a new beacon chain, see `testnet --help`. \
Use `--datadir` to specify a different directory"
.into(),
);
}
// If the `testnet` command was not provided, attempt to load an existing datadir and
// continue with an existing chain.
builder.load_from_datadir()?;
}
};
builder.build(cli_args)
}
/// Process the `testnet` CLI subcommand arguments, updating the `builder`.
fn process_testnet_subcommand(
builder: &mut ConfigBuilder,
cli_args: &ArgMatches,
log: &Logger,
) -> Result<()> {
if cli_args.is_present("random-datadir") {
builder.set_random_datadir()?;
}
if cli_args.is_present("force") {
builder.clean_datadir()?;
}
let is_bootstrap = cli_args.subcommand_name() == Some("bootstrap");
if let Some(path_string) = cli_args.value_of("eth2-config") {
if is_bootstrap {
return Err("Cannot supply --eth2-config when using bootstrap".to_string());
}
let path = path_string
.parse::<PathBuf>()
.map_err(|e| format!("Unable to parse eth2-config path: {:?}", e))?;
builder.load_eth2_config(path)?;
} else {
builder.update_spec_from_subcommand(&cli_args)?;
}
if let Some(slot_time) = cli_args.value_of("slot-time") {
if is_bootstrap {
return Err("Cannot supply --slot-time flag whilst using bootstrap.".into());
}
let slot_time = slot_time
.parse::<u64>()
.map_err(|e| format!("Unable to parse slot-time: {:?}", e))?;
builder.set_slot_time(slot_time);
}
if let Some(path_string) = cli_args.value_of("client-config") {
let path = path_string
.parse::<PathBuf>()
.map_err(|e| format!("Unable to parse client config path: {:?}", e))?;
builder.load_client_config(path)?;
}
info!(
log,
"Creating new datadir";
"path" => format!("{:?}", builder.client_config.data_dir)
);
// When using the testnet command we listen on all addresses.
builder.set_listen_addresses("0.0.0.0".into())?;
warn!(log, "All services listening on 0.0.0.0");
// Start matching on the second subcommand (e.g., `testnet bootstrap ...`).
match cli_args.subcommand() {
("bootstrap", Some(cli_args)) => {
let server = cli_args
.value_of("server")
.ok_or_else(|| "No bootstrap server specified")?;
let port: Option<u16> = cli_args
.value_of("libp2p-port")
.and_then(|s| s.parse::<u16>().ok());
builder.import_bootstrap_libp2p_address(server, port)?;
builder.import_bootstrap_enr_address(server)?;
builder.import_bootstrap_eth2_config(server)?;
builder.set_beacon_chain_start_method(BeaconChainStartMethod::HttpBootstrap {
server: server.to_string(),
port,
})
}
("recent", Some(cli_args)) => {
let validator_count = cli_args
.value_of("validator_count")
.ok_or_else(|| "No validator_count specified")?
.parse::<usize>()
.map_err(|e| format!("Unable to parse validator_count: {:?}", e))?;
let minutes = cli_args
.value_of("minutes")
.ok_or_else(|| "No recent genesis minutes supplied")?
.parse::<u64>()
.map_err(|e| format!("Unable to parse minutes: {:?}", e))?;
builder.set_beacon_chain_start_method(BeaconChainStartMethod::RecentGenesis {
validator_count,
minutes,
})
}
("quick", Some(cli_args)) => {
let validator_count = cli_args
.value_of("validator_count")
.ok_or_else(|| "No validator_count specified")?
.parse::<usize>()
.map_err(|e| format!("Unable to parse validator_count: {:?}", e))?;
let genesis_time = cli_args
.value_of("genesis_time")
.ok_or_else(|| "No genesis time supplied")?
.parse::<u64>()
.map_err(|e| format!("Unable to parse genesis time: {:?}", e))?;
builder.set_beacon_chain_start_method(BeaconChainStartMethod::Generated {
validator_count,
genesis_time,
})
}
("file", Some(cli_args)) => {
let file = cli_args
.value_of("file")
.ok_or_else(|| "No filename specified")?
.parse::<PathBuf>()
.map_err(|e| format!("Unable to parse filename: {:?}", e))?;
let format = cli_args
.value_of("format")
.ok_or_else(|| "No file format specified")?;
let start_method = match format {
"yaml" => BeaconChainStartMethod::Yaml { file },
"ssz" => BeaconChainStartMethod::Ssz { file },
"json" => BeaconChainStartMethod::Json { file },
other => return Err(format!("Unknown genesis file format: {}", other)),
};
builder.set_beacon_chain_start_method(start_method)
}
(cmd, Some(_)) => {
return Err(format!(
"Invalid valid method specified: {}. See 'testnet --help'.",
cmd
))
}
_ => return Err("No testnet method specified. See 'testnet --help'.".into()),
};
builder.write_configs_to_new_datadir()?;
Ok(())
}
/// Allows for building a set of configurations based upon `clap` arguments.
struct ConfigBuilder {
log: Logger,
eth2_config: Eth2Config,
client_config: ClientConfig,
}
impl ConfigBuilder {
/// Create a new builder with default settings.
pub fn new(cli_args: &ArgMatches, log: Logger) -> Result<Self> {
// Read the `--datadir` flag.
//
// If it's not present, try and find the home directory (`~`) and push the default data
// directory onto it.
let data_dir: PathBuf = cli_args
.value_of("datadir")
.map(|string| PathBuf::from(string))
.or_else(|| {
dirs::home_dir().map(|mut home| {
home.push(DEFAULT_DATA_DIR);
home
})
})
.ok_or_else(|| "Unable to find a home directory for the datadir".to_string())?;
let mut client_config = ClientConfig::default();
client_config.data_dir = data_dir;
Ok(Self {
log,
eth2_config: Eth2Config::minimal(),
client_config,
})
}
/// Clears any configuration files that would interfere with writing new configs.
///
/// Moves the following files in `data_dir` into a backup directory:
///
/// - Client config
/// - Eth2 config
/// - The entire database directory
pub fn clean_datadir(&mut self) -> Result<()> {
let backup_dir = {
let mut s = String::from("backup_");
s.push_str(&random_string(6));
self.client_config.data_dir.join(s)
};
fs::create_dir_all(&backup_dir)
.map_err(|e| format!("Unable to create config backup dir: {:?}", e))?;
let move_to_backup_dir = |path: &Path| -> Result<()> {
let file_name = path
.file_name()
.ok_or_else(|| "Invalid path found during datadir clean (no filename).")?;
let mut new = path.to_path_buf();
new.pop();
new.push(backup_dir.clone());
new.push(file_name);
let _ = fs::rename(path, new);
Ok(())
};
move_to_backup_dir(&self.client_config.data_dir.join(CLIENT_CONFIG_FILENAME))?;
move_to_backup_dir(&self.client_config.data_dir.join(ETH2_CONFIG_FILENAME))?;
if let Some(db_path) = self.client_config.db_path() {
move_to_backup_dir(&db_path)?;
}
Ok(())
}
/// Sets the method for starting the beacon chain.
pub fn set_beacon_chain_start_method(&mut self, method: BeaconChainStartMethod) {
self.client_config.beacon_chain_start_method = method;
}
/// Sets the method for starting the beacon chain.
pub fn set_eth1_backend_method(&mut self, method: Eth1BackendMethod) {
self.client_config.eth1_backend_method = method;
}
/// Import the libp2p address for `server` into the list of libp2p nodes to connect with.
///
/// If `port` is `Some`, it is used as the port for the `Multiaddr`. If `port` is `None`,
/// attempts to connect to the `server` via HTTP and retrieve it's libp2p listen port.
pub fn import_bootstrap_libp2p_address(
&mut self,
server: &str,
port: Option<u16>,
) -> Result<()> {
let bootstrapper = Bootstrapper::connect(server.to_string(), &self.log)?;
if let Some(server_multiaddr) = bootstrapper.best_effort_multiaddr(port) {
info!(
self.log,
"Estimated bootstrapper libp2p address";
"multiaddr" => format!("{:?}", server_multiaddr)
);
self.client_config
.network
.libp2p_nodes
.push(server_multiaddr);
} else {
warn!(
self.log,
"Unable to estimate a bootstrapper libp2p address, this node may not find any peers."
);
};
Ok(())
}
/// Import the enr address for `server` into the list of initial enrs (boot nodes).
pub fn import_bootstrap_enr_address(&mut self, server: &str) -> Result<()> {
let bootstrapper = Bootstrapper::connect(server.to_string(), &self.log)?;
if let Ok(enr) = bootstrapper.enr() {
info!(
self.log,
"Loaded bootstrapper libp2p address";
"enr" => format!("{:?}", enr)
);
self.client_config.network.boot_nodes.push(enr);
} else {
warn!(
self.log,
"Unable to estimate a bootstrapper enr address, this node may not find any peers."
);
};
Ok(())
}
/// Set the config data_dir to be an random directory.
///
/// Useful for easily spinning up ephemeral testnets.
pub fn set_random_datadir(&mut self) -> Result<()> {
self.client_config
.data_dir
.push(format!("random_{}", random_string(6)));
self.client_config.network.network_dir = self.client_config.data_dir.join("network");
Ok(())
}
/// Imports an `Eth2Config` from `server`, returning an error if this fails.
pub fn import_bootstrap_eth2_config(&mut self, server: &str) -> Result<()> {
let bootstrapper = Bootstrapper::connect(server.to_string(), &self.log)?;
self.update_eth2_config(bootstrapper.eth2_config()?);
Ok(())
}
fn update_eth2_config(&mut self, eth2_config: Eth2Config) {
self.eth2_config = eth2_config;
}
fn set_slot_time(&mut self, milliseconds_per_slot: u64) {
self.eth2_config.spec.milliseconds_per_slot = milliseconds_per_slot;
}
/// Reads the subcommand and tries to update `self.eth2_config` based up on the `--spec` flag.
///
/// Returns an error if the `--spec` flag is not present in the given `cli_args`.
pub fn update_spec_from_subcommand(&mut self, cli_args: &ArgMatches) -> Result<()> {
// Re-initialise the `Eth2Config`.
//
// If a CLI parameter is set, overwrite any config file present.
// If a parameter is not set, use either the config file present or default to minimal.
let eth2_config = match cli_args.value_of("spec") {
Some("mainnet") => Eth2Config::mainnet(),
Some("minimal") => Eth2Config::minimal(),
Some("interop") => Eth2Config::interop(),
_ => return Err("Unable to determine specification type.".into()),
};
self.client_config.spec_constants = cli_args
.value_of("spec")
.expect("Guarded by prior match statement")
.to_string();
self.eth2_config = eth2_config;
Ok(())
}
/// Writes the configs in `self` to `self.data_dir`.
///
/// Returns an error if `self.data_dir` already exists.
pub fn write_configs_to_new_datadir(&mut self) -> Result<()> {
let db_exists = self
.client_config
.db_path()
.map(|d| d.exists())
.unwrap_or_else(|| false);
// Do not permit creating a new config when the datadir exists.
if db_exists {
return Err("Database already exists. See `-f` or `-r` in `testnet --help`".into());
}
// Create `datadir` and any non-existing parent directories.
fs::create_dir_all(&self.client_config.data_dir).map_err(|e| {
crit!(self.log, "Failed to initialize data dir"; "error" => format!("{}", e));
format!("{}", e)
})?;
let client_config_file = self.client_config.data_dir.join(CLIENT_CONFIG_FILENAME);
if client_config_file.exists() {
return Err(format!(
"Datadir is not clean, {} exists. See `-f` in `testnet --help`.",
CLIENT_CONFIG_FILENAME
));
} else {
// Write the onfig to a TOML file in the datadir.
write_to_file(
self.client_config.data_dir.join(CLIENT_CONFIG_FILENAME),
&self.client_config,
)
.map_err(|e| format!("Unable to write {} file: {:?}", CLIENT_CONFIG_FILENAME, e))?;
}
let eth2_config_file = self.client_config.data_dir.join(ETH2_CONFIG_FILENAME);
if eth2_config_file.exists() {
return Err(format!(
"Datadir is not clean, {} exists. See `-f` in `testnet --help`.",
ETH2_CONFIG_FILENAME
));
} else {
// Write the config to a TOML file in the datadir.
write_to_file(
self.client_config.data_dir.join(ETH2_CONFIG_FILENAME),
&self.eth2_config,
)
.map_err(|e| format!("Unable to write {} file: {:?}", ETH2_CONFIG_FILENAME, e))?;
}
Ok(())
}
/// Attempts to load the client and eth2 configs from `self.data_dir`.
///
/// Returns an error if any files are not found or are invalid.
pub fn load_from_datadir(&mut self) -> Result<()> {
// Check to ensure the datadir exists.
//
// For now we return an error. In the future we may decide to boot a default (e.g.,
// public testnet or mainnet).
if !self.client_config.data_dir.exists() {
return Err(
"No datadir found. Either create a new testnet or specify a different `--datadir`."
.into(),
);
}
// If there is a path to a databse in the config, ensure it exists.
if !self
.client_config
.db_path()
.map(|path| path.exists())
.unwrap_or_else(|| true)
{
return Err(
"No database found in datadir. Use 'testnet -f' to overwrite the existing \
datadir, or specify a different `--datadir`."
.into(),
);
}
self.load_eth2_config(self.client_config.data_dir.join(ETH2_CONFIG_FILENAME))?;
self.load_client_config(self.client_config.data_dir.join(CLIENT_CONFIG_FILENAME))?;
Ok(())
}
/// Attempts to load the client config from `path`.
///
/// Returns an error if any files are not found or are invalid.
pub fn load_client_config(&mut self, path: PathBuf) -> Result<()> {
self.client_config = read_from_file::<ClientConfig>(path.clone())
.map_err(|e| format!("Unable to parse {:?} file: {:?}", path, e))?
.ok_or_else(|| format!("{:?} file does not exist", path))?;
Ok(())
}
/// Attempts to load the eth2 config from `path`.
///
/// Returns an error if any files are not found or are invalid.
pub fn load_eth2_config(&mut self, path: PathBuf) -> Result<()> {
self.eth2_config = read_from_file::<Eth2Config>(path.clone())
.map_err(|e| format!("Unable to parse {:?} file: {:?}", path, e))?
.ok_or_else(|| format!("{:?} file does not exist", path))?;
Ok(())
}
/// Sets all listening addresses to the given `addr`.
pub fn set_listen_addresses(&mut self, addr: String) -> Result<()> {
let addr = addr
.parse::<Ipv4Addr>()
.map_err(|e| format!("Unable to parse default listen address: {:?}", e))?;
self.client_config.network.listen_address = addr.clone().into();
self.client_config.rpc.listen_address = addr.clone();
self.client_config.rest_api.listen_address = addr.clone();
Ok(())
}
/// Consumes self, returning the configs.
///
/// The supplied `cli_args` should be the base-level `clap` cli_args (i.e., not a subcommand
/// cli_args).
pub fn build(mut self, cli_args: &ArgMatches) -> Result<Config> {
self.eth2_config.apply_cli_args(cli_args)?;
self.client_config.apply_cli_args(cli_args, &mut self.log)?;
if let Some(bump) = cli_args.value_of("port-bump") {
let bump = bump
.parse::<u16>()
.map_err(|e| format!("Unable to parse port bump: {}", e))?;
self.client_config.network.libp2p_port += bump;
self.client_config.network.discovery_port += bump;
self.client_config.rpc.port += bump;
self.client_config.rest_api.port += bump;
self.client_config.websocket_server.port += bump;
}
if self.eth2_config.spec_constants != self.client_config.spec_constants {
crit!(self.log, "Specification constants do not match.";
"client_config" => format!("{}", self.client_config.spec_constants),
"eth2_config" => format!("{}", self.eth2_config.spec_constants)
);
return Err("Specification constant mismatch".into());
}
Ok((self.client_config, self.eth2_config, self.log))
}
}
fn random_string(len: usize) -> String {
rand::thread_rng()
.sample_iter(&Alphanumeric)
.take(len)
.collect::<String>()
}

View File

@ -1,12 +1,10 @@
mod config;
mod run;
use clap::{App, Arg};
use client::{ClientConfig, Eth2Config};
use clap::{App, Arg, SubCommand};
use config::get_configs;
use env_logger::{Builder, Env};
use eth2_config::{read_from_file, write_to_file};
use slog::{crit, o, warn, Drain, Level};
use std::fs;
use std::path::PathBuf;
pub const DEFAULT_DATA_DIR: &str = ".lighthouse";
pub const CLIENT_CONFIG_FILENAME: &str = "beacon-node.toml";
@ -30,36 +28,49 @@ fn main() {
.value_name("DIR")
.help("Data directory for keys and databases.")
.takes_value(true)
.global(true)
)
.arg(
Arg::with_name("logfile")
.long("logfile")
.value_name("logfile")
.value_name("FILE")
.help("File path where output will be written.")
.takes_value(true),
)
.arg(
Arg::with_name("network-dir")
.long("network-dir")
.value_name("NETWORK-DIR")
.value_name("DIR")
.help("Data directory for network keys.")
.takes_value(true)
.global(true)
)
/*
* Network parameters.
*/
.arg(
Arg::with_name("port-bump")
.long("port-bump")
.short("b")
.value_name("INCREMENT")
.help("Sets all listening TCP/UDP ports to default values, but with each port increased by \
INCREMENT. Useful when starting multiple nodes on a single machine. Using increments \
in multiples of 10 is recommended.")
.takes_value(true),
)
.arg(
Arg::with_name("listen-address")
.long("listen-address")
.value_name("ADDRESS")
.help("The address lighthouse will listen for UDP and TCP connections. (default 127.0.0.1).")
.takes_value(true),
.takes_value(true)
)
.arg(
Arg::with_name("port")
.long("port")
.value_name("PORT")
.help("The TCP/UDP port to listen on. The UDP port can be modified by the --discovery-port flag.")
.conflicts_with("port-bump")
.takes_value(true),
)
.arg(
@ -72,7 +83,7 @@ fn main() {
Arg::with_name("boot-nodes")
.long("boot-nodes")
.allow_hyphen_values(true)
.value_name("BOOTNODES")
.value_name("ENR-LIST")
.help("One or more comma-delimited base64-encoded ENR's to bootstrap the p2p network.")
.takes_value(true),
)
@ -81,6 +92,7 @@ fn main() {
.long("disc-port")
.value_name("PORT")
.help("The discovery UDP port.")
.conflicts_with("port-bump")
.takes_value(true),
)
.arg(
@ -104,52 +116,93 @@ fn main() {
.help("One or more comma-delimited multiaddrs to manually connect to a libp2p peer without an ENR.")
.takes_value(true),
)
.arg(
Arg::with_name("p2p-priv-key")
.long("p2p-priv-key")
.value_name("HEX")
.help("A secp256k1 secret key, represented as ASCII-encoded hex bytes (with or without 0x prefix).")
.takes_value(true),
)
/*
* gRPC parameters.
*/
.arg(
Arg::with_name("rpc")
.long("rpc")
.value_name("RPC")
.help("Enable the RPC server.")
Arg::with_name("no-grpc")
.long("no-grpc")
.help("Disable the gRPC server.")
.takes_value(false),
)
.arg(
Arg::with_name("rpc-address")
.long("rpc-address")
.value_name("Address")
.value_name("ADDRESS")
.help("Listen address for RPC endpoint.")
.takes_value(true),
)
.arg(
Arg::with_name("rpc-port")
.long("rpc-port")
.value_name("PORT")
.help("Listen port for RPC endpoint.")
.conflicts_with("port-bump")
.takes_value(true),
)
/* Client related arguments */
/* REST API related arguments */
.arg(
Arg::with_name("api")
.long("api")
.value_name("API")
.help("Enable the RESTful HTTP API server.")
Arg::with_name("no-api")
.long("no-api")
.help("Disable RESTful HTTP API server.")
.takes_value(false),
)
.arg(
Arg::with_name("api-address")
.long("api-address")
.value_name("APIADDRESS")
.value_name("ADDRESS")
.help("Set the listen address for the RESTful HTTP API server.")
.takes_value(true),
)
.arg(
Arg::with_name("api-port")
.long("api-port")
.value_name("APIPORT")
.value_name("PORT")
.help("Set the listen TCP port for the RESTful HTTP API server.")
.conflicts_with("port-bump")
.takes_value(true),
)
/* Websocket related arguments */
.arg(
Arg::with_name("no-ws")
.long("no-ws")
.help("Disable websocket server.")
.takes_value(false),
)
.arg(
Arg::with_name("ws-address")
.long("ws-address")
.value_name("ADDRESS")
.help("Set the listen address for the websocket server.")
.conflicts_with_all(&["no-ws"])
.takes_value(true),
)
.arg(
Arg::with_name("ws-port")
.long("ws-port")
.value_name("PORT")
.help("Set the listen TCP port for the websocket server.")
.conflicts_with_all(&["no-ws", "port-bump"])
.takes_value(true),
)
/*
* Eth1 Integration
*/
.arg(
Arg::with_name("eth1-server")
.long("eth1-server")
.value_name("SERVER")
.help("Specifies the server for a web3 connection to the Eth1 chain.")
.takes_value(true)
)
/*
* Database parameters.
*/
@ -160,25 +213,7 @@ fn main() {
.help("Type of database to use.")
.takes_value(true)
.possible_values(&["disk", "memory"])
.default_value("memory"),
)
/*
* Specification/testnet params.
*/
.arg(
Arg::with_name("default-spec")
.long("default-spec")
.value_name("TITLE")
.short("default-spec")
.help("Specifies the default eth2 spec to be used. This will override any spec written to disk and will therefore be used by default in future instances.")
.takes_value(true)
.possible_values(&["mainnet", "minimal", "interop"])
)
.arg(
Arg::with_name("recent-genesis")
.long("recent-genesis")
.short("r")
.help("When present, genesis will be within 30 minutes prior. Only for testing"),
.default_value("disk"),
)
/*
* Logging.
@ -192,22 +227,138 @@ fn main() {
.possible_values(&["info", "debug", "trace", "warn", "error", "crit"])
.default_value("trace"),
)
.arg(
Arg::with_name("verbosity")
.short("v")
.multiple(true)
.help("Sets the verbosity level")
.takes_value(true),
)
/*
* Bootstrap.
* The "testnet" sub-command.
*
* Allows for creating a new datadir with testnet-specific configs.
*/
.arg(
Arg::with_name("bootstrap")
.long("bootstrap")
.value_name("HTTP_SERVER")
.help("Load the genesis state and libp2p address from the HTTP API of another Lighthouse node.")
.takes_value(true)
.subcommand(SubCommand::with_name("testnet")
.about("Create a new Lighthouse datadir using a testnet strategy.")
.arg(
Arg::with_name("spec")
.short("s")
.long("spec")
.value_name("TITLE")
.help("Specifies the default eth2 spec type. Only effective when creating a new datadir.")
.takes_value(true)
.required(true)
.possible_values(&["mainnet", "minimal", "interop"])
.default_value("minimal")
)
.arg(
Arg::with_name("eth2-config")
.long("eth2-config")
.value_name("TOML_FILE")
.help("A existing eth2_spec TOML file (e.g., eth2_spec.toml).")
.takes_value(true)
.conflicts_with("spec")
)
.arg(
Arg::with_name("client-config")
.long("client-config")
.value_name("TOML_FILE")
.help("An existing beacon_node TOML file (e.g., beacon_node.toml).")
.takes_value(true)
)
.arg(
Arg::with_name("random-datadir")
.long("random-datadir")
.short("r")
.help("If present, append a random string to the datadir path. Useful for fast development \
iteration.")
)
.arg(
Arg::with_name("force")
.long("force")
.short("f")
.help("If present, will create new config and database files and move the any existing to a \
backup directory.")
.conflicts_with("random-datadir")
)
.arg(
Arg::with_name("slot-time")
.long("slot-time")
.short("t")
.value_name("MILLISECONDS")
.help("Defines the slot time when creating a new testnet.")
)
/*
* `boostrap`
*
* Start a new node by downloading genesis and network info from another node via the
* HTTP API.
*/
.subcommand(SubCommand::with_name("bootstrap")
.about("Connects to the given HTTP server, downloads a genesis state and attempts to peer with it.")
.arg(Arg::with_name("server")
.value_name("HTTP_SERVER")
.required(true)
.default_value("http://localhost:5052")
.help("A HTTP server, with a http:// prefix"))
.arg(Arg::with_name("libp2p-port")
.short("p")
.long("port")
.value_name("TCP_PORT")
.help("A libp2p listen port used to peer with the bootstrap server. This flag is useful \
when port-fowarding is used: you may connect using a different port than \
the one the server is immediately listening on."))
)
/*
* `recent`
*
* Start a new node, with a specified number of validators with a genesis time in the last
* 30-minutes.
*/
.subcommand(SubCommand::with_name("recent")
.about("Creates a new genesis state where the genesis time was at the previous \
MINUTES boundary (e.g., when MINUTES == 30; 12:00, 12:30, 13:00, etc.)")
.arg(Arg::with_name("validator_count")
.value_name("VALIDATOR_COUNT")
.required(true)
.help("The number of validators in the genesis state"))
.arg(Arg::with_name("minutes")
.long("minutes")
.short("m")
.value_name("MINUTES")
.required(true)
.default_value("15")
.help("The maximum number of minutes that will have elapsed before genesis"))
)
/*
* `quick`
*
* Start a new node, specifying the number of validators and genesis time
*/
.subcommand(SubCommand::with_name("quick")
.about("Creates a new genesis state from the specified validator count and genesis time. \
Compatible with the `quick-start genesis` defined in the eth2.0-pm repo.")
.arg(Arg::with_name("validator_count")
.value_name("VALIDATOR_COUNT")
.required(true)
.help("The number of validators in the genesis state"))
.arg(Arg::with_name("genesis_time")
.value_name("UNIX_EPOCH_SECONDS")
.required(true)
.help("The genesis time for the given state."))
)
/*
* `yaml`
*
* Start a new node, using a genesis state loaded from a YAML file
*/
.subcommand(SubCommand::with_name("file")
.about("Creates a new datadir where the genesis state is read from YAML. May fail to parse \
a file that was generated to a different spec than that specified by --spec.")
.arg(Arg::with_name("format")
.value_name("FORMAT")
.required(true)
.possible_values(&["yaml", "ssz", "json"])
.help("The encoding of the state in the file."))
.arg(Arg::with_name("file")
.value_name("YAML_FILE")
.required(true)
.help("A YAML file from which to read the state"))
)
)
.get_matches();
@ -227,143 +378,34 @@ fn main() {
_ => unreachable!("guarded by clap"),
};
let mut log = slog::Logger::root(drain.fuse(), o!());
let log = slog::Logger::root(drain.fuse(), o!());
if std::mem::size_of::<usize>() != 8 {
crit!(
log,
"Lighthouse only supports 64bit CPUs";
"detected" => format!("{}bit", std::mem::size_of::<usize>() * 8)
);
}
warn!(
log,
"Ethereum 2.0 is pre-release. This software is experimental."
);
let data_dir = match matches
.value_of("datadir")
.and_then(|v| Some(PathBuf::from(v)))
{
Some(v) => v,
None => {
// use the default
let mut default_dir = match dirs::home_dir() {
Some(v) => v,
None => {
crit!(log, "Failed to find a home directory");
return;
}
};
default_dir.push(DEFAULT_DATA_DIR);
default_dir
}
};
let log_clone = log.clone();
// create the directory if needed
match fs::create_dir_all(&data_dir) {
Ok(_) => {}
Err(e) => {
crit!(log, "Failed to initialize data dir"; "error" => format!("{}", e));
return;
}
}
let client_config_path = data_dir.join(CLIENT_CONFIG_FILENAME);
// Attempt to load the `ClientConfig` from disk.
// Load the process-wide configuration.
//
// If file doesn't exist, create a new, default one.
let mut client_config = match read_from_file::<ClientConfig>(client_config_path.clone()) {
Ok(Some(c)) => c,
Ok(None) => {
let default = ClientConfig::default();
if let Err(e) = write_to_file(client_config_path, &default) {
crit!(log, "Failed to write default ClientConfig to file"; "error" => format!("{:?}", e));
return;
}
default
}
// May load this from disk or create a new configuration, depending on the CLI flags supplied.
let (client_config, eth2_config, log) = match get_configs(&matches, log) {
Ok(configs) => configs,
Err(e) => {
crit!(log, "Failed to load a ChainConfig file"; "error" => format!("{:?}", e));
crit!(log_clone, "Failed to load configuration. Exiting"; "error" => e);
return;
}
};
// Ensure the `data_dir` in the config matches that supplied to the CLI.
client_config.data_dir = data_dir.clone();
// Update the client config with any CLI args.
match client_config.apply_cli_args(&matches, &mut log) {
Ok(()) => (),
Err(s) => {
crit!(log, "Failed to parse ClientConfig CLI arguments"; "error" => s);
return;
}
};
let eth2_config_path = data_dir.join(ETH2_CONFIG_FILENAME);
// Initialise the `Eth2Config`.
//
// If a CLI parameter is set, overwrite any config file present.
// If a parameter is not set, use either the config file present or default to minimal.
let cli_config = match matches.value_of("default-spec") {
Some("mainnet") => Some(Eth2Config::mainnet()),
Some("minimal") => Some(Eth2Config::minimal()),
Some("interop") => Some(Eth2Config::interop()),
_ => None,
};
// if a CLI flag is specified, write the new config if it doesn't exist,
// otherwise notify the user that the file will not be written.
let eth2_config_from_file = match read_from_file::<Eth2Config>(eth2_config_path.clone()) {
Ok(config) => config,
Err(e) => {
crit!(log, "Failed to read the Eth2Config from file"; "error" => format!("{:?}", e));
return;
}
};
let mut eth2_config = {
if let Some(cli_config) = cli_config {
if eth2_config_from_file.is_none() {
// write to file if one doesn't exist
if let Err(e) = write_to_file(eth2_config_path, &cli_config) {
crit!(log, "Failed to write default Eth2Config to file"; "error" => format!("{:?}", e));
return;
}
} else {
warn!(
log,
"Eth2Config file exists. Configuration file is ignored, using default"
);
}
cli_config
} else {
// CLI config not specified, read from disk
match eth2_config_from_file {
Some(config) => config,
None => {
// set default to minimal
let eth2_config = Eth2Config::minimal();
if let Err(e) = write_to_file(eth2_config_path, &eth2_config) {
crit!(log, "Failed to write default Eth2Config to file"; "error" => format!("{:?}", e));
return;
}
eth2_config
}
}
}
};
// Update the eth2 config with any CLI flags.
match eth2_config.apply_cli_args(&matches) {
Ok(()) => (),
Err(s) => {
crit!(log, "Failed to parse Eth2Config CLI arguments"; "error" => s);
return;
}
};
// check to ensure the spec constants between the client and eth2_config match
if eth2_config.spec_constants != client_config.spec_constants {
crit!(log, "Specification constants do not match."; "client_config" => format!("{}", client_config.spec_constants), "eth2_config" => format!("{}", eth2_config.spec_constants));
return;
}
// Start the node using a `tokio` executor.
match run::run_beacon_node(client_config, eth2_config, &log) {
Ok(_) => {}

View File

@ -1,19 +1,17 @@
use client::{
error, notifier, BeaconChainTypes, Client, ClientConfig, ClientType, Eth2Config,
InitialiseBeaconChain,
};
use client::{error, notifier, Client, ClientConfig, Eth1BackendMethod, Eth2Config};
use futures::sync::oneshot;
use futures::Future;
use slog::{error, info};
use std::cell::RefCell;
use std::path::Path;
use std::path::PathBuf;
use store::Store;
use store::{DiskStore, MemoryStore};
use tokio::runtime::Builder;
use tokio::runtime::Runtime;
use tokio::runtime::TaskExecutor;
use tokio_timer::clock::Clock;
use types::{InteropEthSpec, MainnetEthSpec, MinimalEthSpec};
use types::{EthSpec, InteropEthSpec, MainnetEthSpec, MinimalEthSpec};
/// Reads the configuration and initializes a `BeaconChain` with the required types and parameters.
///
@ -44,63 +42,29 @@ pub fn run_beacon_node(
info!(
log,
"BeaconNode init";
"p2p_listen_address" => format!("{:?}", &other_client_config.network.listen_address),
"data_dir" => format!("{:?}", other_client_config.data_dir()),
"network_dir" => format!("{:?}", other_client_config.network.network_dir),
"spec_constants" => &spec_constants,
"Starting beacon node";
"p2p_listen_address" => format!("{}", &other_client_config.network.listen_address),
"db_type" => &other_client_config.db_type,
"spec_constants" => &spec_constants,
);
macro_rules! run_client {
($store: ty, $eth_spec: ty) => {
run::<$store, $eth_spec>(&db_path, client_config, eth2_config, executor, runtime, log)
};
}
if let Eth1BackendMethod::Web3 { .. } = client_config.eth1_backend_method {
return Err("Starting from web3 backend is not supported for interop.".into());
}
match (db_type.as_str(), spec_constants.as_str()) {
("disk", "minimal") => run::<ClientType<DiskStore, MinimalEthSpec>>(
&db_path,
client_config,
eth2_config,
executor,
runtime,
log,
),
("memory", "minimal") => run::<ClientType<MemoryStore, MinimalEthSpec>>(
&db_path,
client_config,
eth2_config,
executor,
runtime,
log,
),
("disk", "mainnet") => run::<ClientType<DiskStore, MainnetEthSpec>>(
&db_path,
client_config,
eth2_config,
executor,
runtime,
log,
),
("memory", "mainnet") => run::<ClientType<MemoryStore, MainnetEthSpec>>(
&db_path,
client_config,
eth2_config,
executor,
runtime,
log,
),
("disk", "interop") => run::<ClientType<DiskStore, InteropEthSpec>>(
&db_path,
client_config,
eth2_config,
executor,
runtime,
log,
),
("memory", "interop") => run::<ClientType<MemoryStore, InteropEthSpec>>(
&db_path,
client_config,
eth2_config,
executor,
runtime,
log,
),
("disk", "minimal") => run_client!(DiskStore, MinimalEthSpec),
("disk", "mainnet") => run_client!(DiskStore, MainnetEthSpec),
("disk", "interop") => run_client!(DiskStore, InteropEthSpec),
("memory", "minimal") => run_client!(MemoryStore, MinimalEthSpec),
("memory", "mainnet") => run_client!(MemoryStore, MainnetEthSpec),
("memory", "interop") => run_client!(MemoryStore, InteropEthSpec),
(db_type, spec) => {
error!(log, "Unknown runtime configuration"; "spec_constants" => spec, "db_type" => db_type);
Err("Unknown specification and/or db_type.".into())
@ -109,7 +73,7 @@ pub fn run_beacon_node(
}
/// Performs the type-generic parts of launching a `BeaconChain`.
fn run<T>(
fn run<S, E>(
db_path: &Path,
client_config: ClientConfig,
eth2_config: Eth2Config,
@ -118,12 +82,13 @@ fn run<T>(
log: &slog::Logger,
) -> error::Result<()>
where
T: BeaconChainTypes + InitialiseBeaconChain<T> + Clone,
T::Store: OpenDatabase,
S: Store + Clone + 'static + OpenDatabase,
E: EthSpec,
{
let store = T::Store::open_database(&db_path)?;
let store = S::open_database(&db_path)?;
let client: Client<T> = Client::new(client_config, eth2_config, store, log.clone(), &executor)?;
let client: Client<S, E> =
Client::new(client_config, eth2_config, store, log.clone(), &executor)?;
// run service until ctrl-c
let (ctrlc_send, ctrlc_oneshot) = oneshot::channel();

View File

@ -0,0 +1,20 @@
[package]
name = "websocket_server"
version = "0.1.0"
authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
beacon_chain = { path = "../beacon_chain" }
clap = "2.32.0"
exit-future = "0.1.3"
futures = "0.1.25"
serde = "1.0"
serde_derive = "1.0"
serde_json = "^1.0"
slog = "^2.2.3"
tokio = "0.1.16"
types = { path = "../../eth2/types" }
ws = "0.9"

View File

@ -0,0 +1,45 @@
use clap::ArgMatches;
use serde::{Deserialize, Serialize};
use std::net::Ipv4Addr;
/// The core configuration of a Lighthouse beacon node.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct Config {
pub enabled: bool,
/// The IPv4 address the REST API HTTP server will listen on.
pub listen_address: Ipv4Addr,
/// The port the REST API HTTP server will listen on.
pub port: u16,
}
impl Default for Config {
fn default() -> Self {
Config {
enabled: true,
listen_address: Ipv4Addr::new(127, 0, 0, 1),
port: 5053,
}
}
}
impl Config {
pub fn apply_cli_args(&mut self, args: &ArgMatches) -> Result<(), &'static str> {
if args.is_present("no-ws") {
self.enabled = false;
}
if let Some(rpc_address) = args.value_of("ws-address") {
self.listen_address = rpc_address
.parse::<Ipv4Addr>()
.map_err(|_| "ws-address is not a valid IPv4 address.")?;
}
if let Some(rpc_port) = args.value_of("ws-port") {
self.port = rpc_port
.parse::<u16>()
.map_err(|_| "ws-port is not a valid u16.")?;
}
Ok(())
}
}

View File

@ -0,0 +1,117 @@
use beacon_chain::events::{EventHandler, EventKind};
use futures::Future;
use slog::{debug, error, info, warn, Logger};
use std::marker::PhantomData;
use std::thread;
use tokio::runtime::TaskExecutor;
use types::EthSpec;
use ws::{Sender, WebSocket};
mod config;
pub use config::Config;
pub struct WebSocketSender<T: EthSpec> {
sender: Option<Sender>,
_phantom: PhantomData<T>,
}
impl<T: EthSpec> WebSocketSender<T> {
/// Creates a dummy websocket server that never starts and where all future calls are no-ops.
pub fn dummy() -> Self {
Self {
sender: None,
_phantom: PhantomData,
}
}
pub fn send_string(&self, string: String) -> Result<(), String> {
if let Some(sender) = &self.sender {
sender
.send(string)
.map_err(|e| format!("Unable to broadcast to websocket clients: {:?}", e))
} else {
Ok(())
}
}
}
impl<T: EthSpec> EventHandler<T> for WebSocketSender<T> {
fn register(&self, kind: EventKind<T>) -> Result<(), String> {
self.send_string(
serde_json::to_string(&kind)
.map_err(|e| format!("Unable to serialize event: {:?}", e))?,
)
}
}
pub fn start_server<T: EthSpec>(
config: &Config,
executor: &TaskExecutor,
log: &Logger,
) -> Result<(WebSocketSender<T>, exit_future::Signal), String> {
let server_string = format!("{}:{}", config.listen_address, config.port);
info!(
log,
"Websocket server starting";
"listen_address" => &server_string
);
// Create a server that simply ignores any incoming messages.
let server = WebSocket::new(|_| |_| Ok(()))
.map_err(|e| format!("Failed to initialize websocket server: {:?}", e))?;
let broadcaster = server.broadcaster();
// Produce a signal/channel that can gracefully shutdown the websocket server.
let exit_signal = {
let (exit_signal, exit) = exit_future::signal();
let log_inner = log.clone();
let broadcaster_inner = server.broadcaster();
let exit_future = exit.and_then(move |_| {
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");
}
Ok(())
});
// Place a future on the executor that will shutdown the websocket server when the
// application exits.
executor.spawn(exit_future);
exit_signal
};
let log_inner = log.clone();
let _handle = thread::spawn(move || match server.listen(server_string) {
Ok(_) => {
debug!(
log_inner,
"Websocket server thread stopped";
);
}
Err(e) => {
error!(
log_inner,
"Websocket server failed to start";
"error" => format!("{:?}", e)
);
}
});
Ok((
WebSocketSender {
sender: Some(broadcaster),
_phantom: PhantomData,
},
exit_signal,
))
}

1
book/.gitignore vendored Normal file
View File

@ -0,0 +1 @@
book

6
book/book.toml Normal file
View File

@ -0,0 +1,6 @@
[book]
authors = ["Paul Hauner"]
language = "en"
multilingual = false
src = "src"
title = "Lighthouse"

11
book/src/SUMMARY.md Normal file
View File

@ -0,0 +1,11 @@
# Summary
* [Introduction](./intro.md)
* [Development Environment](./setup.md)
* [Websocket Interface](./websockets.md)
* [Simple Local Testnet](./simple-testnet.md)
* [Interop](./interop.md)
* [Environment](./interop-environment.md)
* [CLI Overview](./interop-cli.md)
* [Scenarios](./interop-scenarios.md)
* [Cheat-sheet](./interop-cheat-sheet.md)

View File

@ -0,0 +1,149 @@
# Interop Cheat-sheet
This document contains a list of tips and tricks that may be useful during
interop testing.
- When starting a beacon node:
- [Specify a boot node by multiaddr](#boot-node-multiaddr)
- [Specify a boot node by ENR](#boot-node-enr)
- [Avoid port clashes when starting multiple nodes](#port-bump)
- [Specify a custom slot time](#slot-time)
- Using the beacon node HTTP API:
- [Pretty-print the genesis state and state root](#http-state)
- [Curl a node's ENR](#http-enr)
- [Curl a node's connected peers](#http-peer-ids)
- [Curl a node's local peer id](#http-peer-id)
- [Curl a node's listening multiaddrs](#http-listen-addresses)
- [Curl a node's beacon chain head](#http-head)
- [Curl a node's finalized checkpoint](#http-finalized)
## Category: CLI
The `--help` command provides detail on the CLI interface. Here are some
interop-specific CLI commands.
<a name="boot-node-multiaddr"></a>
### Specify a boot node by multiaddr
You can specify a static list of multiaddrs when booting Lighthouse using
the `--libp2p-addresses` command.
#### Example:
```
$ ./beacon_node --libp2p-addresses /ip4/192.168.0.1/tcp/9000
```
<a name="boot-node-enr"></a>
### Specify a boot node by ENR
You can specify a static list of Discv5 addresses when booting Lighthouse using
the `--boot-nodes` command.
#### Example:
```
$ ./beacon_node --boot-nodes -IW4QB2Hi8TPuEzQ41Cdf1r2AUU1FFVFDBJdJyOkWk2qXpZfFZQy2YnJIyoT_5fnbtrXUouoskmydZl4pIg90clIkYUDgmlwhH8AAAGDdGNwgiMog3VkcIIjKIlzZWNwMjU2azGhAjg0-DsTkQynhJCRnLLttBK1RS78lmUkLa-wgzAi-Ob5
```
<a name="port-bump"></a>
### Avoid port clashes when starting nodes
Starting a second Lighthouse node on the same machine will fail due to TCP/UDP
port collisions. Use the `-b` (`--port-bump`) flag to increase all listening
ports by some `n`.
#### Example:
Increase all ports by `10` (using multiples of `10` is recommended).
```
$ ./beacon_node -b 10
```
<a name="slot-time"></a>
### Start a testnet with a custom slot time
Lighthouse can run at quite low slot times when there are few validators (e.g.,
`500 ms` slot times should be fine for 8 validators).
#### Example
The `-t` (`--slot-time`) flag specifies the milliseconds per slot.
```
$ ./beacon_node testnet -t 500 recent 8
```
> Note: `bootstrap` loads the slot time via HTTP and therefore conflicts with
> this flag.
## Category: HTTP API
Examples assume there is a Lighthouse node exposing a HTTP API on
`localhost:5052`. Responses are JSON.
<a name="http-state"></a>
### Pretty-print the genesis state and state root
Returns the genesis state and state root in your terminal, in YAML.
```
$ curl --header "Content-Type: application/yaml" "localhost:5052/beacon/state?slot=0"
```
<a name="http-enr"></a>
### Get the node's ENR
```
$ curl localhost:5052/network/enr
"-IW4QFyf1VlY5pZs0xZuvKMRZ9_cdl9WMCDAAJXZiZiuGcfRYoU40VPrYDLQj5prneJIz3zcbTjHp9BbThc-yiymJO8HgmlwhH8AAAGDdGNwgiMog3VkcIIjKIlzZWNwMjU2azGhAjg0-DsTkQynhJCRnLLttBK1RS78lmUkLa-wgzAi-Ob5"%
```
<a name="http-peer-ids"></a>
### Get a list of connected peer ids
```
$ curl localhost:5052/network/peers
["QmeMFRTWfo3KbVG7dEBXGhyRMa29yfmnJBXW84rKuGEhuL"]%
```
<a name="http-peer-id"></a>
### Get the node's peer id
```
curl localhost:5052/network/peer_id
"QmRD1qs2AqNNRdBcGHUGpUGkpih5cmdL32mhh22Sy79xsJ"%
```
<a name="http-listen-addresses"></a>
### Get the list of listening libp2p addresses
Lists all the libp2p multiaddrs that the node is listening on.
```
curl localhost:5052/network/listen_addresses
["/ip4/127.0.0.1/tcp/9000","/ip4/192.168.1.121/tcp/9000","/ip4/172.17.0.1/tcp/9000","/ip4/172.42.0.1/tcp/9000","/ip6/::1/tcp/9000","/ip6/fdd3:c293:1bc::203/tcp/9000","/ip6/fdd3:c293:1bc:0:9aa9:b2ea:c610:44db/tcp/9000"]%
```
<a name="http-head"></a>
### Get the node's beacon chain head
```
curl localhost:5052/beacon/head
{"slot":0,"block_root":"0x827bf71805540aa13f6d8c7d18b41b287b2094a4d7a28cbb8deb061dbf5df4f5","state_root":"0x90a78d73294bc9c7519a64e1912161be0e823eb472012ff54204e15a4d717fa5"}%
```
<a name="http-finalized"></a>
### Get the node's finalized checkpoint
```
curl localhost:5052/beacon/latest_finalized_checkpoint
{"epoch":0,"root":"0x0000000000000000000000000000000000000000000000000000000000000000"}%
```

29
book/src/interop-cli.md Normal file
View File

@ -0,0 +1,29 @@
# Interop CLI Overview
The Lighthouse CLI has two primary tasks:
- **Resuming** an existing database with `$ ./beacon_node`.
- **Creating** a new testnet database using `$ ./beacon_node testnet`.
_See [Scenarios](./interop-scenarios.md) for methods we've anticipated will be
used interop._
## Creating a new database
There are several methods for creating a new beacon node database:
- `quick`: using the `(validator_client, genesis_time)` tuple.
- `recent`: as above but `genesis_time` is set to the start of some recent time
window.
- `file`: loads the genesis file from disk in one of multiple formats.
- `bootstrap`: a Lighthouse-specific method where we connect to a running node
and download it's specification and genesis state via the HTTP API.
See `$ ./beacon_node testnet --help` for more detail.
## Resuming from an existing database
Once a database has been created, it can be resumed by running `$ ./beacon_node`.
Presently, this command will fail if no existing database is found. You must
use the `$ ./beacon_node testnet` command to create a new database.

View File

@ -0,0 +1,30 @@
# Interop Environment
All that is required for inter-op is a built and tested [development
environment](./setup.md).
## Repositories
You will only require the [sigp/lighthouse](http://github.com/sigp/lighthouse)
library.
To allow for faster build/test iterations we will use the
[`interop`](https://github.com/sigp/lighthouse/tree/interop) branch of
[sigp/lighthouse](https://github.com/sigp/lighthouse/tree/interop) for
September 2019 interop. **Please use ensure you `git checkout interop` after
cloning the repo.**
## File System
When lighthouse boots, it will create the following
directories:
- `~/.lighthouse`: database and configuration for the beacon node.
- `~/.lighthouse-validator`: database and configuration for the validator
client.
After building the binaries with `cargo build --release --all`, there will be a
`target/release` directory in the root of the Lighthouse repository. This is
where the `beacon_node` and `validator_client` binaries are located.
You do not need to create any of these directories manually.

View File

@ -0,0 +1,101 @@
# Interop Scenarios
Here we demonstrate some expected interop scenarios.
All scenarios assume a working [development environment](./setup.md) and
commands are based in the `target/release` directory (this is the build dir for
`cargo`).
Additional functions can be found in the [interop
cheat-sheet](./interop-cheat-sheet.md).
### Table of contents
- [Starting from a`validator_count, genesis_time` tuple](#quick-start)
- [Starting a node from a genesis state file](#state-file)
- [Starting a validator client](#val-client)
- [Exporting a genesis state file](#export) from a running Lighthouse
node
<a name="quick-start"></a>
### Start beacon node given a validator count and genesis_time
To start a brand-new beacon node (with no history) use:
```
$ ./beacon_node testnet -f quick 8 <GENESIS_TIME>
```
Where `GENESIS_TIME` is in [unix time](https://duckduckgo.com/?q=unix+time&t=ffab&ia=answer).
> Notes:
>
> - This method conforms the ["Quick-start
genesis"](https://github.com/ethereum/eth2.0-pm/tree/6e41fcf383ebeb5125938850d8e9b4e9888389b4/interop/mocked_start#quick-start-genesis)
method in the `ethereum/eth2.0-pm` repository.
> - The `-f` flag ignores any existing database or configuration, backing them
> up before re-initializing.
> - `8` is the validator count and `1567222226` is the genesis time.
> - See `$ ./beacon_node testnet quick --help` for more configuration options.
<a name="state-file"></a>
### Start Beacon Node given a genesis state file
A genesis state can be read from file using the `testnet file` subcommand.
There are three supported formats:
- `ssz` (default)
- `json`
- `yaml`
Start a new node using `/tmp/genesis.ssz` as the genesis state:
```
$ ./beacon_node testnet --spec minimal -f file ssz /tmp/genesis.ssz
```
> Notes:
>
> - The `-f` flag ignores any existing database or configuration, backing them
> up before re-initializing.
> - See `$ ./beacon_node testnet file --help` for more configuration options.
> - The `--spec` flag is required to allow SSZ parsing of fixed-length lists.
<a name="val-client"></a>
### Start an auto-configured validator client
To start a brand-new validator client (with no history) use:
```
$ ./validator_client testnet -b insecure 0 8
```
> Notes:
>
> - The `-b` flag means the validator client will "bootstrap" specs and config
> from the beacon node.
> - The `insecure` command dictates that the [interop keypairs](https://github.com/ethereum/eth2.0-pm/tree/6e41fcf383ebeb5125938850d8e9b4e9888389b4/interop/mocked_start#pubkeyprivkey-generation)
> will be used.
> - The `0 8` indicates that this validator client should manage 8 validators,
> starting at validator 0 (the first deposited validator).
> - The validator client will try to connect to the beacon node at `localhost`.
> See `--help` to configure that address and other features.
> - The validator client will operate very unsafely in `testnet` mode, happily
> swapping between chains and creating double-votes.
<a name="export"></a>
### Exporting a genesis file
Genesis states can downloaded from a running Lighthouse node via the HTTP API. Three content-types are supported:
- `application/json`
- `application/yaml`
- `application/ssz`
Using `curl`, a genesis state can be downloaded to `/tmp/genesis.ssz`:
```
$ curl --header "Content-Type: application/ssz" "localhost:5052/beacon/state/genesis" -o /tmp/genesis.ssz
```

11
book/src/interop.md Normal file
View File

@ -0,0 +1,11 @@
# Lighthouse Interop Guide
This guide is intended for other Ethereum 2.0 client developers performing
inter-operability testing with Lighthouse.
## Chapters
- Read about the required [development environment](./interop-environment.md).
- Get an [overview](./interop-cli.md) of the Lighthouse CLI.
- See how we expect to handle some [interop scenarios](./interop-scenarios.md).
- See the [interop cheat-sheet](./interop-cheat-sheet.md) for useful CLI tips.

27
book/src/intro.md Normal file
View File

@ -0,0 +1,27 @@
# Lighthouse Documentation
[![Build Status]][Build Link] [![Doc Status]][Doc Link] [![Chat Badge]][Chat Link]
[Build Status]: https://gitlab.sigmaprime.io/sigp/lighthouse/badges/master/build.svg
[Build Link]: https://gitlab.sigmaprime.io/sigp/lighthouse/pipelines
[Chat Badge]: https://img.shields.io/badge/chat-discord-%237289da
[Chat Link]: https://discord.gg/cyAszAh
[Doc Status]:https://img.shields.io/badge/rust--docs-master-orange
[Doc Link]: http://lighthouse-docs.sigmaprime.io/
Lighthouse is an **Ethereum 2.0 client** that connects to other Ethereum 2.0
clients to form a resilient and decentralized proof-of-stake blockchain.
It is written in Rust, maintained by Sigma Prime and funded by the Ethereum
Foundation, Consensys and other individuals and organisations.
## Developer Resources
Documentation is presently targeted at **researchers and developers**. It
assumes significant prior knowledge of Ethereum 2.0.
Topics:
- Get started with [development environment setup](./setup.md).
- See the [interop docs](./interop.md).
- [Run a simple testnet](./simple-testnet.md) in Only Three CLI Commands™.

81
book/src/setup.md Normal file
View File

@ -0,0 +1,81 @@
# Development Environment Setup
Follow this guide to get a Lighthouse development environment up-and-running.
See the [Quick instructions](#quick-instructions) for a summary or the
[Detailed instructions](#detailed-instructions) for clarification.
## Quick instructions
1. Install Rust + Cargo with [rustup](https://rustup.rs/).
1. Install build dependencies using your package manager.
- `$ <package-manager> clang protobuf libssl-dev cmake`
1. Clone the [sigp/lighthouse](https://github.com/sigp/lighthouse).
1. In the root of the repo, run the tests with `cargo test --all --release`.
1. Then, build the binaries with `cargo build --all --release`.
1. Lighthouse is now fully built and tested.
_Note: first-time compilation may take several minutes._
## Detailed instructions
A fully-featured development environment can be achieved with the following
steps:
1. Install [rustup](https://rustup.rs/).
1. Use the command `rustup show` to get information about the Rust
installation. You should see that the active tool-chain is the stable
version.
- Updates can be performed using` rustup update`, Lighthouse generally
requires a recent version of Rust.
1. Install build dependencies (Arch packages are listed here, your
distribution will likely be similar):
- `clang`: required by RocksDB.
- `protobuf`: required for protobuf serialization (gRPC)
- `libssl-dev`: also gRPC
- `cmake`: required for building protobuf
1. Clone the repository with submodules: `git clone
https://github.com/sigp/lighthouse`.
1. Change directory to the root of the repository.
1. Run the test suite with `cargo test --all --release`. The build and test
process can take several minutes. If you experience any failures on
`master`, please raise an
[issue](https://github.com/sigp/lighthouse/issues).
### Notes:
Lighthouse targets Rust `stable` but generally runs on `nightly` too.
#### Note for Windows users:
Perl may also be required to build lighthouse. You can install [Strawberry
Perl](http://strawberryperl.com/), or alternatively use a choco install command
`choco install strawberryperl`.
Additionally, the dependency `protoc-grpcio v0.3.1` is reported to have issues
compiling in Windows. You can specify a known working version by editing
version in `protos/Cargo.toml` section to `protoc-grpcio = "<=0.3.0"`.
## eth2.0-spec-tests
The
[ethereum/eth2.0-spec-tests](https://github.com/ethereum/eth2.0-spec-tests/)
repository contains a large set of tests that verify Lighthouse behaviour
against the Ethereum Foundation specifications.
The `tests/ef_tests` crate runs these tests and it has some interesting
behaviours:
- If the `tests/ef_tests/eth2.0-spec-tests` directory is not present, all tests
indicate a `pass` when they did not actually run.
- If that directory _is_ present, the tests are executed faithfully, failing if
a discrepancy is found.
The `tests/ef_tests/eth2.0-spec-tests` directory is not present by default. To
obtain it, use the Makefile in the root of the repository:
```
make ef_tests
```
_Note: this will download 100+ MB of test files from the [ethereum/eth2.0-spec-tests](https://github.com/ethereum/eth2.0-spec-tests/)._

View File

@ -0,0 +1,81 @@
# Simple Local Testnet
You can setup a local, two-node testnet in **Only Three CLI Commands™**.
Follow the [Quick instructions](#tldr) version if you're confident, or see
[Detailed instructions](#detail) for more.
## Quick instructions
Setup a development environment, build the project and navigate to the
`target/release` directory.
1. Start the first node: `$ ./beacon_node testnet -f recent 8`
1. Start a validator client: `$ ./validator_client testnet -b insecure 0 8`
1. Start another node `$ ./beacon_node -b 10 testnet -f bootstrap http://localhost:5052`
_Repeat #3 to add more nodes._
## Detailed instructions
First, setup a Lighthouse development environment and navigate to the
`target/release` directory (this is where the binaries are located).
## Starting the Beacon Node
Start a new node (creating a fresh database and configuration in `~/.lighthouse`), using:
```
$ ./beacon_node testnet -f recent 8
```
> Notes:
>
> - The `-f` flag ignores any existing database or configuration, backing them
> up before re-initializing.
> - `8` is number of validators with deposits in the genesis state.
> - See `$ ./beacon_node testnet recent --help` for more configuration options,
> including `minimal`/`mainnet` specification.
## Starting the Validator Client
In a new terminal window, start the validator client with:
```
$ ./validator_client testnet -b insecure 0 8
```
> Notes:
>
> - The `-b` flag means the validator client will "bootstrap" specs and config
> from the beacon node.
> - The `insecure` command uses predictable, well-known private keys. Since
> this is just a local testnet, these are fine.
> - The `0 8` indicates that this validator client should manage 8 validators,
> starting at validator 0 (the first deposited validator).
> - The validator client will try to connect to the beacon node at `localhost`.
> See `--help` to configure that address and other features.
## Adding another Beacon Node
You may connect another (non-validating) node to your local network using the
lighthouse `bootstrap` command.
In a new terminal terminal, run:
```
$ ./beacon_node -b 10 testnet -r bootstrap
```
> Notes:
>
> - The `-b` (or `--port-bump`) increases all the listening TCP/UDP ports of
> the new node to `10` higher. Your first node's HTTP server was at TCP
> `5052` but this one will be at `5062`.
> - The `-r` flag creates a new data directory with a random string appended
> (avoids data directory collisions between nodes).
> - The default bootstrap HTTP address is `http://localhost:5052`. The new node
> will download configuration via HTTP before starting sync via libp2p.
> - See `$ ./beacon_node testnet bootstrap --help` for more configuration.

10
book/src/testnets.md Normal file
View File

@ -0,0 +1,10 @@
# Testnets
Lighthouse does not offer a public testnet _yet_. In the meantime, it's easy to
start a local testnet:
- [Run a simple testnet](testnets.html) in Only Three CLI Commands™.
- Developers of other Eth2 clients should see the [interop guide](interop.html).
- The [sigp/lighthouse-docker](https://github.com/sigp/lighthouse-docker) repo
contains a `docker-compose` setup that runs a multi-node network with
built-in metrics and monitoring dashboards, all from your local machine.

108
book/src/websockets.md Normal file
View File

@ -0,0 +1,108 @@
# Websocket Interface
By default, a Lighthouse `beacon_node` exposes a websocket server on `localhost:5053`.
The following CLI flags control the websocket server:
- `--no-ws`: disable the websocket server.
- `--ws-port`: specify the listen port of the server.
- `--ws-address`: specify the listen address of the server.
All clients connected to the websocket server will receive the same stream of events, all triggered
by the `BeaconChain`. Each event is a JSON object with the following schema:
```json
{
"event": "string",
"data": "object"
}
```
## Events
The following events may be emitted:
### Beacon Head Changed
Occurs whenever the canonical head of the beacon chain changes.
```json
{
"event": "beacon_head_changed",
"data": {
"reorg": "boolean",
"current_head_beacon_block_root": "string",
"previous_head_beacon_block_root": "string"
}
}
```
### Beacon Finalization
Occurs whenever the finalized checkpoint of the canonical head changes.
```json
{
"event": "beacon_finalization",
"data": {
"epoch": "number",
"root": "string"
}
}
```
### Beacon Block Imported
Occurs whenever the beacon node imports a valid block.
```json
{
"event": "beacon_block_imported",
"data": {
"block": "object"
}
}
```
### Beacon Block Rejected
Occurs whenever the beacon node rejects a block because it is invalid or an
error occurred during validation.
```json
{
"event": "beacon_block_rejected",
"data": {
"reason": "string",
"block": "object"
}
}
```
### Beacon Attestation Imported
Occurs whenever the beacon node imports a valid attestation.
```json
{
"event": "beacon_attestation_imported",
"data": {
"attestation": "object"
}
}
```
### Beacon Attestation Rejected
Occurs whenever the beacon node rejects an attestation because it is invalid or
an error occurred during validation.
```json
{
"event": "beacon_attestation_rejected",
"data": {
"reason": "string",
"attestation": "object"
}
}
```

File diff suppressed because it is too large Load Diff

View File

@ -46,4 +46,10 @@ pub trait LmdGhost<S: Store, E: EthSpec>: Send + Sync {
/// Returns the latest message for a given validator index.
fn latest_message(&self, validator_index: usize) -> Option<(Hash256, Slot)>;
/// Runs an integrity verification function on fork choice algorithm.
///
/// Returns `Ok(())` if the underlying fork choice has maintained it's integrity,
/// `Err(description)` otherwise.
fn verify_integrity(&self) -> Result<()>;
}

View File

@ -43,16 +43,6 @@ impl<T, E> fmt::Debug for ThreadSafeReducedTree<T, E> {
}
}
impl<T, E> ThreadSafeReducedTree<T, E>
where
T: Store,
E: EthSpec,
{
pub fn verify_integrity(&self) -> std::result::Result<(), String> {
self.core.read().verify_integrity()
}
}
impl<T, E> LmdGhost<T, E> for ThreadSafeReducedTree<T, E>
where
T: Store,
@ -80,7 +70,7 @@ where
fn process_block(&self, block: &BeaconBlock<E>, block_hash: Hash256) -> SuperResult<()> {
self.core
.write()
.add_weightless_node(block.slot, block_hash)
.maybe_add_weightless_node(block.slot, block_hash)
.map_err(|e| format!("process_block failed: {:?}", e))
}
@ -113,6 +103,10 @@ where
fn latest_message(&self, validator_index: usize) -> Option<(Hash256, Slot)> {
self.core.read().latest_message(validator_index)
}
fn verify_integrity(&self) -> std::result::Result<(), String> {
self.core.read().verify_integrity()
}
}
struct ReducedTree<T, E> {
@ -163,15 +157,7 @@ where
/// The given `new_root` must be in the block tree (but not necessarily in the reduced tree).
/// Any nodes which are not a descendant of `new_root` will be removed from the store.
pub fn update_root(&mut self, new_slot: Slot, new_root: Hash256) -> Result<()> {
if !self.nodes.contains_key(&new_root) {
let node = Node {
block_hash: new_root,
voters: vec![],
..Node::default()
};
self.add_node(node)?;
}
self.maybe_add_weightless_node(new_slot, new_root)?;
self.retain_subtree(self.root.0, new_root)?;
@ -247,7 +233,7 @@ where
//
// In this case, we add a weightless node at `start_block_root`.
if !self.nodes.contains_key(&start_block_root) {
self.add_weightless_node(start_block_slot, start_block_root)?;
self.maybe_add_weightless_node(start_block_slot, start_block_root)?;
};
let _root_weight = self.update_weight(start_block_root, weight_fn)?;
@ -325,51 +311,53 @@ where
/// become redundant and removed from the reduced tree.
fn remove_latest_message(&mut self, validator_index: usize) -> Result<()> {
if let Some(vote) = *self.latest_votes.get(validator_index) {
self.get_mut_node(vote.hash)?.remove_voter(validator_index);
let node = self.get_node(vote.hash)?.clone();
if self.nodes.contains_key(&vote.hash) {
self.get_mut_node(vote.hash)?.remove_voter(validator_index);
let node = self.get_node(vote.hash)?.clone();
if let Some(parent_hash) = node.parent_hash {
if node.has_votes() || node.children.len() > 1 {
// A node with votes or more than one child is never removed.
} else if node.children.len() == 1 {
// A node which has only one child may be removed.
//
// Load the child of the node and set it's parent to be the parent of this
// node (viz., graft the node's child to the node's parent)
let child = self.get_mut_node(node.children[0])?;
child.parent_hash = node.parent_hash;
if let Some(parent_hash) = node.parent_hash {
if node.has_votes() || node.children.len() > 1 {
// A node with votes or more than one child is never removed.
} else if node.children.len() == 1 {
// A node which has only one child may be removed.
//
// Load the child of the node and set it's parent to be the parent of this
// node (viz., graft the node's child to the node's parent)
let child = self.get_mut_node(node.children[0])?;
child.parent_hash = node.parent_hash;
// Graft the parent of this node to it's child.
if let Some(parent_hash) = node.parent_hash {
let parent = self.get_mut_node(parent_hash)?;
parent.replace_child(node.block_hash, node.children[0])?;
// Graft the parent of this node to it's child.
if let Some(parent_hash) = node.parent_hash {
let parent = self.get_mut_node(parent_hash)?;
parent.replace_child(node.block_hash, node.children[0])?;
}
self.nodes.remove(&vote.hash);
} else if node.children.is_empty() {
// Remove the to-be-deleted node from it's parent.
if let Some(parent_hash) = node.parent_hash {
self.get_mut_node(parent_hash)?
.remove_child(node.block_hash)?;
}
self.nodes.remove(&vote.hash);
// A node which has no children may be deleted and potentially it's parent
// too.
self.maybe_delete_node(parent_hash)?;
} else {
// It is impossible for a node to have a number of children that is not 0, 1 or
// greater than one.
//
// This code is strictly unnecessary, however we keep it for readability.
unreachable!();
}
self.nodes.remove(&vote.hash);
} else if node.children.is_empty() {
// Remove the to-be-deleted node from it's parent.
if let Some(parent_hash) = node.parent_hash {
self.get_mut_node(parent_hash)?
.remove_child(node.block_hash)?;
}
self.nodes.remove(&vote.hash);
// A node which has no children may be deleted and potentially it's parent
// too.
self.maybe_delete_node(parent_hash)?;
} else {
// It is impossible for a node to have a number of children that is not 0, 1 or
// greater than one.
//
// This code is strictly unnecessary, however we keep it for readability.
unreachable!();
// A node without a parent is the genesis/finalized node and should never be removed.
}
} else {
// A node without a parent is the genesis/finalized node and should never be removed.
}
self.latest_votes.insert(validator_index, Some(vote));
self.latest_votes.insert(validator_index, Some(vote));
}
}
Ok(())
@ -384,25 +372,30 @@ where
/// - it does not have any votes.
fn maybe_delete_node(&mut self, hash: Hash256) -> Result<()> {
let should_delete = {
let node = self.get_node(hash)?.clone();
if let Ok(node) = self.get_node(hash) {
let node = node.clone();
if let Some(parent_hash) = node.parent_hash {
if (node.children.len() == 1) && !node.has_votes() {
let child_hash = node.children[0];
if let Some(parent_hash) = node.parent_hash {
if (node.children.len() == 1) && !node.has_votes() {
let child_hash = node.children[0];
// Graft the single descendant `node` to the `parent` of node.
self.get_mut_node(child_hash)?.parent_hash = Some(parent_hash);
// Graft the single descendant `node` to the `parent` of node.
self.get_mut_node(child_hash)?.parent_hash = Some(parent_hash);
// Detach `node` from `parent`, replacing it with `child`.
self.get_mut_node(parent_hash)?
.replace_child(hash, child_hash)?;
// Detach `node` from `parent`, replacing it with `child`.
self.get_mut_node(parent_hash)?
.replace_child(hash, child_hash)?;
true
true
} else {
false
}
} else {
// A node without a parent is the genesis node and should not be deleted.
false
}
} else {
// A node without a parent is the genesis node and should not be deleted.
// No need to delete a node that does not exist.
false
}
};
@ -430,7 +423,7 @@ where
Ok(())
}
fn add_weightless_node(&mut self, slot: Slot, hash: Hash256) -> Result<()> {
fn maybe_add_weightless_node(&mut self, slot: Slot, hash: Hash256) -> Result<()> {
if slot > self.root_slot() && !self.nodes.contains_key(&hash) {
let node = Node {
block_hash: hash,
@ -477,6 +470,7 @@ where
// descendant of both `node` and `prev_in_tree`.
if self
.iter_ancestors(child_hash)?
.take_while(|(_, slot)| *slot >= self.root_slot())
.any(|(ancestor, _slot)| ancestor == node.block_hash)
{
let child = self.get_mut_node(child_hash)?;
@ -562,6 +556,7 @@ where
fn find_prev_in_tree(&mut self, hash: Hash256) -> Option<Hash256> {
self.iter_ancestors(hash)
.ok()?
.take_while(|(_, slot)| *slot >= self.root_slot())
.find(|(root, _slot)| self.nodes.contains_key(root))
.and_then(|(root, _slot)| Some(root))
}
@ -569,8 +564,12 @@ where
/// For the two given block roots (`a_root` and `b_root`), find the first block they share in
/// the tree. Viz, find the block that these two distinct blocks forked from.
fn find_highest_common_ancestor(&self, a_root: Hash256, b_root: Hash256) -> Result<Hash256> {
let mut a_iter = self.iter_ancestors(a_root)?;
let mut b_iter = self.iter_ancestors(b_root)?;
let mut a_iter = self
.iter_ancestors(a_root)?
.take_while(|(_, slot)| *slot >= self.root_slot());
let mut b_iter = self
.iter_ancestors(b_root)?
.take_while(|(_, slot)| *slot >= self.root_slot());
// Combines the `next()` fns on the `a_iter` and `b_iter` and returns the roots of two
// blocks at the same slot, or `None` if we have gone past genesis or the root of this tree.

View File

@ -4,7 +4,8 @@
extern crate lazy_static;
use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness as BaseBeaconChainHarness, BlockStrategy,
generate_deterministic_keypairs, AttestationStrategy,
BeaconChainHarness as BaseBeaconChainHarness, BlockStrategy,
};
use lmd_ghost::{LmdGhost, ThreadSafeReducedTree as BaseThreadSafeReducedTree};
use rand::{prelude::*, rngs::StdRng};
@ -51,7 +52,7 @@ struct ForkedHarness {
impl ForkedHarness {
/// A new standard instance of with constant parameters.
pub fn new() -> Self {
let harness = BeaconChainHarness::new(VALIDATOR_COUNT);
let harness = BeaconChainHarness::new(generate_deterministic_keypairs(VALIDATOR_COUNT));
// Move past the zero slot.
harness.advance_slot();

View File

@ -16,9 +16,9 @@ use state_processing::per_block_processing::errors::{
};
use state_processing::per_block_processing::{
get_slashable_indices_modular, verify_attestation_for_block_inclusion,
verify_attestation_for_state, verify_attester_slashing, verify_exit,
verify_exit_time_independent_only, verify_proposer_slashing, verify_transfer,
verify_transfer_time_independent_only, VerifySignatures,
verify_attester_slashing, verify_exit, verify_exit_time_independent_only,
verify_proposer_slashing, verify_transfer, verify_transfer_time_independent_only,
VerifySignatures,
};
use std::collections::{btree_map::Entry, hash_map, BTreeMap, HashMap, HashSet};
use std::marker::PhantomData;

View File

@ -15,6 +15,7 @@ serde = "1.0"
serde_derive = "1.0"
lazy_static = "0.1"
serde_yaml = "0.8"
eth2_ssz = { path = "../utils/ssz" }
beacon_chain = { path = "../../beacon_node/beacon_chain" }
store = { path = "../../beacon_node/store" }
lmd_ghost = { path = "../lmd_ghost" }

View File

@ -2,6 +2,7 @@ extern crate env_logger;
use criterion::Criterion;
use criterion::{black_box, criterion_group, criterion_main, Benchmark};
use ssz::Encode;
use state_processing::{test_utils::BlockBuilder, BlockSignatureStrategy, VerifySignatures};
use types::{BeaconBlock, BeaconState, ChainSpec, EthSpec, MainnetEthSpec, MinimalEthSpec, Slot};
@ -393,6 +394,32 @@ fn bench_block<T: EthSpec>(
})
.sample_size(10),
);
let local_block = block.clone();
c.bench(
&title,
Benchmark::new("ssz_serialize_block", move |b| {
b.iter_batched_ref(
|| (),
|_| black_box(local_block.as_ssz_bytes()),
criterion::BatchSize::SmallInput,
)
})
.sample_size(10),
);
let local_block = block.clone();
c.bench(
&title,
Benchmark::new("ssz_block_len", move |b| {
b.iter_batched_ref(
|| (),
|_| black_box(local_block.ssz_bytes_len()),
criterion::BatchSize::SmallInput,
)
})
.sample_size(10),
);
}
criterion_group!(benches, all_benches,);

View File

@ -17,11 +17,9 @@ pub fn get_attesting_indices<T: EthSpec>(
target_relative_epoch,
)?;
/* TODO(freeze): re-enable this?
if bitlist.len() > committee.committee.len() {
if bitlist.len() != committee.committee.len() {
return Err(BeaconStateError::InvalidBitfield);
}
*/
Ok(committee
.committee

View File

@ -3,7 +3,7 @@ use types::*;
/// Return the compact committee root at `relative_epoch`.
///
/// Spec v0.8.0
/// Spec v0.8.3
pub fn get_compact_committees_root<T: EthSpec>(
state: &BeaconState<T>,
relative_epoch: RelativeEpoch,
@ -11,28 +11,13 @@ pub fn get_compact_committees_root<T: EthSpec>(
) -> Result<Hash256, BeaconStateError> {
let mut committees =
FixedVector::<_, T::ShardCount>::from_elem(CompactCommittee::<T>::default());
// FIXME: this is a spec bug, whereby the start shard for the epoch after the next epoch
// is mistakenly used. The start shard from the cache SHOULD work.
// Waiting on a release to fix https://github.com/ethereum/eth2.0-specs/issues/1315
let start_shard = if relative_epoch == RelativeEpoch::Next {
state.next_epoch_start_shard(spec)?
} else {
state.get_epoch_start_shard(relative_epoch)?
};
let start_shard = state.get_epoch_start_shard(relative_epoch)?;
for committee_number in 0..state.get_committee_count(relative_epoch)? {
let shard = (start_shard + committee_number) % T::ShardCount::to_u64();
// FIXME: this is a partial workaround for the above, but it only works in the case
// where there's a committee for every shard in every epoch. It works for the minimal
// tests but not the mainnet ones.
let fake_shard = if relative_epoch == RelativeEpoch::Next {
(shard + 1) % T::ShardCount::to_u64()
} else {
shard
};
for &index in state
.get_crosslink_committee_for_shard(fake_shard, relative_epoch)?
.get_crosslink_committee_for_shard(shard, relative_epoch)?
.committee
{
let validator = state

View File

@ -11,6 +11,8 @@ pub fn get_indexed_attestation<T: EthSpec>(
state: &BeaconState<T>,
attestation: &Attestation<T>,
) -> Result<IndexedAttestation<T>> {
// Note: we rely on both calls to `get_attesting_indices` to check the bitfield lengths
// against the committee length
let attesting_indices =
get_attesting_indices(state, &attestation.data, &attestation.aggregation_bits)?;

View File

@ -42,8 +42,12 @@ pub fn block_proposal_signature_set<'a, T: EthSpec>(
block_signed_root: Option<Hash256>,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>> {
let block_proposer = &state.validators
[state.get_beacon_proposer_index(block.slot, RelativeEpoch::Current, spec)?];
let proposer_index =
state.get_beacon_proposer_index(block.slot, RelativeEpoch::Current, spec)?;
let block_proposer = &state
.validators
.get(proposer_index)
.ok_or_else(|| Error::ValidatorUnknown(proposer_index as u64))?;
let domain = spec.get_domain(
block.slot.epoch(T::slots_per_epoch()),

View File

@ -1,4 +1,5 @@
#![cfg(all(test, not(feature = "fake_crypto")))]
use super::block_processing_builder::BlockProcessingBuilder;
use super::errors::*;
use crate::{per_block_processing, BlockSignatureStrategy};

View File

@ -1,8 +1,5 @@
use crate::common::get_compact_committees_root;
use apply_rewards::process_rewards_and_penalties;
use errors::EpochProcessingError as Error;
use process_slashings::process_slashings;
use registry_updates::process_registry_updates;
use std::collections::HashMap;
use tree_hash::TreeHash;
use types::*;
@ -17,6 +14,10 @@ pub mod tests;
pub mod validator_statuses;
pub mod winning_root;
pub use apply_rewards::process_rewards_and_penalties;
pub use process_slashings::process_slashings;
pub use registry_updates::process_registry_updates;
/// Maps a shard to a winning root.
///
/// It is generated during crosslink processing and later used to reward/penalize validators.
@ -47,15 +48,10 @@ pub fn per_epoch_processing<T: EthSpec>(
process_justification_and_finalization(state, &validator_statuses.total_balances)?;
// Crosslinks.
let winning_root_for_shards = process_crosslinks(state, spec)?;
process_crosslinks(state, spec)?;
// Rewards and Penalties.
process_rewards_and_penalties(
state,
&mut validator_statuses,
&winning_root_for_shards,
spec,
)?;
process_rewards_and_penalties(state, &mut validator_statuses, spec)?;
// Registry Updates.
process_registry_updates(state, spec)?;
@ -159,9 +155,7 @@ pub fn process_justification_and_finalization<T: EthSpec>(
pub fn process_crosslinks<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
) -> Result<WinningRootHashSet, Error> {
let mut winning_root_for_shards: WinningRootHashSet = HashMap::new();
) -> Result<(), Error> {
state.previous_crosslinks = state.current_crosslinks.clone();
for &relative_epoch in &[RelativeEpoch::Previous, RelativeEpoch::Current] {
@ -181,12 +175,11 @@ pub fn process_crosslinks<T: EthSpec>(
if 3 * winning_root.total_attesting_balance >= 2 * total_committee_balance {
state.current_crosslinks[shard as usize] = winning_root.crosslink.clone();
}
winning_root_for_shards.insert(shard, winning_root);
}
}
}
Ok(winning_root_for_shards)
Ok(())
}
/// Finish up an epoch update.
@ -218,45 +211,29 @@ pub fn process_final_updates<T: EthSpec>(
}
}
// Update start shard.
state.start_shard = state.next_epoch_start_shard(spec)?;
// This is a hack to allow us to update index roots and slashed balances for the next epoch.
//
// The indentation here is to make it obvious where the weird stuff happens.
{
state.slot += 1;
// Set active index root
let index_epoch = next_epoch + spec.activation_exit_delay;
let indices_list = VariableList::<usize, T::ValidatorRegistryLimit>::from(
state.get_active_validator_indices(index_epoch),
);
state.set_active_index_root(
index_epoch,
Hash256::from_slice(&indices_list.tree_hash_root()),
spec,
)?;
// Reset slashings
state.set_slashings(next_epoch, 0)?;
// Set randao mix
state.set_randao_mix(next_epoch, *state.get_randao_mix(current_epoch)?)?;
state.slot -= 1;
}
// Set active index root
let index_epoch = next_epoch + spec.activation_exit_delay;
let indices_list = VariableList::<usize, T::ValidatorRegistryLimit>::from(
state.get_active_validator_indices(index_epoch),
);
state.set_active_index_root(
index_epoch,
Hash256::from_slice(&indices_list.tree_hash_root()),
spec,
)?;
// Set committees root
// Note: we do this out-of-order w.r.t. to the spec, because we don't want the slot to be
// incremented. It's safe because the updates to slashings and the RANDAO mix (above) don't
// affect this.
state.set_compact_committee_root(
next_epoch,
get_compact_committees_root(state, RelativeEpoch::Next, spec)?,
spec,
)?;
// Reset slashings
state.set_slashings(next_epoch, 0)?;
// Set randao mix
state.set_randao_mix(next_epoch, *state.get_randao_mix(current_epoch)?)?;
// Set historical root accumulator
if next_epoch.as_u64() % (T::SlotsPerHistoricalRoot::to_u64() / T::slots_per_epoch()) == 0 {
let historical_batch = state.historical_batch();
@ -265,6 +242,9 @@ pub fn process_final_updates<T: EthSpec>(
.push(Hash256::from_slice(&historical_batch.tree_hash_root()))?;
}
// Update start shard.
state.start_shard = state.get_epoch_start_shard(RelativeEpoch::Next)?;
// Rotate current/previous epoch attestations
state.previous_epoch_attestations =
std::mem::replace(&mut state.current_epoch_attestations, VariableList::empty());

View File

@ -1,5 +1,5 @@
use super::validator_statuses::{TotalBalances, ValidatorStatus, ValidatorStatuses};
use super::{Error, WinningRootHashSet};
use super::Error;
use integer_sqrt::IntegerSquareRoot;
use types::*;
@ -36,7 +36,6 @@ impl std::ops::AddAssign for Delta {
pub fn process_rewards_and_penalties<T: EthSpec>(
state: &mut BeaconState<T>,
validator_statuses: &mut ValidatorStatuses,
winning_root_for_shards: &WinningRootHashSet,
spec: &ChainSpec,
) -> Result<(), Error> {
if state.current_epoch() == T::genesis_epoch() {
@ -53,15 +52,13 @@ pub fn process_rewards_and_penalties<T: EthSpec>(
let mut deltas = vec![Delta::default(); state.balances.len()];
get_attestation_deltas(&mut deltas, state, &validator_statuses, spec)?;
// Update statuses with the information from winning roots.
validator_statuses.process_winning_roots(state, spec)?;
get_crosslink_deltas(&mut deltas, state, &validator_statuses, spec)?;
get_proposer_deltas(
&mut deltas,
state,
validator_statuses,
winning_root_for_shards,
spec,
)?;
get_proposer_deltas(&mut deltas, state, validator_statuses, spec)?;
// Apply the deltas, over-flowing but not under-flowing (saturating at 0 instead).
for (i, delta) in deltas.iter().enumerate() {
@ -79,12 +76,8 @@ fn get_proposer_deltas<T: EthSpec>(
deltas: &mut Vec<Delta>,
state: &BeaconState<T>,
validator_statuses: &mut ValidatorStatuses,
winning_root_for_shards: &WinningRootHashSet,
spec: &ChainSpec,
) -> Result<(), Error> {
// Update statuses with the information from winning roots.
validator_statuses.process_winning_roots(state, winning_root_for_shards, spec)?;
for (index, validator) in validator_statuses.statuses.iter().enumerate() {
if validator.is_previous_epoch_attester {
let inclusion = validator

View File

@ -1,4 +1,4 @@
use super::WinningRootHashSet;
use super::{winning_root::winning_root, WinningRootHashSet};
use crate::common::get_attesting_indices;
use types::*;
@ -292,9 +292,29 @@ impl ValidatorStatuses {
pub fn process_winning_roots<T: EthSpec>(
&mut self,
state: &BeaconState<T>,
winning_roots: &WinningRootHashSet,
spec: &ChainSpec,
) -> Result<(), BeaconStateError> {
// We must re-calculate the winning roots here because it is possible that they have
// changed since the first time they were calculated.
//
// This is because we altered the state during the first time we calculated the winning
// roots.
let winning_root_for_shards = {
let mut winning_root_for_shards = WinningRootHashSet::new();
let relative_epoch = RelativeEpoch::Previous;
let epoch = relative_epoch.into_epoch(state.current_epoch());
for offset in 0..state.get_committee_count(relative_epoch)? {
let shard = (state.get_epoch_start_shard(relative_epoch)? + offset)
% T::ShardCount::to_u64();
if let Some(winning_root) = winning_root(state, shard, epoch, spec)? {
winning_root_for_shards.insert(shard, winning_root);
}
}
winning_root_for_shards
};
// Loop through each slot in the previous epoch.
for slot in state.previous_epoch().slot_iter(T::slots_per_epoch()) {
let crosslink_committees_at_slot = state.get_crosslink_committees_at_slot(slot)?;
@ -302,7 +322,7 @@ impl ValidatorStatuses {
// Loop through each committee in the slot.
for c in crosslink_committees_at_slot {
// If there was some winning crosslink root for the committee's shard.
if let Some(winning_root) = winning_roots.get(&c.shard) {
if let Some(winning_root) = winning_root_for_shards.get(&c.shard) {
let total_committee_balance = state.get_total_balance(&c.committee, spec)?;
for &validator_index in &winning_root.attesting_validator_indices {
// Take note of the balance information for the winning root, it will be

View File

@ -3,7 +3,7 @@ use std::collections::{HashMap, HashSet};
use tree_hash::TreeHash;
use types::*;
#[derive(Clone)]
#[derive(Clone, Debug)]
pub struct WinningRoot {
pub crosslink: Crosslink,
pub attesting_validator_indices: Vec<usize>,

View File

@ -1,3 +1,5 @@
#![cfg(not(feature = "fake_crypto"))]
use state_processing::{
per_block_processing, test_utils::BlockBuilder, BlockProcessingError, BlockSignatureStrategy,
};

View File

@ -31,3 +31,4 @@ tree_hash_derive = "0.2"
[dev-dependencies]
env_logger = "0.6.0"
serde_json = "^1.0"

View File

@ -4,25 +4,13 @@ use crate::{Checkpoint, Crosslink, Hash256};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash::TreeHash;
use tree_hash_derive::{SignedRoot, TreeHash};
use tree_hash_derive::TreeHash;
/// The data upon which an attestation is based.
///
/// Spec v0.8.0
#[derive(
Debug,
Clone,
PartialEq,
Eq,
Serialize,
Deserialize,
Hash,
Encode,
Decode,
TreeHash,
TestRandom,
SignedRoot,
Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Hash, Encode, Decode, TreeHash, TestRandom,
)]
pub struct AttestationData {
// LMD GHOST vote

View File

@ -60,6 +60,22 @@ pub enum Error {
SszTypesError(ssz_types::Error),
}
/// Control whether an epoch-indexed field can be indexed at the next epoch or not.
#[derive(Debug, PartialEq, Clone, Copy)]
enum AllowNextEpoch {
True,
False,
}
impl AllowNextEpoch {
fn upper_bound_of(self, current_epoch: Epoch) -> Epoch {
match self {
AllowNextEpoch::True => current_epoch + 1,
AllowNextEpoch::False => current_epoch,
}
}
}
/// The state of the `BeaconChain` at some slot.
///
/// Spec v0.8.0
@ -108,12 +124,12 @@ where
pub start_shard: u64,
pub randao_mixes: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
#[compare_fields(as_slice)]
active_index_roots: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
pub active_index_roots: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
#[compare_fields(as_slice)]
compact_committees_roots: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
pub compact_committees_roots: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
// Slashings
slashings: FixedVector<u64, T::EpochsPerSlashingsVector>,
pub slashings: FixedVector<u64, T::EpochsPerSlashingsVector>,
// Attestations
pub previous_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
@ -282,14 +298,6 @@ impl<T: EthSpec> BeaconState<T> {
Ok(cache.epoch_start_shard())
}
pub fn next_epoch_start_shard(&self, spec: &ChainSpec) -> Result<u64, Error> {
let cache = self.cache(RelativeEpoch::Current)?;
let active_validator_count = cache.active_validator_count();
let shard_delta = T::get_shard_delta(active_validator_count, spec.target_committee_size);
Ok((self.start_shard + shard_delta) % T::ShardCount::to_u64())
}
/// Get the slot of an attestation.
///
/// Note: Utilizes the cache and will fail if the appropriate cache is not initialized.
@ -463,12 +471,16 @@ impl<T: EthSpec> BeaconState<T> {
/// Safely obtains the index for `randao_mixes`
///
/// Spec v0.8.0
fn get_randao_mix_index(&self, epoch: Epoch) -> Result<usize, Error> {
/// Spec v0.8.1
fn get_randao_mix_index(
&self,
epoch: Epoch,
allow_next_epoch: AllowNextEpoch,
) -> Result<usize, Error> {
let current_epoch = self.current_epoch();
let len = T::EpochsPerHistoricalVector::to_u64();
if epoch + len > current_epoch && epoch <= current_epoch {
if current_epoch < epoch + len && epoch <= allow_next_epoch.upper_bound_of(current_epoch) {
Ok(epoch.as_usize() % len as usize)
} else {
Err(Error::EpochOutOfBounds)
@ -496,7 +508,7 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Spec v0.8.1
pub fn get_randao_mix(&self, epoch: Epoch) -> Result<&Hash256, Error> {
let i = self.get_randao_mix_index(epoch)?;
let i = self.get_randao_mix_index(epoch, AllowNextEpoch::False)?;
Ok(&self.randao_mixes[i])
}
@ -504,21 +516,29 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Spec v0.8.1
pub fn set_randao_mix(&mut self, epoch: Epoch, mix: Hash256) -> Result<(), Error> {
let i = self.get_randao_mix_index(epoch)?;
let i = self.get_randao_mix_index(epoch, AllowNextEpoch::True)?;
self.randao_mixes[i] = mix;
Ok(())
}
/// Safely obtains the index for `active_index_roots`, given some `epoch`.
///
/// If `allow_next_epoch` is `True`, then we allow an _extra_ one epoch of lookahead.
///
/// Spec v0.8.1
fn get_active_index_root_index(&self, epoch: Epoch, spec: &ChainSpec) -> Result<usize, Error> {
fn get_active_index_root_index(
&self,
epoch: Epoch,
spec: &ChainSpec,
allow_next_epoch: AllowNextEpoch,
) -> Result<usize, Error> {
let current_epoch = self.current_epoch();
let lookahead = spec.activation_exit_delay;
let lookback = self.active_index_roots.len() as u64 - lookahead;
let epoch_upper_bound = allow_next_epoch.upper_bound_of(current_epoch) + lookahead;
if epoch + lookback > current_epoch && current_epoch + lookahead >= epoch {
if current_epoch < epoch + lookback && epoch <= epoch_upper_bound {
Ok(epoch.as_usize() % self.active_index_roots.len())
} else {
Err(Error::EpochOutOfBounds)
@ -529,7 +549,7 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Spec v0.8.1
pub fn get_active_index_root(&self, epoch: Epoch, spec: &ChainSpec) -> Result<Hash256, Error> {
let i = self.get_active_index_root_index(epoch, spec)?;
let i = self.get_active_index_root_index(epoch, spec, AllowNextEpoch::False)?;
Ok(self.active_index_roots[i])
}
@ -542,7 +562,7 @@ impl<T: EthSpec> BeaconState<T> {
index_root: Hash256,
spec: &ChainSpec,
) -> Result<(), Error> {
let i = self.get_active_index_root_index(epoch, spec)?;
let i = self.get_active_index_root_index(epoch, spec, AllowNextEpoch::True)?;
self.active_index_roots[i] = index_root;
Ok(())
}
@ -556,19 +576,17 @@ impl<T: EthSpec> BeaconState<T> {
/// Safely obtains the index for `compact_committees_roots`, given some `epoch`.
///
/// Spec v0.8.0
/// Spec v0.8.1
fn get_compact_committee_root_index(
&self,
epoch: Epoch,
spec: &ChainSpec,
allow_next_epoch: AllowNextEpoch,
) -> Result<usize, Error> {
let current_epoch = self.current_epoch();
let len = T::EpochsPerHistoricalVector::to_u64();
let lookahead = spec.activation_exit_delay;
let lookback = self.compact_committees_roots.len() as u64 - lookahead;
if epoch + lookback > current_epoch && current_epoch + lookahead >= epoch {
Ok(epoch.as_usize() % self.compact_committees_roots.len())
if current_epoch < epoch + len && epoch <= allow_next_epoch.upper_bound_of(current_epoch) {
Ok(epoch.as_usize() % len as usize)
} else {
Err(Error::EpochOutOfBounds)
}
@ -576,26 +594,21 @@ impl<T: EthSpec> BeaconState<T> {
/// Return the `compact_committee_root` at a recent `epoch`.
///
/// Spec v0.8.0
pub fn get_compact_committee_root(
&self,
epoch: Epoch,
spec: &ChainSpec,
) -> Result<Hash256, Error> {
let i = self.get_compact_committee_root_index(epoch, spec)?;
/// Spec v0.8.1
pub fn get_compact_committee_root(&self, epoch: Epoch) -> Result<Hash256, Error> {
let i = self.get_compact_committee_root_index(epoch, AllowNextEpoch::False)?;
Ok(self.compact_committees_roots[i])
}
/// Set the `compact_committee_root` at a recent `epoch`.
///
/// Spec v0.8.0
/// Spec v0.8.1
pub fn set_compact_committee_root(
&mut self,
epoch: Epoch,
index_root: Hash256,
spec: &ChainSpec,
) -> Result<(), Error> {
let i = self.get_compact_committee_root_index(epoch, spec)?;
let i = self.get_compact_committee_root_index(epoch, AllowNextEpoch::True)?;
self.compact_committees_roots[i] = index_root;
Ok(())
}
@ -646,14 +659,19 @@ impl<T: EthSpec> BeaconState<T> {
/// Safely obtain the index for `slashings`, given some `epoch`.
///
/// Spec v0.8.0
fn get_slashings_index(&self, epoch: Epoch) -> Result<usize, Error> {
/// Spec v0.8.1
fn get_slashings_index(
&self,
epoch: Epoch,
allow_next_epoch: AllowNextEpoch,
) -> Result<usize, Error> {
// We allow the slashings vector to be accessed at any cached epoch at or before
// the current epoch.
if epoch <= self.current_epoch()
&& epoch + T::EpochsPerSlashingsVector::to_u64() >= self.current_epoch() + 1
// the current epoch, or the next epoch if `AllowNextEpoch::True` is passed.
let current_epoch = self.current_epoch();
if current_epoch < epoch + T::EpochsPerSlashingsVector::to_u64()
&& epoch <= allow_next_epoch.upper_bound_of(current_epoch)
{
Ok((epoch.as_u64() % T::EpochsPerSlashingsVector::to_u64()) as usize)
Ok(epoch.as_usize() % T::EpochsPerSlashingsVector::to_usize())
} else {
Err(Error::EpochOutOfBounds)
}
@ -668,17 +686,17 @@ impl<T: EthSpec> BeaconState<T> {
/// Get the total slashed balances for some epoch.
///
/// Spec v0.8.0
/// Spec v0.8.1
pub fn get_slashings(&self, epoch: Epoch) -> Result<u64, Error> {
let i = self.get_slashings_index(epoch)?;
let i = self.get_slashings_index(epoch, AllowNextEpoch::False)?;
Ok(self.slashings[i])
}
/// Set the total slashed balances for some epoch.
///
/// Spec v0.8.0
/// Spec v0.8.1
pub fn set_slashings(&mut self, epoch: Epoch, value: u64) -> Result<(), Error> {
let i = self.get_slashings_index(epoch)?;
let i = self.get_slashings_index(epoch, AllowNextEpoch::True)?;
self.slashings[i] = value;
Ok(())
}

View File

@ -120,6 +120,13 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
fn epochs_per_historical_vector() -> usize {
Self::EpochsPerHistoricalVector::to_usize()
}
/// Returns the `SLOTS_PER_ETH1_VOTING_PERIOD` constant for this specification.
///
/// Spec v0.8.1
fn slots_per_eth1_voting_period() -> usize {
Self::EpochsPerHistoricalVector::to_usize()
}
}
/// Macro to inherit some type values from another EthSpec.

View File

@ -9,7 +9,7 @@ fn default_values() {
let cache = CommitteeCache::default();
assert_eq!(cache.is_initialized_at(Epoch::new(0)), false);
assert_eq!(cache.active_validator_indices(), &[]);
assert!(&cache.active_validator_indices().is_empty());
assert_eq!(cache.get_crosslink_committee_for_shard(0), None);
assert_eq!(cache.get_attestation_duties(0), None);
assert_eq!(cache.active_validator_count(), 0);

View File

@ -90,11 +90,11 @@ fn test_active_index<T: EthSpec>(state_slot: Slot) {
// Test the start and end of the range.
assert_eq!(
state.get_active_index_root_index(*range.start(), &spec),
state.get_active_index_root_index(*range.start(), &spec, AllowNextEpoch::False),
Ok(modulo(*range.start()))
);
assert_eq!(
state.get_active_index_root_index(*range.end(), &spec),
state.get_active_index_root_index(*range.end(), &spec, AllowNextEpoch::False),
Ok(modulo(*range.end()))
);
@ -102,12 +102,12 @@ fn test_active_index<T: EthSpec>(state_slot: Slot) {
if state.current_epoch() > 0 {
// Test is invalid on epoch zero, cannot subtract from zero.
assert_eq!(
state.get_active_index_root_index(*range.start() - 1, &spec),
state.get_active_index_root_index(*range.start() - 1, &spec, AllowNextEpoch::False),
Err(Error::EpochOutOfBounds)
);
}
assert_eq!(
state.get_active_index_root_index(*range.end() + 1, &spec),
state.get_active_index_root_index(*range.end() + 1, &spec, AllowNextEpoch::False),
Err(Error::EpochOutOfBounds)
);
}

View File

@ -58,7 +58,7 @@ pub struct ChainSpec {
/*
* Time parameters
*/
pub seconds_per_slot: u64,
pub milliseconds_per_slot: u64,
pub min_attestation_inclusion_delay: u64,
pub min_seed_lookahead: Epoch,
pub activation_exit_delay: u64,
@ -158,7 +158,7 @@ impl ChainSpec {
/*
* Time parameters
*/
seconds_per_slot: 6,
milliseconds_per_slot: 6_000,
min_attestation_inclusion_delay: 1,
min_seed_lookahead: Epoch::new(1),
activation_exit_delay: 4,
@ -221,7 +221,7 @@ impl ChainSpec {
let boot_nodes = vec![];
Self {
seconds_per_slot: 12,
milliseconds_per_slot: 12_000,
target_committee_size: 4,
shuffle_round_count: 10,
network_id: 13,

View File

@ -3,8 +3,7 @@ use crate::{Epoch, Hash256};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash::TreeHash;
use tree_hash_derive::{SignedRoot, TreeHash};
use tree_hash_derive::TreeHash;
/// Casper FFG checkpoint, used in attestations.
///
@ -22,7 +21,6 @@ use tree_hash_derive::{SignedRoot, TreeHash};
Decode,
TreeHash,
TestRandom,
SignedRoot,
)]
pub struct Checkpoint {
pub epoch: Epoch,

Some files were not shown because too many files have changed in this diff Show More