Merge pull request #979 from sigp/v0.2.0

Lighthouse v0.2.0
This commit is contained in:
Age Manning 2020-04-23 00:53:41 +10:00 committed by GitHub
commit 91648cc230
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
213 changed files with 16771 additions and 7330 deletions

View File

@ -49,15 +49,24 @@ jobs:
- uses: actions/checkout@v1 - uses: actions/checkout@v1
- name: Build the root Dockerfile - name: Build the root Dockerfile
run: docker build . run: docker build .
simulator-ubuntu: eth1-simulator-ubuntu:
runs-on: ubuntu-latest runs-on: ubuntu-latest
needs: cargo-fmt needs: cargo-fmt
steps: steps:
- uses: actions/checkout@v1 - uses: actions/checkout@v1
- name: Install ganache-cli - name: Install ganache-cli
run: sudo npm install -g ganache-cli run: sudo npm install -g ganache-cli
- name: Run the beacon chain sim - name: Run the beacon chain sim that starts from an eth1 contract
run: cargo run --release --bin simulator beacon-chain-sim run: cargo run --release --bin simulator eth1-sim
no-eth1-simulator-ubuntu:
runs-on: ubuntu-latest
needs: cargo-fmt
steps:
- uses: actions/checkout@v1
- name: Install ganache-cli
run: sudo npm install -g ganache-cli
- name: Run the beacon chain sim without an eth1 connection
run: cargo run --release --bin simulator no-eth1-sim
check-benchmarks: check-benchmarks:
runs-on: ubuntu-latest runs-on: ubuntu-latest
needs: cargo-fmt needs: cargo-fmt

2484
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@ -5,6 +5,7 @@ members = [
"eth2/state_processing", "eth2/state_processing",
"eth2/types", "eth2/types",
"eth2/utils/bls", "eth2/utils/bls",
"eth2/utils/clap_utils",
"eth2/utils/compare_fields", "eth2/utils/compare_fields",
"eth2/utils/compare_fields_derive", "eth2/utils/compare_fields_derive",
"eth2/utils/deposit_contract", "eth2/utils/deposit_contract",
@ -13,12 +14,14 @@ members = [
"eth2/utils/eth2_testnet_config", "eth2/utils/eth2_testnet_config",
"eth2/utils/logging", "eth2/utils/logging",
"eth2/utils/eth2_hashing", "eth2/utils/eth2_hashing",
"eth2/utils/hashmap_delay",
"eth2/utils/lighthouse_metrics", "eth2/utils/lighthouse_metrics",
"eth2/utils/merkle_proof", "eth2/utils/merkle_proof",
"eth2/utils/int_to_bytes", "eth2/utils/int_to_bytes",
"eth2/utils/safe_arith", "eth2/utils/safe_arith",
"eth2/utils/serde_hex", "eth2/utils/serde_hex",
"eth2/utils/slot_clock", "eth2/utils/slot_clock",
"eth2/utils/rest_types",
"eth2/utils/ssz", "eth2/utils/ssz",
"eth2/utils/ssz_derive", "eth2/utils/ssz_derive",
"eth2/utils/ssz_types", "eth2/utils/ssz_types",
@ -28,14 +31,15 @@ members = [
"eth2/utils/tree_hash_derive", "eth2/utils/tree_hash_derive",
"eth2/utils/test_random_derive", "eth2/utils/test_random_derive",
"beacon_node", "beacon_node",
"beacon_node/store",
"beacon_node/client",
"beacon_node/rest_api",
"beacon_node/network",
"beacon_node/eth2-libp2p",
"beacon_node/version",
"beacon_node/eth1",
"beacon_node/beacon_chain", "beacon_node/beacon_chain",
"beacon_node/client",
"beacon_node/eth1",
"beacon_node/eth2-libp2p",
"beacon_node/network",
"beacon_node/rest_api",
"beacon_node/store",
"beacon_node/timer",
"beacon_node/version",
"beacon_node/websocket_server", "beacon_node/websocket_server",
"tests/simulator", "tests/simulator",
"tests/ef_tests", "tests/ef_tests",

View File

@ -2,12 +2,16 @@
EF_TESTS = "tests/ef_tests" EF_TESTS = "tests/ef_tests"
# Builds the entire workspace in release (optimized). # Builds the Lighthouse binary in release (optimized).
# #
# Binaries will most likely be found in `./target/release` # Binaries will most likely be found in `./target/release`
install: install:
cargo install --path lighthouse --force --locked cargo install --path lighthouse --force --locked
# Builds the lcli binary in release (optimized).
install-lcli:
cargo install --path lcli --force --locked
# Runs the full workspace tests in **release**, without downloading any additional # Runs the full workspace tests in **release**, without downloading any additional
# test vectors. # test vectors.
test-release: test-release:

View File

@ -24,5 +24,7 @@ hex = "0.3"
validator_client = { path = "../validator_client" } validator_client = { path = "../validator_client" }
rayon = "1.2.0" rayon = "1.2.0"
eth2_testnet_config = { path = "../eth2/utils/eth2_testnet_config" } eth2_testnet_config = { path = "../eth2/utils/eth2_testnet_config" }
web3 = "0.8.0" web3 = "0.10.0"
futures = "0.1.25" futures = "0.1.25"
clap_utils = { path = "../eth2/utils/clap_utils" }
tokio = "0.1.22"

View File

@ -1,3 +1,4 @@
use crate::deposits;
use clap::{App, Arg, SubCommand}; use clap::{App, Arg, SubCommand};
pub fn cli_app<'a, 'b>() -> App<'a, 'b> { pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
@ -7,6 +8,7 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.subcommand( .subcommand(
SubCommand::with_name("validator") SubCommand::with_name("validator")
.about("Generate or manage Etheruem 2.0 validators.") .about("Generate or manage Etheruem 2.0 validators.")
.subcommand(deposits::cli_app())
.subcommand( .subcommand(
SubCommand::with_name("new") SubCommand::with_name("new")
.about("Create a new Ethereum 2.0 validator.") .about("Create a new Ethereum 2.0 validator.")
@ -52,14 +54,6 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.takes_value(true) .takes_value(true)
.help("The password file to unlock the eth1 account (see --index)"), .help("The password file to unlock the eth1 account (see --index)"),
) )
.arg(
Arg::with_name("testnet-dir")
.long("testnet-dir")
.value_name("DIRECTORY")
.takes_value(true)
.help("The directory from which to read the deposit contract /
address. Defaults to the current Lighthouse testnet."),
)
.subcommand( .subcommand(
SubCommand::with_name("insecure") SubCommand::with_name("insecure")
.about("Produce insecure, ephemeral validators. DO NOT USE TO STORE VALUE.") .about("Produce insecure, ephemeral validators. DO NOT USE TO STORE VALUE.")

View File

@ -0,0 +1,255 @@
use clap::{App, Arg, ArgMatches};
use clap_utils;
use environment::Environment;
use futures::{
future::{self, loop_fn, Loop},
Future,
};
use slog::{info, Logger};
use std::fs;
use std::path::PathBuf;
use std::time::{Duration, Instant};
use tokio::timer::Delay;
use types::EthSpec;
use validator_client::validator_directory::ValidatorDirectoryBuilder;
use web3::{
transports::Ipc,
types::{Address, SyncInfo, SyncState},
Transport, Web3,
};
const SYNCING_STATE_RETRY_DELAY: Duration = Duration::from_secs(2);
pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
App::new("deposited")
.about("Creates new Lighthouse validator keys and directories. Each newly-created validator
will have a deposit transaction formed and submitted to the deposit contract via
--eth1-ipc. This application will only write each validator keys to disk if the deposit
transaction returns successfully from the eth1 node. The process exits immediately if any
Eth1 tx fails. Does not wait for Eth1 confirmation blocks, so there is no guarantee that a
deposit will be accepted in the Eth1 chain. Before key generation starts, this application
will wait until the eth1 indicates that it is not syncing via the eth_syncing endpoint")
.arg(
Arg::with_name("validator-dir")
.long("validator-dir")
.value_name("VALIDATOR_DIRECTORY")
.help("The path where the validator directories will be created. Defaults to ~/.lighthouse/validators")
.takes_value(true),
)
.arg(
Arg::with_name("eth1-ipc")
.long("eth1-ipc")
.value_name("ETH1_IPC_PATH")
.help("Path to an Eth1 JSON-RPC IPC endpoint")
.takes_value(true)
.required(true)
)
.arg(
Arg::with_name("from-address")
.long("from-address")
.value_name("FROM_ETH1_ADDRESS")
.help("The address that will submit the eth1 deposit. Must be unlocked on the node
at --eth1-ipc.")
.takes_value(true)
.required(true)
)
.arg(
Arg::with_name("deposit-gwei")
.long("deposit-gwei")
.value_name("DEPOSIT_GWEI")
.help("The GWEI value of the deposit amount. Defaults to the minimum amount
required for an active validator (MAX_EFFECTIVE_BALANCE.")
.takes_value(true),
)
.arg(
Arg::with_name("count")
.long("count")
.value_name("DEPOSIT_COUNT")
.help("The number of deposits to create, regardless of how many already exist")
.conflicts_with("limit")
.takes_value(true),
)
.arg(
Arg::with_name("at-most")
.long("at-most")
.value_name("VALIDATOR_COUNT")
.help("Observe the number of validators in --validator-dir, only creating enough to
ensure reach the given count. Never deletes an existing validator.")
.conflicts_with("count")
.takes_value(true),
)
}
pub fn cli_run<T: EthSpec>(matches: &ArgMatches, mut env: Environment<T>) -> Result<(), String> {
let spec = env.core_context().eth2_config.spec;
let log = env.core_context().log;
let validator_dir = clap_utils::parse_path_with_default_in_home_dir(
matches,
"validator_dir",
PathBuf::new().join(".lighthouse").join("validators"),
)?;
let eth1_ipc_path: PathBuf = clap_utils::parse_required(matches, "eth1-ipc")?;
let from_address: Address = clap_utils::parse_required(matches, "from-address")?;
let deposit_gwei = clap_utils::parse_optional(matches, "deposit-gwei")?
.unwrap_or_else(|| spec.max_effective_balance);
let count: Option<usize> = clap_utils::parse_optional(matches, "count")?;
let at_most: Option<usize> = clap_utils::parse_optional(matches, "at-most")?;
let starting_validator_count = existing_validator_count(&validator_dir)?;
let n = match (count, at_most) {
(Some(_), Some(_)) => Err("Cannot supply --count and --at-most".to_string()),
(None, None) => Err("Must supply either --count or --at-most".to_string()),
(Some(count), None) => Ok(count),
(None, Some(at_most)) => Ok(at_most.saturating_sub(starting_validator_count)),
}?;
if n == 0 {
info!(
log,
"No need to produce and validators, exiting";
"--count" => count,
"--at-most" => at_most,
"existing_validators" => starting_validator_count,
);
return Ok(());
}
let deposit_contract = env
.testnet
.as_ref()
.ok_or_else(|| "Unable to run account manager without a testnet dir".to_string())?
.deposit_contract_address()
.map_err(|e| format!("Unable to parse deposit contract address: {}", e))?;
if deposit_contract == Address::zero() {
return Err("Refusing to deposit to the zero address. Check testnet configuration.".into());
}
let (_event_loop_handle, transport) =
Ipc::new(eth1_ipc_path).map_err(|e| format!("Unable to connect to eth1 IPC: {:?}", e))?;
let web3 = Web3::new(transport);
env.runtime()
.block_on(poll_until_synced(web3.clone(), log.clone()))?;
for i in 0..n {
let tx_hash_log = log.clone();
env.runtime()
.block_on(
ValidatorDirectoryBuilder::default()
.spec(spec.clone())
.custom_deposit_amount(deposit_gwei)
.thread_random_keypairs()
.submit_eth1_deposit(web3.clone(), from_address, deposit_contract)
.map(move |(builder, tx_hash)| {
info!(
tx_hash_log,
"Validator deposited";
"eth1_tx_hash" => format!("{:?}", tx_hash),
"index" => format!("{}/{}", i + 1, n),
);
builder
}),
)?
.create_directory(validator_dir.clone())?
.write_keypair_files()?
.write_eth1_data_file()?
.build()?;
}
let ending_validator_count = existing_validator_count(&validator_dir)?;
let delta = ending_validator_count.saturating_sub(starting_validator_count);
info!(
log,
"Success";
"validators_created_and_deposited" => delta,
);
Ok(())
}
/// Returns the number of validators that exist in the given `validator_dir`.
///
/// This function just assumes any file is a validator directory, making it likely to return a
/// higher number than accurate but never a lower one.
fn existing_validator_count(validator_dir: &PathBuf) -> Result<usize, String> {
fs::read_dir(&validator_dir)
.map(|iter| iter.count())
.map_err(|e| format!("Unable to read {:?}: {}", validator_dir, e))
}
/// Run a poll on the `eth_syncing` endpoint, blocking until the node is synced.
fn poll_until_synced<T>(web3: Web3<T>, log: Logger) -> impl Future<Item = (), Error = String> + Send
where
T: Transport + Send + 'static,
<T as Transport>::Out: Send,
{
loop_fn((web3.clone(), log.clone()), move |(web3, log)| {
web3.clone()
.eth()
.syncing()
.map_err(|e| format!("Unable to read syncing state from eth1 node: {:?}", e))
.and_then::<_, Box<dyn Future<Item = _, Error = _> + Send>>(move |sync_state| {
match sync_state {
SyncState::Syncing(SyncInfo {
current_block,
highest_block,
..
}) => {
info!(
log,
"Waiting for eth1 node to sync";
"est_highest_block" => format!("{}", highest_block),
"current_block" => format!("{}", current_block),
);
Box::new(
Delay::new(Instant::now() + SYNCING_STATE_RETRY_DELAY)
.map_err(|e| format!("Failed to trigger delay: {:?}", e))
.and_then(|_| future::ok(Loop::Continue((web3, log)))),
)
}
SyncState::NotSyncing => Box::new(
web3.clone()
.eth()
.block_number()
.map_err(|e| {
format!("Unable to read block number from eth1 node: {:?}", e)
})
.and_then::<_, Box<dyn Future<Item = _, Error = _> + Send>>(
|block_number| {
if block_number > 0.into() {
info!(
log,
"Eth1 node is synced";
"head_block" => format!("{}", block_number),
);
Box::new(future::ok(Loop::Break((web3, log))))
} else {
Box::new(
Delay::new(Instant::now() + SYNCING_STATE_RETRY_DELAY)
.map_err(|e| {
format!("Failed to trigger delay: {:?}", e)
})
.and_then(|_| {
info!(
log,
"Waiting for eth1 node to sync";
"current_block" => 0,
);
future::ok(Loop::Continue((web3, log)))
}),
)
}
},
),
),
}
})
})
.map(|_| ())
}

View File

@ -1,4 +1,5 @@
mod cli; mod cli;
mod deposits;
use clap::ArgMatches; use clap::ArgMatches;
use deposit_contract::DEPOSIT_GAS; use deposit_contract::DEPOSIT_GAS;
@ -6,7 +7,7 @@ use environment::{Environment, RuntimeContext};
use eth2_testnet_config::Eth2TestnetConfig; use eth2_testnet_config::Eth2TestnetConfig;
use futures::{future, Future, IntoFuture, Stream}; use futures::{future, Future, IntoFuture, Stream};
use rayon::prelude::*; use rayon::prelude::*;
use slog::{crit, error, info, Logger}; use slog::{error, info, Logger};
use std::fs; use std::fs;
use std::fs::File; use std::fs::File;
use std::io::Read; use std::io::Read;
@ -21,20 +22,8 @@ use web3::{
pub use cli::cli_app; pub use cli::cli_app;
/// Run the account manager, logging an error if the operation did not succeed.
pub fn run<T: EthSpec>(matches: &ArgMatches, mut env: Environment<T>) {
let log = env.core_context().log.clone();
match run_account_manager(matches, env) {
Ok(()) => (),
Err(e) => crit!(log, "Account manager failed"; "error" => e),
}
}
/// Run the account manager, returning an error if the operation did not succeed. /// Run the account manager, returning an error if the operation did not succeed.
fn run_account_manager<T: EthSpec>( pub fn run<T: EthSpec>(matches: &ArgMatches, mut env: Environment<T>) -> Result<(), String> {
matches: &ArgMatches,
mut env: Environment<T>,
) -> Result<(), String> {
let context = env.core_context(); let context = env.core_context();
let log = context.log.clone(); let log = context.log.clone();
@ -60,6 +49,7 @@ fn run_account_manager<T: EthSpec>(
match matches.subcommand() { match matches.subcommand() {
("validator", Some(matches)) => match matches.subcommand() { ("validator", Some(matches)) => match matches.subcommand() {
("deposited", Some(matches)) => deposits::cli_run(matches, env)?,
("new", Some(matches)) => run_new_validator_subcommand(matches, datadir, env)?, ("new", Some(matches)) => run_new_validator_subcommand(matches, datadir, env)?,
_ => { _ => {
return Err("Invalid 'validator new' command. See --help.".to_string()); return Err("Invalid 'validator new' command. See --help.".to_string());

View File

@ -1,6 +1,6 @@
[package] [package]
name = "beacon_node" name = "beacon_node"
version = "0.1.0" version = "0.2.0"
authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com"] authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com"]
edition = "2018" edition = "2018"

View File

@ -1,6 +1,6 @@
[package] [package]
name = "beacon_chain" name = "beacon_chain"
version = "0.1.0" version = "0.2.0"
authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com>"] authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com>"]
edition = "2018" edition = "2018"
@ -32,10 +32,10 @@ eth2_ssz_derive = "0.1.0"
state_processing = { path = "../../eth2/state_processing" } state_processing = { path = "../../eth2/state_processing" }
tree_hash = "0.1.0" tree_hash = "0.1.0"
types = { path = "../../eth2/types" } types = { path = "../../eth2/types" }
tokio = "0.1.22"
eth1 = { path = "../eth1" } eth1 = { path = "../eth1" }
websocket_server = { path = "../websocket_server" } websocket_server = { path = "../websocket_server" }
futures = "0.1.25" futures = "0.1.25"
exit-future = "0.1.3"
genesis = { path = "../genesis" } genesis = { path = "../genesis" }
integer-sqrt = "0.1" integer-sqrt = "0.1"
rand = "0.7.2" rand = "0.7.2"

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,808 @@
//! Provides `SignedBeaconBlock` verification logic.
//!
//! Specifically, it provides the following:
//!
//! - Verification for gossip blocks (i.e., should we gossip some block from the network).
//! - Verification for normal blocks (e.g., some block received on the RPC during a parent lookup).
//! - Verification for chain segments (e.g., some chain of blocks received on the RPC during a
//! sync).
//!
//! The primary source of complexity here is that we wish to avoid doing duplicate work as a block
//! moves through the verification process. For example, if some block is verified for gossip, we
//! do not wish to re-verify the block proposal signature or re-hash the block. Or, if we've
//! verified the signatures of a block during a chain segment import, we do not wish to verify each
//! signature individually again.
//!
//! The incremental processing steps (e.g., signatures verified but not the state transition) is
//! represented as a sequence of wrapper-types around the block. There is a linear progression of
//! types, starting at a `SignedBeaconBlock` and finishing with a `Fully VerifiedBlock` (see
//! diagram below).
//!
//! ```ignore
//! START
//! |
//! ▼
//! SignedBeaconBlock
//! |---------------
//! | |
//! | ▼
//! | GossipVerifiedBlock
//! | |
//! |---------------
//! |
//! ▼
//! SignatureVerifiedBlock
//! |
//! ▼
//! FullyVerifiedBlock
//! |
//! ▼
//! END
//!
//! ```
use crate::validator_pubkey_cache::ValidatorPubkeyCache;
use crate::{
beacon_chain::{
BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
},
metrics, BeaconChain, BeaconChainError, BeaconChainTypes, BeaconSnapshot,
};
use parking_lot::RwLockReadGuard;
use slot_clock::SlotClock;
use state_processing::{
block_signature_verifier::{
BlockSignatureVerifier, Error as BlockSignatureVerifierError, G1Point,
},
per_block_processing, per_slot_processing, BlockProcessingError, BlockSignatureStrategy,
SlotProcessingError,
};
use std::borrow::Cow;
use store::{Error as DBError, StateBatch};
use types::{
BeaconBlock, BeaconState, BeaconStateError, ChainSpec, CloneConfig, EthSpec, Hash256,
RelativeEpoch, SignedBeaconBlock, Slot,
};
mod block_processing_outcome;
pub use block_processing_outcome::BlockProcessingOutcome;
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
const MAXIMUM_BLOCK_SLOT_NUMBER: u64 = 4_294_967_296; // 2^32
/// Returned when a block was not verified. A block is not verified for two reasons:
///
/// - The block is malformed/invalid (indicated by all results other than `BeaconChainError`.
/// - We encountered an error whilst trying to verify the block (a `BeaconChainError`).
#[derive(Debug, PartialEq)]
pub enum BlockError {
/// The parent block was unknown.
ParentUnknown(Hash256),
/// The block slot is greater than the present slot.
FutureSlot {
present_slot: Slot,
block_slot: Slot,
},
/// The block state_root does not match the generated state.
StateRootMismatch { block: Hash256, local: Hash256 },
/// The block was a genesis block, these blocks cannot be re-imported.
GenesisBlock,
/// The slot is finalized, no need to import.
WouldRevertFinalizedSlot {
block_slot: Slot,
finalized_slot: Slot,
},
/// Block is already known, no need to re-import.
BlockIsAlreadyKnown,
/// The block slot exceeds the MAXIMUM_BLOCK_SLOT_NUMBER.
BlockSlotLimitReached,
/// The proposal signature in invalid.
ProposalSignatureInvalid,
/// A signature in the block is invalid (exactly which is unknown).
InvalidSignature,
/// The provided block is from an earlier slot than its parent.
BlockIsNotLaterThanParent { block_slot: Slot, state_slot: Slot },
/// At least one block in the chain segment did not have it's parent root set to the root of
/// the prior block.
NonLinearParentRoots,
/// The slots of the blocks in the chain segment were not strictly increasing. I.e., a child
/// had lower slot than a parent.
NonLinearSlots,
/// The block failed the specification's `per_block_processing` function, it is invalid.
PerBlockProcessingError(BlockProcessingError),
/// There was an error whilst processing the block. It is not necessarily invalid.
BeaconChainError(BeaconChainError),
}
impl From<BlockSignatureVerifierError> for BlockError {
fn from(e: BlockSignatureVerifierError) -> Self {
BlockError::BeaconChainError(BeaconChainError::BlockSignatureVerifierError(e))
}
}
impl From<BeaconChainError> for BlockError {
fn from(e: BeaconChainError) -> Self {
BlockError::BeaconChainError(e)
}
}
impl From<BeaconStateError> for BlockError {
fn from(e: BeaconStateError) -> Self {
BlockError::BeaconChainError(BeaconChainError::BeaconStateError(e))
}
}
impl From<SlotProcessingError> for BlockError {
fn from(e: SlotProcessingError) -> Self {
BlockError::BeaconChainError(BeaconChainError::SlotProcessingError(e))
}
}
impl From<DBError> for BlockError {
fn from(e: DBError) -> Self {
BlockError::BeaconChainError(BeaconChainError::DBError(e))
}
}
/// Verify all signatures (except deposit signatures) on all blocks in the `chain_segment`. If all
/// signatures are valid, the `chain_segment` is mapped to a `Vec<SignatureVerifiedBlock>` that can
/// later be transformed into a `FullyVerifiedBlock` without re-checking the signatures. If any
/// signature in the block is invalid, an `Err` is returned (it is not possible to known _which_
/// signature was invalid).
///
/// ## Errors
///
/// The given `chain_segment` must span no more than two epochs, otherwise an error will be
/// returned.
pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
chain_segment: Vec<(Hash256, SignedBeaconBlock<T::EthSpec>)>,
chain: &BeaconChain<T>,
) -> Result<Vec<SignatureVerifiedBlock<T>>, BlockError> {
let (mut parent, slot) = if let Some(block) = chain_segment.first().map(|(_, block)| block) {
let parent = load_parent(&block.message, chain)?;
(parent, block.slot())
} else {
return Ok(vec![]);
};
let highest_slot = chain_segment
.last()
.map(|(_, block)| block.slot())
.unwrap_or_else(|| slot);
let state = cheap_state_advance_to_obtain_committees(
&mut parent.beacon_state,
highest_slot,
&chain.spec,
)?;
let pubkey_cache = get_validator_pubkey_cache(chain)?;
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
for (block_root, block) in &chain_segment {
signature_verifier.include_all_signatures(block, Some(*block_root))?;
}
if signature_verifier.verify().is_err() {
return Err(BlockError::InvalidSignature);
}
drop(pubkey_cache);
let mut signature_verified_blocks = chain_segment
.into_iter()
.map(|(block_root, block)| SignatureVerifiedBlock {
block,
block_root,
parent: None,
})
.collect::<Vec<_>>();
if let Some(signature_verified_block) = signature_verified_blocks.first_mut() {
signature_verified_block.parent = Some(parent);
}
Ok(signature_verified_blocks)
}
/// A wrapper around a `SignedBeaconBlock` that indicates it has been approved for re-gossiping on
/// the p2p network.
pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
pub block: SignedBeaconBlock<T::EthSpec>,
block_root: Hash256,
parent: BeaconSnapshot<T::EthSpec>,
}
/// A wrapper around a `SignedBeaconBlock` that indicates that all signatures (except the deposit
/// signatures) have been verified.
pub struct SignatureVerifiedBlock<T: BeaconChainTypes> {
block: SignedBeaconBlock<T::EthSpec>,
block_root: Hash256,
parent: Option<BeaconSnapshot<T::EthSpec>>,
}
/// A wrapper around a `SignedBeaconBlock` that indicates that this block is fully verified and
/// ready to import into the `BeaconChain`. The validation includes:
///
/// - Parent is known
/// - Signatures
/// - State root check
/// - Per block processing
///
/// Note: a `FullyVerifiedBlock` is not _forever_ valid to be imported, it may later become invalid
/// due to finality or some other event. A `FullyVerifiedBlock` should be imported into the
/// `BeaconChain` immediately after it is instantiated.
pub struct FullyVerifiedBlock<T: BeaconChainTypes> {
pub block: SignedBeaconBlock<T::EthSpec>,
pub block_root: Hash256,
pub state: BeaconState<T::EthSpec>,
pub parent_block: SignedBeaconBlock<T::EthSpec>,
pub intermediate_states: StateBatch<T::EthSpec>,
}
/// Implemented on types that can be converted into a `FullyVerifiedBlock`.
///
/// Used to allow functions to accept blocks at various stages of verification.
pub trait IntoFullyVerifiedBlock<T: BeaconChainTypes> {
fn into_fully_verified_block(
self,
chain: &BeaconChain<T>,
) -> Result<FullyVerifiedBlock<T>, BlockError>;
fn block(&self) -> &SignedBeaconBlock<T::EthSpec>;
}
impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
/// Instantiates `Self`, a wrapper that indicates the given `block` is safe to be re-gossiped
/// on the p2p network.
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn new(
block: SignedBeaconBlock<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError> {
// Do not gossip or process blocks from future slots.
let present_slot_with_tolerance = chain
.slot_clock
.now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY)
.ok_or_else(|| BeaconChainError::UnableToReadSlot)?;
if block.slot() > present_slot_with_tolerance {
return Err(BlockError::FutureSlot {
present_slot: present_slot_with_tolerance,
block_slot: block.slot(),
});
}
// Do not gossip a block from a finalized slot.
check_block_against_finalized_slot(&block.message, chain)?;
// TODO: add check for the `(block.proposer_index, block.slot)` tuple once we have v0.11.0
let mut parent = load_parent(&block.message, chain)?;
let block_root = get_block_root(&block);
let state = cheap_state_advance_to_obtain_committees(
&mut parent.beacon_state,
block.slot(),
&chain.spec,
)?;
let pubkey_cache = get_validator_pubkey_cache(chain)?;
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
signature_verifier.include_block_proposal(&block, Some(block_root))?;
if signature_verifier.verify().is_ok() {
Ok(Self {
block,
block_root,
parent,
})
} else {
Err(BlockError::ProposalSignatureInvalid)
}
}
pub fn block_root(&self) -> Hash256 {
self.block_root
}
}
impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for GossipVerifiedBlock<T> {
/// Completes verification of the wrapped `block`.
fn into_fully_verified_block(
self,
chain: &BeaconChain<T>,
) -> Result<FullyVerifiedBlock<T>, BlockError> {
let fully_verified = SignatureVerifiedBlock::from_gossip_verified_block(self, chain)?;
fully_verified.into_fully_verified_block(chain)
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
&self.block
}
}
impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
/// Instantiates `Self`, a wrapper that indicates that all signatures (except the deposit
/// signatures) are valid (i.e., signed by the correct public keys).
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn new(
block: SignedBeaconBlock<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError> {
let mut parent = load_parent(&block.message, chain)?;
let block_root = get_block_root(&block);
let state = cheap_state_advance_to_obtain_committees(
&mut parent.beacon_state,
block.slot(),
&chain.spec,
)?;
let pubkey_cache = get_validator_pubkey_cache(chain)?;
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
signature_verifier.include_all_signatures(&block, Some(block_root))?;
if signature_verifier.verify().is_ok() {
Ok(Self {
block,
block_root,
parent: Some(parent),
})
} else {
Err(BlockError::InvalidSignature)
}
}
/// Finishes signature verification on the provided `GossipVerifedBlock`. Does not re-verify
/// the proposer signature.
pub fn from_gossip_verified_block(
from: GossipVerifiedBlock<T>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError> {
let mut parent = from.parent;
let block = from.block;
let state = cheap_state_advance_to_obtain_committees(
&mut parent.beacon_state,
block.slot(),
&chain.spec,
)?;
let pubkey_cache = get_validator_pubkey_cache(chain)?;
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
signature_verifier.include_all_signatures_except_proposal(&block)?;
if signature_verifier.verify().is_ok() {
Ok(Self {
block,
block_root: from.block_root,
parent: Some(parent),
})
} else {
Err(BlockError::InvalidSignature)
}
}
}
impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignatureVerifiedBlock<T> {
/// Completes verification of the wrapped `block`.
fn into_fully_verified_block(
self,
chain: &BeaconChain<T>,
) -> Result<FullyVerifiedBlock<T>, BlockError> {
let block = self.block;
let parent = self
.parent
.map(Result::Ok)
.unwrap_or_else(|| load_parent(&block.message, chain))?;
FullyVerifiedBlock::from_signature_verified_components(
block,
self.block_root,
parent,
chain,
)
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
&self.block
}
}
impl<T: BeaconChainTypes> IntoFullyVerifiedBlock<T> for SignedBeaconBlock<T::EthSpec> {
/// Verifies the `SignedBeaconBlock` by first transforming it into a `SignatureVerifiedBlock`
/// and then using that implementation of `IntoFullyVerifiedBlock` to complete verification.
fn into_fully_verified_block(
self,
chain: &BeaconChain<T>,
) -> Result<FullyVerifiedBlock<T>, BlockError> {
SignatureVerifiedBlock::new(self, chain)?.into_fully_verified_block(chain)
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
&self
}
}
impl<T: BeaconChainTypes> FullyVerifiedBlock<T> {
/// Instantiates `Self`, a wrapper that indicates that the given `block` is fully valid. See
/// the struct-level documentation for more information.
///
/// Note: this function does not verify block signatures, it assumes they are valid. Signature
/// verification must be done upstream (e.g., via a `SignatureVerifiedBlock`
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn from_signature_verified_components(
block: SignedBeaconBlock<T::EthSpec>,
block_root: Hash256,
parent: BeaconSnapshot<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError> {
// Reject any block if its parent is not known to fork choice.
//
// A block that is not in fork choice is either:
//
// - Not yet imported: we should reject this block because we should only import a child
// after its parent has been fully imported.
// - Pre-finalized: if the parent block is _prior_ to finalization, we should ignore it
// because it will revert finalization. Note that the finalized block is stored in fork
// choice, so we will not reject any child of the finalized block (this is relevant during
// genesis).
if !chain.fork_choice.contains_block(&block.parent_root()) {
return Err(BlockError::ParentUnknown(block.parent_root()));
}
/*
* Perform cursory checks to see if the block is even worth processing.
*/
check_block_relevancy(&block, Some(block_root), chain)?;
/*
* Advance the given `parent.beacon_state` to the slot of the given `block`.
*/
let catchup_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_CATCHUP_STATE);
// Keep a batch of any states that were "skipped" (block-less) in between the parent state
// slot and the block slot. These will be stored in the database.
let mut intermediate_states = StateBatch::new();
// The block must have a higher slot than its parent.
if block.slot() <= parent.beacon_state.slot {
return Err(BlockError::BlockIsNotLaterThanParent {
block_slot: block.slot(),
state_slot: parent.beacon_state.slot,
});
}
// Transition the parent state to the block slot.
let mut state = parent.beacon_state;
let distance = block.slot().as_u64().saturating_sub(state.slot.as_u64());
for i in 0..distance {
let state_root = if i == 0 {
parent.beacon_block.state_root()
} else {
// This is a new state we've reached, so stage it for storage in the DB.
// Computing the state root here is time-equivalent to computing it during slot
// processing, but we get early access to it.
let state_root = state.update_tree_hash_cache()?;
intermediate_states.add_state(state_root, &state)?;
state_root
};
per_slot_processing(&mut state, Some(state_root), &chain.spec)?;
}
metrics::stop_timer(catchup_timer);
/*
* Build the committee caches on the state.
*/
let committee_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_COMMITTEE);
state.build_committee_cache(RelativeEpoch::Previous, &chain.spec)?;
state.build_committee_cache(RelativeEpoch::Current, &chain.spec)?;
metrics::stop_timer(committee_timer);
/*
* Perform `per_block_processing` on the block and state, returning early if the block is
* invalid.
*/
let core_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_CORE);
if let Err(err) = per_block_processing(
&mut state,
&block,
Some(block_root),
// Signatures were verified earlier in this function.
BlockSignatureStrategy::NoVerification,
&chain.spec,
) {
match err {
// Capture `BeaconStateError` so that we can easily distinguish between a block
// that's invalid and one that caused an internal error.
BlockProcessingError::BeaconStateError(e) => return Err(e.into()),
other => return Err(BlockError::PerBlockProcessingError(other)),
}
};
metrics::stop_timer(core_timer);
/*
* Calculate the state root of the newly modified state
*/
let state_root_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_STATE_ROOT);
let state_root = state.update_tree_hash_cache()?;
metrics::stop_timer(state_root_timer);
/*
* Check to ensure the state root on the block matches the one we have calculated.
*/
if block.state_root() != state_root {
return Err(BlockError::StateRootMismatch {
block: block.state_root(),
local: state_root,
});
}
Ok(Self {
block,
block_root,
state,
parent_block: parent.beacon_block,
intermediate_states,
})
}
}
/// Returns `Ok(())` if the block is later than the finalized slot on `chain`.
///
/// Returns an error if the block is earlier or equal to the finalized slot, or there was an error
/// verifying that condition.
fn check_block_against_finalized_slot<T: BeaconChainTypes>(
block: &BeaconBlock<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<(), BlockError> {
let finalized_slot = chain
.head_info()?
.finalized_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch());
if block.slot <= finalized_slot {
Err(BlockError::WouldRevertFinalizedSlot {
block_slot: block.slot,
finalized_slot,
})
} else {
Ok(())
}
}
/// Performs simple, cheap checks to ensure that the block is relevant to be imported.
///
/// `Ok(block_root)` is returned if the block passes these checks and should progress with
/// verification (viz., it is relevant).
///
/// Returns an error if the block fails one of these checks (viz., is not relevant) or an error is
/// experienced whilst attempting to verify.
pub fn check_block_relevancy<T: BeaconChainTypes>(
signed_block: &SignedBeaconBlock<T::EthSpec>,
block_root: Option<Hash256>,
chain: &BeaconChain<T>,
) -> Result<Hash256, BlockError> {
let block = &signed_block.message;
// Do not process blocks from the future.
if block.slot > chain.slot()? {
return Err(BlockError::FutureSlot {
present_slot: chain.slot()?,
block_slot: block.slot,
});
}
// Do not re-process the genesis block.
if block.slot == 0 {
return Err(BlockError::GenesisBlock);
}
// This is an artificial (non-spec) restriction that provides some protection from overflow
// abuses.
if block.slot >= MAXIMUM_BLOCK_SLOT_NUMBER {
return Err(BlockError::BlockSlotLimitReached);
}
// Do not process a block from a finalized slot.
check_block_against_finalized_slot(block, chain)?;
let block_root = block_root.unwrap_or_else(|| get_block_root(&signed_block));
// Check if the block is already known. We know it is post-finalization, so it is
// sufficient to check the fork choice.
if chain.fork_choice.contains_block(&block_root) {
return Err(BlockError::BlockIsAlreadyKnown);
}
Ok(block_root)
}
/// Returns the canonical root of the given `block`.
///
/// Use this function to ensure that we report the block hashing time Prometheus metric.
pub fn get_block_root<E: EthSpec>(block: &SignedBeaconBlock<E>) -> Hash256 {
let block_root_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_BLOCK_ROOT);
let block_root = block.canonical_root();
metrics::stop_timer(block_root_timer);
block_root
}
/// Load the parent snapshot (block and state) of the given `block`.
///
/// Returns `Err(BlockError::ParentUnknown)` if the parent is not found, or if an error occurs
/// whilst attempting the operation.
fn load_parent<T: BeaconChainTypes>(
block: &BeaconBlock<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<BeaconSnapshot<T::EthSpec>, BlockError> {
let db_read_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_DB_READ);
// Reject any block if its parent is not known to fork choice.
//
// A block that is not in fork choice is either:
//
// - Not yet imported: we should reject this block because we should only import a child
// after its parent has been fully imported.
// - Pre-finalized: if the parent block is _prior_ to finalization, we should ignore it
// because it will revert finalization. Note that the finalized block is stored in fork
// choice, so we will not reject any child of the finalized block (this is relevant during
// genesis).
if !chain.fork_choice.contains_block(&block.parent_root) {
return Err(BlockError::ParentUnknown(block.parent_root));
}
// Load the parent block and state from disk, returning early if it's not available.
let result = chain
.snapshot_cache
.try_write_for(BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT)
.and_then(|mut snapshot_cache| snapshot_cache.try_remove(block.parent_root))
.map(|snapshot| Ok(Some(snapshot)))
.unwrap_or_else(|| {
// Load the blocks parent block from the database, returning invalid if that block is not
// found.
//
// We don't return a DBInconsistent error here since it's possible for a block to
// exist in fork choice but not in the database yet. In such a case we simply
// indicate that we don't yet know the parent.
let parent_block = if let Some(block) = chain.get_block(&block.parent_root)? {
block
} else {
return Ok(None);
};
// Load the parent blocks state from the database, returning an error if it is not found.
// It is an error because if we know the parent block we should also know the parent state.
let parent_state_root = parent_block.state_root();
let parent_state = chain
.get_state(&parent_state_root, Some(parent_block.slot()))?
.ok_or_else(|| {
BeaconChainError::DBInconsistent(format!(
"Missing state {:?}",
parent_state_root
))
})?;
Ok(Some(BeaconSnapshot {
beacon_block: parent_block,
beacon_block_root: block.parent_root,
beacon_state: parent_state,
beacon_state_root: parent_state_root,
}))
})
.map_err(BlockError::BeaconChainError)?
.ok_or_else(|| BlockError::ParentUnknown(block.parent_root));
metrics::stop_timer(db_read_timer);
result
}
/// Performs a cheap (time-efficient) state advancement so the committees for `slot` can be
/// obtained from `state`.
///
/// The state advancement is "cheap" since it does not generate state roots. As a result, the
/// returned state might be holistically invalid but the committees will be correct (since they do
/// not rely upon state roots).
///
/// If the given `state` can already serve the `slot`, the committees will be built on the `state`
/// and `Cow::Borrowed(state)` will be returned. Otherwise, the state will be cloned, cheaply
/// advanced and then returned as a `Cow::Owned`. The end result is that the given `state` is never
/// mutated to be invalid (in fact, it is never changed beyond a simple committee cache build).
fn cheap_state_advance_to_obtain_committees<'a, E: EthSpec>(
state: &'a mut BeaconState<E>,
block_slot: Slot,
spec: &ChainSpec,
) -> Result<Cow<'a, BeaconState<E>>, BlockError> {
let block_epoch = block_slot.epoch(E::slots_per_epoch());
if state.current_epoch() == block_epoch {
state.build_committee_cache(RelativeEpoch::Current, spec)?;
Ok(Cow::Borrowed(state))
} else if state.slot > block_slot {
Err(BlockError::BlockIsNotLaterThanParent {
block_slot,
state_slot: state.slot,
})
} else {
let mut state = state.clone_with(CloneConfig::committee_caches_only());
while state.current_epoch() < block_epoch {
// Don't calculate state roots since they aren't required for calculating
// shuffling (achieved by providing Hash256::zero()).
per_slot_processing(&mut state, Some(Hash256::zero()), spec).map_err(|e| {
BlockError::BeaconChainError(BeaconChainError::SlotProcessingError(e))
})?;
}
state.build_committee_cache(RelativeEpoch::Current, spec)?;
Ok(Cow::Owned(state))
}
}
/// Obtains a read-locked `ValidatorPubkeyCache` from the `chain`.
fn get_validator_pubkey_cache<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
) -> Result<RwLockReadGuard<ValidatorPubkeyCache>, BlockError> {
chain
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or_else(|| BeaconChainError::ValidatorPubkeyCacheLockTimeout)
.map_err(BlockError::BeaconChainError)
}
/// Produces an _empty_ `BlockSignatureVerifier`.
///
/// The signature verifier is empty because it does not yet have any of this block's signatures
/// added to it. Use `Self::apply_to_signature_verifier` to apply the signatures.
fn get_signature_verifier<'a, E: EthSpec>(
state: &'a BeaconState<E>,
validator_pubkey_cache: &'a ValidatorPubkeyCache,
spec: &'a ChainSpec,
) -> BlockSignatureVerifier<'a, E, impl Fn(usize) -> Option<Cow<'a, G1Point>> + Clone> {
BlockSignatureVerifier::new(
state,
move |validator_index| {
// Disallow access to any validator pubkeys that are not in the current beacon
// state.
if validator_index < state.validators.len() {
validator_pubkey_cache
.get(validator_index)
.map(|pk| Cow::Borrowed(pk.as_point()))
} else {
None
}
},
spec,
)
}

View File

@ -0,0 +1,105 @@
use crate::{BeaconChainError, BlockError};
use state_processing::BlockProcessingError;
use types::{Hash256, Slot};
/// This is a legacy object that is being kept around to reduce merge conflicts.
///
/// As soon as this is merged into master, it should be removed as soon as possible.
#[derive(Debug, PartialEq)]
pub enum BlockProcessingOutcome {
/// Block was valid and imported into the block graph.
Processed {
block_root: Hash256,
},
InvalidSignature,
/// The proposal signature in invalid.
ProposalSignatureInvalid,
/// The parent block was unknown.
ParentUnknown(Hash256),
/// The block slot is greater than the present slot.
FutureSlot {
present_slot: Slot,
block_slot: Slot,
},
/// The block state_root does not match the generated state.
StateRootMismatch {
block: Hash256,
local: Hash256,
},
/// The block was a genesis block, these blocks cannot be re-imported.
GenesisBlock,
/// The slot is finalized, no need to import.
WouldRevertFinalizedSlot {
block_slot: Slot,
finalized_slot: Slot,
},
/// Block is already known, no need to re-import.
BlockIsAlreadyKnown,
/// The block slot exceeds the MAXIMUM_BLOCK_SLOT_NUMBER.
BlockSlotLimitReached,
/// The provided block is from an earlier slot than its parent.
BlockIsNotLaterThanParent {
block_slot: Slot,
state_slot: Slot,
},
/// At least one block in the chain segement did not have it's parent root set to the root of
/// the prior block.
NonLinearParentRoots,
/// The slots of the blocks in the chain segment were not strictly increasing. I.e., a child
/// had lower slot than a parent.
NonLinearSlots,
/// The block could not be applied to the state, it is invalid.
PerBlockProcessingError(BlockProcessingError),
}
impl BlockProcessingOutcome {
pub fn shim(
result: Result<Hash256, BlockError>,
) -> Result<BlockProcessingOutcome, BeaconChainError> {
match result {
Ok(block_root) => Ok(BlockProcessingOutcome::Processed { block_root }),
Err(BlockError::ParentUnknown(root)) => Ok(BlockProcessingOutcome::ParentUnknown(root)),
Err(BlockError::FutureSlot {
present_slot,
block_slot,
}) => Ok(BlockProcessingOutcome::FutureSlot {
present_slot,
block_slot,
}),
Err(BlockError::StateRootMismatch { block, local }) => {
Ok(BlockProcessingOutcome::StateRootMismatch { block, local })
}
Err(BlockError::GenesisBlock) => Ok(BlockProcessingOutcome::GenesisBlock),
Err(BlockError::WouldRevertFinalizedSlot {
block_slot,
finalized_slot,
}) => Ok(BlockProcessingOutcome::WouldRevertFinalizedSlot {
block_slot,
finalized_slot,
}),
Err(BlockError::BlockIsAlreadyKnown) => Ok(BlockProcessingOutcome::BlockIsAlreadyKnown),
Err(BlockError::BlockSlotLimitReached) => {
Ok(BlockProcessingOutcome::BlockSlotLimitReached)
}
Err(BlockError::ProposalSignatureInvalid) => {
Ok(BlockProcessingOutcome::ProposalSignatureInvalid)
}
Err(BlockError::InvalidSignature) => Ok(BlockProcessingOutcome::InvalidSignature),
Err(BlockError::BlockIsNotLaterThanParent {
block_slot,
state_slot,
}) => Ok(BlockProcessingOutcome::BlockIsNotLaterThanParent {
block_slot,
state_slot,
}),
Err(BlockError::NonLinearParentRoots) => {
Ok(BlockProcessingOutcome::NonLinearParentRoots)
}
Err(BlockError::NonLinearSlots) => Ok(BlockProcessingOutcome::NonLinearSlots),
Err(BlockError::PerBlockProcessingError(e)) => {
Ok(BlockProcessingOutcome::PerBlockProcessingError(e))
}
Err(BlockError::BeaconChainError(e)) => Err(e),
}
}
}

View File

@ -88,6 +88,7 @@ pub struct BeaconChainBuilder<T: BeaconChainTypes> {
pubkey_cache_path: Option<PathBuf>, pubkey_cache_path: Option<PathBuf>,
validator_pubkey_cache: Option<ValidatorPubkeyCache>, validator_pubkey_cache: Option<ValidatorPubkeyCache>,
spec: ChainSpec, spec: ChainSpec,
disabled_forks: Vec<String>,
log: Option<Logger>, log: Option<Logger>,
} }
@ -122,6 +123,7 @@ where
head_tracker: None, head_tracker: None,
pubkey_cache_path: None, pubkey_cache_path: None,
data_dir: None, data_dir: None,
disabled_forks: Vec::new(),
validator_pubkey_cache: None, validator_pubkey_cache: None,
spec: TEthSpec::default_spec(), spec: TEthSpec::default_spec(),
log: None, log: None,
@ -168,6 +170,12 @@ where
self self
} }
/// Sets a list of hard-coded forks that will not be activated.
pub fn disabled_forks(mut self, disabled_forks: Vec<String>) -> Self {
self.disabled_forks = disabled_forks;
self
}
/// Attempt to load an existing eth1 cache from the builder's `Store`. /// Attempt to load an existing eth1 cache from the builder's `Store`.
pub fn get_persisted_eth1_backend(&self) -> Result<Option<SszEth1>, String> { pub fn get_persisted_eth1_backend(&self) -> Result<Option<SszEth1>, String> {
let store = self let store = self
@ -421,7 +429,10 @@ where
op_pool: self op_pool: self
.op_pool .op_pool
.ok_or_else(|| "Cannot build without op pool".to_string())?, .ok_or_else(|| "Cannot build without op pool".to_string())?,
// TODO: allow for persisting and loading the pool from disk.
naive_aggregation_pool: <_>::default(),
eth1_chain: self.eth1_chain, eth1_chain: self.eth1_chain,
genesis_validators_root: canonical_head.beacon_state.genesis_validators_root,
canonical_head: TimeoutRwLock::new(canonical_head.clone()), canonical_head: TimeoutRwLock::new(canonical_head.clone()),
genesis_block_root: self genesis_block_root: self
.genesis_block_root .genesis_block_root
@ -433,12 +444,13 @@ where
.event_handler .event_handler
.ok_or_else(|| "Cannot build without an event handler".to_string())?, .ok_or_else(|| "Cannot build without an event handler".to_string())?,
head_tracker: Arc::new(self.head_tracker.unwrap_or_default()), head_tracker: Arc::new(self.head_tracker.unwrap_or_default()),
block_processing_cache: TimeoutRwLock::new(SnapshotCache::new( snapshot_cache: TimeoutRwLock::new(SnapshotCache::new(
DEFAULT_SNAPSHOT_CACHE_SIZE, DEFAULT_SNAPSHOT_CACHE_SIZE,
canonical_head, canonical_head,
)), )),
shuffling_cache: TimeoutRwLock::new(ShufflingCache::new()), shuffling_cache: TimeoutRwLock::new(ShufflingCache::new()),
validator_pubkey_cache: TimeoutRwLock::new(validator_pubkey_cache), validator_pubkey_cache: TimeoutRwLock::new(validator_pubkey_cache),
disabled_forks: self.disabled_forks,
log: log.clone(), log: log.clone(),
}; };
@ -661,7 +673,7 @@ mod test {
#[test] #[test]
fn recent_genesis() { fn recent_genesis() {
let validator_count = 8; let validator_count = 1;
let genesis_time = 13_371_337; let genesis_time = 13_371_337;
let log = get_logger(); let log = get_logger();

View File

@ -1,11 +1,14 @@
use crate::eth1_chain::Error as Eth1ChainError; use crate::eth1_chain::Error as Eth1ChainError;
use crate::fork_choice::Error as ForkChoiceError; use crate::fork_choice::Error as ForkChoiceError;
use crate::naive_aggregation_pool::Error as NaiveAggregationError;
use operation_pool::OpPoolError; use operation_pool::OpPoolError;
use ssz::DecodeError; use ssz::DecodeError;
use ssz_types::Error as SszTypesError; use ssz_types::Error as SszTypesError;
use state_processing::per_block_processing::errors::AttestationValidationError; use state_processing::{
use state_processing::BlockProcessingError; block_signature_verifier::Error as BlockSignatureVerifierError,
use state_processing::SlotProcessingError; per_block_processing::errors::AttestationValidationError,
signature_sets::Error as SignatureSetError, BlockProcessingError, SlotProcessingError,
};
use std::time::Duration; use std::time::Duration;
use types::*; use types::*;
@ -57,15 +60,20 @@ pub enum BeaconChainError {
IncorrectStateForAttestation(RelativeEpochError), IncorrectStateForAttestation(RelativeEpochError),
InvalidValidatorPubkeyBytes(DecodeError), InvalidValidatorPubkeyBytes(DecodeError),
ValidatorPubkeyCacheIncomplete(usize), ValidatorPubkeyCacheIncomplete(usize),
SignatureSetError(state_processing::signature_sets::Error), SignatureSetError(SignatureSetError),
BlockSignatureVerifierError(state_processing::block_signature_verifier::Error), BlockSignatureVerifierError(state_processing::block_signature_verifier::Error),
DuplicateValidatorPublicKey, DuplicateValidatorPublicKey,
ValidatorPubkeyCacheFileError(String), ValidatorPubkeyCacheFileError(String),
OpPoolError(OpPoolError),
NaiveAggregationError(NaiveAggregationError),
} }
easy_from_to!(SlotProcessingError, BeaconChainError); easy_from_to!(SlotProcessingError, BeaconChainError);
easy_from_to!(AttestationValidationError, BeaconChainError); easy_from_to!(AttestationValidationError, BeaconChainError);
easy_from_to!(SszTypesError, BeaconChainError); easy_from_to!(SszTypesError, BeaconChainError);
easy_from_to!(OpPoolError, BeaconChainError);
easy_from_to!(NaiveAggregationError, BeaconChainError);
easy_from_to!(BlockSignatureVerifierError, BeaconChainError);
#[derive(Debug, PartialEq)] #[derive(Debug, PartialEq)]
pub enum BlockProductionError { pub enum BlockProductionError {

View File

@ -1,7 +1,6 @@
use crate::metrics; use crate::metrics;
use eth1::{Config as Eth1Config, Eth1Block, Service as HttpService}; use eth1::{Config as Eth1Config, Eth1Block, Service as HttpService};
use eth2_hashing::hash; use eth2_hashing::hash;
use exit_future::Exit;
use futures::Future; use futures::Future;
use slog::{debug, error, trace, Logger}; use slog::{debug, error, trace, Logger};
use ssz::{Decode, Encode}; use ssz::{Decode, Encode};
@ -287,7 +286,10 @@ impl<T: EthSpec, S: Store<T>> CachingEth1Backend<T, S> {
} }
/// Starts the routine which connects to the external eth1 node and updates the caches. /// Starts the routine which connects to the external eth1 node and updates the caches.
pub fn start(&self, exit: Exit) -> impl Future<Item = (), Error = ()> { pub fn start(
&self,
exit: tokio::sync::oneshot::Receiver<()>,
) -> impl Future<Item = (), Error = ()> {
self.core.auto_update(exit) self.core.auto_update(exit)
} }

View File

@ -4,6 +4,7 @@ extern crate lazy_static;
mod beacon_chain; mod beacon_chain;
mod beacon_snapshot; mod beacon_snapshot;
mod block_verification;
pub mod builder; pub mod builder;
mod errors; mod errors;
pub mod eth1_chain; pub mod eth1_chain;
@ -12,6 +13,7 @@ mod fork_choice;
mod head_tracker; mod head_tracker;
mod metrics; mod metrics;
pub mod migrate; pub mod migrate;
mod naive_aggregation_pool;
mod persisted_beacon_chain; mod persisted_beacon_chain;
mod shuffling_cache; mod shuffling_cache;
mod snapshot_cache; mod snapshot_cache;
@ -20,11 +22,12 @@ mod timeout_rw_lock;
mod validator_pubkey_cache; mod validator_pubkey_cache;
pub use self::beacon_chain::{ pub use self::beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BlockProcessingOutcome, AttestationProcessingOutcome, AttestationType, BeaconChain, BeaconChainTypes,
StateSkipConfig, ChainSegmentResult, StateSkipConfig,
}; };
pub use self::beacon_snapshot::BeaconSnapshot; pub use self::beacon_snapshot::BeaconSnapshot;
pub use self::errors::{BeaconChainError, BlockProductionError}; pub use self::errors::{BeaconChainError, BlockProductionError};
pub use block_verification::{BlockError, BlockProcessingOutcome, GossipVerifiedBlock};
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend}; pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};
pub use events::EventHandler; pub use events::EventHandler;
pub use fork_choice::ForkChoice; pub use fork_choice::ForkChoice;

View File

@ -0,0 +1,480 @@
use parking_lot::RwLock;
use std::collections::HashMap;
use types::{Attestation, AttestationData, EthSpec, Slot};
/// The number of slots that will be stored in the pool.
///
/// For example, if `SLOTS_RETAINED == 3` and the pool is pruned at slot `6`, then all attestations
/// at slots less than `4` will be dropped and any future attestation with a slot less than `4`
/// will be refused.
const SLOTS_RETAINED: usize = 3;
/// The maximum number of distinct `AttestationData` that will be stored in each slot.
///
/// This is a DoS protection measure.
const MAX_ATTESTATIONS_PER_SLOT: usize = 16_384;
/// Returned upon successfully inserting an attestation into the pool.
#[derive(Debug, PartialEq)]
pub enum InsertOutcome {
/// The `attestation.data` had not been seen before and was added to the pool.
NewAttestationData { committee_index: usize },
/// A validator signature for the given `attestation.data` was already known. No changes were
/// made.
SignatureAlreadyKnown { committee_index: usize },
/// The `attestation.data` was known, but a signature for the given validator was not yet
/// known. The signature was aggregated into the pool.
SignatureAggregated { committee_index: usize },
}
#[derive(Debug, PartialEq)]
pub enum Error {
/// The given `attestation.data.slot` was too low to be stored. No changes were made.
SlotTooLow {
slot: Slot,
lowest_permissible_slot: Slot,
},
/// The given `attestation.aggregation_bits` field was empty.
NoAggregationBitsSet,
/// The given `attestation.aggregation_bits` field had more than one signature. The number of
/// signatures found is included.
MoreThanOneAggregationBitSet(usize),
/// We have reached the maximum number of unique `AttestationData` that can be stored in a
/// slot. This is a DoS protection function.
ReachedMaxAttestationsPerSlot(usize),
/// The given `attestation.aggregation_bits` field had a different length to the one currently
/// stored. This indicates a fairly serious error somewhere in the code that called this
/// function.
InconsistentBitfieldLengths,
/// The function to obtain a map index failed, this is an internal error.
InvalidMapIndex(usize),
/// The given `attestation` was for the incorrect slot. This is an internal error.
IncorrectSlot { expected: Slot, attestation: Slot },
}
/// A collection of `Attestation` objects, keyed by their `attestation.data`. Enforces that all
/// `attestation` are from the same slot.
struct AggregatedAttestationMap<E: EthSpec> {
map: HashMap<AttestationData, Attestation<E>>,
slot: Slot,
}
impl<E: EthSpec> AggregatedAttestationMap<E> {
/// Create an empty collection that will only contain attestation for the given `slot`.
pub fn new(slot: Slot) -> Self {
Self {
slot,
map: <_>::default(),
}
}
/// Insert an attestation into `self`, aggregating it into the pool.
///
/// The given attestation (`a`) must only have one signature and be from the slot that `self`
/// was initialized with.
pub fn insert(&mut self, a: &Attestation<E>) -> Result<InsertOutcome, Error> {
if a.data.slot != self.slot {
return Err(Error::IncorrectSlot {
expected: self.slot,
attestation: a.data.slot,
});
}
let set_bits = a
.aggregation_bits
.iter()
.enumerate()
.filter(|(_i, bit)| *bit)
.map(|(i, _bit)| i)
.collect::<Vec<_>>();
let committee_index = set_bits
.first()
.copied()
.ok_or_else(|| Error::NoAggregationBitsSet)?;
if set_bits.len() > 1 {
return Err(Error::MoreThanOneAggregationBitSet(set_bits.len()));
}
if let Some(existing_attestation) = self.map.get_mut(&a.data) {
if existing_attestation
.aggregation_bits
.get(committee_index)
.map_err(|_| Error::InconsistentBitfieldLengths)?
{
Ok(InsertOutcome::SignatureAlreadyKnown { committee_index })
} else {
existing_attestation.aggregate(a);
Ok(InsertOutcome::SignatureAggregated { committee_index })
}
} else {
if self.map.len() >= MAX_ATTESTATIONS_PER_SLOT {
return Err(Error::ReachedMaxAttestationsPerSlot(
MAX_ATTESTATIONS_PER_SLOT,
));
}
self.map.insert(a.data.clone(), a.clone());
Ok(InsertOutcome::NewAttestationData { committee_index })
}
}
/// Returns an aggregated `Attestation` with the given `data`, if any.
///
/// The given `a.data.slot` must match the slot that `self` was initialized with.
pub fn get(&self, data: &AttestationData) -> Result<Option<Attestation<E>>, Error> {
if data.slot != self.slot {
return Err(Error::IncorrectSlot {
expected: self.slot,
attestation: data.slot,
});
}
Ok(self.map.get(data).cloned())
}
}
/// A pool of `Attestation` that is specially designed to store "unaggregated" attestations from
/// the native aggregation scheme.
///
/// **The `NaiveAggregationPool` does not do any signature or attestation verification. It assumes
/// that all `Attestation` objects provided are valid.**
///
/// ## Details
///
/// The pool sorts the `Attestation` by `attestation.data.slot`, then by `attestation.data`.
///
/// As each unaggregated attestation is added it is aggregated with any existing `attestation` with
/// the same `AttestationData`. Considering that the pool only accepts attestations with a single
/// signature, there should only ever be a single aggregated `Attestation` for any given
/// `AttestationData`.
///
/// The pool has a capacity for `SLOTS_RETAINED` slots, when a new `attestation.data.slot` is
/// provided, the oldest slot is dropped and replaced with the new slot. The pool can also be
/// pruned by supplying a `current_slot`; all existing attestations with a slot lower than
/// `current_slot - SLOTS_RETAINED` will be removed and any future attestation with a slot lower
/// than that will also be refused. Pruning is done automatically based upon the attestations it
/// receives and it can be triggered manually.
pub struct NaiveAggregationPool<E: EthSpec> {
lowest_permissible_slot: RwLock<Slot>,
maps: RwLock<Vec<AggregatedAttestationMap<E>>>,
}
impl<E: EthSpec> Default for NaiveAggregationPool<E> {
fn default() -> Self {
Self {
lowest_permissible_slot: RwLock::new(Slot::new(0)),
maps: RwLock::new(vec![]),
}
}
}
impl<E: EthSpec> NaiveAggregationPool<E> {
/// Insert an attestation into `self`, aggregating it into the pool.
///
/// The given attestation (`a`) must only have one signature and have an
/// `attestation.data.slot` that is not lower than `self.lowest_permissible_slot`.
///
/// The pool may be pruned if the given `attestation.data` has a slot higher than any
/// previously seen.
pub fn insert(&self, attestation: &Attestation<E>) -> Result<InsertOutcome, Error> {
let lowest_permissible_slot = *self.lowest_permissible_slot.read();
// Reject any attestations that are too old.
if attestation.data.slot < lowest_permissible_slot {
return Err(Error::SlotTooLow {
slot: attestation.data.slot,
lowest_permissible_slot,
});
}
// Prune the pool if this attestation indicates that the current slot has advanced.
if (lowest_permissible_slot + SLOTS_RETAINED as u64) < attestation.data.slot + 1 {
self.prune(attestation.data.slot)
}
let index = self.get_map_index(attestation.data.slot);
self.maps
.write()
.get_mut(index)
.ok_or_else(|| Error::InvalidMapIndex(index))?
.insert(attestation)
}
/// Returns an aggregated `Attestation` with the given `data`, if any.
pub fn get(&self, data: &AttestationData) -> Result<Option<Attestation<E>>, Error> {
self.maps
.read()
.iter()
.find(|map| map.slot == data.slot)
.map(|map| map.get(data))
.unwrap_or_else(|| Ok(None))
}
/// Removes any attestations with a slot lower than `current_slot` and bars any future
/// attestations with a slot lower than `current_slot - SLOTS_RETAINED`.
pub fn prune(&self, current_slot: Slot) {
// Taking advantage of saturating subtraction on `Slot`.
let lowest_permissible_slot = current_slot - Slot::from(SLOTS_RETAINED);
self.maps
.write()
.retain(|map| map.slot >= lowest_permissible_slot);
*self.lowest_permissible_slot.write() = lowest_permissible_slot;
}
/// Returns the index of `self.maps` that matches `slot`.
///
/// If there is no existing map for this slot one will be created. If `self.maps.len() >=
/// SLOTS_RETAINED`, the map with the lowest slot will be replaced.
fn get_map_index(&self, slot: Slot) -> usize {
let mut maps = self.maps.write();
if let Some(index) = maps.iter().position(|map| map.slot == slot) {
return index;
}
if maps.len() < SLOTS_RETAINED || maps.is_empty() {
let index = maps.len();
maps.push(AggregatedAttestationMap::new(slot));
return index;
}
let index = maps
.iter()
.enumerate()
.min_by_key(|(_i, map)| map.slot)
.map(|(i, _map)| i)
.expect("maps cannot be empty due to previous .is_empty() check");
maps[index] = AggregatedAttestationMap::new(slot);
index
}
}
#[cfg(test)]
mod tests {
use super::*;
use ssz_types::BitList;
use types::{
test_utils::{generate_deterministic_keypair, test_random_instance},
Fork, Hash256,
};
type E = types::MainnetEthSpec;
fn get_attestation(slot: Slot) -> Attestation<E> {
let mut a: Attestation<E> = test_random_instance();
a.data.slot = slot;
a.aggregation_bits = BitList::with_capacity(4).expect("should create bitlist");
a
}
fn sign(a: &mut Attestation<E>, i: usize, genesis_validators_root: Hash256) {
a.sign(
&generate_deterministic_keypair(i).sk,
i,
&Fork::default(),
genesis_validators_root,
&E::default_spec(),
)
.expect("should sign attestation");
}
fn unset_bit(a: &mut Attestation<E>, i: usize) {
a.aggregation_bits
.set(i, false)
.expect("should unset aggregation bit")
}
#[test]
fn single_attestation() {
let mut a = get_attestation(Slot::new(0));
let pool = NaiveAggregationPool::default();
assert_eq!(
pool.insert(&a),
Err(Error::NoAggregationBitsSet),
"should not accept attestation without any signatures"
);
sign(&mut a, 0, Hash256::random());
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept new attestation"
);
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::SignatureAlreadyKnown { committee_index: 0 }),
"should acknowledge duplicate signature"
);
let retrieved = pool
.get(&a.data)
.expect("should not error while getting attestation")
.expect("should get an attestation");
assert_eq!(
retrieved, a,
"retrieved attestation should equal the one inserted"
);
sign(&mut a, 1, Hash256::random());
assert_eq!(
pool.insert(&a),
Err(Error::MoreThanOneAggregationBitSet(2)),
"should not accept attestation with multiple signatures"
);
}
#[test]
fn multiple_attestations() {
let mut a_0 = get_attestation(Slot::new(0));
let mut a_1 = a_0.clone();
let genesis_validators_root = Hash256::random();
sign(&mut a_0, 0, genesis_validators_root);
sign(&mut a_1, 1, genesis_validators_root);
let pool = NaiveAggregationPool::default();
assert_eq!(
pool.insert(&a_0),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept a_0"
);
assert_eq!(
pool.insert(&a_1),
Ok(InsertOutcome::SignatureAggregated { committee_index: 1 }),
"should accept a_1"
);
let retrieved = pool
.get(&a_0.data)
.expect("should not error while getting attestation")
.expect("should get an attestation");
let mut a_01 = a_0.clone();
a_01.aggregate(&a_1);
assert_eq!(
retrieved, a_01,
"retrieved attestation should be aggregated"
);
/*
* Throw a different attestation data in there and ensure it isn't aggregated
*/
let mut a_different = a_0.clone();
let different_root = Hash256::from_low_u64_be(1337);
unset_bit(&mut a_different, 0);
sign(&mut a_different, 2, genesis_validators_root);
assert!(a_different.data.beacon_block_root != different_root);
a_different.data.beacon_block_root = different_root;
assert_eq!(
pool.insert(&a_different),
Ok(InsertOutcome::NewAttestationData { committee_index: 2 }),
"should accept a_different"
);
assert_eq!(
pool.get(&a_0.data)
.expect("should not error while getting attestation")
.expect("should get an attestation"),
retrieved,
"should not have aggregated different attestation data"
);
}
#[test]
fn auto_pruning() {
let mut base = get_attestation(Slot::new(0));
sign(&mut base, 0, Hash256::random());
let pool = NaiveAggregationPool::default();
for i in 0..SLOTS_RETAINED * 2 {
let slot = Slot::from(i);
let mut a = base.clone();
a.data.slot = slot;
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept new attestation"
);
if i < SLOTS_RETAINED {
let len = i + 1;
assert_eq!(
pool.maps.read().len(),
len,
"the pool should have length {}",
len
);
} else {
assert_eq!(
pool.maps.read().len(),
SLOTS_RETAINED,
"the pool should have length SLOTS_RETAINED"
);
let mut pool_slots = pool
.maps
.read()
.iter()
.map(|map| map.slot)
.collect::<Vec<_>>();
pool_slots.sort_unstable();
for (j, pool_slot) in pool_slots.iter().enumerate() {
let expected_slot = slot - (SLOTS_RETAINED - 1 - j) as u64;
assert_eq!(
*pool_slot, expected_slot,
"the slot of the map should be {}",
expected_slot
)
}
}
}
}
#[test]
fn max_attestations() {
let mut base = get_attestation(Slot::new(0));
sign(&mut base, 0, Hash256::random());
let pool = NaiveAggregationPool::default();
for i in 0..=MAX_ATTESTATIONS_PER_SLOT {
let mut a = base.clone();
a.data.beacon_block_root = Hash256::from_low_u64_be(i as u64);
if i < MAX_ATTESTATIONS_PER_SLOT {
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept attestation below limit"
);
} else {
assert_eq!(
pool.insert(&a),
Err(Error::ReachedMaxAttestationsPerSlot(
MAX_ATTESTATIONS_PER_SLOT
)),
"should not accept attestation above limit"
);
}
}
}
}

View File

@ -7,8 +7,7 @@ use crate::{
builder::{BeaconChainBuilder, Witness}, builder::{BeaconChainBuilder, Witness},
eth1_chain::CachingEth1Backend, eth1_chain::CachingEth1Backend,
events::NullEventHandler, events::NullEventHandler,
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BlockProcessingOutcome, AttestationProcessingOutcome, AttestationType, BeaconChain, BeaconChainTypes, StateSkipConfig,
StateSkipConfig,
}; };
use genesis::interop_genesis_state; use genesis::interop_genesis_state;
use rayon::prelude::*; use rayon::prelude::*;
@ -263,20 +262,15 @@ where
let (block, new_state) = self.build_block(state.clone(), slot, block_strategy); let (block, new_state) = self.build_block(state.clone(), slot, block_strategy);
let outcome = self let block_root = self
.chain .chain
.process_block(block) .process_block(block)
.expect("should not error during block processing"); .expect("should not error during block processing");
self.chain.fork_choice().expect("should find head"); self.chain.fork_choice().expect("should find head");
if let BlockProcessingOutcome::Processed { block_root } = outcome { head_block_root = Some(block_root);
head_block_root = Some(block_root); self.add_free_attestations(&attestation_strategy, &new_state, block_root, slot);
self.add_free_attestations(&attestation_strategy, &new_state, block_root, slot);
} else {
panic!("block should be successfully processed: {:?}", outcome);
}
state = new_state; state = new_state;
slot += 1; slot += 1;
@ -310,20 +304,16 @@ where
let (block, new_state) = self.build_block(state.clone(), slot, block_strategy); let (block, new_state) = self.build_block(state.clone(), slot, block_strategy);
let outcome = self let block_root = self
.chain .chain
.process_block(block) .process_block(block)
.expect("should not error during block processing"); .expect("should not error during block processing");
self.chain.fork_choice().expect("should find head"); self.chain.fork_choice().expect("should find head");
if let BlockProcessingOutcome::Processed { block_root } = outcome { let attestation_strategy = AttestationStrategy::SomeValidators(validators.to_vec());
let attestation_strategy = AttestationStrategy::SomeValidators(validators.to_vec()); self.add_free_attestations(&attestation_strategy, &new_state, block_root, slot);
self.add_free_attestations(&attestation_strategy, &new_state, block_root, slot); (block_root.into(), new_state)
(block_root.into(), new_state)
} else {
panic!("block should be successfully processed: {:?}", outcome);
}
} }
/// `add_block()` repeated `num_blocks` times. /// `add_block()` repeated `num_blocks` times.
@ -478,7 +468,7 @@ where
.for_each(|attestation| { .for_each(|attestation| {
match self match self
.chain .chain
.process_attestation(attestation) .process_attestation(attestation, AttestationType::Aggregated)
.expect("should not error during attestation processing") .expect("should not error during attestation processing")
{ {
// PastEpoch can occur if we fork over several epochs // PastEpoch can occur if we fork over several epochs

View File

@ -6,7 +6,7 @@ extern crate lazy_static;
use beacon_chain::test_utils::{ use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, HarnessType, AttestationStrategy, BeaconChainHarness, BlockStrategy, HarnessType,
}; };
use beacon_chain::AttestationProcessingOutcome; use beacon_chain::{AttestationProcessingOutcome, AttestationType};
use state_processing::per_slot_processing; use state_processing::per_slot_processing;
use types::{ use types::{
test_utils::generate_deterministic_keypair, AggregateSignature, BitList, EthSpec, Hash256, test_utils::generate_deterministic_keypair, AggregateSignature, BitList, EthSpec, Hash256,
@ -56,7 +56,7 @@ fn attestation_validity() {
.expect("should get at least one attestation"); .expect("should get at least one attestation");
assert_eq!( assert_eq!(
chain.process_attestation(valid_attestation.clone()), chain.process_attestation(valid_attestation.clone(), AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::Processed), Ok(AttestationProcessingOutcome::Processed),
"should accept valid attestation" "should accept valid attestation"
); );
@ -71,7 +71,7 @@ fn attestation_validity() {
assert_eq!( assert_eq!(
harness harness
.chain .chain
.process_attestation(epoch_mismatch_attestation), .process_attestation(epoch_mismatch_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::BadTargetEpoch), Ok(AttestationProcessingOutcome::BadTargetEpoch),
"should not accept attestation where the slot is not in the same epoch as the target" "should not accept attestation where the slot is not in the same epoch as the target"
); );
@ -85,7 +85,9 @@ fn attestation_validity() {
early_attestation.data.slot = (current_epoch + 1).start_slot(MainnetEthSpec::slots_per_epoch()); early_attestation.data.slot = (current_epoch + 1).start_slot(MainnetEthSpec::slots_per_epoch());
assert_eq!( assert_eq!(
harness.chain.process_attestation(early_attestation), harness
.chain
.process_attestation(early_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::FutureEpoch { Ok(AttestationProcessingOutcome::FutureEpoch {
attestation_epoch: current_epoch + 1, attestation_epoch: current_epoch + 1,
current_epoch current_epoch
@ -118,7 +120,9 @@ fn attestation_validity() {
.expect("should get at least one late attestation"); .expect("should get at least one late attestation");
assert_eq!( assert_eq!(
harness.chain.process_attestation(late_attestation), harness
.chain
.process_attestation(late_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::PastEpoch { Ok(AttestationProcessingOutcome::PastEpoch {
attestation_epoch: current_epoch - 2, attestation_epoch: current_epoch - 2,
current_epoch current_epoch
@ -134,7 +138,9 @@ fn attestation_validity() {
bad_target_attestation.data.target.root = Hash256::from_low_u64_be(42); bad_target_attestation.data.target.root = Hash256::from_low_u64_be(42);
assert_eq!( assert_eq!(
harness.chain.process_attestation(bad_target_attestation), harness
.chain
.process_attestation(bad_target_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::UnknownTargetRoot( Ok(AttestationProcessingOutcome::UnknownTargetRoot(
Hash256::from_low_u64_be(42) Hash256::from_low_u64_be(42)
)), )),
@ -149,7 +155,9 @@ fn attestation_validity() {
future_block_attestation.data.slot -= 1; future_block_attestation.data.slot -= 1;
assert_eq!( assert_eq!(
harness.chain.process_attestation(future_block_attestation), harness
.chain
.process_attestation(future_block_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::AttestsToFutureBlock { Ok(AttestationProcessingOutcome::AttestsToFutureBlock {
block: current_slot, block: current_slot,
attestation: current_slot - 1 attestation: current_slot - 1
@ -165,7 +173,9 @@ fn attestation_validity() {
bad_head_attestation.data.beacon_block_root = Hash256::from_low_u64_be(42); bad_head_attestation.data.beacon_block_root = Hash256::from_low_u64_be(42);
assert_eq!( assert_eq!(
harness.chain.process_attestation(bad_head_attestation), harness
.chain
.process_attestation(bad_head_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::UnknownHeadBlock { Ok(AttestationProcessingOutcome::UnknownHeadBlock {
beacon_block_root: Hash256::from_low_u64_be(42) beacon_block_root: Hash256::from_low_u64_be(42)
}), }),
@ -183,7 +193,9 @@ fn attestation_validity() {
bad_signature_attestation.signature = agg_sig; bad_signature_attestation.signature = agg_sig;
assert_eq!( assert_eq!(
harness.chain.process_attestation(bad_signature_attestation), harness
.chain
.process_attestation(bad_signature_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::InvalidSignature), Ok(AttestationProcessingOutcome::InvalidSignature),
"should not accept bad_signature attestation" "should not accept bad_signature attestation"
); );
@ -199,7 +211,7 @@ fn attestation_validity() {
assert_eq!( assert_eq!(
harness harness
.chain .chain
.process_attestation(empty_bitfield_attestation), .process_attestation(empty_bitfield_attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::EmptyAggregationBitfield), Ok(AttestationProcessingOutcome::EmptyAggregationBitfield),
"should not accept empty_bitfield attestation" "should not accept empty_bitfield attestation"
); );
@ -247,7 +259,9 @@ fn attestation_that_skips_epochs() {
.expect("should get at least one attestation"); .expect("should get at least one attestation");
assert_eq!( assert_eq!(
harness.chain.process_attestation(attestation), harness
.chain
.process_attestation(attestation, AttestationType::Aggregated),
Ok(AttestationProcessingOutcome::Processed), Ok(AttestationProcessingOutcome::Processed),
"should process attestation that skips slots" "should process attestation that skips slots"
); );

View File

@ -0,0 +1,588 @@
#![cfg(not(debug_assertions))]
#[macro_use]
extern crate lazy_static;
use beacon_chain::{
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, HarnessType},
BeaconSnapshot, BlockError,
};
use types::{
test_utils::generate_deterministic_keypair, AggregateSignature, AttestationData,
AttesterSlashing, Checkpoint, Deposit, DepositData, Epoch, EthSpec, Hash256,
IndexedAttestation, Keypair, MainnetEthSpec, ProposerSlashing, Signature, SignedBeaconBlock,
SignedBeaconBlockHeader, SignedVoluntaryExit, Slot, VoluntaryExit, DEPOSIT_TREE_DEPTH,
};
type E = MainnetEthSpec;
// Should ideally be divisible by 3.
pub const VALIDATOR_COUNT: usize = 24;
pub const CHAIN_SEGMENT_LENGTH: usize = 64 * 5;
lazy_static! {
/// A cached set of keys.
static ref KEYPAIRS: Vec<Keypair> = types::test_utils::generate_deterministic_keypairs(VALIDATOR_COUNT);
/// A cached set of valid blocks
static ref CHAIN_SEGMENT: Vec<BeaconSnapshot<E>> = get_chain_segment();
}
fn get_chain_segment() -> Vec<BeaconSnapshot<E>> {
let harness = get_harness(VALIDATOR_COUNT);
harness.extend_chain(
CHAIN_SEGMENT_LENGTH,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness
.chain
.chain_dump()
.expect("should dump chain")
.into_iter()
.skip(1)
.collect()
}
fn get_harness(validator_count: usize) -> BeaconChainHarness<HarnessType<E>> {
let harness = BeaconChainHarness::new(MainnetEthSpec, KEYPAIRS[0..validator_count].to_vec());
harness.advance_slot();
harness
}
fn chain_segment_blocks() -> Vec<SignedBeaconBlock<E>> {
CHAIN_SEGMENT
.iter()
.map(|snapshot| snapshot.beacon_block.clone())
.collect()
}
fn junk_signature() -> Signature {
let kp = generate_deterministic_keypair(VALIDATOR_COUNT);
let message = &[42, 42];
Signature::new(message, &kp.sk)
}
fn junk_aggregate_signature() -> AggregateSignature {
let mut agg_sig = AggregateSignature::new();
agg_sig.add(&junk_signature());
agg_sig
}
fn update_proposal_signatures(
snapshots: &mut [BeaconSnapshot<E>],
harness: &BeaconChainHarness<HarnessType<E>>,
) {
for snapshot in snapshots {
let spec = &harness.chain.spec;
let slot = snapshot.beacon_block.slot();
let state = &snapshot.beacon_state;
let proposer_index = state
.get_beacon_proposer_index(slot, spec)
.expect("should find proposer index");
let keypair = harness
.keypairs
.get(proposer_index)
.expect("proposer keypair should be available");
snapshot.beacon_block = snapshot.beacon_block.message.clone().sign(
&keypair.sk,
&state.fork,
state.genesis_validators_root,
spec,
);
}
}
fn update_parent_roots(snapshots: &mut [BeaconSnapshot<E>]) {
for i in 0..snapshots.len() {
let root = snapshots[i].beacon_block.canonical_root();
if let Some(child) = snapshots.get_mut(i + 1) {
child.beacon_block.message.parent_root = root
}
}
}
#[test]
fn chain_segment_full_segment() {
let harness = get_harness(VALIDATOR_COUNT);
let blocks = chain_segment_blocks();
harness
.chain
.slot_clock
.set_slot(blocks.last().unwrap().slot().as_u64());
// Sneak in a little check to ensure we can process empty chain segments.
harness
.chain
.process_chain_segment(vec![])
.to_block_error()
.expect("should import empty chain segment");
harness
.chain
.process_chain_segment(blocks.clone())
.to_block_error()
.expect("should import chain segment");
harness.chain.fork_choice().expect("should run fork choice");
assert_eq!(
harness
.chain
.head_info()
.expect("should get harness b head")
.block_root,
blocks.last().unwrap().canonical_root(),
"harness should have last block as head"
);
}
#[test]
fn chain_segment_varying_chunk_size() {
for chunk_size in &[1, 2, 3, 5, 31, 32, 33, 42] {
let harness = get_harness(VALIDATOR_COUNT);
let blocks = chain_segment_blocks();
harness
.chain
.slot_clock
.set_slot(blocks.last().unwrap().slot().as_u64());
for chunk in blocks.chunks(*chunk_size) {
harness
.chain
.process_chain_segment(chunk.to_vec())
.to_block_error()
.expect(&format!(
"should import chain segment of len {}",
chunk_size
));
}
harness.chain.fork_choice().expect("should run fork choice");
assert_eq!(
harness
.chain
.head_info()
.expect("should get harness b head")
.block_root,
blocks.last().unwrap().canonical_root(),
"harness should have last block as head"
);
}
}
#[test]
fn chain_segment_non_linear_parent_roots() {
let harness = get_harness(VALIDATOR_COUNT);
harness
.chain
.slot_clock
.set_slot(CHAIN_SEGMENT.last().unwrap().beacon_block.slot().as_u64());
/*
* Test with a block removed.
*/
let mut blocks = chain_segment_blocks();
blocks.remove(2);
assert_eq!(
harness
.chain
.process_chain_segment(blocks.clone())
.to_block_error(),
Err(BlockError::NonLinearParentRoots),
"should not import chain with missing parent"
);
/*
* Test with a modified parent root.
*/
let mut blocks = chain_segment_blocks();
blocks[3].message.parent_root = Hash256::zero();
assert_eq!(
harness
.chain
.process_chain_segment(blocks.clone())
.to_block_error(),
Err(BlockError::NonLinearParentRoots),
"should not import chain with a broken parent root link"
);
}
#[test]
fn chain_segment_non_linear_slots() {
let harness = get_harness(VALIDATOR_COUNT);
harness
.chain
.slot_clock
.set_slot(CHAIN_SEGMENT.last().unwrap().beacon_block.slot().as_u64());
/*
* Test where a child is lower than the parent.
*/
let mut blocks = chain_segment_blocks();
blocks[3].message.slot = Slot::new(0);
assert_eq!(
harness
.chain
.process_chain_segment(blocks.clone())
.to_block_error(),
Err(BlockError::NonLinearSlots),
"should not import chain with a parent that has a lower slot than its child"
);
/*
* Test where a child is equal to the parent.
*/
let mut blocks = chain_segment_blocks();
blocks[3].message.slot = blocks[2].message.slot;
assert_eq!(
harness
.chain
.process_chain_segment(blocks.clone())
.to_block_error(),
Err(BlockError::NonLinearSlots),
"should not import chain with a parent that has an equal slot to its child"
);
}
#[test]
fn invalid_signatures() {
let mut checked_attestation = false;
for &block_index in &[0, 1, 32, 64, 68 + 1, 129, CHAIN_SEGMENT.len() - 1] {
let harness = get_harness(VALIDATOR_COUNT);
harness
.chain
.slot_clock
.set_slot(CHAIN_SEGMENT.last().unwrap().beacon_block.slot().as_u64());
// Import all the ancestors before the `block_index` block.
let ancestor_blocks = CHAIN_SEGMENT
.iter()
.take(block_index)
.map(|snapshot| snapshot.beacon_block.clone())
.collect();
harness
.chain
.process_chain_segment(ancestor_blocks)
.to_block_error()
.expect("should import all blocks prior to the one being tested");
// For the given snapshots, test the following:
//
// - The `process_chain_segment` function returns `InvalidSignature`.
// - The `process_block` function returns `InvalidSignature` when importing the
// `SignedBeaconBlock` directly.
// - The `verify_block_for_gossip` function does _not_ return an error.
// - The `process_block` function returns `InvalidSignature` when verifying the
// `GossipVerifiedBlock`.
let assert_invalid_signature = |snapshots: &[BeaconSnapshot<E>], item: &str| {
let blocks = snapshots
.iter()
.map(|snapshot| snapshot.beacon_block.clone())
.collect();
// Ensure the block will be rejected if imported in a chain segment.
assert_eq!(
harness.chain.process_chain_segment(blocks).to_block_error(),
Err(BlockError::InvalidSignature),
"should not import chain segment with an invalid {} signature",
item
);
// Ensure the block will be rejected if imported on its own (without gossip checking).
assert_eq!(
harness
.chain
.process_block(snapshots[block_index].beacon_block.clone()),
Err(BlockError::InvalidSignature),
"should not import individual block with an invalid {} signature",
item
);
let gossip_verified = harness
.chain
.verify_block_for_gossip(snapshots[block_index].beacon_block.clone())
.expect("should obtain gossip verified block");
assert_eq!(
harness.chain.process_block(gossip_verified),
Err(BlockError::InvalidSignature),
"should not import gossip verified block with an invalid {} signature",
item
);
};
/*
* Block proposal
*/
let mut snapshots = CHAIN_SEGMENT.clone();
snapshots[block_index].beacon_block.signature = junk_signature();
let blocks = snapshots
.iter()
.map(|snapshot| snapshot.beacon_block.clone())
.collect();
// Ensure the block will be rejected if imported in a chain segment.
assert_eq!(
harness.chain.process_chain_segment(blocks).to_block_error(),
Err(BlockError::InvalidSignature),
"should not import chain segment with an invalid gossip signature",
);
// Ensure the block will be rejected if imported on its own (without gossip checking).
assert_eq!(
harness
.chain
.process_block(snapshots[block_index].beacon_block.clone()),
Err(BlockError::InvalidSignature),
"should not import individual block with an invalid gossip signature",
);
/*
* Randao reveal
*/
let mut snapshots = CHAIN_SEGMENT.clone();
snapshots[block_index]
.beacon_block
.message
.body
.randao_reveal = junk_signature();
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature(&snapshots, "randao");
/*
* Proposer slashing
*/
let mut snapshots = CHAIN_SEGMENT.clone();
let proposer_slashing = ProposerSlashing {
signed_header_1: SignedBeaconBlockHeader {
message: snapshots[block_index].beacon_block.message.block_header(),
signature: junk_signature(),
},
signed_header_2: SignedBeaconBlockHeader {
message: snapshots[block_index].beacon_block.message.block_header(),
signature: junk_signature(),
},
};
snapshots[block_index]
.beacon_block
.message
.body
.proposer_slashings
.push(proposer_slashing)
.expect("should update proposer slashing");
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature(&snapshots, "proposer slashing");
/*
* Attester slashing
*/
let mut snapshots = CHAIN_SEGMENT.clone();
let indexed_attestation = IndexedAttestation {
attesting_indices: vec![0].into(),
data: AttestationData {
slot: Slot::new(0),
index: 0,
beacon_block_root: Hash256::zero(),
source: Checkpoint {
epoch: Epoch::new(0),
root: Hash256::zero(),
},
target: Checkpoint {
epoch: Epoch::new(0),
root: Hash256::zero(),
},
},
signature: junk_aggregate_signature(),
};
let attester_slashing = AttesterSlashing {
attestation_1: indexed_attestation.clone(),
attestation_2: indexed_attestation,
};
snapshots[block_index]
.beacon_block
.message
.body
.attester_slashings
.push(attester_slashing)
.expect("should update attester slashing");
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature(&snapshots, "attester slashing");
/*
* Attestation
*/
let mut snapshots = CHAIN_SEGMENT.clone();
if let Some(attestation) = snapshots[block_index]
.beacon_block
.message
.body
.attestations
.get_mut(0)
{
attestation.signature = junk_aggregate_signature();
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature(&snapshots, "attestation");
checked_attestation = true;
}
/*
* Deposit
*
* Note: an invalid deposit signature is permitted!
*/
let mut snapshots = CHAIN_SEGMENT.clone();
let deposit = Deposit {
proof: vec![Hash256::zero(); DEPOSIT_TREE_DEPTH + 1].into(),
data: DepositData {
pubkey: Keypair::random().pk.into(),
withdrawal_credentials: Hash256::zero(),
amount: 0,
signature: junk_signature().into(),
},
};
snapshots[block_index]
.beacon_block
.message
.body
.deposits
.push(deposit)
.expect("should update deposit");
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
let blocks = snapshots
.iter()
.map(|snapshot| snapshot.beacon_block.clone())
.collect();
assert!(
harness.chain.process_chain_segment(blocks).to_block_error()
!= Err(BlockError::InvalidSignature),
"should not throw an invalid signature error for a bad deposit signature"
);
/*
* Voluntary exit
*/
let mut snapshots = CHAIN_SEGMENT.clone();
let epoch = snapshots[block_index].beacon_state.current_epoch();
snapshots[block_index]
.beacon_block
.message
.body
.voluntary_exits
.push(SignedVoluntaryExit {
message: VoluntaryExit {
epoch,
validator_index: 0,
},
signature: junk_signature(),
})
.expect("should update deposit");
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
assert_invalid_signature(&snapshots, "voluntary exit");
}
assert!(
checked_attestation,
"the test should check an attestation signature"
)
}
fn unwrap_err<T, E>(result: Result<T, E>) -> E {
match result {
Ok(_) => panic!("called unwrap_err on Ok"),
Err(e) => e,
}
}
#[test]
fn gossip_verification() {
let harness = get_harness(VALIDATOR_COUNT);
let block_index = CHAIN_SEGMENT_LENGTH - 2;
harness
.chain
.slot_clock
.set_slot(CHAIN_SEGMENT[block_index].beacon_block.slot().as_u64());
// Import the ancestors prior to the block we're testing.
for snapshot in &CHAIN_SEGMENT[0..block_index] {
let gossip_verified = harness
.chain
.verify_block_for_gossip(snapshot.beacon_block.clone())
.expect("should obtain gossip verified block");
harness
.chain
.process_block(gossip_verified)
.expect("should import valid gossip verified block");
}
/*
* Block with invalid signature
*/
let mut block = CHAIN_SEGMENT[block_index].beacon_block.clone();
block.signature = junk_signature();
assert_eq!(
unwrap_err(harness.chain.verify_block_for_gossip(block)),
BlockError::ProposalSignatureInvalid,
"should not import a block with an invalid proposal signature"
);
/*
* Block from a future slot.
*/
let mut block = CHAIN_SEGMENT[block_index].beacon_block.clone();
let block_slot = block.message.slot + 1;
block.message.slot = block_slot;
assert_eq!(
unwrap_err(harness.chain.verify_block_for_gossip(block)),
BlockError::FutureSlot {
present_slot: block_slot - 1,
block_slot
},
"should not import a block with a future slot"
);
/*
* Block from a finalized slot.
*/
let mut block = CHAIN_SEGMENT[block_index].beacon_block.clone();
let finalized_slot = harness
.chain
.head_info()
.expect("should get head info")
.finalized_checkpoint
.epoch
.start_slot(E::slots_per_epoch());
block.message.slot = finalized_slot;
assert_eq!(
unwrap_err(harness.chain.verify_block_for_gossip(block)),
BlockError::WouldRevertFinalizedSlot {
block_slot: finalized_slot,
finalized_slot
},
"should not import a block with a finalized slot"
);
}

View File

@ -7,7 +7,7 @@ use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, DiskHarnessType, AttestationStrategy, BeaconChainHarness, BlockStrategy, DiskHarnessType,
}; };
use beacon_chain::BeaconSnapshot; use beacon_chain::BeaconSnapshot;
use beacon_chain::{AttestationProcessingOutcome, StateSkipConfig}; use beacon_chain::{AttestationProcessingOutcome, AttestationType, StateSkipConfig};
use rand::Rng; use rand::Rng;
use sloggers::{null::NullLoggerBuilder, Build}; use sloggers::{null::NullLoggerBuilder, Build};
use std::collections::HashMap; use std::collections::HashMap;
@ -312,7 +312,7 @@ fn epoch_boundary_state_attestation_processing() {
.epoch; .epoch;
let res = harness let res = harness
.chain .chain
.process_attestation_internal(attestation.clone()); .process_attestation_internal(attestation.clone(), AttestationType::Aggregated);
let current_epoch = harness.chain.epoch().expect("should get epoch"); let current_epoch = harness.chain.epoch().expect("should get epoch");
let attestation_epoch = attestation.data.target.epoch; let attestation_epoch = attestation.data.target.epoch;

View File

@ -3,13 +3,10 @@
#[macro_use] #[macro_use]
extern crate lazy_static; extern crate lazy_static;
use beacon_chain::AttestationProcessingOutcome; use beacon_chain::test_utils::{
use beacon_chain::{ AttestationStrategy, BeaconChainHarness, BlockStrategy, HarnessType, OP_POOL_DB_KEY,
test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, HarnessType, OP_POOL_DB_KEY,
},
BlockProcessingOutcome,
}; };
use beacon_chain::{AttestationProcessingOutcome, AttestationType};
use operation_pool::PersistedOperationPool; use operation_pool::PersistedOperationPool;
use state_processing::{ use state_processing::{
per_slot_processing, per_slot_processing::Error as SlotProcessingError, EpochProcessingError, per_slot_processing, per_slot_processing::Error as SlotProcessingError, EpochProcessingError,
@ -452,7 +449,9 @@ fn attestations_with_increasing_slots() {
for attestation in attestations { for attestation in attestations {
let attestation_epoch = attestation.data.target.epoch; let attestation_epoch = attestation.data.target.epoch;
let res = harness.chain.process_attestation(attestation); let res = harness
.chain
.process_attestation(attestation, AttestationType::Aggregated);
if attestation_epoch + 1 < current_epoch { if attestation_epoch + 1 < current_epoch {
assert_eq!( assert_eq!(
@ -562,15 +561,13 @@ fn run_skip_slot_test(skip_slots: u64) {
.head() .head()
.expect("should get head") .expect("should get head")
.beacon_block .beacon_block
.clone() .clone(),
), ),
Ok(BlockProcessingOutcome::Processed { Ok(harness_a
block_root: harness_a .chain
.chain .head()
.head() .expect("should get head")
.expect("should get head") .beacon_block_root)
.beacon_block_root
})
); );
harness_b harness_b

View File

@ -1,6 +1,6 @@
[package] [package]
name = "client" name = "client"
version = "0.1.0" version = "0.2.0"
authors = ["Age Manning <Age@AgeManning.com>"] authors = ["Age Manning <Age@AgeManning.com>"]
edition = "2018" edition = "2018"
@ -12,6 +12,7 @@ toml = "^0.5"
beacon_chain = { path = "../beacon_chain" } beacon_chain = { path = "../beacon_chain" }
store = { path = "../store" } store = { path = "../store" }
network = { path = "../network" } network = { path = "../network" }
timer = { path = "../timer" }
eth2-libp2p = { path = "../eth2-libp2p" } eth2-libp2p = { path = "../eth2-libp2p" }
rest_api = { path = "../rest_api" } rest_api = { path = "../rest_api" }
parking_lot = "0.9.0" parking_lot = "0.9.0"
@ -29,7 +30,6 @@ slog = { version = "2.5.2", features = ["max_level_trace"] }
slog-async = "2.3.0" slog-async = "2.3.0"
tokio = "0.1.22" tokio = "0.1.22"
dirs = "2.0.2" dirs = "2.0.2"
exit-future = "0.1.4"
futures = "0.1.29" futures = "0.1.29"
reqwest = "0.9.22" reqwest = "0.9.22"
url = "2.1.0" url = "2.1.0"

View File

@ -12,10 +12,10 @@ use beacon_chain::{
use environment::RuntimeContext; use environment::RuntimeContext;
use eth1::{Config as Eth1Config, Service as Eth1Service}; use eth1::{Config as Eth1Config, Service as Eth1Service};
use eth2_config::Eth2Config; use eth2_config::Eth2Config;
use exit_future::Signal; use eth2_libp2p::NetworkGlobals;
use futures::{future, Future, IntoFuture}; use futures::{future, Future, IntoFuture};
use genesis::{interop_genesis_state, Eth1GenesisService}; use genesis::{interop_genesis_state, Eth1GenesisService};
use network::{NetworkConfig, NetworkMessage, Service as NetworkService}; use network::{NetworkConfig, NetworkMessage, NetworkService};
use slog::info; use slog::info;
use ssz::Decode; use ssz::Decode;
use std::net::SocketAddr; use std::net::SocketAddr;
@ -51,10 +51,10 @@ pub struct ClientBuilder<T: BeaconChainTypes> {
beacon_chain_builder: Option<BeaconChainBuilder<T>>, beacon_chain_builder: Option<BeaconChainBuilder<T>>,
beacon_chain: Option<Arc<BeaconChain<T>>>, beacon_chain: Option<Arc<BeaconChain<T>>>,
eth1_service: Option<Eth1Service>, eth1_service: Option<Eth1Service>,
exit_signals: Vec<Signal>, exit_channels: Vec<tokio::sync::oneshot::Sender<()>>,
event_handler: Option<T::EventHandler>, event_handler: Option<T::EventHandler>,
libp2p_network: Option<Arc<NetworkService<T>>>, network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
libp2p_network_send: Option<UnboundedSender<NetworkMessage>>, network_send: Option<UnboundedSender<NetworkMessage<T::EthSpec>>>,
http_listen_addr: Option<SocketAddr>, http_listen_addr: Option<SocketAddr>,
websocket_listen_addr: Option<SocketAddr>, websocket_listen_addr: Option<SocketAddr>,
eth_spec_instance: T::EthSpec, eth_spec_instance: T::EthSpec,
@ -85,10 +85,10 @@ where
beacon_chain_builder: None, beacon_chain_builder: None,
beacon_chain: None, beacon_chain: None,
eth1_service: None, eth1_service: None,
exit_signals: vec![], exit_channels: vec![],
event_handler: None, event_handler: None,
libp2p_network: None, network_globals: None,
libp2p_network_send: None, network_send: None,
http_listen_addr: None, http_listen_addr: None,
websocket_listen_addr: None, websocket_listen_addr: None,
eth_spec_instance, eth_spec_instance,
@ -120,6 +120,7 @@ where
let runtime_context = self.runtime_context.clone(); let runtime_context = self.runtime_context.clone();
let eth_spec_instance = self.eth_spec_instance.clone(); let eth_spec_instance = self.eth_spec_instance.clone();
let data_dir = config.data_dir.clone(); let data_dir = config.data_dir.clone();
let disabled_forks = config.disabled_forks.clone();
future::ok(()) future::ok(())
.and_then(move |()| { .and_then(move |()| {
@ -141,7 +142,8 @@ where
.store(store) .store(store)
.store_migrator(store_migrator) .store_migrator(store_migrator)
.data_dir(data_dir) .data_dir(data_dir)
.custom_spec(spec.clone()); .custom_spec(spec.clone())
.disabled_forks(disabled_forks);
Ok((builder, spec, context)) Ok((builder, spec, context))
}) })
@ -237,24 +239,55 @@ where
}) })
} }
/// Immediately starts the libp2p networking stack. /// Immediately starts the networking stack.
pub fn libp2p_network(mut self, config: &NetworkConfig) -> Result<Self, String> { pub fn network(mut self, config: &NetworkConfig) -> Result<Self, String> {
let beacon_chain = self let beacon_chain = self
.beacon_chain .beacon_chain
.clone() .clone()
.ok_or_else(|| "libp2p_network requires a beacon chain")?; .ok_or_else(|| "network requires a beacon chain")?;
let context = self let context = self
.runtime_context .runtime_context
.as_ref() .as_ref()
.ok_or_else(|| "libp2p_network requires a runtime_context")? .ok_or_else(|| "network requires a runtime_context")?
.service_context("network".into()); .service_context("network".into());
let (network, network_send) = let (network_globals, network_send, network_exit) =
NetworkService::new(beacon_chain, config, &context.executor, context.log) NetworkService::start(beacon_chain, config, &context.executor, context.log)
.map_err(|e| format!("Failed to start libp2p network: {:?}", e))?; .map_err(|e| format!("Failed to start network: {:?}", e))?;
self.libp2p_network = Some(network); self.network_globals = Some(network_globals);
self.libp2p_network_send = Some(network_send); self.network_send = Some(network_send);
self.exit_channels.push(network_exit);
Ok(self)
}
/// Immediately starts the timer service.
fn timer(mut self) -> Result<Self, String> {
let context = self
.runtime_context
.as_ref()
.ok_or_else(|| "node timer requires a runtime_context")?
.service_context("node_timer".into());
let beacon_chain = self
.beacon_chain
.clone()
.ok_or_else(|| "node timer requires a beacon chain")?;
let milliseconds_per_slot = self
.chain_spec
.as_ref()
.ok_or_else(|| "node timer requires a chain spec".to_string())?
.milliseconds_per_slot;
let timer_exit = timer::spawn(
&context.executor,
beacon_chain,
milliseconds_per_slot,
context.log,
)
.map_err(|e| format!("Unable to start node timer: {}", e))?;
self.exit_channels.push(timer_exit);
Ok(self) Ok(self)
} }
@ -274,21 +307,21 @@ where
.as_ref() .as_ref()
.ok_or_else(|| "http_server requires a runtime_context")? .ok_or_else(|| "http_server requires a runtime_context")?
.service_context("http".into()); .service_context("http".into());
let network = self let network_globals = self
.libp2p_network .network_globals
.clone() .clone()
.ok_or_else(|| "http_server requires a libp2p network")?; .ok_or_else(|| "http_server requires a libp2p network")?;
let network_send = self let network_send = self
.libp2p_network_send .network_send
.clone() .clone()
.ok_or_else(|| "http_server requires a libp2p network sender")?; .ok_or_else(|| "http_server requires a libp2p network sender")?;
let network_info = rest_api::NetworkInfo { let network_info = rest_api::NetworkInfo {
network_service: network, network_globals,
network_chan: network_send, network_chan: network_send,
}; };
let (exit_signal, listening_addr) = rest_api::start_server( let (exit_channel, listening_addr) = rest_api::start_server(
&client_config.rest_api, &client_config.rest_api,
&context.executor, &context.executor,
beacon_chain, beacon_chain,
@ -304,7 +337,7 @@ where
) )
.map_err(|e| format!("Failed to start HTTP API: {:?}", e))?; .map_err(|e| format!("Failed to start HTTP API: {:?}", e))?;
self.exit_signals.push(exit_signal); self.exit_channels.push(exit_channel);
self.http_listen_addr = Some(listening_addr); self.http_listen_addr = Some(listening_addr);
Ok(self) Ok(self)
@ -321,8 +354,8 @@ where
.beacon_chain .beacon_chain
.clone() .clone()
.ok_or_else(|| "slot_notifier requires a beacon chain")?; .ok_or_else(|| "slot_notifier requires a beacon chain")?;
let network = self let network_globals = self
.libp2p_network .network_globals
.clone() .clone()
.ok_or_else(|| "slot_notifier requires a libp2p network")?; .ok_or_else(|| "slot_notifier requires a libp2p network")?;
let milliseconds_per_slot = self let milliseconds_per_slot = self
@ -331,10 +364,15 @@ where
.ok_or_else(|| "slot_notifier requires a chain spec".to_string())? .ok_or_else(|| "slot_notifier requires a chain spec".to_string())?
.milliseconds_per_slot; .milliseconds_per_slot;
let exit_signal = spawn_notifier(context, beacon_chain, network, milliseconds_per_slot) let exit_channel = spawn_notifier(
.map_err(|e| format!("Unable to start slot notifier: {}", e))?; context,
beacon_chain,
network_globals,
milliseconds_per_slot,
)
.map_err(|e| format!("Unable to start slot notifier: {}", e))?;
self.exit_signals.push(exit_signal); self.exit_channels.push(exit_channel);
Ok(self) Ok(self)
} }
@ -349,10 +387,10 @@ where
{ {
Client { Client {
beacon_chain: self.beacon_chain, beacon_chain: self.beacon_chain,
libp2p_network: self.libp2p_network, network_globals: self.network_globals,
http_listen_addr: self.http_listen_addr, http_listen_addr: self.http_listen_addr,
websocket_listen_addr: self.websocket_listen_addr, websocket_listen_addr: self.websocket_listen_addr,
_exit_signals: self.exit_signals, _exit_channels: self.exit_channels,
} }
} }
} }
@ -392,7 +430,8 @@ where
self.beacon_chain_builder = None; self.beacon_chain_builder = None;
self.event_handler = None; self.event_handler = None;
Ok(self) // a beacon chain requires a timer
self.timer()
} }
} }
@ -422,7 +461,7 @@ where
.ok_or_else(|| "websocket_event_handler requires a runtime_context")? .ok_or_else(|| "websocket_event_handler requires a runtime_context")?
.service_context("ws".into()); .service_context("ws".into());
let (sender, exit_signal, listening_addr): ( let (sender, exit_channel, listening_addr): (
WebSocketSender<TEthSpec>, WebSocketSender<TEthSpec>,
Option<_>, Option<_>,
Option<_>, Option<_>,
@ -434,8 +473,8 @@ where
(WebSocketSender::dummy(), None, None) (WebSocketSender::dummy(), None, None)
}; };
if let Some(signal) = exit_signal { if let Some(channel) = exit_channel {
self.exit_signals.push(signal); self.exit_channels.push(channel);
} }
self.event_handler = Some(sender); self.event_handler = Some(sender);
self.websocket_listen_addr = listening_addr; self.websocket_listen_addr = listening_addr;
@ -641,8 +680,8 @@ where
self.eth1_service = None; self.eth1_service = None;
let exit = { let exit = {
let (tx, rx) = exit_future::signal(); let (tx, rx) = tokio::sync::oneshot::channel();
self.exit_signals.push(tx); self.exit_channels.push(tx);
rx rx
}; };

View File

@ -53,6 +53,8 @@ pub struct Config {
/// This is the method used for the 2019 client interop in Canada. /// This is the method used for the 2019 client interop in Canada.
pub dummy_eth1_backend: bool, pub dummy_eth1_backend: bool,
pub sync_eth1_chain: bool, pub sync_eth1_chain: bool,
/// A list of hard-coded forks that will be disabled.
pub disabled_forks: Vec<String>,
#[serde(skip)] #[serde(skip)]
/// The `genesis` field is not serialized or deserialized by `serde` to ensure it is defined /// The `genesis` field is not serialized or deserialized by `serde` to ensure it is defined
/// via the CLI at runtime, instead of from a configuration file saved to disk. /// via the CLI at runtime, instead of from a configuration file saved to disk.
@ -81,6 +83,7 @@ impl Default for Config {
dummy_eth1_backend: false, dummy_eth1_backend: false,
sync_eth1_chain: false, sync_eth1_chain: false,
eth1: <_>::default(), eth1: <_>::default(),
disabled_forks: Vec::new(),
} }
} }
} }

View File

@ -8,9 +8,7 @@ pub mod builder;
pub mod error; pub mod error;
use beacon_chain::BeaconChain; use beacon_chain::BeaconChain;
use eth2_libp2p::{Enr, Multiaddr}; use eth2_libp2p::{Enr, Multiaddr, NetworkGlobals};
use exit_future::Signal;
use network::Service as NetworkService;
use std::net::SocketAddr; use std::net::SocketAddr;
use std::sync::Arc; use std::sync::Arc;
@ -24,11 +22,11 @@ pub use eth2_config::Eth2Config;
/// Holds references to running services, cleanly shutting them down when dropped. /// Holds references to running services, cleanly shutting them down when dropped.
pub struct Client<T: BeaconChainTypes> { pub struct Client<T: BeaconChainTypes> {
beacon_chain: Option<Arc<BeaconChain<T>>>, beacon_chain: Option<Arc<BeaconChain<T>>>,
libp2p_network: Option<Arc<NetworkService<T>>>, network_globals: Option<Arc<NetworkGlobals<T::EthSpec>>>,
http_listen_addr: Option<SocketAddr>, http_listen_addr: Option<SocketAddr>,
websocket_listen_addr: Option<SocketAddr>, websocket_listen_addr: Option<SocketAddr>,
/// Exit signals will "fire" when dropped, causing each service to exit gracefully. /// Exit channels will complete/error when dropped, causing each service to exit gracefully.
_exit_signals: Vec<Signal>, _exit_channels: Vec<tokio::sync::oneshot::Sender<()>>,
} }
impl<T: BeaconChainTypes> Client<T> { impl<T: BeaconChainTypes> Client<T> {
@ -49,16 +47,16 @@ impl<T: BeaconChainTypes> Client<T> {
/// Returns the port of the client's libp2p stack, if it was started. /// Returns the port of the client's libp2p stack, if it was started.
pub fn libp2p_listen_port(&self) -> Option<u16> { pub fn libp2p_listen_port(&self) -> Option<u16> {
self.libp2p_network.as_ref().map(|n| n.listen_port()) self.network_globals.as_ref().map(|n| n.listen_port_tcp())
} }
/// Returns the list of libp2p addresses the client is listening to. /// Returns the list of libp2p addresses the client is listening to.
pub fn libp2p_listen_addresses(&self) -> Option<Vec<Multiaddr>> { pub fn libp2p_listen_addresses(&self) -> Option<Vec<Multiaddr>> {
self.libp2p_network.as_ref().map(|n| n.listen_multiaddrs()) self.network_globals.as_ref().map(|n| n.listen_multiaddrs())
} }
/// Returns the local libp2p ENR of this node, for network discovery. /// Returns the local libp2p ENR of this node, for network discovery.
pub fn enr(&self) -> Option<Enr> { pub fn enr(&self) -> Option<Enr> {
self.libp2p_network.as_ref()?.local_enr() self.network_globals.as_ref().map(|n| n.local_enr())
} }
} }

View File

@ -1,9 +1,8 @@
use crate::metrics; use crate::metrics;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use environment::RuntimeContext; use environment::RuntimeContext;
use exit_future::Signal; use eth2_libp2p::NetworkGlobals;
use futures::{Future, Stream}; use futures::{Future, Stream};
use network::Service as NetworkService;
use parking_lot::Mutex; use parking_lot::Mutex;
use slog::{debug, error, info, warn}; use slog::{debug, error, info, warn};
use slot_clock::SlotClock; use slot_clock::SlotClock;
@ -30,9 +29,9 @@ const SPEEDO_OBSERVATIONS: usize = 4;
pub fn spawn_notifier<T: BeaconChainTypes>( pub fn spawn_notifier<T: BeaconChainTypes>(
context: RuntimeContext<T::EthSpec>, context: RuntimeContext<T::EthSpec>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
milliseconds_per_slot: u64, milliseconds_per_slot: u64,
) -> Result<Signal, String> { ) -> Result<tokio::sync::oneshot::Sender<()>, String> {
let log_1 = context.log.clone(); let log_1 = context.log.clone();
let log_2 = context.log.clone(); let log_2 = context.log.clone();
let log_3 = context.log.clone(); let log_3 = context.log.clone();
@ -59,6 +58,7 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
let log = log_2.clone(); let log = log_2.clone();
let connected_peer_count = network.connected_peers(); let connected_peer_count = network.connected_peers();
let sync_state = network.sync_state();
let head_info = beacon_chain.head_info() let head_info = beacon_chain.head_info()
.map_err(|e| error!( .map_err(|e| error!(
@ -68,7 +68,6 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
))?; ))?;
let head_slot = head_info.slot; let head_slot = head_info.slot;
let head_epoch = head_slot.epoch(T::EthSpec::slots_per_epoch());
let current_slot = beacon_chain.slot().map_err(|e| { let current_slot = beacon_chain.slot().map_err(|e| {
error!( error!(
log, log,
@ -102,15 +101,17 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
"head_block" => format!("{}", head_root), "head_block" => format!("{}", head_root),
"head_slot" => head_slot, "head_slot" => head_slot,
"current_slot" => current_slot, "current_slot" => current_slot,
"sync_state" =>format!("{}", sync_state)
); );
if head_epoch + 1 < current_epoch {
// Log if we are syncing
if sync_state.is_syncing() {
let distance = format!( let distance = format!(
"{} slots ({})", "{} slots ({})",
head_distance.as_u64(), head_distance.as_u64(),
slot_distance_pretty(head_distance, slot_duration) slot_distance_pretty(head_distance, slot_duration)
); );
info!( info!(
log, log,
"Syncing"; "Syncing";
@ -119,15 +120,21 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
"speed" => sync_speed_pretty(speedo.slots_per_second()), "speed" => sync_speed_pretty(speedo.slots_per_second()),
"est_time" => estimated_time_pretty(speedo.estimated_time_till_slot(current_slot)), "est_time" => estimated_time_pretty(speedo.estimated_time_till_slot(current_slot)),
); );
} else {
return Ok(()); if sync_state.is_synced() {
};
macro_rules! not_quite_synced_log {
($message: expr) => {
info!( info!(
log_2, log_2,
$message; "Synced";
"peers" => peer_count_pretty(connected_peer_count),
"finalized_root" => format!("{}", finalized_root),
"finalized_epoch" => finalized_epoch,
"epoch" => current_epoch,
"slot" => current_slot,
);
} else {
info!(
log_2,
"Searching for peers";
"peers" => peer_count_pretty(connected_peer_count), "peers" => peer_count_pretty(connected_peer_count),
"finalized_root" => format!("{}", finalized_root), "finalized_root" => format!("{}", finalized_root),
"finalized_epoch" => finalized_epoch, "finalized_epoch" => finalized_epoch,
@ -136,23 +143,6 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
); );
} }
} }
if head_epoch + 1 == current_epoch {
not_quite_synced_log!("Synced to previous epoch")
} else if head_slot != current_slot {
not_quite_synced_log!("Synced to current epoch")
} else {
info!(
log_2,
"Synced";
"peers" => peer_count_pretty(connected_peer_count),
"finalized_root" => format!("{}", finalized_root),
"finalized_epoch" => finalized_epoch,
"epoch" => current_epoch,
"slot" => current_slot,
);
};
Ok(()) Ok(())
}) })
.then(move |result| { .then(move |result| {
@ -167,10 +157,11 @@ pub fn spawn_notifier<T: BeaconChainTypes>(
Ok(()) Ok(())
} } }); } } });
let (exit_signal, exit) = exit_future::signal(); let (exit_signal, exit) = tokio::sync::oneshot::channel();
context context
.executor .executor
.spawn(exit.until(interval_future).map(|_| ())); .spawn(interval_future.select(exit).map(|_| ()).map_err(|_| ()));
Ok(exit_signal) Ok(exit_signal)
} }

View File

@ -1,6 +1,6 @@
[package] [package]
name = "eth1" name = "eth1"
version = "0.1.0" version = "0.2.0"
authors = ["Paul Hauner <paul@paulhauner.com>"] authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018" edition = "2018"
@ -8,7 +8,7 @@ edition = "2018"
eth1_test_rig = { path = "../../tests/eth1_test_rig" } eth1_test_rig = { path = "../../tests/eth1_test_rig" }
environment = { path = "../../lighthouse/environment" } environment = { path = "../../lighthouse/environment" }
toml = "^0.5" toml = "^0.5"
web3 = "0.8.0" web3 = "0.10.0"
[dependencies] [dependencies]
reqwest = "0.9" reqwest = "0.9"
@ -26,7 +26,6 @@ parking_lot = "0.7"
slog = "^2.2.3" slog = "^2.2.3"
tokio = "0.1.22" tokio = "0.1.22"
state_processing = { path = "../../eth2/state_processing" } state_processing = { path = "../../eth2/state_processing" }
exit-future = "0.1.4"
libflate = "0.1" libflate = "0.1"
lighthouse_metrics = { path = "../../eth2/utils/lighthouse_metrics"} lighthouse_metrics = { path = "../../eth2/utils/lighthouse_metrics"}
lazy_static = "1.4.0" lazy_static = "1.4.0"

View File

@ -6,7 +6,6 @@ use crate::{
inner::{DepositUpdater, Inner}, inner::{DepositUpdater, Inner},
DepositLog, DepositLog,
}; };
use exit_future::Exit;
use futures::{ use futures::{
future::{loop_fn, Loop}, future::{loop_fn, Loop},
stream, Future, Stream, stream, Future, Stream,
@ -314,7 +313,10 @@ impl Service {
/// - Err(_) if there is an error. /// - Err(_) if there is an error.
/// ///
/// Emits logs for debugging and errors. /// Emits logs for debugging and errors.
pub fn auto_update(&self, exit: Exit) -> impl Future<Item = (), Error = ()> { pub fn auto_update(
&self,
exit: tokio::sync::oneshot::Receiver<()>,
) -> impl Future<Item = (), Error = ()> {
let service = self.clone(); let service = self.clone();
let log = self.log.clone(); let log = self.log.clone();
let update_interval = Duration::from_millis(self.config().auto_update_interval_millis); let update_interval = Duration::from_millis(self.config().auto_update_interval_millis);
@ -360,7 +362,7 @@ impl Service {
}) })
}); });
exit.until(loop_future).map(|_: Option<()>| ()) loop_future.select(exit).map(|_| ()).map_err(|_| ())
} }
/// Contacts the remote eth1 node and attempts to import deposit logs up to the configured /// Contacts the remote eth1 node and attempts to import deposit logs up to the configured

View File

@ -1,6 +1,6 @@
[package] [package]
name = "eth2-libp2p" name = "eth2-libp2p"
version = "0.1.0" version = "0.2.0"
authors = ["Age Manning <Age@AgeManning.com>"] authors = ["Age Manning <Age@AgeManning.com>"]
edition = "2018" edition = "2018"
@ -8,10 +8,11 @@ edition = "2018"
hex = "0.3" hex = "0.3"
# rust-libp2p is presently being sourced from a Sigma Prime fork of the # rust-libp2p is presently being sourced from a Sigma Prime fork of the
# `libp2p/rust-libp2p` repository. # `libp2p/rust-libp2p` repository.
libp2p = { git = "https://github.com/SigP/rust-libp2p", rev = "49c95c4c4242f1c9f08558a3daac5e9ecac290d5" } libp2p = { git = "https://github.com/SigP/rust-libp2p", rev = "37b7e9349cf3e724da02bbd4b5dd6c054c2d56d3" }
enr = { git = "https://github.com/SigP/rust-libp2p/", rev = "49c95c4c4242f1c9f08558a3daac5e9ecac290d5", features = ["serde"] }
types = { path = "../../eth2/types" } types = { path = "../../eth2/types" }
serde = "1.0.102" hashmap_delay = { path = "../../eth2/utils/hashmap_delay" }
eth2_ssz_types = { path = "../../eth2/utils/ssz_types" }
serde = { version = "1.0.102", features = ["derive"] }
serde_derive = "1.0.102" serde_derive = "1.0.102"
eth2_ssz = "0.1.2" eth2_ssz = "0.1.2"
eth2_ssz_derive = "0.1.0" eth2_ssz_derive = "0.1.0"
@ -31,6 +32,8 @@ lru = "0.4.3"
parking_lot = "0.9.0" parking_lot = "0.9.0"
sha2 = "0.8.0" sha2 = "0.8.0"
base64 = "0.11.0" base64 = "0.11.0"
snap = "1"
void = "1.0.2"
[dev-dependencies] [dev-dependencies]
slog-stdlog = "4.0.0" slog-stdlog = "4.0.0"

View File

@ -1,228 +1,207 @@
use crate::discovery::Discovery; use crate::discovery::{enr::Eth2Enr, Discovery};
use crate::rpc::{RPCEvent, RPCMessage, RPC}; use crate::peer_manager::{PeerManager, PeerManagerEvent};
use crate::{error, GossipTopic, NetworkConfig, NetworkGlobals, Topic, TopicHash}; use crate::rpc::*;
use enr::Enr; use crate::types::{GossipEncoding, GossipKind, GossipTopic};
use crate::{error, Enr, NetworkConfig, NetworkGlobals, PubsubMessage, TopicHash};
use futures::prelude::*; use futures::prelude::*;
use libp2p::{ use libp2p::{
core::identity::Keypair, core::{identity::Keypair, ConnectedPoint},
discv5::Discv5Event, discv5::Discv5Event,
gossipsub::{Gossipsub, GossipsubEvent, MessageId}, gossipsub::{Gossipsub, GossipsubEvent, MessageId},
identify::{Identify, IdentifyEvent}, identify::{Identify, IdentifyEvent},
ping::{Ping, PingConfig, PingEvent},
swarm::{NetworkBehaviourAction, NetworkBehaviourEventProcess}, swarm::{NetworkBehaviourAction, NetworkBehaviourEventProcess},
tokio_io::{AsyncRead, AsyncWrite}, tokio_io::{AsyncRead, AsyncWrite},
NetworkBehaviour, PeerId, NetworkBehaviour, PeerId,
}; };
use lru::LruCache; use lru::LruCache;
use slog::{debug, o}; use slog::{crit, debug, o, warn};
use std::num::NonZeroU32; use std::marker::PhantomData;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use types::{EnrForkId, EthSpec, SubnetId};
const MAX_IDENTIFY_ADDRESSES: usize = 20; const MAX_IDENTIFY_ADDRESSES: usize = 10;
/// Builds the network behaviour that manages the core protocols of eth2. /// Builds the network behaviour that manages the core protocols of eth2.
/// This core behaviour is managed by `Behaviour` which adds peer management to all core /// This core behaviour is managed by `Behaviour` which adds peer management to all core
/// behaviours. /// behaviours.
#[derive(NetworkBehaviour)] #[derive(NetworkBehaviour)]
#[behaviour(out_event = "BehaviourEvent", poll_method = "poll")] #[behaviour(out_event = "BehaviourEvent<TSpec>", poll_method = "poll")]
pub struct Behaviour<TSubstream: AsyncRead + AsyncWrite> { pub struct Behaviour<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec> {
/// The routing pub-sub mechanism for eth2. /// The routing pub-sub mechanism for eth2.
gossipsub: Gossipsub<TSubstream>, gossipsub: Gossipsub<TSubstream>,
/// The Eth2 RPC specified in the wire-0 protocol. /// The Eth2 RPC specified in the wire-0 protocol.
eth2_rpc: RPC<TSubstream>, eth2_rpc: RPC<TSubstream, TSpec>,
/// Keep regular connection to peers and disconnect if absent. /// Keep regular connection to peers and disconnect if absent.
// TODO: Remove Libp2p ping in favour of discv5 ping.
ping: Ping<TSubstream>,
// TODO: Using id for initial interop. This will be removed by mainnet. // TODO: Using id for initial interop. This will be removed by mainnet.
/// Provides IP addresses and peer information. /// Provides IP addresses and peer information.
identify: Identify<TSubstream>, identify: Identify<TSubstream>,
/// Discovery behaviour. /// Discovery behaviour.
discovery: Discovery<TSubstream>, discovery: Discovery<TSubstream, TSpec>,
/// The peer manager that keeps track of peer's reputation and status.
#[behaviour(ignore)]
peer_manager: PeerManager<TSpec>,
/// The events generated by this behaviour to be consumed in the swarm poll. /// The events generated by this behaviour to be consumed in the swarm poll.
#[behaviour(ignore)] #[behaviour(ignore)]
events: Vec<BehaviourEvent>, events: Vec<BehaviourEvent<TSpec>>,
/// The current meta data of the node, so respond to pings and get metadata
#[behaviour(ignore)]
meta_data: MetaData<TSpec>,
/// A cache of recently seen gossip messages. This is used to filter out any possible /// A cache of recently seen gossip messages. This is used to filter out any possible
/// duplicates that may still be seen over gossipsub. /// duplicates that may still be seen over gossipsub.
#[behaviour(ignore)] #[behaviour(ignore)]
// TODO: Remove this
seen_gossip_messages: LruCache<MessageId, ()>, seen_gossip_messages: LruCache<MessageId, ()>,
/// A collections of variables accessible outside the network service.
#[behaviour(ignore)]
network_globals: Arc<NetworkGlobals<TSpec>>,
#[behaviour(ignore)]
/// Keeps track of the current EnrForkId for upgrading gossipsub topics.
// NOTE: This can be accessed via the network_globals ENR. However we keep it here for quick
// lookups for every gossipsub message send.
enr_fork_id: EnrForkId,
#[behaviour(ignore)] #[behaviour(ignore)]
/// Logger for behaviour actions. /// Logger for behaviour actions.
log: slog::Logger, log: slog::Logger,
} }
impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> { /// Implements the combined behaviour for the libp2p service.
impl<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec> Behaviour<TSubstream, TSpec> {
pub fn new( pub fn new(
local_key: &Keypair, local_key: &Keypair,
net_conf: &NetworkConfig, net_conf: &NetworkConfig,
network_globals: Arc<NetworkGlobals>, network_globals: Arc<NetworkGlobals<TSpec>>,
log: &slog::Logger, log: &slog::Logger,
) -> error::Result<Self> { ) -> error::Result<Self> {
let local_peer_id = local_key.public().into_peer_id(); let local_peer_id = local_key.public().into_peer_id();
let behaviour_log = log.new(o!()); let behaviour_log = log.new(o!());
let ping_config = PingConfig::new()
.with_timeout(Duration::from_secs(30))
.with_interval(Duration::from_secs(20))
.with_max_failures(NonZeroU32::new(2).expect("2 != 0"))
.with_keep_alive(false);
let identify = Identify::new( let identify = Identify::new(
"lighthouse/libp2p".into(), "lighthouse/libp2p".into(),
version::version(), version::version(),
local_key.public(), local_key.public(),
); );
let enr_fork_id = network_globals
.local_enr
.read()
.eth2()
.expect("Local ENR must have a fork id");
let attnets = network_globals
.local_enr
.read()
.bitfield::<TSpec>()
.expect("Local ENR must have subnet bitfield");
let meta_data = MetaData {
seq_number: 1,
attnets,
};
Ok(Behaviour { Ok(Behaviour {
eth2_rpc: RPC::new(log.clone()), eth2_rpc: RPC::new(log.clone()),
gossipsub: Gossipsub::new(local_peer_id, net_conf.gs_config.clone()), gossipsub: Gossipsub::new(local_peer_id, net_conf.gs_config.clone()),
discovery: Discovery::new(local_key, net_conf, network_globals, log)?, discovery: Discovery::new(local_key, net_conf, network_globals.clone(), log)?,
ping: Ping::new(ping_config),
identify, identify,
peer_manager: PeerManager::new(network_globals.clone(), log),
events: Vec::new(), events: Vec::new(),
seen_gossip_messages: LruCache::new(100_000), seen_gossip_messages: LruCache::new(100_000),
meta_data,
network_globals,
enr_fork_id,
log: behaviour_log, log: behaviour_log,
}) })
} }
pub fn discovery(&self) -> &Discovery<TSubstream> { /// Obtain a reference to the discovery protocol.
pub fn discovery(&self) -> &Discovery<TSubstream, TSpec> {
&self.discovery &self.discovery
} }
/// Obtain a reference to the gossipsub protocol.
pub fn gs(&self) -> &Gossipsub<TSubstream> { pub fn gs(&self) -> &Gossipsub<TSubstream> {
&self.gossipsub &self.gossipsub
} }
}
// Implement the NetworkBehaviourEventProcess trait so that we can derive NetworkBehaviour for Behaviour
impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<GossipsubEvent>
for Behaviour<TSubstream>
{
fn inject_event(&mut self, event: GossipsubEvent) {
match event {
GossipsubEvent::Message(propagation_source, id, gs_msg) => {
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.
if self.seen_gossip_messages.put(id.clone(), ()).is_none() {
// if this message isn't a duplicate, notify the network
self.events.push(BehaviourEvent::GossipMessage {
id,
source: propagation_source,
topics: gs_msg.topics,
message: msg,
});
} else {
debug!(self.log, "A duplicate message was received"; "message" => format!("{:?}", msg));
}
}
GossipsubEvent::Subscribed { peer_id, topic } => {
self.events
.push(BehaviourEvent::PeerSubscribed(peer_id, topic));
}
GossipsubEvent::Unsubscribed { .. } => {}
}
}
}
impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<RPCMessage>
for Behaviour<TSubstream>
{
fn inject_event(&mut self, event: RPCMessage) {
match event {
RPCMessage::PeerDialed(peer_id) => {
self.events.push(BehaviourEvent::PeerDialed(peer_id))
}
RPCMessage::PeerDisconnected(peer_id) => {
self.events.push(BehaviourEvent::PeerDisconnected(peer_id))
}
RPCMessage::RPC(peer_id, rpc_event) => {
self.events.push(BehaviourEvent::RPC(peer_id, rpc_event))
}
}
}
}
impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<PingEvent>
for Behaviour<TSubstream>
{
fn inject_event(&mut self, _event: PingEvent) {
// not interested in ping responses at the moment.
}
}
impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
/// Consumes the events list when polled.
fn poll<TBehaviourIn>(
&mut self,
) -> Async<NetworkBehaviourAction<TBehaviourIn, BehaviourEvent>> {
if !self.events.is_empty() {
return Async::Ready(NetworkBehaviourAction::GenerateEvent(self.events.remove(0)));
}
Async::NotReady
}
}
impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<IdentifyEvent>
for Behaviour<TSubstream>
{
fn inject_event(&mut self, event: IdentifyEvent) {
match event {
IdentifyEvent::Received {
peer_id,
mut info,
observed_addr,
} => {
if info.listen_addrs.len() > MAX_IDENTIFY_ADDRESSES {
debug!(
self.log,
"More than 20 addresses have been identified, truncating"
);
info.listen_addrs.truncate(MAX_IDENTIFY_ADDRESSES);
}
debug!(self.log, "Identified Peer"; "peer" => format!("{}", peer_id),
"protocol_version" => info.protocol_version,
"agent_version" => info.agent_version,
"listening_ addresses" => format!("{:?}", info.listen_addrs),
"observed_address" => format!("{:?}", observed_addr),
"protocols" => format!("{:?}", info.protocols)
);
}
IdentifyEvent::Sent { .. } => {}
IdentifyEvent::Error { .. } => {}
}
}
}
impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<Discv5Event>
for Behaviour<TSubstream>
{
fn inject_event(&mut self, _event: Discv5Event) {
// discv5 has no events to inject
}
}
/// Implements the combined behaviour for the libp2p service.
impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
/* Pubsub behaviour functions */ /* Pubsub behaviour functions */
/// Subscribes to a gossipsub topic kind, letting the network service determine the
/// encoding and fork version.
pub fn subscribe_kind(&mut self, kind: GossipKind) -> bool {
let gossip_topic = GossipTopic::new(
kind,
GossipEncoding::default(),
self.enr_fork_id.fork_digest,
);
self.subscribe(gossip_topic)
}
/// Unsubscribes from a gossipsub topic kind, letting the network service determine the
/// encoding and fork version.
pub fn unsubscribe_kind(&mut self, kind: GossipKind) -> bool {
let gossip_topic = GossipTopic::new(
kind,
GossipEncoding::default(),
self.enr_fork_id.fork_digest,
);
self.unsubscribe(gossip_topic)
}
/// Subscribes to a specific subnet id;
pub fn subscribe_to_subnet(&mut self, subnet_id: SubnetId) -> bool {
let topic = GossipTopic::new(
subnet_id.into(),
GossipEncoding::default(),
self.enr_fork_id.fork_digest,
);
self.subscribe(topic)
}
/// Un-Subscribes from a specific subnet id;
pub fn unsubscribe_from_subnet(&mut self, subnet_id: SubnetId) -> bool {
let topic = GossipTopic::new(
subnet_id.into(),
GossipEncoding::default(),
self.enr_fork_id.fork_digest,
);
self.unsubscribe(topic)
}
/// Subscribes to a gossipsub topic. /// Subscribes to a gossipsub topic.
pub fn subscribe(&mut self, topic: Topic) -> bool { fn subscribe(&mut self, topic: GossipTopic) -> bool {
self.gossipsub.subscribe(topic) // update the network globals
self.network_globals
.gossipsub_subscriptions
.write()
.insert(topic.clone());
let topic_str: String = topic.clone().into();
debug!(self.log, "Subscribed to topic"; "topic" => topic_str);
self.gossipsub.subscribe(topic.into())
} }
/// Unsubscribe from a gossipsub topic. /// Unsubscribe from a gossipsub topic.
pub fn unsubscribe(&mut self, topic: Topic) -> bool { fn unsubscribe(&mut self, topic: GossipTopic) -> bool {
self.gossipsub.unsubscribe(topic) // update the network globals
self.network_globals
.gossipsub_subscriptions
.write()
.remove(&topic);
// unsubscribe from the topic
self.gossipsub.unsubscribe(topic.into())
} }
/// Publishes a message on the pubsub (gossipsub) behaviour. /// Publishes a list of messages on the pubsub (gossipsub) behaviour, choosing the encoding.
pub fn publish(&mut self, topics: &[Topic], message: PubsubMessage) { pub fn publish(&mut self, messages: Vec<PubsubMessage<TSpec>>) {
let message_data = message.into_data(); for message in messages {
for topic in topics { for topic in message.topics(GossipEncoding::default(), self.enr_fork_id.fork_digest) {
self.gossipsub.publish(topic, message_data.clone()); match message.encode(GossipEncoding::default()) {
Ok(message_data) => {
self.gossipsub.publish(&topic.into(), message_data);
}
Err(e) => crit!(self.log, "Could not publish message"; "error" => e),
}
}
} }
} }
@ -236,15 +215,11 @@ impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
/* Eth2 RPC behaviour functions */ /* Eth2 RPC behaviour functions */
/// Sends an RPC Request/Response via the RPC protocol. /// Sends an RPC Request/Response via the RPC protocol.
pub fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCEvent) { pub fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCEvent<TSpec>) {
self.eth2_rpc.send_rpc(peer_id, rpc_event); self.eth2_rpc.send_rpc(peer_id, rpc_event);
} }
/* Discovery / Peer management functions */ /* Discovery / Peer management functions */
/// Return the list of currently connected peers.
pub fn connected_peers(&self) -> usize {
self.discovery.connected_peers()
}
/// Notify discovery that the peer has been banned. /// Notify discovery that the peer has been banned.
pub fn peer_banned(&mut self, peer_id: PeerId) { pub fn peer_banned(&mut self, peer_id: PeerId) {
@ -265,18 +240,322 @@ impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
pub fn add_enr(&mut self, enr: Enr) { pub fn add_enr(&mut self, enr: Enr) {
self.discovery.add_enr(enr); self.discovery.add_enr(enr);
} }
/// Updates a subnet value to the ENR bitfield.
///
/// The `value` is `true` if a subnet is being added and false otherwise.
pub fn update_enr_subnet(&mut self, subnet_id: SubnetId, value: bool) {
if let Err(e) = self.discovery.update_enr_bitfield(subnet_id, value) {
crit!(self.log, "Could not update ENR bitfield"; "error" => e);
}
// update the local meta data which informs our peers of the update during PINGS
self.update_metadata();
}
/// A request to search for peers connected to a long-lived subnet.
pub fn peers_request(&mut self, subnet_id: SubnetId) {
self.discovery.peers_request(subnet_id);
}
/// Updates the local ENR's "eth2" field with the latest EnrForkId.
pub fn update_fork_version(&mut self, enr_fork_id: EnrForkId) {
self.discovery.update_eth2_enr(enr_fork_id.clone());
// unsubscribe from all gossip topics and re-subscribe to their new fork counterparts
let subscribed_topics = self
.network_globals
.gossipsub_subscriptions
.read()
.iter()
.cloned()
.collect::<Vec<GossipTopic>>();
// unsubscribe from all topics
for topic in &subscribed_topics {
self.unsubscribe(topic.clone());
}
// re-subscribe modifying the fork version
for mut topic in subscribed_topics {
*topic.digest() = enr_fork_id.fork_digest;
self.subscribe(topic);
}
// update the local reference
self.enr_fork_id = enr_fork_id;
}
/* Private internal functions */
/// Updates the current meta data of the node.
fn update_metadata(&mut self) {
self.meta_data.seq_number += 1;
self.meta_data.attnets = self
.discovery
.local_enr()
.bitfield::<TSpec>()
.expect("Local discovery must have bitfield");
}
/// Sends a PING/PONG request/response to a peer.
fn send_ping(&mut self, id: RequestId, peer_id: PeerId) {
let pong_response = RPCEvent::Response(
id,
RPCErrorResponse::Success(RPCResponse::Pong(crate::rpc::methods::Ping {
data: self.meta_data.seq_number,
})),
);
self.send_rpc(peer_id, pong_response);
}
/// Sends a METADATA request to a peer.
fn send_meta_data_request(&mut self, peer_id: PeerId) {
let metadata_request =
RPCEvent::Request(RequestId::from(0usize), RPCRequest::MetaData(PhantomData));
self.send_rpc(peer_id, metadata_request);
}
/// Sends a METADATA response to a peer.
fn send_meta_data_response(&mut self, id: RequestId, peer_id: PeerId) {
let metadata_response = RPCEvent::Response(
id,
RPCErrorResponse::Success(RPCResponse::MetaData(self.meta_data.clone())),
);
self.send_rpc(peer_id, metadata_response);
}
}
// Implement the NetworkBehaviourEventProcess trait so that we can derive NetworkBehaviour for Behaviour
impl<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec>
NetworkBehaviourEventProcess<GossipsubEvent> for Behaviour<TSubstream, TSpec>
{
fn inject_event(&mut self, event: GossipsubEvent) {
match event {
GossipsubEvent::Message(propagation_source, id, gs_msg) => {
// Note: We are keeping track here of the peer that sent us the message, not the
// peer that originally published the message.
if self.seen_gossip_messages.put(id.clone(), ()).is_none() {
match PubsubMessage::decode(&gs_msg.topics, &gs_msg.data) {
Err(e) => {
debug!(self.log, "Could not decode gossipsub message"; "error" => format!("{}", e))
}
Ok(msg) => {
// if this message isn't a duplicate, notify the network
self.events.push(BehaviourEvent::PubsubMessage {
id,
source: propagation_source,
topics: gs_msg.topics,
message: msg,
});
}
}
} else {
match PubsubMessage::<TSpec>::decode(&gs_msg.topics, &gs_msg.data) {
Err(e) => {
debug!(self.log, "Could not decode gossipsub message"; "error" => format!("{}", e))
}
Ok(msg) => {
crit!(self.log, "A duplicate gossipsub message was received"; "message_source" => format!("{}", gs_msg.source), "propagated_peer" => format!("{}",propagation_source), "message" => format!("{}", msg));
}
}
}
}
GossipsubEvent::Subscribed { peer_id, topic } => {
self.events
.push(BehaviourEvent::PeerSubscribed(peer_id, topic));
}
GossipsubEvent::Unsubscribed { .. } => {}
}
}
}
impl<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec>
NetworkBehaviourEventProcess<RPCMessage<TSpec>> for Behaviour<TSubstream, TSpec>
{
fn inject_event(&mut self, event: RPCMessage<TSpec>) {
match event {
// TODO: These are temporary methods to give access to injected behaviour
// events to the
// peer manager. After a behaviour re-write remove these:
RPCMessage::PeerConnectedHack(peer_id, connected_point) => {
match connected_point {
ConnectedPoint::Dialer { .. } => self.peer_manager.connect_outgoing(&peer_id),
ConnectedPoint::Listener { .. } => self.peer_manager.connect_ingoing(&peer_id),
};
// Find ENR info about a peer if possible.
if let Some(enr) = self.discovery.enr_of_peer(&peer_id) {
let bitfield = match enr.bitfield::<TSpec>() {
Ok(v) => v,
Err(e) => {
warn!(self.log, "Peer has invalid ENR bitfield";
"peer_id" => format!("{}", peer_id),
"error" => format!("{:?}", e));
return;
}
};
// use this as a baseline, until we get the actual meta-data
let meta_data = MetaData {
seq_number: 0,
attnets: bitfield,
};
// TODO: Shift to the peer manager
self.network_globals
.peers
.write()
.add_metadata(&peer_id, meta_data);
}
}
RPCMessage::PeerDisconnectedHack(peer_id, _connected_point) => {
self.peer_manager.notify_disconnect(&peer_id)
}
RPCMessage::PeerDialed(peer_id) => {
self.events.push(BehaviourEvent::PeerDialed(peer_id))
}
RPCMessage::PeerDisconnected(peer_id) => {
self.events.push(BehaviourEvent::PeerDisconnected(peer_id))
}
RPCMessage::RPC(peer_id, rpc_event) => {
// The METADATA and PING RPC responses are handled within the behaviour and not
// propagated
// TODO: Improve the RPC types to better handle this logic discrepancy
match rpc_event {
RPCEvent::Request(id, RPCRequest::Ping(ping)) => {
// inform the peer manager and send the response
self.peer_manager.ping_request(&peer_id, ping.data);
self.send_ping(id, peer_id);
}
RPCEvent::Request(id, RPCRequest::MetaData(_)) => {
// send the requested meta-data
self.send_meta_data_response(id, peer_id);
}
RPCEvent::Response(_, RPCErrorResponse::Success(RPCResponse::Pong(ping))) => {
self.peer_manager.pong_response(&peer_id, ping.data);
}
RPCEvent::Response(
_,
RPCErrorResponse::Success(RPCResponse::MetaData(meta_data)),
) => {
self.peer_manager.meta_data_response(&peer_id, meta_data);
}
RPCEvent::Request(_, RPCRequest::Status(_))
| RPCEvent::Response(_, RPCErrorResponse::Success(RPCResponse::Status(_))) => {
// inform the peer manager that we have received a status from a peer
self.peer_manager.peer_statusd(&peer_id);
// propagate the STATUS message upwards
self.events.push(BehaviourEvent::RPC(peer_id, rpc_event));
}
_ => {
// propagate all other RPC messages upwards
self.events.push(BehaviourEvent::RPC(peer_id, rpc_event))
}
}
}
}
}
}
impl<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec> Behaviour<TSubstream, TSpec> {
/// Consumes the events list when polled.
fn poll<TBehaviourIn>(
&mut self,
) -> Async<NetworkBehaviourAction<TBehaviourIn, BehaviourEvent<TSpec>>> {
// check the peer manager for events
loop {
match self.peer_manager.poll() {
Ok(Async::Ready(Some(event))) => match event {
PeerManagerEvent::Status(peer_id) => {
// it's time to status. We don't keep a beacon chain reference here, so we inform
// the network to send a status to this peer
return Async::Ready(NetworkBehaviourAction::GenerateEvent(
BehaviourEvent::StatusPeer(peer_id),
));
}
PeerManagerEvent::Ping(peer_id) => {
// send a ping to this peer
self.send_ping(RequestId::from(0usize), peer_id);
}
PeerManagerEvent::MetaData(peer_id) => {
self.send_meta_data_request(peer_id);
}
PeerManagerEvent::_DisconnectPeer(_peer_id) => {
//TODO: Implement
}
PeerManagerEvent::_BanPeer(_peer_id) => {
//TODO: Implement
}
},
Ok(Async::NotReady) => break,
Ok(Async::Ready(None)) | Err(_) => {
crit!(self.log, "Error polling peer manager");
break;
}
}
}
if !self.events.is_empty() {
return Async::Ready(NetworkBehaviourAction::GenerateEvent(self.events.remove(0)));
}
Async::NotReady
}
}
impl<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec> NetworkBehaviourEventProcess<IdentifyEvent>
for Behaviour<TSubstream, TSpec>
{
fn inject_event(&mut self, event: IdentifyEvent) {
match event {
IdentifyEvent::Received {
peer_id,
mut info,
observed_addr,
} => {
if info.listen_addrs.len() > MAX_IDENTIFY_ADDRESSES {
debug!(
self.log,
"More than 10 addresses have been identified, truncating"
);
info.listen_addrs.truncate(MAX_IDENTIFY_ADDRESSES);
}
// send peer info to the peer manager.
self.peer_manager.identify(&peer_id, &info);
debug!(self.log, "Identified Peer"; "peer" => format!("{}", peer_id),
"protocol_version" => info.protocol_version,
"agent_version" => info.agent_version,
"listening_ addresses" => format!("{:?}", info.listen_addrs),
"observed_address" => format!("{:?}", observed_addr),
"protocols" => format!("{:?}", info.protocols)
);
}
IdentifyEvent::Sent { .. } => {}
IdentifyEvent::Error { .. } => {}
}
}
}
impl<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec> NetworkBehaviourEventProcess<Discv5Event>
for Behaviour<TSubstream, TSpec>
{
fn inject_event(&mut self, _event: Discv5Event) {
// discv5 has no events to inject
}
} }
/// The types of events than can be obtained from polling the behaviour. /// The types of events than can be obtained from polling the behaviour.
pub enum BehaviourEvent { #[derive(Debug)]
pub enum BehaviourEvent<TSpec: EthSpec> {
/// A received RPC event and the peer that it was received from. /// A received RPC event and the peer that it was received from.
RPC(PeerId, RPCEvent), RPC(PeerId, RPCEvent<TSpec>),
/// We have completed an initial connection to a new peer. /// We have completed an initial connection to a new peer.
PeerDialed(PeerId), PeerDialed(PeerId),
/// A peer has disconnected. /// A peer has disconnected.
PeerDisconnected(PeerId), PeerDisconnected(PeerId),
/// A gossipsub message has been received. /// A gossipsub message has been received.
GossipMessage { PubsubMessage {
/// The gossipsub message id. Used when propagating blocks after validation. /// The gossipsub message id. Used when propagating blocks after validation.
id: MessageId, id: MessageId,
/// The peer from which we received this message, not the peer that published it. /// The peer from which we received this message, not the peer that published it.
@ -284,60 +563,10 @@ pub enum BehaviourEvent {
/// The topics that this message was sent on. /// The topics that this message was sent on.
topics: Vec<TopicHash>, topics: Vec<TopicHash>,
/// The message itself. /// The message itself.
message: PubsubMessage, message: PubsubMessage<TSpec>,
}, },
/// Subscribed to peer for given topic /// Subscribed to peer for given topic
PeerSubscribed(PeerId, TopicHash), PeerSubscribed(PeerId, TopicHash),
} /// Inform the network to send a Status to this peer.
StatusPeer(PeerId),
/// Messages that are passed to and from the pubsub (Gossipsub) behaviour. These are encoded and
/// decoded upstream.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub enum PubsubMessage {
/// Gossipsub message providing notification of a new block.
Block(Vec<u8>),
/// Gossipsub message providing notification of a new attestation.
Attestation(Vec<u8>),
/// Gossipsub message providing notification of a voluntary exit.
VoluntaryExit(Vec<u8>),
/// Gossipsub message providing notification of a new proposer slashing.
ProposerSlashing(Vec<u8>),
/// Gossipsub message providing notification of a new attester slashing.
AttesterSlashing(Vec<u8>),
/// Gossipsub message from an unknown topic.
Unknown(Vec<u8>),
}
impl PubsubMessage {
/* Note: This is assuming we are not hashing topics. If we choose to hash topics, these will
* need to be modified.
*
* Also note that a message can be associated with many topics. As soon as one of the topics is
* known we match. If none of the topics are known we return an unknown state.
*/
fn from_topics(topics: &[TopicHash], data: Vec<u8>) -> Self {
for topic in topics {
match GossipTopic::from(topic.as_str()) {
GossipTopic::BeaconBlock => return PubsubMessage::Block(data),
GossipTopic::BeaconAttestation => return PubsubMessage::Attestation(data),
GossipTopic::VoluntaryExit => return PubsubMessage::VoluntaryExit(data),
GossipTopic::ProposerSlashing => return PubsubMessage::ProposerSlashing(data),
GossipTopic::AttesterSlashing => return PubsubMessage::AttesterSlashing(data),
GossipTopic::Shard => return PubsubMessage::Unknown(data),
GossipTopic::Unknown(_) => continue,
}
}
PubsubMessage::Unknown(data)
}
fn into_data(self) -> Vec<u8> {
match self {
PubsubMessage::Block(data)
| PubsubMessage::Attestation(data)
| PubsubMessage::VoluntaryExit(data)
| PubsubMessage::ProposerSlashing(data)
| PubsubMessage::AttesterSlashing(data)
| PubsubMessage::Unknown(data) => data,
}
}
} }

View File

@ -1,5 +1,6 @@
use crate::topics::GossipTopic; use crate::types::GossipKind;
use enr::Enr; use crate::Enr;
use libp2p::discv5::{Discv5Config, Discv5ConfigBuilder};
use libp2p::gossipsub::{GossipsubConfig, GossipsubConfigBuilder, GossipsubMessage, MessageId}; use libp2p::gossipsub::{GossipsubConfig, GossipsubConfigBuilder, GossipsubMessage, MessageId};
use libp2p::Multiaddr; use libp2p::Multiaddr;
use serde_derive::{Deserialize, Serialize}; use serde_derive::{Deserialize, Serialize};
@ -7,6 +8,8 @@ use sha2::{Digest, Sha256};
use std::path::PathBuf; use std::path::PathBuf;
use std::time::Duration; use std::time::Duration;
pub const GOSSIP_MAX_SIZE: usize = 1_048_576;
#[derive(Clone, Debug, Serialize, Deserialize)] #[derive(Clone, Debug, Serialize, Deserialize)]
#[serde(default)] #[serde(default)]
/// Network configuration for lighthouse. /// Network configuration for lighthouse.
@ -20,13 +23,19 @@ pub struct Config {
/// The TCP port that libp2p listens on. /// The TCP port that libp2p listens on.
pub libp2p_port: u16, pub libp2p_port: u16,
/// The address to broadcast to peers about which address we are listening on. None indicates
/// that no discovery address has been set in the CLI args.
pub discovery_address: Option<std::net::IpAddr>,
/// UDP port that discovery listens on. /// UDP port that discovery listens on.
pub discovery_port: u16, pub discovery_port: u16,
/// The address to broadcast to peers about which address we are listening on. None indicates
/// that no discovery address has been set in the CLI args.
pub enr_address: Option<std::net::IpAddr>,
/// The udp port to broadcast to peers in order to reach back for discovery.
pub enr_udp_port: Option<u16>,
/// The tcp port to broadcast to peers in order to reach back for libp2p services.
pub enr_tcp_port: Option<u16>,
/// Target number of connected peers. /// Target number of connected peers.
pub max_peers: usize, pub max_peers: usize,
@ -40,6 +49,10 @@ pub struct Config {
#[serde(skip)] #[serde(skip)]
pub gs_config: GossipsubConfig, pub gs_config: GossipsubConfig,
/// Discv5 configuration parameters.
#[serde(skip)]
pub discv5_config: Discv5Config,
/// List of nodes to initially connect to. /// List of nodes to initially connect to.
pub boot_nodes: Vec<Enr>, pub boot_nodes: Vec<Enr>,
@ -50,7 +63,7 @@ pub struct Config {
pub client_version: String, pub client_version: String,
/// List of extra topics to initially subscribe to as strings. /// List of extra topics to initially subscribe to as strings.
pub topics: Vec<GossipTopic>, pub topics: Vec<GossipKind>,
/// Introduces randomization in network propagation of messages. This should only be set for /// Introduces randomization in network propagation of messages. This should only be set for
/// testing purposes and will likely be removed in future versions. /// testing purposes and will likely be removed in future versions.
@ -67,11 +80,11 @@ impl Default for Config {
// The default topics that we will initially subscribe to // The default topics that we will initially subscribe to
let topics = vec![ let topics = vec![
GossipTopic::BeaconBlock, GossipKind::BeaconBlock,
GossipTopic::BeaconAttestation, GossipKind::BeaconAggregateAndProof,
GossipTopic::VoluntaryExit, GossipKind::VoluntaryExit,
GossipTopic::ProposerSlashing, GossipKind::ProposerSlashing,
GossipTopic::AttesterSlashing, GossipKind::AttesterSlashing,
]; ];
// The function used to generate a gossipsub message id // The function used to generate a gossipsub message id
@ -83,23 +96,43 @@ impl Default for Config {
)) ))
}; };
// gossipsub configuration
// Note: The topics by default are sent as plain strings. Hashes are an optional
// parameter.
let gs_config = GossipsubConfigBuilder::new()
.max_transmit_size(GOSSIP_MAX_SIZE)
.heartbeat_interval(Duration::from_secs(20)) // TODO: Reduce for mainnet
.manual_propagation() // require validation before propagation
.no_source_id()
.message_id_fn(gossip_message_id)
.build();
// discv5 configuration
let discv5_config = Discv5ConfigBuilder::new()
.request_timeout(Duration::from_secs(4))
.request_retries(2)
.enr_update(true) // update IP based on PONG responses
.enr_peer_update_min(2) // prevents NAT's should be raised for mainnet
.query_parallelism(5)
.query_timeout(Duration::from_secs(60))
.query_peer_timeout(Duration::from_secs(2))
.ip_limit(false) // limits /24 IP's in buckets. Enable for mainnet
.ping_interval(Duration::from_secs(300))
.build();
// NOTE: Some of these get overridden by the corresponding CLI default values.
Config { Config {
network_dir, network_dir,
listen_address: "127.0.0.1".parse().expect("valid ip address"), listen_address: "0.0.0.0".parse().expect("valid ip address"),
libp2p_port: 9000, libp2p_port: 9000,
discovery_address: None,
discovery_port: 9000, discovery_port: 9000,
max_peers: 10, enr_address: None,
enr_udp_port: None,
enr_tcp_port: None,
max_peers: 50,
secret_key_hex: None, secret_key_hex: None,
// Note: The topics by default are sent as plain strings. Hashes are an optional gs_config,
// parameter. discv5_config,
gs_config: GossipsubConfigBuilder::new()
.max_transmit_size(1_048_576)
.heartbeat_interval(Duration::from_secs(20)) // TODO: Reduce for mainnet
.manual_propagation() // require validation before propagation
.no_source_id()
.message_id_fn(gossip_message_id)
.build(),
boot_nodes: vec![], boot_nodes: vec![],
libp2p_nodes: vec![], libp2p_nodes: vec![],
client_version: version::version(), client_version: version::version(),

View File

@ -1,412 +0,0 @@
use crate::metrics;
use crate::{error, NetworkConfig, NetworkGlobals};
/// This manages the discovery and management of peers.
///
/// Currently using discv5 for peer discovery.
///
use futures::prelude::*;
use libp2p::core::{identity::Keypair, ConnectedPoint, Multiaddr, PeerId};
use libp2p::discv5::{Discv5, Discv5Event};
use libp2p::enr::{Enr, EnrBuilder, NodeId};
use libp2p::multiaddr::Protocol;
use libp2p::swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler};
use slog::{debug, info, warn};
use std::collections::HashSet;
use std::fs::File;
use std::io::prelude::*;
use std::path::Path;
use std::str::FromStr;
use std::sync::{atomic::Ordering, Arc};
use std::time::{Duration, Instant};
use tokio::io::{AsyncRead, AsyncWrite};
use tokio::timer::Delay;
/// Maximum seconds before searching for extra peers.
const MAX_TIME_BETWEEN_PEER_SEARCHES: u64 = 120;
/// Initial delay between peer searches.
const INITIAL_SEARCH_DELAY: u64 = 5;
/// Local ENR storage filename.
const ENR_FILENAME: &str = "enr.dat";
/// Lighthouse discovery behaviour. This provides peer management and discovery using the Discv5
/// libp2p protocol.
pub struct Discovery<TSubstream> {
/// The currently banned peers.
banned_peers: HashSet<PeerId>,
/// The target number of connected peers on the libp2p interface.
max_peers: usize,
/// The directory where the ENR is stored.
enr_dir: String,
/// The delay between peer discovery searches.
peer_discovery_delay: Delay,
/// Tracks the last discovery delay. The delay is doubled each round until the max
/// time is reached.
past_discovery_delay: u64,
/// The TCP port for libp2p. Used to convert an updated IP address to a multiaddr. Note: This
/// assumes that the external TCP port is the same as the internal TCP port if behind a NAT.
//TODO: Improve NAT handling limit the above restriction
tcp_port: u16,
/// The discovery behaviour used to discover new peers.
discovery: Discv5<TSubstream>,
/// A collection of network constants that can be read from other threads.
network_globals: Arc<NetworkGlobals>,
/// Logger for the discovery behaviour.
log: slog::Logger,
}
impl<TSubstream> Discovery<TSubstream> {
pub fn new(
local_key: &Keypair,
config: &NetworkConfig,
network_globals: Arc<NetworkGlobals>,
log: &slog::Logger,
) -> error::Result<Self> {
let log = log.clone();
// checks if current ENR matches that found on disk
let local_enr = load_enr(local_key, config, &log)?;
*network_globals.local_enr.write() = Some(local_enr.clone());
let enr_dir = match config.network_dir.to_str() {
Some(path) => String::from(path),
None => String::from(""),
};
info!(log, "ENR Initialised"; "enr" => local_enr.to_base64(), "seq" => local_enr.seq());
debug!(log, "Discv5 Node ID Initialised"; "node_id" => format!("{}",local_enr.node_id()));
// the last parameter enables IP limiting. 2 Nodes on the same /24 subnet per bucket and 10
// nodes on the same /24 subnet per table.
// TODO: IP filtering is currently disabled for the DHT. Enable for production
let mut discovery = Discv5::new(local_enr, local_key.clone(), config.listen_address, false)
.map_err(|e| format!("Discv5 service failed. Error: {:?}", e))?;
// Add bootnodes to routing table
for bootnode_enr in config.boot_nodes.clone() {
debug!(
log,
"Adding node to routing table";
"node_id" => format!("{}",
bootnode_enr.node_id())
);
discovery.add_enr(bootnode_enr);
}
Ok(Self {
banned_peers: HashSet::new(),
max_peers: config.max_peers,
peer_discovery_delay: Delay::new(Instant::now()),
past_discovery_delay: INITIAL_SEARCH_DELAY,
tcp_port: config.libp2p_port,
discovery,
network_globals,
log,
enr_dir,
})
}
/// Return the nodes local ENR.
pub fn local_enr(&self) -> &Enr {
self.discovery.local_enr()
}
/// Manually search for peers. This restarts the discovery round, sparking multiple rapid
/// queries.
pub fn discover_peers(&mut self) {
self.past_discovery_delay = INITIAL_SEARCH_DELAY;
self.find_peers();
}
/// Add an ENR to the routing table of the discovery mechanism.
pub fn add_enr(&mut self, enr: Enr) {
self.discovery.add_enr(enr);
}
/// The current number of connected libp2p peers.
pub fn connected_peers(&self) -> usize {
self.network_globals.connected_peers.load(Ordering::Relaxed)
}
/// The current number of connected libp2p peers.
pub fn connected_peer_set(&self) -> Vec<PeerId> {
self.network_globals
.connected_peer_set
.read()
.iter()
.cloned()
.collect::<Vec<_>>()
}
/// The peer has been banned. Add this peer to the banned list to prevent any future
/// re-connections.
// TODO: Remove the peer from the DHT if present
pub fn peer_banned(&mut self, peer_id: PeerId) {
self.banned_peers.insert(peer_id);
}
pub fn peer_unbanned(&mut self, peer_id: &PeerId) {
self.banned_peers.remove(peer_id);
}
/// Returns an iterator over all enr entries in the DHT.
pub fn enr_entries(&mut self) -> impl Iterator<Item = &Enr> {
self.discovery.enr_entries()
}
/// Search for new peers using the underlying discovery mechanism.
fn find_peers(&mut self) {
// pick a random NodeId
let random_node = NodeId::random();
debug!(self.log, "Searching for peers");
self.discovery.find_node(random_node);
}
}
// Redirect all behaviour events to underlying discovery behaviour.
impl<TSubstream> NetworkBehaviour for Discovery<TSubstream>
where
TSubstream: AsyncRead + AsyncWrite,
{
type ProtocolsHandler = <Discv5<TSubstream> as NetworkBehaviour>::ProtocolsHandler;
type OutEvent = <Discv5<TSubstream> as NetworkBehaviour>::OutEvent;
fn new_handler(&mut self) -> Self::ProtocolsHandler {
NetworkBehaviour::new_handler(&mut self.discovery)
}
fn addresses_of_peer(&mut self, peer_id: &PeerId) -> Vec<Multiaddr> {
// Let discovery track possible known peers.
self.discovery.addresses_of_peer(peer_id)
}
fn inject_connected(&mut self, peer_id: PeerId, _endpoint: ConnectedPoint) {
self.network_globals
.connected_peer_set
.write()
.insert(peer_id);
self.network_globals.connected_peers.store(
self.network_globals.connected_peer_set.read().len(),
Ordering::Relaxed,
);
// 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);
}
fn inject_disconnected(&mut self, peer_id: &PeerId, _endpoint: ConnectedPoint) {
self.network_globals
.connected_peer_set
.write()
.remove(peer_id);
self.network_globals.connected_peers.store(
self.network_globals.connected_peer_set.read().len(),
Ordering::Relaxed,
);
metrics::inc_counter(&metrics::PEER_DISCONNECT_EVENT_COUNT);
metrics::set_gauge(&metrics::PEERS_CONNECTED, self.connected_peers() as i64);
}
fn inject_replaced(
&mut self,
_peer_id: PeerId,
_closed: ConnectedPoint,
_opened: ConnectedPoint,
) {
// discv5 doesn't implement
}
fn inject_node_event(
&mut self,
_peer_id: PeerId,
_event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent,
) {
// discv5 doesn't implement
}
fn poll(
&mut self,
params: &mut impl PollParameters,
) -> Async<
NetworkBehaviourAction<
<Self::ProtocolsHandler as ProtocolsHandler>::InEvent,
Self::OutEvent,
>,
> {
// search for peers if it is time
loop {
match self.peer_discovery_delay.poll() {
Ok(Async::Ready(_)) => {
if self.network_globals.connected_peers.load(Ordering::Relaxed) < self.max_peers
{
self.find_peers();
}
// Set to maximum, and update to earlier, once we get our results back.
self.peer_discovery_delay.reset(
Instant::now() + Duration::from_secs(MAX_TIME_BETWEEN_PEER_SEARCHES),
);
}
Ok(Async::NotReady) => break,
Err(e) => {
warn!(self.log, "Discovery peer search failed"; "error" => format!("{:?}", e));
}
}
}
// Poll discovery
loop {
match self.discovery.poll(params) {
Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) => {
match event {
Discv5Event::Discovered(_enr) => {
// not concerned about FINDNODE results, rather the result of an entire
// query.
}
Discv5Event::SocketUpdated(socket) => {
info!(self.log, "Address updated"; "ip" => format!("{}",socket.ip()), "udp_port" => format!("{}", socket.port()));
metrics::inc_counter(&metrics::ADDRESS_UPDATE_COUNT);
let mut address = Multiaddr::from(socket.ip());
address.push(Protocol::Tcp(self.tcp_port));
let enr = self.discovery.local_enr();
save_enr_to_disc(Path::new(&self.enr_dir), enr, &self.log);
return Async::Ready(NetworkBehaviourAction::ReportObservedAddr {
address,
});
}
Discv5Event::FindNodeResult { closer_peers, .. } => {
debug!(self.log, "Discovery query completed"; "peers_found" => closer_peers.len());
// update the time to the next query
if self.past_discovery_delay < MAX_TIME_BETWEEN_PEER_SEARCHES {
self.past_discovery_delay *= 2;
}
let delay = std::cmp::max(
self.past_discovery_delay,
MAX_TIME_BETWEEN_PEER_SEARCHES,
);
self.peer_discovery_delay
.reset(Instant::now() + Duration::from_secs(delay));
if closer_peers.is_empty() {
debug!(self.log, "Discovery random query found no peers");
}
for peer_id in closer_peers {
// if we need more peers, attempt a connection
if self.network_globals.connected_peers.load(Ordering::Relaxed)
< self.max_peers
&& self
.network_globals
.connected_peer_set
.read()
.get(&peer_id)
.is_none()
&& !self.banned_peers.contains(&peer_id)
{
debug!(self.log, "Peer discovered"; "peer_id"=> format!("{:?}", peer_id));
return Async::Ready(NetworkBehaviourAction::DialPeer {
peer_id,
});
}
}
}
_ => {}
}
}
// discv5 does not output any other NetworkBehaviourAction
Async::Ready(_) => {}
Async::NotReady => break,
}
}
Async::NotReady
}
}
/// Loads an ENR from file if it exists and matches the current NodeId and sequence number. If none
/// exists, generates a new one.
///
/// If an ENR exists, with the same NodeId and IP address, we use the disk-generated one as its
/// ENR sequence will be equal or higher than a newly generated one.
fn load_enr(
local_key: &Keypair,
config: &NetworkConfig,
log: &slog::Logger,
) -> Result<Enr, String> {
// Build the local ENR.
// Note: Discovery should update the ENR record's IP to the external IP as seen by the
// majority of our peers.
let mut local_enr = EnrBuilder::new("v4")
.ip(config
.discovery_address
.unwrap_or_else(|| "127.0.0.1".parse().expect("valid ip")))
.tcp(config.libp2p_port)
.udp(config.discovery_port)
.build(&local_key)
.map_err(|e| format!("Could not build Local ENR: {:?}", e))?;
let enr_f = config.network_dir.join(ENR_FILENAME);
if let Ok(mut enr_file) = File::open(enr_f.clone()) {
let mut enr_string = String::new();
match enr_file.read_to_string(&mut enr_string) {
Err(_) => debug!(log, "Could not read ENR from file"),
Ok(_) => {
match Enr::from_str(&enr_string) {
Ok(enr) => {
if enr.node_id() == local_enr.node_id() {
if (config.discovery_address.is_none()
|| enr.ip().map(Into::into) == config.discovery_address)
&& enr.tcp() == Some(config.libp2p_port)
&& enr.udp() == Some(config.discovery_port)
{
debug!(log, "ENR loaded from file"; "file" => format!("{:?}", enr_f));
// the stored ENR has the same configuration, use it
return Ok(enr);
}
// same node id, different configuration - update the sequence number
let new_seq_no = enr.seq().checked_add(1).ok_or_else(|| "ENR sequence number on file is too large. Remove it to generate a new NodeId")?;
local_enr.set_seq(new_seq_no, local_key).map_err(|e| {
format!("Could not update ENR sequence number: {:?}", e)
})?;
debug!(log, "ENR sequence number increased"; "seq" => new_seq_no);
}
}
Err(e) => {
warn!(log, "ENR from file could not be decoded"; "error" => format!("{:?}", e));
}
}
}
}
}
save_enr_to_disc(&config.network_dir, &local_enr, log);
Ok(local_enr)
}
fn save_enr_to_disc(dir: &Path, enr: &Enr, log: &slog::Logger) {
let _ = std::fs::create_dir_all(dir);
match File::create(dir.join(Path::new(ENR_FILENAME)))
.and_then(|mut f| f.write_all(&enr.to_base64().as_bytes()))
{
Ok(_) => {
debug!(log, "ENR written to disk");
}
Err(e) => {
warn!(
log,
"Could not write ENR to file"; "file" => format!("{:?}{:?}",dir, ENR_FILENAME), "error" => format!("{}", e)
);
}
}
}

View File

@ -0,0 +1,172 @@
//! Helper functions and an extension trait for Ethereum 2 ENRs.
pub use libp2p::{core::identity::Keypair, discv5::enr::CombinedKey};
use super::ENR_FILENAME;
use crate::types::{Enr, EnrBitfield};
use crate::NetworkConfig;
use libp2p::discv5::enr::EnrBuilder;
use slog::{debug, warn};
use ssz::{Decode, Encode};
use ssz_types::BitVector;
use std::convert::TryInto;
use std::fs::File;
use std::io::prelude::*;
use std::path::Path;
use std::str::FromStr;
use types::{EnrForkId, EthSpec};
/// The ENR field specifying the fork id.
pub const ETH2_ENR_KEY: &'static str = "eth2";
/// The ENR field specifying the subnet bitfield.
pub const BITFIELD_ENR_KEY: &'static str = "attnets";
/// Extension trait for ENR's within Eth2.
pub trait Eth2Enr {
/// The subnet bitfield associated with the ENR.
fn bitfield<TSpec: EthSpec>(&self) -> Result<EnrBitfield<TSpec>, &'static str>;
fn eth2(&self) -> Result<EnrForkId, &'static str>;
}
impl Eth2Enr for Enr {
fn bitfield<TSpec: EthSpec>(&self) -> Result<EnrBitfield<TSpec>, &'static str> {
let bitfield_bytes = self
.get(BITFIELD_ENR_KEY)
.ok_or_else(|| "ENR bitfield non-existent")?;
BitVector::<TSpec::SubnetBitfieldLength>::from_ssz_bytes(bitfield_bytes)
.map_err(|_| "Could not decode the ENR SSZ bitfield")
}
fn eth2(&self) -> Result<EnrForkId, &'static str> {
let eth2_bytes = self
.get(ETH2_ENR_KEY)
.ok_or_else(|| "ENR has no eth2 field")?;
EnrForkId::from_ssz_bytes(eth2_bytes).map_err(|_| "Could not decode EnrForkId")
}
}
/// Loads an ENR from file if it exists and matches the current NodeId and sequence number. If none
/// exists, generates a new one.
///
/// If an ENR exists, with the same NodeId, this function checks to see if the loaded ENR from
/// disk is suitable to use, otherwise we increment our newly generated ENR's sequence number.
pub fn build_or_load_enr<T: EthSpec>(
local_key: Keypair,
config: &NetworkConfig,
enr_fork_id: EnrForkId,
log: &slog::Logger,
) -> Result<Enr, String> {
// Build the local ENR.
// Note: Discovery should update the ENR record's IP to the external IP as seen by the
// majority of our peers, if the CLI doesn't expressly forbid it.
let enr_key: CombinedKey = local_key
.try_into()
.map_err(|_| "Invalid key type for ENR records")?;
let mut local_enr = build_enr::<T>(&enr_key, config, enr_fork_id)?;
let enr_f = config.network_dir.join(ENR_FILENAME);
if let Ok(mut enr_file) = File::open(enr_f.clone()) {
let mut enr_string = String::new();
match enr_file.read_to_string(&mut enr_string) {
Err(_) => debug!(log, "Could not read ENR from file"),
Ok(_) => {
match Enr::from_str(&enr_string) {
Ok(disk_enr) => {
// if the same node id, then we may need to update our sequence number
if local_enr.node_id() == disk_enr.node_id() {
if compare_enr(&local_enr, &disk_enr) {
debug!(log, "ENR loaded from disk"; "file" => format!("{:?}", enr_f));
// the stored ENR has the same configuration, use it
return Ok(disk_enr);
}
// same node id, different configuration - update the sequence number
let new_seq_no = disk_enr.seq().checked_add(1).ok_or_else(|| "ENR sequence number on file is too large. Remove it to generate a new NodeId")?;
local_enr.set_seq(new_seq_no, &enr_key).map_err(|e| {
format!("Could not update ENR sequence number: {:?}", e)
})?;
debug!(log, "ENR sequence number increased"; "seq" => new_seq_no);
}
}
Err(e) => {
warn!(log, "ENR from file could not be decoded"; "error" => format!("{:?}", e));
}
}
}
}
}
save_enr_to_disk(&config.network_dir, &local_enr, log);
Ok(local_enr)
}
/// Builds a lighthouse ENR given a `NetworkConfig`.
pub fn build_enr<T: EthSpec>(
enr_key: &CombinedKey,
config: &NetworkConfig,
enr_fork_id: EnrForkId,
) -> Result<Enr, String> {
let mut builder = EnrBuilder::new("v4");
if let Some(enr_address) = config.enr_address {
builder.ip(enr_address);
}
if let Some(udp_port) = config.enr_udp_port {
builder.udp(udp_port);
}
// we always give it our listening tcp port
// TODO: Add uPnP support to map udp and tcp ports
let tcp_port = config.enr_tcp_port.unwrap_or_else(|| config.libp2p_port);
builder.tcp(tcp_port);
// set the `eth2` field on our ENR
builder.add_value(ETH2_ENR_KEY.into(), enr_fork_id.as_ssz_bytes());
// set the "attnets" field on our ENR
let bitfield = BitVector::<T::SubnetBitfieldLength>::new();
builder.add_value(BITFIELD_ENR_KEY.into(), bitfield.as_ssz_bytes());
builder
.tcp(config.libp2p_port)
.build(enr_key)
.map_err(|e| format!("Could not build Local ENR: {:?}", e))
}
/// Defines the conditions under which we use the locally built ENR or the one stored on disk.
/// If this function returns true, we use the `disk_enr`.
fn compare_enr(local_enr: &Enr, disk_enr: &Enr) -> bool {
// take preference over disk_enr address if one is not specified
(local_enr.ip().is_none() || local_enr.ip() == disk_enr.ip())
// tcp ports must match
&& local_enr.tcp() == disk_enr.tcp()
// must match on the same fork
&& local_enr.get(ETH2_ENR_KEY) == disk_enr.get(ETH2_ENR_KEY)
// take preference over disk udp port if one is not specified
&& (local_enr.udp().is_none() || local_enr.udp() == disk_enr.udp())
// we need the BITFIELD_ENR_KEY key to match, otherwise we use a new ENR. This will likely only
// be true for non-validating nodes
&& local_enr.get(BITFIELD_ENR_KEY) == disk_enr.get(BITFIELD_ENR_KEY)
}
/// Saves an ENR to disk
pub fn save_enr_to_disk(dir: &Path, enr: &Enr, log: &slog::Logger) {
let _ = std::fs::create_dir_all(dir);
match File::create(dir.join(Path::new(ENR_FILENAME)))
.and_then(|mut f| f.write_all(&enr.to_base64().as_bytes()))
{
Ok(_) => {
debug!(log, "ENR written to disk");
}
Err(e) => {
warn!(
log,
"Could not write ENR to file"; "file" => format!("{:?}{:?}",dir, ENR_FILENAME), "error" => format!("{}", e)
);
}
}
}

View File

@ -0,0 +1,489 @@
///! This manages the discovery and management of peers.
pub(crate) mod enr;
// Allow external use of the lighthouse ENR builder
pub use enr::{build_enr, CombinedKey, Keypair};
use crate::metrics;
use crate::{error, Enr, NetworkConfig, NetworkGlobals};
use enr::{Eth2Enr, BITFIELD_ENR_KEY, ETH2_ENR_KEY};
use futures::prelude::*;
use libp2p::core::{ConnectedPoint, Multiaddr, PeerId};
use libp2p::discv5::enr::NodeId;
use libp2p::discv5::{Discv5, Discv5Event};
use libp2p::multiaddr::Protocol;
use libp2p::swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler};
use slog::{crit, debug, info, warn};
use ssz::{Decode, Encode};
use ssz_types::BitVector;
use std::collections::{HashSet, VecDeque};
use std::net::SocketAddr;
use std::path::Path;
use std::sync::Arc;
use std::time::{Duration, Instant};
use tokio::io::{AsyncRead, AsyncWrite};
use tokio::timer::Delay;
use types::{EnrForkId, EthSpec, SubnetId};
/// Maximum seconds before searching for extra peers.
const MAX_TIME_BETWEEN_PEER_SEARCHES: u64 = 120;
/// Initial delay between peer searches.
const INITIAL_SEARCH_DELAY: u64 = 5;
/// Local ENR storage filename.
pub const ENR_FILENAME: &str = "enr.dat";
/// Number of peers we'd like to have connected to a given long-lived subnet.
const TARGET_SUBNET_PEERS: u64 = 3;
/// Lighthouse discovery behaviour. This provides peer management and discovery using the Discv5
/// libp2p protocol.
pub struct Discovery<TSubstream, TSpec: EthSpec> {
/// Events to be processed by the behaviour.
events: VecDeque<NetworkBehaviourAction<void::Void, Discv5Event>>,
/// The currently banned peers.
banned_peers: HashSet<PeerId>,
/// The target number of connected peers on the libp2p interface.
max_peers: usize,
/// The directory where the ENR is stored.
enr_dir: String,
/// The delay between peer discovery searches.
peer_discovery_delay: Delay,
/// Tracks the last discovery delay. The delay is doubled each round until the max
/// time is reached.
past_discovery_delay: u64,
/// The TCP port for libp2p. Used to convert an updated IP address to a multiaddr. Note: This
/// assumes that the external TCP port is the same as the internal TCP port if behind a NAT.
//TODO: Improve NAT handling limit the above restriction
tcp_port: u16,
/// The discovery behaviour used to discover new peers.
discovery: Discv5<TSubstream>,
/// A collection of network constants that can be read from other threads.
network_globals: Arc<NetworkGlobals<TSpec>>,
/// Logger for the discovery behaviour.
log: slog::Logger,
}
impl<TSubstream, TSpec: EthSpec> Discovery<TSubstream, TSpec> {
pub fn new(
local_key: &Keypair,
config: &NetworkConfig,
network_globals: Arc<NetworkGlobals<TSpec>>,
log: &slog::Logger,
) -> error::Result<Self> {
let log = log.clone();
let enr_dir = match config.network_dir.to_str() {
Some(path) => String::from(path),
None => String::from(""),
};
let local_enr = network_globals.local_enr.read().clone();
info!(log, "ENR Initialised"; "enr" => local_enr.to_base64(), "seq" => local_enr.seq(), "id"=> format!("{}",local_enr.node_id()), "ip" => format!("{:?}", local_enr.ip()), "udp"=> format!("{:?}", local_enr.udp()), "tcp" => format!("{:?}", local_enr.tcp()));
let listen_socket = SocketAddr::new(config.listen_address, config.discovery_port);
let mut discovery = Discv5::new(
local_enr,
local_key.clone(),
config.discv5_config.clone(),
listen_socket,
)
.map_err(|e| format!("Discv5 service failed. Error: {:?}", e))?;
// Add bootnodes to routing table
for bootnode_enr in config.boot_nodes.clone() {
debug!(
log,
"Adding node to routing table";
"node_id" => format!("{}", bootnode_enr.node_id()),
"peer_id" => format!("{}", bootnode_enr.peer_id()),
"ip" => format!("{:?}", bootnode_enr.ip()),
"udp" => format!("{:?}", bootnode_enr.udp()),
"tcp" => format!("{:?}", bootnode_enr.tcp())
);
let _ = discovery.add_enr(bootnode_enr).map_err(|e| {
warn!(
log,
"Could not add peer to the local routing table";
"error" => format!("{}", e)
)
});
}
Ok(Self {
events: VecDeque::with_capacity(16),
banned_peers: HashSet::new(),
max_peers: config.max_peers,
peer_discovery_delay: Delay::new(Instant::now()),
past_discovery_delay: INITIAL_SEARCH_DELAY,
tcp_port: config.libp2p_port,
discovery,
network_globals,
log,
enr_dir,
})
}
/// Return the nodes local ENR.
pub fn local_enr(&self) -> &Enr {
self.discovery.local_enr()
}
/// Manually search for peers. This restarts the discovery round, sparking multiple rapid
/// queries.
pub fn discover_peers(&mut self) {
self.past_discovery_delay = INITIAL_SEARCH_DELAY;
self.find_peers();
}
/// Add an ENR to the routing table of the discovery mechanism.
pub fn add_enr(&mut self, enr: Enr) {
let _ = self.discovery.add_enr(enr).map_err(|e| {
warn!(
self.log,
"Could not add peer to the local routing table";
"error" => format!("{}", e)
)
});
}
/// The peer has been banned. Add this peer to the banned list to prevent any future
/// re-connections.
// TODO: Remove the peer from the DHT if present
pub fn peer_banned(&mut self, peer_id: PeerId) {
self.banned_peers.insert(peer_id);
}
pub fn peer_unbanned(&mut self, peer_id: &PeerId) {
self.banned_peers.remove(peer_id);
}
/// Returns an iterator over all enr entries in the DHT.
pub fn enr_entries(&mut self) -> impl Iterator<Item = &Enr> {
self.discovery.enr_entries()
}
/// Returns the ENR of a known peer if it exists.
pub fn enr_of_peer(&mut self, peer_id: &PeerId) -> Option<Enr> {
self.discovery.enr_of_peer(peer_id)
}
/// Adds/Removes a subnet from the ENR Bitfield
pub fn update_enr_bitfield(&mut self, subnet_id: SubnetId, value: bool) -> Result<(), String> {
let id = *subnet_id as usize;
let local_enr = self.discovery.local_enr();
let mut current_bitfield = local_enr.bitfield::<TSpec>()?;
if id >= current_bitfield.len() {
return Err(format!(
"Subnet id: {} is outside the ENR bitfield length: {}",
id,
current_bitfield.len()
));
}
if current_bitfield
.get(id)
.map_err(|_| String::from("Subnet ID out of bounds"))?
== value
{
return Err(format!(
"Subnet id: {} already in the local ENR already has value: {}",
id, value
));
}
// set the subnet bitfield in the ENR
current_bitfield
.set(id, value)
.map_err(|_| String::from("Subnet ID out of bounds, could not set subnet ID"))?;
// insert the bitfield into the ENR record
let _ = self
.discovery
.enr_insert(BITFIELD_ENR_KEY, current_bitfield.as_ssz_bytes());
// replace the global version
*self.network_globals.local_enr.write() = self.discovery.local_enr().clone();
Ok(())
}
/// Updates the `eth2` field of our local ENR.
pub fn update_eth2_enr(&mut self, enr_fork_id: EnrForkId) {
// to avoid having a reference to the spec constant, for the logging we assume
// FAR_FUTURE_EPOCH is u64::max_value()
let next_fork_epoch_log = if enr_fork_id.next_fork_epoch == u64::max_value() {
String::from("No other fork")
} else {
format!("{:?}", enr_fork_id.next_fork_epoch)
};
info!(self.log, "Updating the ENR fork version";
"fork_digest" => format!("{:?}", enr_fork_id.fork_digest),
"next_fork_version" => format!("{:?}", enr_fork_id.next_fork_version),
"next_fork_epoch" => next_fork_epoch_log,
);
let _ = self
.discovery
.enr_insert(ETH2_ENR_KEY.into(), enr_fork_id.as_ssz_bytes())
.map_err(|e| {
warn!(
self.log,
"Could not update eth2 ENR field";
"error" => format!("{:?}", e)
)
});
// replace the global version with discovery version
*self.network_globals.local_enr.write() = self.discovery.local_enr().clone();
}
/// A request to find peers on a given subnet.
// TODO: This logic should be improved with added sophistication in peer management
// This currently checks for currently connected peers and if we don't have
// PEERS_WANTED_BEFORE_DISCOVERY connected to a given subnet we search for more.
pub fn peers_request(&mut self, subnet_id: SubnetId) {
let peers_on_subnet = self
.network_globals
.peers
.read()
.peers_on_subnet(&subnet_id)
.count() as u64;
if peers_on_subnet < TARGET_SUBNET_PEERS {
let target_peers = TARGET_SUBNET_PEERS - peers_on_subnet;
debug!(self.log, "Searching for peers for subnet";
"subnet_id" => *subnet_id,
"connected_peers_on_subnet" => peers_on_subnet,
"target_subnet_peers" => TARGET_SUBNET_PEERS,
"peers_to_find" => target_peers
);
let log_clone = self.log.clone();
let subnet_predicate = move |enr: &Enr| {
if let Some(bitfield_bytes) = enr.get(BITFIELD_ENR_KEY) {
let bitfield = match BitVector::<TSpec::SubnetBitfieldLength>::from_ssz_bytes(
bitfield_bytes,
) {
Ok(v) => v,
Err(e) => {
warn!(log_clone, "Could not decode ENR bitfield for peer"; "peer_id" => format!("{}", enr.peer_id()), "error" => format!("{:?}", e));
return false;
}
};
return bitfield.get(*subnet_id as usize).unwrap_or_else(|_| {
debug!(log_clone, "Peer found but not on desired subnet"; "peer_id" => format!("{}", enr.peer_id()));
false
});
}
false
};
// start the query
self.start_query(subnet_predicate, target_peers as usize);
} else {
debug!(self.log, "Discovery ignored";
"reason" => "Already connected to desired peers",
"connected_peers_on_subnet" => peers_on_subnet,
"target_subnet_peers" => TARGET_SUBNET_PEERS,
);
}
}
/* Internal Functions */
/// Run a standard query to search for more peers.
///
/// This searches for the standard kademlia bucket size (16) peers.
fn find_peers(&mut self) {
debug!(self.log, "Searching for peers");
self.start_query(|_| true, 16);
}
/// Search for a specified number of new peers using the underlying discovery mechanism.
///
/// This can optionally search for peers for a given predicate. Regardless of the predicate
/// given, this will only search for peers on the same enr_fork_id as specified in the local
/// ENR.
fn start_query<F>(&mut self, enr_predicate: F, num_nodes: usize)
where
F: Fn(&Enr) -> bool + Send + 'static + Clone,
{
// pick a random NodeId
let random_node = NodeId::random();
let enr_fork_id = match self.local_enr().eth2() {
Ok(v) => v,
Err(e) => {
crit!(self.log, "Local ENR has no fork id"; "error" => e);
return;
}
};
// predicate for finding nodes with a matching fork
let eth2_fork_predicate = move |enr: &Enr| enr.eth2() == Ok(enr_fork_id.clone());
let predicate = move |enr: &Enr| eth2_fork_predicate(enr) && enr_predicate(enr);
// general predicate
self.discovery
.find_enr_predicate(random_node, predicate, num_nodes);
}
}
// Redirect all behaviour events to underlying discovery behaviour.
impl<TSubstream, TSpec: EthSpec> NetworkBehaviour for Discovery<TSubstream, TSpec>
where
TSubstream: AsyncRead + AsyncWrite,
{
type ProtocolsHandler = <Discv5<TSubstream> as NetworkBehaviour>::ProtocolsHandler;
type OutEvent = <Discv5<TSubstream> as NetworkBehaviour>::OutEvent;
fn new_handler(&mut self) -> Self::ProtocolsHandler {
NetworkBehaviour::new_handler(&mut self.discovery)
}
fn addresses_of_peer(&mut self, peer_id: &PeerId) -> Vec<Multiaddr> {
// Let discovery track possible known peers.
self.discovery.addresses_of_peer(peer_id)
}
fn inject_connected(&mut self, _peer_id: PeerId, _endpoint: ConnectedPoint) {}
fn inject_disconnected(&mut self, _peer_id: &PeerId, _endpoint: ConnectedPoint) {}
fn inject_replaced(
&mut self,
_peer_id: PeerId,
_closed: ConnectedPoint,
_opened: ConnectedPoint,
) {
// discv5 doesn't implement
}
fn inject_node_event(
&mut self,
_peer_id: PeerId,
_event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent,
) {
// discv5 doesn't implement
}
fn poll(
&mut self,
params: &mut impl PollParameters,
) -> Async<
NetworkBehaviourAction<
<Self::ProtocolsHandler as ProtocolsHandler>::InEvent,
Self::OutEvent,
>,
> {
// search for peers if it is time
loop {
match self.peer_discovery_delay.poll() {
Ok(Async::Ready(_)) => {
if self.network_globals.connected_peers() < self.max_peers {
self.find_peers();
}
// Set to maximum, and update to earlier, once we get our results back.
self.peer_discovery_delay.reset(
Instant::now() + Duration::from_secs(MAX_TIME_BETWEEN_PEER_SEARCHES),
);
}
Ok(Async::NotReady) => break,
Err(e) => {
warn!(self.log, "Discovery peer search failed"; "error" => format!("{:?}", e));
}
}
}
// Poll discovery
loop {
match self.discovery.poll(params) {
Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) => {
match event {
Discv5Event::Discovered(_enr) => {
// peers that get discovered during a query but are not contactable or
// don't match a predicate can end up here. For debugging purposes we
// log these to see if we are unnecessarily dropping discovered peers
/*
if enr.eth2() == self.local_enr().eth2() {
trace!(self.log, "Peer found in process of query"; "peer_id" => format!("{}", enr.peer_id()), "tcp_socket" => enr.tcp_socket());
} else {
// this is temporary warning for debugging the DHT
warn!(self.log, "Found peer during discovery not on correct fork"; "peer_id" => format!("{}", enr.peer_id()), "tcp_socket" => enr.tcp_socket());
}
*/
}
Discv5Event::SocketUpdated(socket) => {
info!(self.log, "Address updated"; "ip" => format!("{}",socket.ip()), "udp_port" => format!("{}", socket.port()));
metrics::inc_counter(&metrics::ADDRESS_UPDATE_COUNT);
let mut address = Multiaddr::from(socket.ip());
address.push(Protocol::Tcp(self.tcp_port));
let enr = self.discovery.local_enr();
enr::save_enr_to_disk(Path::new(&self.enr_dir), enr, &self.log);
return Async::Ready(NetworkBehaviourAction::ReportObservedAddr {
address,
});
}
Discv5Event::FindNodeResult { closer_peers, .. } => {
debug!(self.log, "Discovery query completed"; "peers_found" => closer_peers.len());
// update the time to the next query
if self.past_discovery_delay < MAX_TIME_BETWEEN_PEER_SEARCHES {
self.past_discovery_delay *= 2;
}
let delay = std::cmp::max(
self.past_discovery_delay,
MAX_TIME_BETWEEN_PEER_SEARCHES,
);
self.peer_discovery_delay
.reset(Instant::now() + Duration::from_secs(delay));
for peer_id in closer_peers {
// if we need more peers, attempt a connection
if self.network_globals.connected_or_dialing_peers()
< self.max_peers
&& !self
.network_globals
.peers
.read()
.is_connected_or_dialing(&peer_id)
&& !self.banned_peers.contains(&peer_id)
{
debug!(self.log, "Connecting to discovered peer"; "peer_id"=> format!("{:?}", peer_id));
self.network_globals.peers.write().dialing_peer(&peer_id);
self.events
.push_back(NetworkBehaviourAction::DialPeer { peer_id });
}
}
}
_ => {}
}
}
// discv5 does not output any other NetworkBehaviourAction
Async::Ready(_) => {}
Async::NotReady => break,
}
}
// process any queued events
if let Some(event) = self.events.pop_front() {
return Async::Ready(event);
}
Async::NotReady
}
}

View File

@ -1,30 +0,0 @@
//! A collection of variables that are accessible outside of the network thread itself.
use crate::{Enr, Multiaddr, PeerId};
use parking_lot::RwLock;
use std::collections::HashSet;
use std::sync::atomic::AtomicUsize;
pub struct NetworkGlobals {
/// The current local ENR.
pub local_enr: RwLock<Option<Enr>>,
/// The local peer_id.
pub peer_id: RwLock<PeerId>,
/// Listening multiaddrs.
pub listen_multiaddrs: RwLock<Vec<Multiaddr>>,
/// Current number of connected libp2p peers.
pub connected_peers: AtomicUsize,
/// The collection of currently connected peers.
pub connected_peer_set: RwLock<HashSet<PeerId>>,
}
impl NetworkGlobals {
pub fn new(peer_id: PeerId) -> Self {
NetworkGlobals {
local_enr: RwLock::new(None),
peer_id: RwLock::new(peer_id),
listen_multiaddrs: RwLock::new(Vec::new()),
connected_peers: AtomicUsize::new(0),
connected_peer_set: RwLock::new(HashSet::new()),
}
}
}

View File

@ -7,26 +7,19 @@ extern crate lazy_static;
pub mod behaviour; pub mod behaviour;
mod config; mod config;
mod discovery; pub mod discovery;
pub mod error;
mod globals;
mod metrics; mod metrics;
mod peer_manager;
pub mod rpc; pub mod rpc;
mod service; mod service;
mod topics; pub mod types;
pub use behaviour::PubsubMessage; pub use crate::types::{error, Enr, GossipTopic, NetworkGlobals, PubsubMessage};
pub use behaviour::BehaviourEvent;
pub use config::Config as NetworkConfig; pub use config::Config as NetworkConfig;
pub use globals::NetworkGlobals;
pub use libp2p::enr::Enr;
pub use libp2p::gossipsub::{MessageId, Topic, TopicHash}; pub use libp2p::gossipsub::{MessageId, Topic, TopicHash};
pub use libp2p::multiaddr; pub use libp2p::{multiaddr, Multiaddr};
pub use libp2p::Multiaddr; pub use libp2p::{PeerId, Swarm};
pub use libp2p::{ pub use peer_manager::{PeerDB, PeerInfo, PeerSyncStatus, SyncInfo};
gossipsub::{GossipsubConfig, GossipsubConfigBuilder},
PeerId, Swarm,
};
pub use rpc::RPCEvent; pub use rpc::RPCEvent;
pub use service::Libp2pEvent; pub use service::{Service, NETWORK_KEY_FILENAME};
pub use service::Service;
pub use topics::GossipTopic;

View File

@ -0,0 +1,139 @@
//! Known Ethereum 2.0 clients and their fingerprints.
//!
//! Currently using identify to fingerprint.
use libp2p::identify::IdentifyInfo;
use serde::Serialize;
/// Various client and protocol information related to a node.
#[derive(Clone, Debug, Serialize)]
pub struct Client {
/// The client's name (Ex: lighthouse, prism, nimbus, etc)
pub kind: ClientKind,
/// The client's version.
pub version: String,
/// The OS version of the client.
pub os_version: String,
/// The libp2p protocol version.
pub protocol_version: String,
/// Identify agent string
pub agent_string: Option<String>,
}
#[derive(Clone, Debug, Serialize)]
pub enum ClientKind {
/// A lighthouse node (the best kind).
Lighthouse,
/// A Nimbus node.
Nimbus,
/// A Teku node.
Teku,
/// A Prysm node.
Prysm,
/// An unknown client.
Unknown,
}
impl Default for Client {
fn default() -> Self {
Client {
kind: ClientKind::Unknown,
version: "unknown".into(),
os_version: "unknown".into(),
protocol_version: "unknown".into(),
agent_string: None,
}
}
}
impl Client {
/// Builds a `Client` from `IdentifyInfo`.
pub fn from_identify_info(info: &IdentifyInfo) -> Self {
let (kind, version, os_version) = client_from_agent_version(&info.agent_version);
Client {
kind,
version,
os_version,
protocol_version: info.protocol_version.clone(),
agent_string: Some(info.agent_version.clone()),
}
}
}
impl std::fmt::Display for Client {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self.kind {
ClientKind::Lighthouse => write!(
f,
"Lighthouse: version: {}, os_version: {}",
self.version, self.os_version
),
ClientKind::Teku => write!(
f,
"Teku: version: {}, os_version: {}",
self.version, self.os_version
),
ClientKind::Nimbus => write!(
f,
"Nimbus: version: {}, os_version: {}",
self.version, self.os_version
),
ClientKind::Prysm => write!(
f,
"Prysm: version: {}, os_version: {}",
self.version, self.os_version
),
ClientKind::Unknown => {
if let Some(agent_string) = &self.agent_string {
write!(f, "Unknown: {}", agent_string)
} else {
write!(f, "Unknown")
}
}
}
}
}
// helper function to identify clients from their agent_version. Returns the client
// kind and it's associated version and the OS kind.
fn client_from_agent_version(agent_version: &str) -> (ClientKind, String, String) {
let mut agent_split = agent_version.split("/");
match agent_split.next() {
Some("Lighthouse") => {
let kind = ClientKind::Lighthouse;
let mut version = String::from("unknown");
let mut os_version = version.clone();
if let Some(agent_version) = agent_split.next() {
version = agent_version.into();
if let Some(agent_os_version) = agent_split.next() {
os_version = agent_os_version.into();
}
}
(kind, version, os_version)
}
Some("teku") => {
let kind = ClientKind::Teku;
let mut version = String::from("unknown");
let mut os_version = version.clone();
if let Some(_) = agent_split.next() {
if let Some(agent_version) = agent_split.next() {
version = agent_version.into();
if let Some(agent_os_version) = agent_split.next() {
os_version = agent_os_version.into();
}
}
}
(kind, version, os_version)
}
Some("github.com") => {
let kind = ClientKind::Prysm;
let unknown = String::from("unknown");
(kind, unknown.clone(), unknown)
}
_ => {
let unknown = String::from("unknown");
(ClientKind::Unknown, unknown.clone(), unknown)
}
}
}

View File

@ -0,0 +1,341 @@
//! Implementation of a Lighthouse's peer management system.
pub use self::peerdb::*;
use crate::metrics;
use crate::rpc::MetaData;
use crate::{NetworkGlobals, PeerId};
use futures::prelude::*;
use futures::Stream;
use hashmap_delay::HashSetDelay;
use libp2p::identify::IdentifyInfo;
use slog::{crit, debug, error, warn};
use smallvec::SmallVec;
use std::sync::Arc;
use std::time::{Duration, Instant};
use types::EthSpec;
mod client;
mod peer_info;
mod peer_sync_status;
mod peerdb;
pub use peer_info::PeerInfo;
pub use peer_sync_status::{PeerSyncStatus, SyncInfo};
/// The minimum reputation before a peer is disconnected.
// Most likely this needs tweaking
const _MINIMUM_REPUTATION_BEFORE_BAN: Rep = 20;
/// The time in seconds between re-status's peers.
const STATUS_INTERVAL: u64 = 300;
/// The time in seconds between PING events. We do not send a ping if the other peer as PING'd us within
/// this time frame (Seconds)
const PING_INTERVAL: u64 = 30;
/// The main struct that handles peer's reputation and connection status.
pub struct PeerManager<TSpec: EthSpec> {
/// Storage of network globals to access the PeerDB.
network_globals: Arc<NetworkGlobals<TSpec>>,
/// A queue of events that the `PeerManager` is waiting to produce.
events: SmallVec<[PeerManagerEvent; 5]>,
/// A collection of peers awaiting to be Ping'd.
ping_peers: HashSetDelay<PeerId>,
/// A collection of peers awaiting to be Status'd.
status_peers: HashSetDelay<PeerId>,
/// Last updated moment.
last_updated: Instant,
/// The logger associated with the `PeerManager`.
log: slog::Logger,
}
/// A collection of actions a peer can perform which will adjust its reputation
/// Each variant has an associated reputation change.
pub enum PeerAction {
/// The peer timed out on an RPC request/response.
_TimedOut = -10,
/// The peer sent and invalid request/response or encoding.
_InvalidMessage = -20,
/// The peer sent something objectively malicious.
_Malicious = -50,
/// Received an expected message.
_ValidMessage = 20,
/// Peer disconnected.
Disconnected = -30,
}
/// The events that the PeerManager outputs (requests).
pub enum PeerManagerEvent {
/// Sends a STATUS to a peer.
Status(PeerId),
/// Sends a PING to a peer.
Ping(PeerId),
/// Request METADATA from a peer.
MetaData(PeerId),
/// The peer should be disconnected.
_DisconnectPeer(PeerId),
/// The peer should be disconnected and banned.
_BanPeer(PeerId),
}
impl<TSpec: EthSpec> PeerManager<TSpec> {
pub fn new(network_globals: Arc<NetworkGlobals<TSpec>>, log: &slog::Logger) -> Self {
PeerManager {
network_globals,
events: SmallVec::new(),
last_updated: Instant::now(),
ping_peers: HashSetDelay::new(Duration::from_secs(PING_INTERVAL)),
status_peers: HashSetDelay::new(Duration::from_secs(STATUS_INTERVAL)),
log: log.clone(),
}
}
/* Public accessible functions */
/// A ping request has been received.
// NOTE: The behaviour responds with a PONG automatically
// TODO: Update last seen
pub fn ping_request(&mut self, peer_id: &PeerId, seq: u64) {
if let Some(peer_info) = self.network_globals.peers.read().peer_info(peer_id) {
// received a ping
// reset the to-ping timer for this peer
self.ping_peers.insert(peer_id.clone());
// if the sequence number is unknown send update the meta data of the peer.
if let Some(meta_data) = &peer_info.meta_data {
if meta_data.seq_number < seq {
debug!(self.log, "Requesting new metadata from peer"; "peer_id" => format!("{}", peer_id), "known_seq_no" => meta_data.seq_number, "ping_seq_no" => seq);
self.events
.push(PeerManagerEvent::MetaData(peer_id.clone()));
}
} else {
// if we don't know the meta-data, request it
debug!(self.log, "Requesting first metadata from peer"; "peer_id" => format!("{}", peer_id));
self.events
.push(PeerManagerEvent::MetaData(peer_id.clone()));
}
} else {
crit!(self.log, "Received a PING from an unknown peer"; "peer_id" => format!("{}", peer_id));
}
}
/// A PONG has been returned from a peer.
// TODO: Update last seen
pub fn pong_response(&mut self, peer_id: &PeerId, seq: u64) {
if let Some(peer_info) = self.network_globals.peers.read().peer_info(peer_id) {
// received a pong
// if the sequence number is unknown send update the meta data of the peer.
if let Some(meta_data) = &peer_info.meta_data {
if meta_data.seq_number < seq {
debug!(self.log, "Requesting new metadata from peer"; "peer_id" => format!("{}", peer_id), "known_seq_no" => meta_data.seq_number, "pong_seq_no" => seq);
self.events
.push(PeerManagerEvent::MetaData(peer_id.clone()));
}
} else {
// if we don't know the meta-data, request it
debug!(self.log, "Requesting first metadata from peer"; "peer_id" => format!("{}", peer_id));
self.events
.push(PeerManagerEvent::MetaData(peer_id.clone()));
}
} else {
crit!(self.log, "Received a PONG from an unknown peer"; "peer_id" => format!("{}", peer_id));
}
}
/// Received a metadata response from a peer.
// TODO: Update last seen
pub fn meta_data_response(&mut self, peer_id: &PeerId, meta_data: MetaData<TSpec>) {
if let Some(peer_info) = self.network_globals.peers.write().peer_info_mut(peer_id) {
if let Some(known_meta_data) = &peer_info.meta_data {
if known_meta_data.seq_number < meta_data.seq_number {
debug!(self.log, "Updating peer's metadata"; "peer_id" => format!("{}", peer_id), "known_seq_no" => known_meta_data.seq_number, "new_seq_no" => meta_data.seq_number);
} else {
warn!(self.log, "Received old metadata"; "peer_id" => format!("{}", peer_id), "known_seq_no" => known_meta_data.seq_number, "new_seq_no" => meta_data.seq_number);
}
} else {
// we have no meta-data for this peer, update
debug!(self.log, "Obtained peer's metadata"; "peer_id" => format!("{}", peer_id), "new_seq_no" => meta_data.seq_number);
peer_info.meta_data = Some(meta_data);
}
} else {
crit!(self.log, "Received METADATA from an unknown peer"; "peer_id" => format!("{}", peer_id));
}
}
/// A STATUS message has been received from a peer. This resets the status timer.
pub fn peer_statusd(&mut self, peer_id: &PeerId) {
self.status_peers.insert(peer_id.clone());
}
/// Checks the reputation of a peer and if it is too low, bans it and
/// sends the corresponding event. Informs if it got banned
fn _gets_banned(&mut self, peer_id: &PeerId) -> bool {
// if the peer was already banned don't inform again
let mut peerdb = self.network_globals.peers.write();
if let Some(connection_status) = peerdb.connection_status(peer_id) {
if peerdb.reputation(peer_id) < _MINIMUM_REPUTATION_BEFORE_BAN
&& !connection_status.is_banned()
{
peerdb.ban(peer_id);
self.events
.push(PeerManagerEvent::_BanPeer(peer_id.clone()));
return true;
}
}
false
}
/// Requests that a peer get disconnected.
pub fn _disconnect_peer(&mut self, peer_id: &PeerId) {
self.events
.push(PeerManagerEvent::_DisconnectPeer(peer_id.clone()));
}
/// Updates the state of the peer as disconnected.
pub fn notify_disconnect(&mut self, peer_id: &PeerId) {
self.update_reputations();
{
let mut peerdb = self.network_globals.peers.write();
peerdb.disconnect(peer_id);
peerdb.add_reputation(peer_id, PeerAction::Disconnected as Rep);
}
// remove the ping and status timer for the peer
self.ping_peers.remove(peer_id);
self.status_peers.remove(peer_id);
metrics::inc_counter(&metrics::PEER_DISCONNECT_EVENT_COUNT);
metrics::set_gauge(
&metrics::PEERS_CONNECTED,
self.network_globals.connected_peers() as i64,
);
}
/// Sets a peer as connected as long as their reputation allows it
/// Informs if the peer was accepted
pub fn connect_ingoing(&mut self, peer_id: &PeerId) -> bool {
self.connect_peer(peer_id, false)
}
/// Sets a peer as connected as long as their reputation allows it
/// Informs if the peer was accepted
pub fn connect_outgoing(&mut self, peer_id: &PeerId) -> bool {
self.connect_peer(peer_id, true)
}
/// Provides a given peer's reputation if it exists.
pub fn _get_peer_rep(&self, peer_id: &PeerId) -> Rep {
self.network_globals.peers.read().reputation(peer_id)
}
/// Updates the reputation of known peers according to their connection
/// status and the time that has passed.
pub fn update_reputations(&mut self) {
let now = Instant::now();
let elapsed = (now - self.last_updated).as_secs();
// 0 seconds means now - last_updated < 0, but (most likely) not = 0.
// In this case, do nothing (updating last_updated would propagate
// rounding errors)
if elapsed > 0 {
self.last_updated = now;
// TODO decide how reputations change with time. If they get too low
// set the peers as banned
}
}
/// Reports a peer for some action.
///
/// If the peer doesn't exist, log a warning and insert defaults.
pub fn _report_peer(&mut self, peer_id: &PeerId, action: PeerAction) {
self.update_reputations();
self.network_globals
.peers
.write()
.add_reputation(peer_id, action as Rep);
self.update_reputations();
}
/// Updates `PeerInfo` with `identify` information.
pub fn identify(&mut self, peer_id: &PeerId, info: &IdentifyInfo) {
if let Some(peer_info) = self.network_globals.peers.write().peer_info_mut(peer_id) {
peer_info.client = client::Client::from_identify_info(info);
peer_info.listening_addresses = info.listen_addrs.clone();
} else {
crit!(self.log, "Received an Identify response from an unknown peer"; "peer_id" => format!("{}", peer_id));
}
}
/* Internal functions */
/// Registers a peer as connected. The `ingoing` parameter determines if the peer is being
/// dialed or connecting to us.
///
/// This is called by `connect_ingoing` and `connect_outgoing`.
///
/// This informs if the peer was accepted in to the db or not.
// TODO: Drop peers if over max_peer limit
fn connect_peer(&mut self, peer_id: &PeerId, outgoing: bool) -> bool {
// TODO: Call this on a timer
self.update_reputations();
{
let mut peerdb = self.network_globals.peers.write();
if peerdb.connection_status(peer_id).map(|c| c.is_banned()) == Some(true) {
// don't connect if the peer is banned
return false;
}
if outgoing {
peerdb.connect_outgoing(peer_id);
} else {
peerdb.connect_outgoing(peer_id);
}
}
// start a ping and status timer for the peer
self.ping_peers.insert(peer_id.clone());
self.status_peers.insert(peer_id.clone());
// increment prometheus metrics
metrics::inc_counter(&metrics::PEER_CONNECT_EVENT_COUNT);
metrics::set_gauge(
&metrics::PEERS_CONNECTED,
self.network_globals.connected_peers() as i64,
);
true
}
/// Notifies the peer manager that this peer is being dialed.
pub fn _dialing_peer(&mut self, peer_id: &PeerId) {
self.network_globals.peers.write().dialing_peer(peer_id);
}
}
impl<TSpec: EthSpec> Stream for PeerManager<TSpec> {
type Item = PeerManagerEvent;
type Error = ();
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
// poll the timeouts for pings and status'
while let Async::Ready(Some(peer_id)) = self.ping_peers.poll().map_err(|e| {
error!(self.log, "Failed to check for peers to ping"; "error" => format!("{}",e));
})? {
debug!(self.log, "Pinging peer"; "peer_id" => format!("{}", peer_id));
self.events.push(PeerManagerEvent::Ping(peer_id));
}
while let Async::Ready(Some(peer_id)) = self.status_peers.poll().map_err(|e| {
error!(self.log, "Failed to check for peers to status"; "error" => format!("{}",e));
})? {
debug!(self.log, "Sending Status to peer"; "peer_id" => format!("{}", peer_id));
self.events.push(PeerManagerEvent::Status(peer_id));
}
if !self.events.is_empty() {
return Ok(Async::Ready(Some(self.events.remove(0))));
} else {
self.events.shrink_to_fit();
}
Ok(Async::NotReady)
}
}

View File

@ -0,0 +1,217 @@
use super::client::Client;
use super::peerdb::{Rep, DEFAULT_REPUTATION};
use super::PeerSyncStatus;
use crate::rpc::MetaData;
use crate::Multiaddr;
use serde::{
ser::{SerializeStructVariant, Serializer},
Serialize,
};
use std::time::Instant;
use types::{EthSpec, SubnetId};
use PeerConnectionStatus::*;
/// Information about a given connected peer.
#[derive(Clone, Debug, Serialize)]
#[serde(bound = "T: EthSpec")]
pub struct PeerInfo<T: EthSpec> {
/// The connection status of the peer
_status: PeerStatus,
/// The peers reputation
pub reputation: Rep,
/// Client managing this peer
pub client: Client,
/// Connection status of this peer
pub connection_status: PeerConnectionStatus,
/// The known listening addresses of this peer.
pub listening_addresses: Vec<Multiaddr>,
/// The current syncing state of the peer. The state may be determined after it's initial
/// connection.
pub sync_status: PeerSyncStatus,
/// The ENR subnet bitfield of the peer. This may be determined after it's initial
/// connection.
pub meta_data: Option<MetaData<T>>,
}
impl<TSpec: EthSpec> Default for PeerInfo<TSpec> {
fn default() -> PeerInfo<TSpec> {
PeerInfo {
_status: Default::default(),
reputation: DEFAULT_REPUTATION,
client: Client::default(),
connection_status: Default::default(),
listening_addresses: vec![],
sync_status: PeerSyncStatus::Unknown,
meta_data: None,
}
}
}
impl<T: EthSpec> PeerInfo<T> {
/// Returns if the peer is subscribed to a given `SubnetId`
pub fn on_subnet(&self, subnet_id: SubnetId) -> bool {
if let Some(meta_data) = &self.meta_data {
return meta_data
.attnets
.get(*subnet_id as usize)
.unwrap_or_else(|_| false);
}
false
}
}
#[derive(Clone, Debug, Serialize)]
/// The current health status of the peer.
pub enum PeerStatus {
/// The peer is healthy.
Healthy,
/// The peer is clogged. It has not been responding to requests on time.
_Clogged,
}
impl Default for PeerStatus {
fn default() -> Self {
PeerStatus::Healthy
}
}
/// Connection Status of the peer.
#[derive(Debug, Clone)]
pub enum PeerConnectionStatus {
/// The peer is connected.
Connected {
/// number of ingoing connections.
n_in: u8,
/// number of outgoing connections.
n_out: u8,
},
/// The peer has disconnected.
Disconnected {
/// last time the peer was connected or discovered.
since: Instant,
},
/// The peer has been banned and is disconnected.
Banned {
/// moment when the peer was banned.
since: Instant,
},
/// We are currently dialing this peer.
Dialing {
/// time since we last communicated with the peer.
since: Instant,
},
}
/// Serialization for http requests.
impl Serialize for PeerConnectionStatus {
fn serialize<S: Serializer>(&self, serializer: S) -> Result<S::Ok, S::Error> {
match self {
Connected { n_in, n_out } => {
let mut s = serializer.serialize_struct_variant("", 0, "Connected", 2)?;
s.serialize_field("in", n_in)?;
s.serialize_field("out", n_out)?;
s.end()
}
Disconnected { since } => {
let mut s = serializer.serialize_struct_variant("", 1, "Disconnected", 1)?;
s.serialize_field("since", &since.elapsed().as_secs())?;
s.end()
}
Banned { since } => {
let mut s = serializer.serialize_struct_variant("", 2, "Banned", 1)?;
s.serialize_field("since", &since.elapsed().as_secs())?;
s.end()
}
Dialing { since } => {
let mut s = serializer.serialize_struct_variant("", 3, "Dialing", 1)?;
s.serialize_field("since", &since.elapsed().as_secs())?;
s.end()
}
}
}
}
impl Default for PeerConnectionStatus {
fn default() -> Self {
PeerConnectionStatus::Dialing {
since: Instant::now(),
}
}
}
impl PeerConnectionStatus {
/// Checks if the status is connected
pub fn is_connected(&self) -> bool {
match self {
PeerConnectionStatus::Connected { .. } => true,
_ => false,
}
}
/// Checks if the status is connected
pub fn is_dialing(&self) -> bool {
match self {
PeerConnectionStatus::Dialing { .. } => true,
_ => false,
}
}
/// Checks if the status is banned
pub fn is_banned(&self) -> bool {
match self {
PeerConnectionStatus::Banned { .. } => true,
_ => false,
}
}
/// Checks if the status is disconnected
pub fn is_disconnected(&self) -> bool {
match self {
Disconnected { .. } => true,
_ => false,
}
}
/// Modifies the status to Connected and increases the number of ingoing
/// connections by one
pub fn connect_ingoing(&mut self) {
match self {
Connected { n_in, .. } => *n_in += 1,
Disconnected { .. } | Banned { .. } | Dialing { .. } => {
*self = Connected { n_in: 1, n_out: 0 }
}
}
}
/// Modifies the status to Connected and increases the number of outgoing
/// connections by one
pub fn connect_outgoing(&mut self) {
match self {
Connected { n_out, .. } => *n_out += 1,
Disconnected { .. } | Banned { .. } | Dialing { .. } => {
*self = Connected { n_in: 0, n_out: 1 }
}
}
}
/// Modifies the status to Disconnected and sets the last seen instant to now
pub fn disconnect(&mut self) {
*self = Disconnected {
since: Instant::now(),
};
}
/// Modifies the status to Banned
pub fn ban(&mut self) {
*self = Banned {
since: Instant::now(),
};
}
pub fn connections(&self) -> (u8, u8) {
match self {
Connected { n_in, n_out } => (*n_in, *n_out),
_ => (0, 0),
}
}
}

View File

@ -0,0 +1,104 @@
//! Handles individual sync status for peers.
use serde::Serialize;
use types::{Epoch, Hash256, Slot};
#[derive(Clone, Debug, Serialize)]
/// The current sync status of the peer.
pub enum PeerSyncStatus {
/// At the current state as our node or ahead of us.
Synced { info: SyncInfo },
/// The peer has greater knowledge about the canonical chain than we do.
Advanced { info: SyncInfo },
/// Is behind our current head and not useful for block downloads.
Behind { info: SyncInfo },
/// Not currently known as a STATUS handshake has not occurred.
Unknown,
}
/// This is stored inside the PeerSyncStatus and is very similar to `PeerSyncInfo` in the
/// `Network` crate.
#[derive(Clone, Debug, Serialize)]
pub struct SyncInfo {
pub status_head_slot: Slot,
pub status_head_root: Hash256,
pub status_finalized_epoch: Epoch,
pub status_finalized_root: Hash256,
}
impl PeerSyncStatus {
/// Returns true if the peer has advanced knowledge of the chain.
pub fn is_advanced(&self) -> bool {
match self {
PeerSyncStatus::Advanced { .. } => true,
_ => false,
}
}
/// Returns true if the peer is up to date with the current chain.
pub fn is_synced(&self) -> bool {
match self {
PeerSyncStatus::Synced { .. } => true,
_ => false,
}
}
/// Returns true if the peer is behind the current chain.
pub fn is_behind(&self) -> bool {
match self {
PeerSyncStatus::Behind { .. } => true,
_ => false,
}
}
/// Updates the sync state given a fully synced peer.
/// Returns true if the state has changed.
pub fn update_synced(&mut self, info: SyncInfo) -> bool {
let new_state = PeerSyncStatus::Synced { info };
match self {
PeerSyncStatus::Synced { .. } => {
*self = new_state;
false // state was not updated
}
_ => {
*self = new_state;
true
}
}
}
/// Updates the sync state given a peer that is further ahead in the chain than us.
/// Returns true if the state has changed.
pub fn update_ahead(&mut self, info: SyncInfo) -> bool {
let new_state = PeerSyncStatus::Advanced { info };
match self {
PeerSyncStatus::Advanced { .. } => {
*self = new_state;
false // state was not updated
}
_ => {
*self = new_state;
true
}
}
}
/// Updates the sync state given a peer that is behind us in the chain.
/// Returns true if the state has changed.
pub fn update_behind(&mut self, info: SyncInfo) -> bool {
let new_state = PeerSyncStatus::Behind { info };
match self {
PeerSyncStatus::Behind { .. } => {
*self = new_state;
false // state was not updated
}
_ => {
*self = new_state;
true
}
}
}
}

View File

@ -0,0 +1,499 @@
use super::peer_info::{PeerConnectionStatus, PeerInfo};
use super::peer_sync_status::PeerSyncStatus;
use crate::rpc::methods::MetaData;
use crate::PeerId;
use slog::{crit, warn};
use std::collections::HashMap;
use std::time::Instant;
use types::{EthSpec, SubnetId};
/// A peer's reputation.
pub type Rep = i32;
/// Max number of disconnected nodes to remember
const MAX_DC_PEERS: usize = 30;
/// The default starting reputation for an unknown peer.
pub const DEFAULT_REPUTATION: Rep = 50;
/// Storage of known peers, their reputation and information
pub struct PeerDB<TSpec: EthSpec> {
/// The collection of known connected peers, their status and reputation
peers: HashMap<PeerId, PeerInfo<TSpec>>,
/// Tracking of number of disconnected nodes
n_dc: usize,
/// PeerDB's logger
log: slog::Logger,
}
impl<TSpec: EthSpec> PeerDB<TSpec> {
pub fn new(log: &slog::Logger) -> Self {
Self {
log: log.clone(),
n_dc: 0,
peers: HashMap::new(),
}
}
/* Getters */
/// Gives the reputation of a peer, or DEFAULT_REPUTATION if it is unknown.
pub fn reputation(&self, peer_id: &PeerId) -> Rep {
self.peers
.get(peer_id)
.map_or(DEFAULT_REPUTATION, |info| info.reputation)
}
/// Returns an iterator over all peers in the db.
pub fn peers(&self) -> impl Iterator<Item = (&PeerId, &PeerInfo<TSpec>)> {
self.peers.iter()
}
/// Gives the ids of all known peers.
pub fn peer_ids(&self) -> impl Iterator<Item = &PeerId> {
self.peers.keys()
}
/// Returns a peer's info, if known.
pub fn peer_info(&self, peer_id: &PeerId) -> Option<&PeerInfo<TSpec>> {
self.peers.get(peer_id)
}
/// Returns a mutable reference to a peer's info if known.
pub fn peer_info_mut(&mut self, peer_id: &PeerId) -> Option<&mut PeerInfo<TSpec>> {
self.peers.get_mut(peer_id)
}
/// Returns true if the peer is synced at least to our current head.
pub fn peer_synced(&self, peer_id: &PeerId) -> bool {
match self.peers.get(peer_id).map(|info| &info.sync_status) {
Some(PeerSyncStatus::Synced { .. }) => true,
Some(_) => false,
None => false,
}
}
/// Gives the ids of all known connected peers.
pub fn connected_peers(&self) -> impl Iterator<Item = (&PeerId, &PeerInfo<TSpec>)> {
self.peers
.iter()
.filter(|(_, info)| info.connection_status.is_connected())
}
/// Gives the ids of all known connected peers.
pub fn connected_peer_ids(&self) -> impl Iterator<Item = &PeerId> {
self.peers
.iter()
.filter(|(_, info)| info.connection_status.is_connected())
.map(|(peer_id, _)| peer_id)
}
/// Connected or dialing peers
pub fn connected_or_dialing_peers(&self) -> impl Iterator<Item = &PeerId> {
self.peers
.iter()
.filter(|(_, info)| {
info.connection_status.is_connected() || info.connection_status.is_dialing()
})
.map(|(peer_id, _)| peer_id)
}
/// Gives the `peer_id` of all known connected and synced peers.
pub fn synced_peers(&self) -> impl Iterator<Item = &PeerId> {
self.peers
.iter()
.filter(|(_, info)| {
if info.sync_status.is_synced() || info.sync_status.is_advanced() {
return info.connection_status.is_connected();
}
false
})
.map(|(peer_id, _)| peer_id)
}
/// Gives an iterator of all peers on a given subnet.
pub fn peers_on_subnet(&self, subnet_id: &SubnetId) -> impl Iterator<Item = &PeerId> {
let subnet_id_filter = subnet_id.clone();
self.peers
.iter()
.filter(move |(_, info)| {
info.connection_status.is_connected() && info.on_subnet(subnet_id_filter)
})
.map(|(peer_id, _)| peer_id)
}
/// Gives the ids of all known disconnected peers.
pub fn disconnected_peers(&self) -> impl Iterator<Item = &PeerId> {
self.peers
.iter()
.filter(|(_, info)| info.connection_status.is_disconnected())
.map(|(peer_id, _)| peer_id)
}
/// Gives the ids of all known banned peers.
pub fn banned_peers(&self) -> impl Iterator<Item = &PeerId> {
self.peers
.iter()
.filter(|(_, info)| info.connection_status.is_banned())
.map(|(peer_id, _)| peer_id)
}
/// Returns a vector containing peers (their ids and info), sorted by
/// reputation from highest to lowest, and filtered using `is_status`
pub fn best_peers_by_status<F>(&self, is_status: F) -> Vec<(&PeerId, &PeerInfo<TSpec>)>
where
F: Fn(&PeerConnectionStatus) -> bool,
{
let mut by_status = self
.peers
.iter()
.filter(|(_, info)| is_status(&info.connection_status))
.collect::<Vec<_>>();
by_status.sort_by_key(|(_, info)| Rep::max_value() - info.reputation);
by_status
}
/// Returns the peer with highest reputation that satisfies `is_status`
pub fn best_by_status<F>(&self, is_status: F) -> Option<&PeerId>
where
F: Fn(&PeerConnectionStatus) -> bool,
{
self.peers
.iter()
.filter(|(_, info)| is_status(&info.connection_status))
.max_by_key(|(_, info)| info.reputation)
.map(|(id, _)| id)
}
/// Returns the peer's connection status. Returns unknown if the peer is not in the DB.
pub fn connection_status(&self, peer_id: &PeerId) -> Option<PeerConnectionStatus> {
self.peer_info(peer_id)
.map(|info| info.connection_status.clone())
}
/// Returns if the peer is already connected.
pub fn is_connected(&self, peer_id: &PeerId) -> bool {
if let Some(PeerConnectionStatus::Connected { .. }) = self.connection_status(peer_id) {
true
} else {
false
}
}
/// If we are connected or currently dialing the peer returns true.
pub fn is_connected_or_dialing(&self, peer_id: &PeerId) -> bool {
match self.connection_status(peer_id) {
Some(PeerConnectionStatus::Connected { .. })
| Some(PeerConnectionStatus::Dialing { .. }) => true,
_ => false,
}
}
/* Setters */
/// A peer is being dialed.
pub fn dialing_peer(&mut self, peer_id: &PeerId) {
let info = self
.peers
.entry(peer_id.clone())
.or_insert_with(|| Default::default());
if info.connection_status.is_disconnected() {
self.n_dc -= 1;
}
info.connection_status = PeerConnectionStatus::Dialing {
since: Instant::now(),
};
}
/// Sets a peer as connected with an ingoing connection.
pub fn connect_ingoing(&mut self, peer_id: &PeerId) {
let info = self
.peers
.entry(peer_id.clone())
.or_insert_with(|| Default::default());
if info.connection_status.is_disconnected() {
self.n_dc -= 1;
}
info.connection_status.connect_ingoing();
}
/// Sets a peer as connected with an outgoing connection.
pub fn connect_outgoing(&mut self, peer_id: &PeerId) {
let info = self
.peers
.entry(peer_id.clone())
.or_insert_with(|| Default::default());
if info.connection_status.is_disconnected() {
self.n_dc -= 1;
}
info.connection_status.connect_outgoing();
}
/// Sets the peer as disconnected.
pub fn disconnect(&mut self, peer_id: &PeerId) {
let log_ref = &self.log;
let info = self.peers.entry(peer_id.clone()).or_insert_with(|| {
warn!(log_ref, "Disconnecting unknown peer";
"peer_id" => format!("{:?}",peer_id));
PeerInfo::default()
});
if !info.connection_status.is_disconnected() {
info.connection_status.disconnect();
self.n_dc += 1;
}
self.shrink_to_fit();
}
/// Drops the peers with the lowest reputation so that the number of
/// disconnected peers is less than MAX_DC_PEERS
pub fn shrink_to_fit(&mut self) {
// for caution, but the difference should never be > 1
while self.n_dc > MAX_DC_PEERS {
let to_drop = self
.peers
.iter()
.filter(|(_, info)| info.connection_status.is_disconnected())
.min_by_key(|(_, info)| info.reputation)
.map(|(id, _)| id.clone())
.unwrap(); // should be safe since n_dc > MAX_DC_PEERS > 0
self.peers.remove(&to_drop);
self.n_dc -= 1;
}
}
/// Sets a peer as banned
pub fn ban(&mut self, peer_id: &PeerId) {
let log_ref = &self.log;
let info = self.peers.entry(peer_id.clone()).or_insert_with(|| {
warn!(log_ref, "Banning unknown peer";
"peer_id" => format!("{:?}",peer_id));
PeerInfo::default()
});
if info.connection_status.is_disconnected() {
self.n_dc -= 1;
}
info.connection_status.ban();
}
/// Add the meta data of a peer.
pub fn add_metadata(&mut self, peer_id: &PeerId, meta_data: MetaData<TSpec>) {
if let Some(peer_info) = self.peers.get_mut(peer_id) {
peer_info.meta_data = Some(meta_data);
} else {
warn!(self.log, "Tried to add meta data for a non-existant peer"; "peer_id" => format!("{}", peer_id));
}
}
/// Sets the reputation of peer.
pub fn set_reputation(&mut self, peer_id: &PeerId, rep: Rep) {
if let Some(peer_info) = self.peers.get_mut(peer_id) {
peer_info.reputation = rep;
} else {
crit!(self.log, "Tried to modify reputation for an unknown peer"; "peer_id" => format!("{}",peer_id));
}
}
/// Sets the syncing status of a peer.
pub fn set_sync_status(&mut self, peer_id: &PeerId, sync_status: PeerSyncStatus) {
if let Some(peer_info) = self.peers.get_mut(peer_id) {
peer_info.sync_status = sync_status;
} else {
crit!(self.log, "Tried to the sync status for an unknown peer"; "peer_id" => format!("{}",peer_id));
}
}
/// Adds to a peer's reputation by `change`. If the reputation exceeds Rep's
/// upper (lower) bounds, it stays at the maximum (minimum) value.
pub fn add_reputation(&mut self, peer_id: &PeerId, change: Rep) {
let log_ref = &self.log;
let info = self.peers.entry(peer_id.clone()).or_insert_with(|| {
warn!(log_ref, "Adding to the reputation of an unknown peer";
"peer_id" => format!("{:?}",peer_id));
PeerInfo::default()
});
info.reputation = info.reputation.saturating_add(change);
}
}
#[cfg(test)]
mod tests {
use super::*;
use slog::{o, Drain};
use types::MinimalEthSpec;
type M = MinimalEthSpec;
pub fn build_log(level: slog::Level, enabled: bool) -> slog::Logger {
let decorator = slog_term::TermDecorator::new().build();
let drain = slog_term::FullFormat::new(decorator).build().fuse();
let drain = slog_async::Async::new(drain).build().fuse();
if enabled {
slog::Logger::root(drain.filter_level(level).fuse(), o!())
} else {
slog::Logger::root(drain.filter(|_| false).fuse(), o!())
}
}
fn get_db() -> PeerDB<M> {
let log = build_log(slog::Level::Debug, true);
PeerDB::new(&log)
}
#[test]
fn test_peer_connected_successfully() {
let mut pdb = get_db();
let random_peer = PeerId::random();
let (n_in, n_out) = (10, 20);
for _ in 0..n_in {
pdb.connect_ingoing(&random_peer);
}
for _ in 0..n_out {
pdb.connect_outgoing(&random_peer);
}
// the peer is known
let peer_info = pdb.peer_info(&random_peer);
assert!(peer_info.is_some());
// this is the only peer
assert_eq!(pdb.peers().count(), 1);
// the peer has the default reputation
assert_eq!(pdb.reputation(&random_peer), DEFAULT_REPUTATION);
// it should be connected, and therefore not counted as disconnected
assert_eq!(pdb.n_dc, 0);
assert!(peer_info.unwrap().connection_status.is_connected());
assert_eq!(
peer_info.unwrap().connection_status.connections(),
(n_in, n_out)
);
}
#[test]
fn test_set_reputation() {
let mut pdb = get_db();
let random_peer = PeerId::random();
pdb.connect_ingoing(&random_peer);
let mut rep = Rep::min_value();
pdb.set_reputation(&random_peer, rep);
assert_eq!(pdb.reputation(&random_peer), rep);
rep = Rep::max_value();
pdb.set_reputation(&random_peer, rep);
assert_eq!(pdb.reputation(&random_peer), rep);
rep = Rep::max_value() / 100;
pdb.set_reputation(&random_peer, rep);
assert_eq!(pdb.reputation(&random_peer), rep);
}
#[test]
fn test_reputation_change() {
let mut pdb = get_db();
// 0 change does not change de reputation
let random_peer = PeerId::random();
let change: Rep = 0;
pdb.connect_ingoing(&random_peer);
pdb.add_reputation(&random_peer, change);
assert_eq!(pdb.reputation(&random_peer), DEFAULT_REPUTATION);
// overflowing change is capped
let random_peer = PeerId::random();
let change = Rep::max_value();
pdb.connect_ingoing(&random_peer);
pdb.add_reputation(&random_peer, change);
assert_eq!(pdb.reputation(&random_peer), Rep::max_value());
}
#[test]
fn test_disconnected_are_bounded() {
let mut pdb = get_db();
for _ in 0..MAX_DC_PEERS + 1 {
let p = PeerId::random();
pdb.connect_ingoing(&p);
}
assert_eq!(pdb.n_dc, 0);
for p in pdb.connected_peer_ids().cloned().collect::<Vec<_>>() {
pdb.disconnect(&p);
}
assert_eq!(pdb.n_dc, MAX_DC_PEERS);
}
#[test]
fn test_best_peers() {
let mut pdb = get_db();
let p0 = PeerId::random();
let p1 = PeerId::random();
let p2 = PeerId::random();
pdb.connect_ingoing(&p0);
pdb.connect_ingoing(&p1);
pdb.connect_ingoing(&p2);
pdb.set_reputation(&p0, 70);
pdb.set_reputation(&p1, 100);
pdb.set_reputation(&p2, 50);
let best_peers = pdb.best_peers_by_status(PeerConnectionStatus::is_connected);
assert!(vec![&p1, &p0, &p2]
.into_iter()
.eq(best_peers.into_iter().map(|p| p.0)));
}
#[test]
fn test_the_best_peer() {
let mut pdb = get_db();
let p0 = PeerId::random();
let p1 = PeerId::random();
let p2 = PeerId::random();
pdb.connect_ingoing(&p0);
pdb.connect_ingoing(&p1);
pdb.connect_ingoing(&p2);
pdb.set_reputation(&p0, 70);
pdb.set_reputation(&p1, 100);
pdb.set_reputation(&p2, 50);
let the_best = pdb.best_by_status(PeerConnectionStatus::is_connected);
assert!(the_best.is_some());
// Consistency check
let best_peers = pdb.best_peers_by_status(PeerConnectionStatus::is_connected);
assert_eq!(the_best, best_peers.into_iter().map(|p| p.0).next());
}
#[test]
fn test_disconnected_consistency() {
let mut pdb = get_db();
let random_peer = PeerId::random();
pdb.connect_ingoing(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.connect_ingoing(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.disconnect(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.connect_outgoing(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.disconnect(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.ban(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.disconnect(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.disconnect(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
pdb.disconnect(&random_peer);
assert_eq!(pdb.n_dc, pdb.disconnected_peers().count());
}
}

View File

@ -3,7 +3,9 @@
use crate::rpc::{ErrorMessage, RPCErrorResponse, RPCRequest, RPCResponse}; use crate::rpc::{ErrorMessage, RPCErrorResponse, RPCRequest, RPCResponse};
use libp2p::bytes::BufMut; use libp2p::bytes::BufMut;
use libp2p::bytes::BytesMut; use libp2p::bytes::BytesMut;
use std::marker::PhantomData;
use tokio::codec::{Decoder, Encoder}; use tokio::codec::{Decoder, Encoder};
use types::EthSpec;
pub trait OutboundCodec: Encoder + Decoder { pub trait OutboundCodec: Encoder + Decoder {
type ErrorType; type ErrorType;
@ -17,43 +19,53 @@ pub trait OutboundCodec: Encoder + Decoder {
/* Global Inbound Codec */ /* Global Inbound Codec */
// This deals with Decoding RPC Requests from other peers and encoding our responses // This deals with Decoding RPC Requests from other peers and encoding our responses
pub struct BaseInboundCodec<TCodec> pub struct BaseInboundCodec<TCodec, TSpec>
where where
TCodec: Encoder + Decoder, TCodec: Encoder + Decoder,
TSpec: EthSpec,
{ {
/// Inner codec for handling various encodings /// Inner codec for handling various encodings
inner: TCodec, inner: TCodec,
phantom: PhantomData<TSpec>,
} }
impl<TCodec> BaseInboundCodec<TCodec> impl<TCodec, TSpec> BaseInboundCodec<TCodec, TSpec>
where where
TCodec: Encoder + Decoder, TCodec: Encoder + Decoder,
TSpec: EthSpec,
{ {
pub fn new(codec: TCodec) -> Self { pub fn new(codec: TCodec) -> Self {
BaseInboundCodec { inner: codec } BaseInboundCodec {
inner: codec,
phantom: PhantomData,
}
} }
} }
/* Global Outbound Codec */ /* Global Outbound Codec */
// This deals with Decoding RPC Responses from other peers and encoding our requests // This deals with Decoding RPC Responses from other peers and encoding our requests
pub struct BaseOutboundCodec<TOutboundCodec> pub struct BaseOutboundCodec<TOutboundCodec, TSpec>
where where
TOutboundCodec: OutboundCodec, TOutboundCodec: OutboundCodec,
TSpec: EthSpec,
{ {
/// Inner codec for handling various encodings. /// Inner codec for handling various encodings.
inner: TOutboundCodec, inner: TOutboundCodec,
/// Keeps track of the current response code for a chunk. /// Keeps track of the current response code for a chunk.
current_response_code: Option<u8>, current_response_code: Option<u8>,
phantom: PhantomData<TSpec>,
} }
impl<TOutboundCodec> BaseOutboundCodec<TOutboundCodec> impl<TOutboundCodec, TSpec> BaseOutboundCodec<TOutboundCodec, TSpec>
where where
TSpec: EthSpec,
TOutboundCodec: OutboundCodec, TOutboundCodec: OutboundCodec,
{ {
pub fn new(codec: TOutboundCodec) -> Self { pub fn new(codec: TOutboundCodec) -> Self {
BaseOutboundCodec { BaseOutboundCodec {
inner: codec, inner: codec,
current_response_code: None, current_response_code: None,
phantom: PhantomData,
} }
} }
} }
@ -63,11 +75,12 @@ where
/* Base Inbound Codec */ /* Base Inbound Codec */
// This Encodes RPC Responses sent to external peers // This Encodes RPC Responses sent to external peers
impl<TCodec> Encoder for BaseInboundCodec<TCodec> impl<TCodec, TSpec> Encoder for BaseInboundCodec<TCodec, TSpec>
where where
TCodec: Decoder + Encoder<Item = RPCErrorResponse>, TSpec: EthSpec,
TCodec: Decoder + Encoder<Item = RPCErrorResponse<TSpec>>,
{ {
type Item = RPCErrorResponse; type Item = RPCErrorResponse<TSpec>;
type Error = <TCodec as Encoder>::Error; type Error = <TCodec as Encoder>::Error;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
@ -82,11 +95,12 @@ where
} }
// This Decodes RPC Requests from external peers // This Decodes RPC Requests from external peers
impl<TCodec> Decoder for BaseInboundCodec<TCodec> impl<TCodec, TSpec> Decoder for BaseInboundCodec<TCodec, TSpec>
where where
TCodec: Encoder + Decoder<Item = RPCRequest>, TSpec: EthSpec,
TCodec: Encoder + Decoder<Item = RPCRequest<TSpec>>,
{ {
type Item = RPCRequest; type Item = RPCRequest<TSpec>;
type Error = <TCodec as Decoder>::Error; type Error = <TCodec as Decoder>::Error;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> { fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
@ -97,11 +111,12 @@ where
/* Base Outbound Codec */ /* Base Outbound Codec */
// This Encodes RPC Requests sent to external peers // This Encodes RPC Requests sent to external peers
impl<TCodec> Encoder for BaseOutboundCodec<TCodec> impl<TCodec, TSpec> Encoder for BaseOutboundCodec<TCodec, TSpec>
where where
TCodec: OutboundCodec + Encoder<Item = RPCRequest>, TSpec: EthSpec,
TCodec: OutboundCodec + Encoder<Item = RPCRequest<TSpec>>,
{ {
type Item = RPCRequest; type Item = RPCRequest<TSpec>;
type Error = <TCodec as Encoder>::Error; type Error = <TCodec as Encoder>::Error;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
@ -110,11 +125,12 @@ where
} }
// This decodes RPC Responses received from external peers // This decodes RPC Responses received from external peers
impl<TCodec> Decoder for BaseOutboundCodec<TCodec> impl<TCodec, TSpec> Decoder for BaseOutboundCodec<TCodec, TSpec>
where where
TCodec: OutboundCodec<ErrorType = ErrorMessage> + Decoder<Item = RPCResponse>, TSpec: EthSpec,
TCodec: OutboundCodec<ErrorType = ErrorMessage> + Decoder<Item = RPCResponse<TSpec>>,
{ {
type Item = RPCErrorResponse; type Item = RPCErrorResponse<TSpec>;
type Error = <TCodec as Decoder>::Error; type Error = <TCodec as Decoder>::Error;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> { fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
@ -130,7 +146,7 @@ where
}); });
let inner_result = { let inner_result = {
if RPCErrorResponse::is_response(response_code) { if RPCErrorResponse::<TSpec>::is_response(response_code) {
// decode an actual response and mutates the buffer if enough bytes have been read // decode an actual response and mutates the buffer if enough bytes have been read
// returning the result. // returning the result.
self.inner self.inner

View File

@ -1,62 +1,71 @@
pub(crate) mod base; pub(crate) mod base;
pub(crate) mod ssz; pub(crate) mod ssz;
pub(crate) mod ssz_snappy;
use self::base::{BaseInboundCodec, BaseOutboundCodec}; use self::base::{BaseInboundCodec, BaseOutboundCodec};
use self::ssz::{SSZInboundCodec, SSZOutboundCodec}; use self::ssz::{SSZInboundCodec, SSZOutboundCodec};
use self::ssz_snappy::{SSZSnappyInboundCodec, SSZSnappyOutboundCodec};
use crate::rpc::protocol::RPCError; use crate::rpc::protocol::RPCError;
use crate::rpc::{RPCErrorResponse, RPCRequest}; use crate::rpc::{RPCErrorResponse, RPCRequest};
use libp2p::bytes::BytesMut; use libp2p::bytes::BytesMut;
use tokio::codec::{Decoder, Encoder}; use tokio::codec::{Decoder, Encoder};
use types::EthSpec;
// Known types of codecs // Known types of codecs
pub enum InboundCodec { pub enum InboundCodec<TSpec: EthSpec> {
SSZ(BaseInboundCodec<SSZInboundCodec>), SSZSnappy(BaseInboundCodec<SSZSnappyInboundCodec<TSpec>, TSpec>),
SSZ(BaseInboundCodec<SSZInboundCodec<TSpec>, TSpec>),
} }
pub enum OutboundCodec { pub enum OutboundCodec<TSpec: EthSpec> {
SSZ(BaseOutboundCodec<SSZOutboundCodec>), SSZSnappy(BaseOutboundCodec<SSZSnappyOutboundCodec<TSpec>, TSpec>),
SSZ(BaseOutboundCodec<SSZOutboundCodec<TSpec>, TSpec>),
} }
impl Encoder for InboundCodec { impl<T: EthSpec> Encoder for InboundCodec<T> {
type Item = RPCErrorResponse; type Item = RPCErrorResponse<T>;
type Error = RPCError; type Error = RPCError;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
match self { match self {
InboundCodec::SSZ(codec) => codec.encode(item, dst), InboundCodec::SSZ(codec) => codec.encode(item, dst),
InboundCodec::SSZSnappy(codec) => codec.encode(item, dst),
} }
} }
} }
impl Decoder for InboundCodec { impl<TSpec: EthSpec> Decoder for InboundCodec<TSpec> {
type Item = RPCRequest; type Item = RPCRequest<TSpec>;
type Error = RPCError; type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> { fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
match self { match self {
InboundCodec::SSZ(codec) => codec.decode(src), InboundCodec::SSZ(codec) => codec.decode(src),
InboundCodec::SSZSnappy(codec) => codec.decode(src),
} }
} }
} }
impl Encoder for OutboundCodec { impl<TSpec: EthSpec> Encoder for OutboundCodec<TSpec> {
type Item = RPCRequest; type Item = RPCRequest<TSpec>;
type Error = RPCError; type Error = RPCError;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
match self { match self {
OutboundCodec::SSZ(codec) => codec.encode(item, dst), OutboundCodec::SSZ(codec) => codec.encode(item, dst),
OutboundCodec::SSZSnappy(codec) => codec.encode(item, dst),
} }
} }
} }
impl Decoder for OutboundCodec { impl<T: EthSpec> Decoder for OutboundCodec<T> {
type Item = RPCErrorResponse; type Item = RPCErrorResponse<T>;
type Error = RPCError; type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> { fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
match self { match self {
OutboundCodec::SSZ(codec) => codec.decode(src), OutboundCodec::SSZ(codec) => codec.decode(src),
OutboundCodec::SSZSnappy(codec) => codec.decode(src),
} }
} }
} }

View File

@ -1,52 +1,54 @@
use crate::rpc::methods::*; use crate::rpc::methods::*;
use crate::rpc::{ use crate::rpc::{
codec::base::OutboundCodec, codec::base::OutboundCodec,
protocol::{ protocol::{Encoding, Protocol, ProtocolId, RPCError, Version},
ProtocolId, RPCError, RPC_BLOCKS_BY_RANGE, RPC_BLOCKS_BY_ROOT, RPC_GOODBYE, RPC_STATUS,
},
}; };
use crate::rpc::{ErrorMessage, RPCErrorResponse, RPCRequest, RPCResponse}; use crate::rpc::{ErrorMessage, RPCErrorResponse, RPCRequest, RPCResponse};
use libp2p::bytes::{BufMut, Bytes, BytesMut}; use libp2p::bytes::{BufMut, Bytes, BytesMut};
use ssz::{Decode, Encode}; use ssz::{Decode, Encode};
use std::marker::PhantomData;
use tokio::codec::{Decoder, Encoder}; use tokio::codec::{Decoder, Encoder};
use types::{EthSpec, SignedBeaconBlock};
use unsigned_varint::codec::UviBytes; use unsigned_varint::codec::UviBytes;
/* Inbound Codec */ /* Inbound Codec */
pub struct SSZInboundCodec { pub struct SSZInboundCodec<TSpec: EthSpec> {
inner: UviBytes, inner: UviBytes,
protocol: ProtocolId, protocol: ProtocolId,
phantom: PhantomData<TSpec>,
} }
impl SSZInboundCodec { impl<T: EthSpec> SSZInboundCodec<T> {
pub fn new(protocol: ProtocolId, max_packet_size: usize) -> Self { pub fn new(protocol: ProtocolId, max_packet_size: usize) -> Self {
let mut uvi_codec = UviBytes::default(); let mut uvi_codec = UviBytes::default();
uvi_codec.set_max_len(max_packet_size); uvi_codec.set_max_len(max_packet_size);
// this encoding only applies to ssz. // this encoding only applies to ssz.
debug_assert!(protocol.encoding.as_str() == "ssz"); debug_assert_eq!(protocol.encoding, Encoding::SSZ);
SSZInboundCodec { SSZInboundCodec {
inner: uvi_codec, inner: uvi_codec,
protocol, protocol,
phantom: PhantomData,
} }
} }
} }
// Encoder for inbound streams: Encodes RPC Responses sent to peers. // Encoder for inbound streams: Encodes RPC Responses sent to peers.
impl Encoder for SSZInboundCodec { impl<TSpec: EthSpec> Encoder for SSZInboundCodec<TSpec> {
type Item = RPCErrorResponse; type Item = RPCErrorResponse<TSpec>;
type Error = RPCError; type Error = RPCError;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
let bytes = match item { let bytes = match item {
RPCErrorResponse::Success(resp) => { RPCErrorResponse::Success(resp) => match resp {
match resp { RPCResponse::Status(res) => res.as_ssz_bytes(),
RPCResponse::Status(res) => res.as_ssz_bytes(), RPCResponse::BlocksByRange(res) => res.as_ssz_bytes(),
RPCResponse::BlocksByRange(res) => res, // already raw bytes RPCResponse::BlocksByRoot(res) => res.as_ssz_bytes(),
RPCResponse::BlocksByRoot(res) => res, // already raw bytes RPCResponse::Pong(res) => res.data.as_ssz_bytes(),
} RPCResponse::MetaData(res) => res.as_ssz_bytes(),
} },
RPCErrorResponse::InvalidRequest(err) => err.as_ssz_bytes(), RPCErrorResponse::InvalidRequest(err) => err.as_ssz_bytes(),
RPCErrorResponse::ServerError(err) => err.as_ssz_bytes(), RPCErrorResponse::ServerError(err) => err.as_ssz_bytes(),
RPCErrorResponse::Unknown(err) => err.as_ssz_bytes(), RPCErrorResponse::Unknown(err) => err.as_ssz_bytes(),
@ -70,38 +72,49 @@ impl Encoder for SSZInboundCodec {
} }
// Decoder for inbound streams: Decodes RPC requests from peers // Decoder for inbound streams: Decodes RPC requests from peers
impl Decoder for SSZInboundCodec { impl<TSpec: EthSpec> Decoder for SSZInboundCodec<TSpec> {
type Item = RPCRequest; type Item = RPCRequest<TSpec>;
type Error = RPCError; type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> { fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
match self.inner.decode(src).map_err(RPCError::from) { match self.inner.decode(src).map_err(RPCError::from) {
Ok(Some(packet)) => match self.protocol.message_name.as_str() { Ok(Some(packet)) => match self.protocol.message_name {
RPC_STATUS => match self.protocol.version.as_str() { Protocol::Status => match self.protocol.version {
"1" => Ok(Some(RPCRequest::Status(StatusMessage::from_ssz_bytes( Version::V1 => Ok(Some(RPCRequest::Status(StatusMessage::from_ssz_bytes(
&packet, &packet,
)?))), )?))),
_ => unreachable!("Cannot negotiate an unknown version"),
}, },
RPC_GOODBYE => match self.protocol.version.as_str() { Protocol::Goodbye => match self.protocol.version {
"1" => Ok(Some(RPCRequest::Goodbye(GoodbyeReason::from_ssz_bytes( Version::V1 => Ok(Some(RPCRequest::Goodbye(GoodbyeReason::from_ssz_bytes(
&packet, &packet,
)?))), )?))),
_ => unreachable!("Cannot negotiate an unknown version"),
}, },
RPC_BLOCKS_BY_RANGE => match self.protocol.version.as_str() { Protocol::BlocksByRange => match self.protocol.version {
"1" => Ok(Some(RPCRequest::BlocksByRange( Version::V1 => Ok(Some(RPCRequest::BlocksByRange(
BlocksByRangeRequest::from_ssz_bytes(&packet)?, BlocksByRangeRequest::from_ssz_bytes(&packet)?,
))), ))),
_ => unreachable!("Cannot negotiate an unknown version"),
}, },
RPC_BLOCKS_BY_ROOT => match self.protocol.version.as_str() { Protocol::BlocksByRoot => match self.protocol.version {
"1" => Ok(Some(RPCRequest::BlocksByRoot(BlocksByRootRequest { Version::V1 => Ok(Some(RPCRequest::BlocksByRoot(BlocksByRootRequest {
block_roots: Vec::from_ssz_bytes(&packet)?, block_roots: Vec::from_ssz_bytes(&packet)?,
}))), }))),
_ => unreachable!("Cannot negotiate an unknown version"),
}, },
_ => unreachable!("Cannot negotiate an unknown protocol"), Protocol::Ping => match self.protocol.version {
Version::V1 => Ok(Some(RPCRequest::Ping(Ping {
data: u64::from_ssz_bytes(&packet)?,
}))),
},
Protocol::MetaData => match self.protocol.version {
Version::V1 => {
if packet.len() > 0 {
Err(RPCError::Custom(
"Get metadata request should be empty".into(),
))
} else {
Ok(Some(RPCRequest::MetaData(PhantomData)))
}
}
},
}, },
Ok(None) => Ok(None), Ok(None) => Ok(None),
Err(e) => Err(e), Err(e) => Err(e),
@ -111,29 +124,31 @@ impl Decoder for SSZInboundCodec {
/* Outbound Codec: Codec for initiating RPC requests */ /* Outbound Codec: Codec for initiating RPC requests */
pub struct SSZOutboundCodec { pub struct SSZOutboundCodec<TSpec: EthSpec> {
inner: UviBytes, inner: UviBytes,
protocol: ProtocolId, protocol: ProtocolId,
phantom: PhantomData<TSpec>,
} }
impl SSZOutboundCodec { impl<TSpec: EthSpec> SSZOutboundCodec<TSpec> {
pub fn new(protocol: ProtocolId, max_packet_size: usize) -> Self { pub fn new(protocol: ProtocolId, max_packet_size: usize) -> Self {
let mut uvi_codec = UviBytes::default(); let mut uvi_codec = UviBytes::default();
uvi_codec.set_max_len(max_packet_size); uvi_codec.set_max_len(max_packet_size);
// this encoding only applies to ssz. // this encoding only applies to ssz.
debug_assert!(protocol.encoding.as_str() == "ssz"); debug_assert_eq!(protocol.encoding, Encoding::SSZ);
SSZOutboundCodec { SSZOutboundCodec {
inner: uvi_codec, inner: uvi_codec,
protocol, protocol,
phantom: PhantomData,
} }
} }
} }
// Encoder for outbound streams: Encodes RPC Requests to peers // Encoder for outbound streams: Encodes RPC Requests to peers
impl Encoder for SSZOutboundCodec { impl<TSpec: EthSpec> Encoder for SSZOutboundCodec<TSpec> {
type Item = RPCRequest; type Item = RPCRequest<TSpec>;
type Error = RPCError; type Error = RPCError;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
@ -142,6 +157,8 @@ impl Encoder for SSZOutboundCodec {
RPCRequest::Goodbye(req) => req.as_ssz_bytes(), RPCRequest::Goodbye(req) => req.as_ssz_bytes(),
RPCRequest::BlocksByRange(req) => req.as_ssz_bytes(), RPCRequest::BlocksByRange(req) => req.as_ssz_bytes(),
RPCRequest::BlocksByRoot(req) => req.block_roots.as_ssz_bytes(), RPCRequest::BlocksByRoot(req) => req.block_roots.as_ssz_bytes(),
RPCRequest::Ping(req) => req.as_ssz_bytes(),
RPCRequest::MetaData(_) => return Ok(()), // no metadata to encode
}; };
// length-prefix // length-prefix
self.inner self.inner
@ -155,8 +172,8 @@ impl Encoder for SSZOutboundCodec {
// The majority of the decoding has now been pushed upstream due to the changing specification. // The majority of the decoding has now been pushed upstream due to the changing specification.
// We prefer to decode blocks and attestations with extra knowledge about the chain to perform // We prefer to decode blocks and attestations with extra knowledge about the chain to perform
// faster verification checks before decoding entire blocks/attestations. // faster verification checks before decoding entire blocks/attestations.
impl Decoder for SSZOutboundCodec { impl<TSpec: EthSpec> Decoder for SSZOutboundCodec<TSpec> {
type Item = RPCResponse; type Item = RPCResponse<TSpec>;
type Error = RPCError; type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> { fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
@ -164,23 +181,35 @@ impl Decoder for SSZOutboundCodec {
// the object is empty. We return the empty object if this is the case // the object is empty. We return the empty object if this is the case
// clear the buffer and return an empty object // clear the buffer and return an empty object
src.clear(); src.clear();
match self.protocol.message_name.as_str() { match self.protocol.message_name {
RPC_STATUS => match self.protocol.version.as_str() { Protocol::Status => match self.protocol.version {
"1" => Err(RPCError::Custom( Version::V1 => Err(RPCError::Custom(
"Status stream terminated unexpectedly".into(), "Status stream terminated unexpectedly".into(),
)), // cannot have an empty HELLO message. The stream has terminated unexpectedly )), // cannot have an empty HELLO message. The stream has terminated unexpectedly
_ => unreachable!("Cannot negotiate an unknown version"),
}, },
RPC_GOODBYE => Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response")), Protocol::Goodbye => {
RPC_BLOCKS_BY_RANGE => match self.protocol.version.as_str() { Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response"))
"1" => Ok(Some(RPCResponse::BlocksByRange(Vec::new()))), }
_ => unreachable!("Cannot negotiate an unknown version"), Protocol::BlocksByRange => match self.protocol.version {
Version::V1 => Err(RPCError::Custom(
"Status stream terminated unexpectedly, empty block".into(),
)), // cannot have an empty block message.
}, },
RPC_BLOCKS_BY_ROOT => match self.protocol.version.as_str() { Protocol::BlocksByRoot => match self.protocol.version {
"1" => Ok(Some(RPCResponse::BlocksByRoot(Vec::new()))), Version::V1 => Err(RPCError::Custom(
_ => unreachable!("Cannot negotiate an unknown version"), "Status stream terminated unexpectedly, empty block".into(),
)), // cannot have an empty block message.
},
Protocol::Ping => match self.protocol.version {
Version::V1 => Err(RPCError::Custom(
"PING stream terminated unexpectedly".into(),
)), // cannot have an empty block message.
},
Protocol::MetaData => match self.protocol.version {
Version::V1 => Err(RPCError::Custom(
"Metadata stream terminated unexpectedly".into(),
)), // cannot have an empty block message.
}, },
_ => unreachable!("Cannot negotiate an unknown protocol"),
} }
} else { } else {
match self.inner.decode(src).map_err(RPCError::from) { match self.inner.decode(src).map_err(RPCError::from) {
@ -188,25 +217,35 @@ impl Decoder for SSZOutboundCodec {
// take the bytes from the buffer // take the bytes from the buffer
let raw_bytes = packet.take(); let raw_bytes = packet.take();
match self.protocol.message_name.as_str() { match self.protocol.message_name {
RPC_STATUS => match self.protocol.version.as_str() { Protocol::Status => match self.protocol.version {
"1" => Ok(Some(RPCResponse::Status(StatusMessage::from_ssz_bytes( Version::V1 => Ok(Some(RPCResponse::Status(
&raw_bytes, StatusMessage::from_ssz_bytes(&raw_bytes)?,
)?))), ))),
_ => unreachable!("Cannot negotiate an unknown version"),
}, },
RPC_GOODBYE => { Protocol::Goodbye => {
Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response")) Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response"))
} }
RPC_BLOCKS_BY_RANGE => match self.protocol.version.as_str() { Protocol::BlocksByRange => match self.protocol.version {
"1" => Ok(Some(RPCResponse::BlocksByRange(raw_bytes.to_vec()))), Version::V1 => Ok(Some(RPCResponse::BlocksByRange(Box::new(
_ => unreachable!("Cannot negotiate an unknown version"), SignedBeaconBlock::from_ssz_bytes(&raw_bytes)?,
)))),
}, },
RPC_BLOCKS_BY_ROOT => match self.protocol.version.as_str() { Protocol::BlocksByRoot => match self.protocol.version {
"1" => Ok(Some(RPCResponse::BlocksByRoot(raw_bytes.to_vec()))), Version::V1 => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
_ => unreachable!("Cannot negotiate an unknown version"), SignedBeaconBlock::from_ssz_bytes(&raw_bytes)?,
)))),
},
Protocol::Ping => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::Pong(Ping {
data: u64::from_ssz_bytes(&raw_bytes)?,
}))),
},
Protocol::MetaData => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::MetaData(
MetaData::from_ssz_bytes(&raw_bytes)?,
))),
}, },
_ => unreachable!("Cannot negotiate an unknown protocol"),
} }
} }
Ok(None) => Ok(None), // waiting for more bytes Ok(None) => Ok(None), // waiting for more bytes
@ -216,7 +255,7 @@ impl Decoder for SSZOutboundCodec {
} }
} }
impl OutboundCodec for SSZOutboundCodec { impl<TSpec: EthSpec> OutboundCodec for SSZOutboundCodec<TSpec> {
type ErrorType = ErrorMessage; type ErrorType = ErrorMessage;
fn decode_error(&mut self, src: &mut BytesMut) -> Result<Option<Self::ErrorType>, RPCError> { fn decode_error(&mut self, src: &mut BytesMut) -> Result<Option<Self::ErrorType>, RPCError> {

View File

@ -0,0 +1,357 @@
use crate::rpc::methods::*;
use crate::rpc::{
codec::base::OutboundCodec,
protocol::{Encoding, Protocol, ProtocolId, RPCError, Version},
};
use crate::rpc::{ErrorMessage, RPCErrorResponse, RPCRequest, RPCResponse};
use libp2p::bytes::BytesMut;
use snap::read::FrameDecoder;
use snap::write::FrameEncoder;
use ssz::{Decode, Encode};
use std::io::Cursor;
use std::io::ErrorKind;
use std::io::{Read, Write};
use std::marker::PhantomData;
use tokio::codec::{Decoder, Encoder};
use types::{EthSpec, SignedBeaconBlock};
use unsigned_varint::codec::Uvi;
/* Inbound Codec */
pub struct SSZSnappyInboundCodec<TSpec: EthSpec> {
protocol: ProtocolId,
inner: Uvi<usize>,
len: Option<usize>,
/// Maximum bytes that can be sent in one req/resp chunked responses.
max_packet_size: usize,
phantom: PhantomData<TSpec>,
}
impl<T: EthSpec> SSZSnappyInboundCodec<T> {
pub fn new(protocol: ProtocolId, max_packet_size: usize) -> Self {
let uvi_codec = Uvi::default();
// this encoding only applies to ssz_snappy.
debug_assert_eq!(protocol.encoding, Encoding::SSZSnappy);
SSZSnappyInboundCodec {
inner: uvi_codec,
protocol,
len: None,
phantom: PhantomData,
max_packet_size,
}
}
}
// Encoder for inbound streams: Encodes RPC Responses sent to peers.
impl<TSpec: EthSpec> Encoder for SSZSnappyInboundCodec<TSpec> {
type Item = RPCErrorResponse<TSpec>;
type Error = RPCError;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
let bytes = match item {
RPCErrorResponse::Success(resp) => match resp {
RPCResponse::Status(res) => res.as_ssz_bytes(),
RPCResponse::BlocksByRange(res) => res.as_ssz_bytes(),
RPCResponse::BlocksByRoot(res) => res.as_ssz_bytes(),
RPCResponse::Pong(res) => res.data.as_ssz_bytes(),
RPCResponse::MetaData(res) => res.as_ssz_bytes(),
},
RPCErrorResponse::InvalidRequest(err) => err.as_ssz_bytes(),
RPCErrorResponse::ServerError(err) => err.as_ssz_bytes(),
RPCErrorResponse::Unknown(err) => err.as_ssz_bytes(),
RPCErrorResponse::StreamTermination(_) => {
unreachable!("Code error - attempting to encode a stream termination")
}
};
// SSZ encoded bytes should be within `max_packet_size`
if bytes.len() > self.max_packet_size {
return Err(RPCError::Custom(
"attempting to encode data > max_packet_size".into(),
));
}
// Inserts the length prefix of the uncompressed bytes into dst
// encoded as a unsigned varint
self.inner
.encode(bytes.len(), dst)
.map_err(RPCError::from)?;
let mut writer = FrameEncoder::new(Vec::new());
writer.write_all(&bytes).map_err(RPCError::from)?;
writer.flush().map_err(RPCError::from)?;
// Write compressed bytes to `dst`
dst.extend_from_slice(writer.get_ref());
Ok(())
}
}
// Decoder for inbound streams: Decodes RPC requests from peers
impl<TSpec: EthSpec> Decoder for SSZSnappyInboundCodec<TSpec> {
type Item = RPCRequest<TSpec>;
type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
if self.len.is_none() {
// Decode the length of the uncompressed bytes from an unsigned varint
match self.inner.decode(src).map_err(RPCError::from)? {
Some(length) => {
self.len = Some(length);
}
None => return Ok(None), // need more bytes to decode length
}
};
let length = self.len.expect("length should be Some");
// Should not attempt to decode rpc chunks with length > max_packet_size
if length > self.max_packet_size {
return Err(RPCError::Custom(
"attempting to decode data > max_packet_size".into(),
));
}
let mut reader = FrameDecoder::new(Cursor::new(&src));
let mut decoded_buffer = vec![0; length];
match reader.read_exact(&mut decoded_buffer) {
Ok(()) => {
// `n` is how many bytes the reader read in the compressed stream
let n = reader.get_ref().position();
self.len = None;
src.split_to(n as usize);
match self.protocol.message_name {
Protocol::Status => match self.protocol.version {
Version::V1 => Ok(Some(RPCRequest::Status(StatusMessage::from_ssz_bytes(
&decoded_buffer,
)?))),
},
Protocol::Goodbye => match self.protocol.version {
Version::V1 => Ok(Some(RPCRequest::Goodbye(
GoodbyeReason::from_ssz_bytes(&decoded_buffer)?,
))),
},
Protocol::BlocksByRange => match self.protocol.version {
Version::V1 => Ok(Some(RPCRequest::BlocksByRange(
BlocksByRangeRequest::from_ssz_bytes(&decoded_buffer)?,
))),
},
Protocol::BlocksByRoot => match self.protocol.version {
Version::V1 => Ok(Some(RPCRequest::BlocksByRoot(BlocksByRootRequest {
block_roots: Vec::from_ssz_bytes(&decoded_buffer)?,
}))),
},
Protocol::Ping => match self.protocol.version {
Version::V1 => Ok(Some(RPCRequest::Ping(Ping::from_ssz_bytes(
&decoded_buffer,
)?))),
},
Protocol::MetaData => match self.protocol.version {
Version::V1 => {
if decoded_buffer.len() > 0 {
Err(RPCError::Custom(
"Get metadata request should be empty".into(),
))
} else {
Ok(Some(RPCRequest::MetaData(PhantomData)))
}
}
},
}
}
Err(e) => match e.kind() {
// Haven't received enough bytes to decode yet
// TODO: check if this is the only Error variant where we return `Ok(None)`
ErrorKind::UnexpectedEof => {
return Ok(None);
}
_ => return Err(e).map_err(RPCError::from),
},
}
}
}
/* Outbound Codec: Codec for initiating RPC requests */
pub struct SSZSnappyOutboundCodec<TSpec: EthSpec> {
inner: Uvi<usize>,
len: Option<usize>,
protocol: ProtocolId,
/// Maximum bytes that can be sent in one req/resp chunked responses.
max_packet_size: usize,
phantom: PhantomData<TSpec>,
}
impl<TSpec: EthSpec> SSZSnappyOutboundCodec<TSpec> {
pub fn new(protocol: ProtocolId, max_packet_size: usize) -> Self {
let uvi_codec = Uvi::default();
// this encoding only applies to ssz_snappy.
debug_assert_eq!(protocol.encoding, Encoding::SSZSnappy);
SSZSnappyOutboundCodec {
inner: uvi_codec,
protocol,
max_packet_size,
len: None,
phantom: PhantomData,
}
}
}
// Encoder for outbound streams: Encodes RPC Requests to peers
impl<TSpec: EthSpec> Encoder for SSZSnappyOutboundCodec<TSpec> {
type Item = RPCRequest<TSpec>;
type Error = RPCError;
fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> {
let bytes = match item {
RPCRequest::Status(req) => req.as_ssz_bytes(),
RPCRequest::Goodbye(req) => req.as_ssz_bytes(),
RPCRequest::BlocksByRange(req) => req.as_ssz_bytes(),
RPCRequest::BlocksByRoot(req) => req.block_roots.as_ssz_bytes(),
RPCRequest::Ping(req) => req.as_ssz_bytes(),
RPCRequest::MetaData(_) => return Ok(()), // no metadata to encode
};
// SSZ encoded bytes should be within `max_packet_size`
if bytes.len() > self.max_packet_size {
return Err(RPCError::Custom(
"attempting to encode data > max_packet_size".into(),
));
}
// Inserts the length prefix of the uncompressed bytes into dst
// encoded as a unsigned varint
self.inner
.encode(bytes.len(), dst)
.map_err(RPCError::from)?;
let mut writer = FrameEncoder::new(Vec::new());
writer.write_all(&bytes).map_err(RPCError::from)?;
writer.flush().map_err(RPCError::from)?;
// Write compressed bytes to `dst`
dst.extend_from_slice(writer.get_ref());
Ok(())
}
}
// Decoder for outbound streams: Decodes RPC responses from peers.
//
// The majority of the decoding has now been pushed upstream due to the changing specification.
// We prefer to decode blocks and attestations with extra knowledge about the chain to perform
// faster verification checks before decoding entire blocks/attestations.
impl<TSpec: EthSpec> Decoder for SSZSnappyOutboundCodec<TSpec> {
type Item = RPCResponse<TSpec>;
type Error = RPCError;
fn decode(&mut self, src: &mut BytesMut) -> Result<Option<Self::Item>, Self::Error> {
if self.len.is_none() {
// Decode the length of the uncompressed bytes from an unsigned varint
match self.inner.decode(src).map_err(RPCError::from)? {
Some(length) => {
self.len = Some(length as usize);
}
None => return Ok(None), // need more bytes to decode length
}
};
let length = self.len.expect("length should be Some");
// Should not attempt to decode rpc chunks with length > max_packet_size
if length > self.max_packet_size {
return Err(RPCError::Custom(
"attempting to decode data > max_packet_size".into(),
));
}
let mut reader = FrameDecoder::new(Cursor::new(&src));
let mut decoded_buffer = vec![0; length];
match reader.read_exact(&mut decoded_buffer) {
Ok(()) => {
// `n` is how many bytes the reader read in the compressed stream
let n = reader.get_ref().position();
self.len = None;
src.split_to(n as usize);
match self.protocol.message_name {
Protocol::Status => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::Status(
StatusMessage::from_ssz_bytes(&decoded_buffer)?,
))),
},
Protocol::Goodbye => {
Err(RPCError::InvalidProtocol("GOODBYE doesn't have a response"))
}
Protocol::BlocksByRange => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::BlocksByRange(Box::new(
SignedBeaconBlock::from_ssz_bytes(&decoded_buffer)?,
)))),
},
Protocol::BlocksByRoot => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
SignedBeaconBlock::from_ssz_bytes(&decoded_buffer)?,
)))),
},
Protocol::Ping => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::Pong(Ping {
data: u64::from_ssz_bytes(&decoded_buffer)?,
}))),
},
Protocol::MetaData => match self.protocol.version {
Version::V1 => Ok(Some(RPCResponse::MetaData(MetaData::from_ssz_bytes(
&decoded_buffer,
)?))),
},
}
}
Err(e) => match e.kind() {
// Haven't received enough bytes to decode yet
// TODO: check if this is the only Error variant where we return `Ok(None)`
ErrorKind::UnexpectedEof => {
return Ok(None);
}
_ => return Err(e).map_err(RPCError::from),
},
}
}
}
impl<TSpec: EthSpec> OutboundCodec for SSZSnappyOutboundCodec<TSpec> {
type ErrorType = ErrorMessage;
fn decode_error(&mut self, src: &mut BytesMut) -> Result<Option<Self::ErrorType>, RPCError> {
if self.len.is_none() {
// Decode the length of the uncompressed bytes from an unsigned varint
match self.inner.decode(src).map_err(RPCError::from)? {
Some(length) => {
self.len = Some(length as usize);
}
None => return Ok(None), // need more bytes to decode length
}
};
let length = self.len.expect("length should be Some");
// Should not attempt to decode rpc chunks with length > max_packet_size
if length > self.max_packet_size {
return Err(RPCError::Custom(
"attempting to decode data > max_packet_size".into(),
));
}
let mut reader = FrameDecoder::new(Cursor::new(&src));
let mut decoded_buffer = vec![0; length];
match reader.read_exact(&mut decoded_buffer) {
Ok(()) => {
// `n` is how many bytes the reader read in the compressed stream
let n = reader.get_ref().position();
self.len = None;
src.split_to(n as usize);
Ok(Some(ErrorMessage::from_ssz_bytes(&decoded_buffer)?))
}
Err(e) => match e.kind() {
// Haven't received enough bytes to decode yet
// TODO: check if this is the only Error variant where we return `Ok(None)`
ErrorKind::UnexpectedEof => {
return Ok(None);
}
_ => return Err(e).map_err(RPCError::from),
},
}
}
}

View File

@ -12,12 +12,13 @@ use libp2p::core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError};
use libp2p::swarm::protocols_handler::{ use libp2p::swarm::protocols_handler::{
KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol,
}; };
use slog::{crit, debug, error, warn}; use slog::{crit, debug, error, trace, warn};
use smallvec::SmallVec; use smallvec::SmallVec;
use std::collections::hash_map::Entry; use std::collections::hash_map::Entry;
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
use tokio::io::{AsyncRead, AsyncWrite}; use tokio::io::{AsyncRead, AsyncWrite};
use tokio::timer::{delay_queue, DelayQueue}; use tokio::timer::{delay_queue, DelayQueue};
use types::EthSpec;
//TODO: Implement close() on the substream types to improve the poll code. //TODO: Implement close() on the substream types to improve the poll code.
//TODO: Implement check_timeout() on the substream types //TODO: Implement check_timeout() on the substream types
@ -36,42 +37,50 @@ type InboundRequestId = RequestId;
type OutboundRequestId = RequestId; type OutboundRequestId = RequestId;
/// Implementation of `ProtocolsHandler` for the RPC protocol. /// Implementation of `ProtocolsHandler` for the RPC protocol.
pub struct RPCHandler<TSubstream> pub struct RPCHandler<TSubstream, TSpec>
where where
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
/// The upgrade for inbound substreams. /// The upgrade for inbound substreams.
listen_protocol: SubstreamProtocol<RPCProtocol>, listen_protocol: SubstreamProtocol<RPCProtocol<TSpec>>,
/// If something bad happened and we should shut down the handler with an error. /// If something bad happened and we should shut down the handler with an error.
pending_error: Vec<(RequestId, ProtocolsHandlerUpgrErr<RPCError>)>, pending_error: Vec<(RequestId, ProtocolsHandlerUpgrErr<RPCError>)>,
/// Queue of events to produce in `poll()`. /// Queue of events to produce in `poll()`.
events_out: SmallVec<[RPCEvent; 4]>, events_out: SmallVec<[RPCEvent<TSpec>; 4]>,
/// Queue of outbound substreams to open. /// Queue of outbound substreams to open.
dial_queue: SmallVec<[RPCEvent; 4]>, dial_queue: SmallVec<[RPCEvent<TSpec>; 4]>,
/// Current number of concurrent outbound substreams being opened. /// Current number of concurrent outbound substreams being opened.
dial_negotiated: u32, dial_negotiated: u32,
/// Current inbound substreams awaiting processing. /// Current inbound substreams awaiting processing.
inbound_substreams: inbound_substreams: FnvHashMap<
FnvHashMap<InboundRequestId, (InboundSubstreamState<TSubstream>, Option<delay_queue::Key>)>, InboundRequestId,
(
InboundSubstreamState<TSubstream, TSpec>,
Option<delay_queue::Key>,
),
>,
/// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout. /// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout.
inbound_substreams_delay: DelayQueue<InboundRequestId>, inbound_substreams_delay: DelayQueue<InboundRequestId>,
/// Map of outbound substreams that need to be driven to completion. The `RequestId` is /// Map of outbound substreams that need to be driven to completion. The `RequestId` is
/// maintained by the application sending the request. /// maintained by the application sending the request.
outbound_substreams: outbound_substreams: FnvHashMap<
FnvHashMap<OutboundRequestId, (OutboundSubstreamState<TSubstream>, delay_queue::Key)>, OutboundRequestId,
(OutboundSubstreamState<TSubstream, TSpec>, delay_queue::Key),
>,
/// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout. /// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout.
outbound_substreams_delay: DelayQueue<OutboundRequestId>, outbound_substreams_delay: DelayQueue<OutboundRequestId>,
/// Map of outbound items that are queued as the stream processes them. /// Map of outbound items that are queued as the stream processes them.
queued_outbound_items: FnvHashMap<RequestId, Vec<RPCErrorResponse>>, queued_outbound_items: FnvHashMap<RequestId, Vec<RPCErrorResponse<TSpec>>>,
/// Sequential ID for waiting substreams. For inbound substreams, this is also the inbound request ID. /// Sequential ID for waiting substreams. For inbound substreams, this is also the inbound request ID.
current_inbound_substream_id: RequestId, current_inbound_substream_id: RequestId,
@ -97,14 +106,15 @@ where
} }
/// State of an outbound substream. Either waiting for a response, or in the process of sending. /// State of an outbound substream. Either waiting for a response, or in the process of sending.
pub enum InboundSubstreamState<TSubstream> pub enum InboundSubstreamState<TSubstream, TSpec>
where where
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
/// A response has been sent, pending writing and flush. /// A response has been sent, pending writing and flush.
ResponsePendingSend { ResponsePendingSend {
/// The substream used to send the response /// The substream used to send the response
substream: futures::sink::Send<InboundFramed<TSubstream>>, substream: futures::sink::Send<InboundFramed<TSubstream, TSpec>>,
/// Whether a stream termination is requested. If true the stream will be closed after /// Whether a stream termination is requested. If true the stream will be closed after
/// this send. Otherwise it will transition to an idle state until a stream termination is /// this send. Otherwise it will transition to an idle state until a stream termination is
/// requested or a timeout is reached. /// requested or a timeout is reached.
@ -112,40 +122,41 @@ where
}, },
/// The response stream is idle and awaiting input from the application to send more chunked /// The response stream is idle and awaiting input from the application to send more chunked
/// responses. /// responses.
ResponseIdle(InboundFramed<TSubstream>), ResponseIdle(InboundFramed<TSubstream, TSpec>),
/// The substream is attempting to shutdown. /// The substream is attempting to shutdown.
Closing(InboundFramed<TSubstream>), Closing(InboundFramed<TSubstream, TSpec>),
/// Temporary state during processing /// Temporary state during processing
Poisoned, Poisoned,
} }
pub enum OutboundSubstreamState<TSubstream> { pub enum OutboundSubstreamState<TSubstream, TSpec: EthSpec> {
/// A request has been sent, and we are awaiting a response. This future is driven in the /// A request has been sent, and we are awaiting a response. This future is driven in the
/// handler because GOODBYE requests can be handled and responses dropped instantly. /// handler because GOODBYE requests can be handled and responses dropped instantly.
RequestPendingResponse { RequestPendingResponse {
/// The framed negotiated substream. /// The framed negotiated substream.
substream: OutboundFramed<TSubstream>, substream: OutboundFramed<TSubstream, TSpec>,
/// Keeps track of the actual request sent. /// Keeps track of the actual request sent.
request: RPCRequest, request: RPCRequest<TSpec>,
}, },
/// Closing an outbound substream> /// Closing an outbound substream>
Closing(OutboundFramed<TSubstream>), Closing(OutboundFramed<TSubstream, TSpec>),
/// Temporary state during processing /// Temporary state during processing
Poisoned, Poisoned,
} }
impl<TSubstream> InboundSubstreamState<TSubstream> impl<TSubstream, TSpec> InboundSubstreamState<TSubstream, TSpec>
where where
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
/// Moves the substream state to closing and informs the connected peer. The /// Moves the substream state to closing and informs the connected peer. The
/// `queued_outbound_items` must be given as a parameter to add stream termination messages to /// `queued_outbound_items` must be given as a parameter to add stream termination messages to
/// the outbound queue. /// the outbound queue.
pub fn close(&mut self, outbound_queue: &mut Vec<RPCErrorResponse>) { pub fn close(&mut self, outbound_queue: &mut Vec<RPCErrorResponse<TSpec>>) {
// When terminating a stream, report the stream termination to the requesting user via // When terminating a stream, report the stream termination to the requesting user via
// an RPC error // an RPC error
let error = RPCErrorResponse::ServerError(ErrorMessage { let error = RPCErrorResponse::ServerError(ErrorMessage {
error_message: b"Request timed out".to_vec(), error_message: "Request timed out".as_bytes().to_vec(),
}); });
// The stream termination type is irrelevant, this will terminate the // The stream termination type is irrelevant, this will terminate the
@ -163,16 +174,11 @@ where
*self = InboundSubstreamState::ResponsePendingSend { substream, closing } *self = InboundSubstreamState::ResponsePendingSend { substream, closing }
} }
InboundSubstreamState::ResponseIdle(mut substream) => { InboundSubstreamState::ResponseIdle(substream) => {
// check if the stream is already closed *self = InboundSubstreamState::ResponsePendingSend {
if let Ok(Async::Ready(None)) = substream.poll() { substream: substream.send(error),
*self = InboundSubstreamState::Closing(substream); closing: true,
} else { };
*self = InboundSubstreamState::ResponsePendingSend {
substream: substream.send(error),
closing: true,
};
}
} }
InboundSubstreamState::Closing(substream) => { InboundSubstreamState::Closing(substream) => {
// let the stream close // let the stream close
@ -185,12 +191,13 @@ where
} }
} }
impl<TSubstream> RPCHandler<TSubstream> impl<TSubstream, TSpec> RPCHandler<TSubstream, TSpec>
where where
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
pub fn new( pub fn new(
listen_protocol: SubstreamProtocol<RPCProtocol>, listen_protocol: SubstreamProtocol<RPCProtocol<TSpec>>,
inactive_timeout: Duration, inactive_timeout: Duration,
log: &slog::Logger, log: &slog::Logger,
) -> Self { ) -> Self {
@ -224,7 +231,7 @@ where
/// ///
/// > **Note**: If you modify the protocol, modifications will only applies to future inbound /// > **Note**: If you modify the protocol, modifications will only applies to future inbound
/// > substreams, not the ones already being negotiated. /// > substreams, not the ones already being negotiated.
pub fn listen_protocol_ref(&self) -> &SubstreamProtocol<RPCProtocol> { pub fn listen_protocol_ref(&self) -> &SubstreamProtocol<RPCProtocol<TSpec>> {
&self.listen_protocol &self.listen_protocol
} }
@ -232,29 +239,30 @@ where
/// ///
/// > **Note**: If you modify the protocol, modifications will only applies to future inbound /// > **Note**: If you modify the protocol, modifications will only applies to future inbound
/// > substreams, not the ones already being negotiated. /// > substreams, not the ones already being negotiated.
pub fn listen_protocol_mut(&mut self) -> &mut SubstreamProtocol<RPCProtocol> { pub fn listen_protocol_mut(&mut self) -> &mut SubstreamProtocol<RPCProtocol<TSpec>> {
&mut self.listen_protocol &mut self.listen_protocol
} }
/// Opens an outbound substream with a request. /// Opens an outbound substream with a request.
pub fn send_request(&mut self, rpc_event: RPCEvent) { pub fn send_request(&mut self, rpc_event: RPCEvent<TSpec>) {
self.keep_alive = KeepAlive::Yes; self.keep_alive = KeepAlive::Yes;
self.dial_queue.push(rpc_event); self.dial_queue.push(rpc_event);
} }
} }
impl<TSubstream> ProtocolsHandler for RPCHandler<TSubstream> impl<TSubstream, TSpec> ProtocolsHandler for RPCHandler<TSubstream, TSpec>
where where
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
type InEvent = RPCEvent; type InEvent = RPCEvent<TSpec>;
type OutEvent = RPCEvent; type OutEvent = RPCEvent<TSpec>;
type Error = ProtocolsHandlerUpgrErr<RPCError>; type Error = ProtocolsHandlerUpgrErr<RPCError>;
type Substream = TSubstream; type Substream = TSubstream;
type InboundProtocol = RPCProtocol; type InboundProtocol = RPCProtocol<TSpec>;
type OutboundProtocol = RPCRequest; type OutboundProtocol = RPCRequest<TSpec>;
type OutboundOpenInfo = RPCEvent; // Keep track of the id and the request type OutboundOpenInfo = RPCEvent<TSpec>; // Keep track of the id and the request
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> { fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol> {
self.listen_protocol.clone() self.listen_protocol.clone()
@ -262,7 +270,7 @@ where
fn inject_fully_negotiated_inbound( fn inject_fully_negotiated_inbound(
&mut self, &mut self,
out: <RPCProtocol as InboundUpgrade<TSubstream>>::Output, out: <RPCProtocol<TSpec> as InboundUpgrade<TSubstream>>::Output,
) { ) {
// update the keep alive timeout if there are no more remaining outbound streams // update the keep alive timeout if there are no more remaining outbound streams
if let KeepAlive::Until(_) = self.keep_alive { if let KeepAlive::Until(_) = self.keep_alive {
@ -294,7 +302,7 @@ where
fn inject_fully_negotiated_outbound( fn inject_fully_negotiated_outbound(
&mut self, &mut self,
out: <RPCRequest as OutboundUpgrade<TSubstream>>::Output, out: <RPCRequest<TSpec> as OutboundUpgrade<TSubstream>>::Output,
rpc_event: Self::OutboundOpenInfo, rpc_event: Self::OutboundOpenInfo,
) { ) {
self.dial_negotiated -= 1; self.dial_negotiated -= 1;
@ -310,7 +318,22 @@ where
// add the stream to substreams if we expect a response, otherwise drop the stream. // add the stream to substreams if we expect a response, otherwise drop the stream.
match rpc_event { match rpc_event {
RPCEvent::Request(id, request) if request.expect_response() => { RPCEvent::Request(mut id, request) if request.expect_response() => {
// outbound requests can be sent from various aspects of lighthouse which don't
// track request ids. In the future these will be flagged as None, currently they
// are flagged as 0. These can overlap. In this case, we pick the highest request
// Id available
if id == 0 && self.outbound_substreams.get(&id).is_some() {
// have duplicate outbound request with no id. Pick one that will not collide
let mut new_id = std::usize::MAX;
while self.outbound_substreams.get(&new_id).is_some() {
// panic all outbound substreams are full
new_id -= 1;
}
trace!(self.log, "New outbound stream id created"; "id" => new_id);
id = RequestId::from(new_id);
}
// new outbound request. Store the stream and tag the output. // new outbound request. Store the stream and tag the output.
let delay_key = self let delay_key = self
.outbound_substreams_delay .outbound_substreams_delay
@ -323,7 +346,7 @@ where
.outbound_substreams .outbound_substreams
.insert(id, (awaiting_stream, delay_key)) .insert(id, (awaiting_stream, delay_key))
{ {
warn!(self.log, "Duplicate outbound substream id"; "id" => format!("{:?}", id)); crit!(self.log, "Duplicate outbound substream id"; "id" => format!("{:?}", id));
} }
} }
_ => { // a response is not expected, drop the stream for all other requests _ => { // a response is not expected, drop the stream for all other requests
@ -393,7 +416,7 @@ where
} }
} }
None => { None => {
debug!(self.log, "Stream has expired. Response not sent"; "response" => format!("{}",response)); warn!(self.log, "Stream has expired. Response not sent"; "response" => format!("{}",response));
} }
}; };
} }
@ -465,12 +488,14 @@ where
} }
ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(err)) => { ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(err)) => {
// IO/Decode/Custom Error, report to the application // IO/Decode/Custom Error, report to the application
debug!(self.log, "Upgrade Error"; "error" => format!("{}",err));
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom( return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(
RPCEvent::Error(request_id, err), RPCEvent::Error(request_id, err),
))); )));
} }
ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(err)) => { ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(err)) => {
// Error during negotiation // Error during negotiation
debug!(self.log, "Upgrade Error"; "error" => format!("{}",err));
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom( return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(
RPCEvent::Error(request_id, RPCError::Custom(format!("{}", err))), RPCEvent::Error(request_id, RPCError::Custom(format!("{}", err))),
))); )));
@ -488,10 +513,11 @@ where
} }
// purge expired inbound substreams and send an error // purge expired inbound substreams and send an error
while let Async::Ready(Some(stream_id)) = self while let Async::Ready(Some(stream_id)) =
.inbound_substreams_delay self.inbound_substreams_delay.poll().map_err(|e| {
.poll() warn!(self.log, "Inbound substream poll failed"; "error" => format!("{:?}", e));
.map_err(|_| ProtocolsHandlerUpgrErr::Timer)? ProtocolsHandlerUpgrErr::Timer
})?
{ {
let rpc_id = stream_id.get_ref(); let rpc_id = stream_id.get_ref();
@ -509,10 +535,11 @@ where
} }
// purge expired outbound substreams // purge expired outbound substreams
if let Async::Ready(Some(stream_id)) = self if let Async::Ready(Some(stream_id)) =
.outbound_substreams_delay self.outbound_substreams_delay.poll().map_err(|e| {
.poll() warn!(self.log, "Outbound substream poll failed"; "error" => format!("{:?}", e));
.map_err(|_| ProtocolsHandlerUpgrErr::Timer)? ProtocolsHandlerUpgrErr::Timer
})?
{ {
self.outbound_substreams.remove(stream_id.get_ref()); self.outbound_substreams.remove(stream_id.get_ref());
// notify the user // notify the user
@ -748,11 +775,11 @@ where
} }
// Check for new items to send to the peer and update the underlying stream // Check for new items to send to the peer and update the underlying stream
fn apply_queued_responses<TSubstream: AsyncRead + AsyncWrite>( fn apply_queued_responses<TSubstream: AsyncRead + AsyncWrite, TSpec: EthSpec>(
raw_substream: InboundFramed<TSubstream>, raw_substream: InboundFramed<TSubstream, TSpec>,
queued_outbound_items: &mut Option<&mut Vec<RPCErrorResponse>>, queued_outbound_items: &mut Option<&mut Vec<RPCErrorResponse<TSpec>>>,
new_items_to_send: &mut bool, new_items_to_send: &mut bool,
) -> InboundSubstreamState<TSubstream> { ) -> InboundSubstreamState<TSubstream, TSpec> {
match queued_outbound_items { match queued_outbound_items {
Some(ref mut queue) if !queue.is_empty() => { Some(ref mut queue) if !queue.is_empty() => {
*new_items_to_send = true; *new_items_to_send = true;

View File

@ -1,7 +1,9 @@
//! Available RPC methods types and ids. //! Available RPC methods types and ids.
use crate::types::EnrBitfield;
use serde::Serialize;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use types::{Epoch, Hash256, Slot}; use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/* Request/Response data structures for RPC methods */ /* Request/Response data structures for RPC methods */
@ -13,7 +15,7 @@ pub type RequestId = usize;
#[derive(Encode, Decode, Clone, Debug, PartialEq)] #[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct StatusMessage { pub struct StatusMessage {
/// The fork version of the chain we are broadcasting. /// The fork version of the chain we are broadcasting.
pub fork_version: [u8; 4], pub fork_digest: [u8; 4],
/// Latest finalized root. /// Latest finalized root.
pub finalized_root: Hash256, pub finalized_root: Hash256,
@ -28,6 +30,23 @@ pub struct StatusMessage {
pub head_slot: Slot, pub head_slot: Slot,
} }
/// The PING request/response message.
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct Ping {
/// The metadata sequence number.
pub data: u64,
}
/// The METADATA response structure.
#[derive(Encode, Decode, Clone, Debug, PartialEq, Serialize)]
#[serde(bound = "T: EthSpec")]
pub struct MetaData<T: EthSpec> {
/// A sequential counter indicating when data gets modified.
pub seq_number: u64,
/// The persistent subnet bitfield.
pub attnets: EnrBitfield<T>,
}
/// The reason given for a `Goodbye` message. /// The reason given for a `Goodbye` message.
/// ///
/// Note: any unknown `u64::into(n)` will resolve to `Goodbye::Unknown` for any unknown `n`, /// Note: any unknown `u64::into(n)` will resolve to `Goodbye::Unknown` for any unknown `n`,
@ -101,9 +120,6 @@ impl ssz::Decode for GoodbyeReason {
/// Request a number of beacon block roots from a peer. /// Request a number of beacon block roots from a peer.
#[derive(Encode, Decode, Clone, Debug, PartialEq)] #[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct BlocksByRangeRequest { pub struct BlocksByRangeRequest {
/// The hash tree root of a block on the requested chain.
pub head_block_root: Hash256,
/// The starting slot to request blocks. /// The starting slot to request blocks.
pub start_slot: u64, pub start_slot: u64,
@ -129,16 +145,22 @@ pub struct BlocksByRootRequest {
// Collection of enums and structs used by the Codecs to encode/decode RPC messages // Collection of enums and structs used by the Codecs to encode/decode RPC messages
#[derive(Debug, Clone, PartialEq)] #[derive(Debug, Clone, PartialEq)]
pub enum RPCResponse { pub enum RPCResponse<T: EthSpec> {
/// A HELLO message. /// A HELLO message.
Status(StatusMessage), Status(StatusMessage),
/// A response to a get BLOCKS_BY_RANGE request. A None response signifies the end of the /// A response to a get BLOCKS_BY_RANGE request. A None response signifies the end of the
/// batch. /// batch.
BlocksByRange(Vec<u8>), BlocksByRange(Box<SignedBeaconBlock<T>>),
/// A response to a get BLOCKS_BY_ROOT request. /// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Vec<u8>), BlocksByRoot(Box<SignedBeaconBlock<T>>),
/// A PONG response to a PING request.
Pong(Ping),
/// A response to a META_DATA request.
MetaData(MetaData<T>),
} }
/// Indicates which response is being terminated by a stream termination response. /// Indicates which response is being terminated by a stream termination response.
@ -152,9 +174,9 @@ pub enum ResponseTermination {
} }
#[derive(Debug)] #[derive(Debug)]
pub enum RPCErrorResponse { pub enum RPCErrorResponse<T: EthSpec> {
/// The response is a successful. /// The response is a successful.
Success(RPCResponse), Success(RPCResponse<T>),
/// The response was invalid. /// The response was invalid.
InvalidRequest(ErrorMessage), InvalidRequest(ErrorMessage),
@ -169,7 +191,7 @@ pub enum RPCErrorResponse {
StreamTermination(ResponseTermination), StreamTermination(ResponseTermination),
} }
impl RPCErrorResponse { impl<T: EthSpec> RPCErrorResponse<T> {
/// Used to encode the response in the codec. /// Used to encode the response in the codec.
pub fn as_u8(&self) -> Option<u8> { pub fn as_u8(&self) -> Option<u8> {
match self { match self {
@ -205,6 +227,8 @@ impl RPCErrorResponse {
RPCResponse::Status(_) => false, RPCResponse::Status(_) => false,
RPCResponse::BlocksByRange(_) => true, RPCResponse::BlocksByRange(_) => true,
RPCResponse::BlocksByRoot(_) => true, RPCResponse::BlocksByRoot(_) => true,
RPCResponse::Pong(_) => false,
RPCResponse::MetaData(_) => false,
}, },
RPCErrorResponse::InvalidRequest(_) => true, RPCErrorResponse::InvalidRequest(_) => true,
RPCErrorResponse::ServerError(_) => true, RPCErrorResponse::ServerError(_) => true,
@ -238,21 +262,27 @@ impl ErrorMessage {
impl std::fmt::Display for StatusMessage { impl std::fmt::Display for StatusMessage {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "Status 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) write!(f, "Status Message: Fork Digest: {:?}, Finalized Root: {}, Finalized Epoch: {}, Head Root: {}, Head Slot: {}", self.fork_digest, self.finalized_root, self.finalized_epoch, self.head_root, self.head_slot)
} }
} }
impl std::fmt::Display for RPCResponse { impl<T: EthSpec> std::fmt::Display for RPCResponse<T> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self { match self {
RPCResponse::Status(status) => write!(f, "{}", status), RPCResponse::Status(status) => write!(f, "{}", status),
RPCResponse::BlocksByRange(_) => write!(f, "<BlocksByRange>"), RPCResponse::BlocksByRange(block) => {
RPCResponse::BlocksByRoot(_) => write!(f, "<BlocksByRoot>"), write!(f, "BlocksByRange: Block slot: {}", block.message.slot)
}
RPCResponse::BlocksByRoot(block) => {
write!(f, "BlocksByRoot: BLock slot: {}", block.message.slot)
}
RPCResponse::Pong(ping) => write!(f, "Pong: {}", ping.data),
RPCResponse::MetaData(metadata) => write!(f, "Metadata: {}", metadata.seq_number),
} }
} }
} }
impl std::fmt::Display for RPCErrorResponse { impl<T: EthSpec> std::fmt::Display for RPCErrorResponse<T> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self { match self {
RPCErrorResponse::Success(res) => write!(f, "{}", res), RPCErrorResponse::Success(res) => write!(f, "{}", res),
@ -279,8 +309,8 @@ impl std::fmt::Display for BlocksByRangeRequest {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!( write!(
f, f,
"Head Block Root: {}, Start Slot: {}, Count: {}, Step: {}", "Start Slot: {}, Count: {}, Step: {}",
self.head_block_root, self.start_slot, self.count, self.step self.start_slot, self.count, self.step
) )
} }
} }

View File

@ -13,13 +13,15 @@ use libp2p::swarm::{
}; };
use libp2p::{Multiaddr, PeerId}; use libp2p::{Multiaddr, PeerId};
pub use methods::{ pub use methods::{
ErrorMessage, RPCErrorResponse, RPCResponse, RequestId, ResponseTermination, StatusMessage, ErrorMessage, MetaData, RPCErrorResponse, RPCResponse, RequestId, ResponseTermination,
StatusMessage,
}; };
pub use protocol::{RPCError, RPCProtocol, RPCRequest}; pub use protocol::{RPCError, RPCProtocol, RPCRequest};
use slog::o; use slog::{debug, o};
use std::marker::PhantomData; use std::marker::PhantomData;
use std::time::Duration; use std::time::Duration;
use tokio::io::{AsyncRead, AsyncWrite}; use tokio::io::{AsyncRead, AsyncWrite};
use types::EthSpec;
pub(crate) mod codec; pub(crate) mod codec;
mod handler; mod handler;
@ -28,19 +30,19 @@ mod protocol;
/// The return type used in the behaviour and the resultant event from the protocols handler. /// The return type used in the behaviour and the resultant event from the protocols handler.
#[derive(Debug)] #[derive(Debug)]
pub enum RPCEvent { pub enum RPCEvent<T: EthSpec> {
/// An inbound/outbound request for RPC protocol. The first parameter is a sequential /// An inbound/outbound request for RPC protocol. The first parameter is a sequential
/// id which tracks an awaiting substream for the response. /// id which tracks an awaiting substream for the response.
Request(RequestId, RPCRequest), Request(RequestId, RPCRequest<T>),
/// A response that is being sent or has been received from the RPC protocol. The first parameter returns /// A response that is being sent or has been received from the RPC protocol. The first parameter returns
/// that which was sent with the corresponding request, the second is a single chunk of a /// that which was sent with the corresponding request, the second is a single chunk of a
/// response. /// response.
Response(RequestId, RPCErrorResponse), Response(RequestId, RPCErrorResponse<T>),
/// An Error occurred. /// An Error occurred.
Error(RequestId, RPCError), Error(RequestId, RPCError),
} }
impl RPCEvent { impl<T: EthSpec> RPCEvent<T> {
pub fn id(&self) -> usize { pub fn id(&self) -> usize {
match *self { match *self {
RPCEvent::Request(id, _) => id, RPCEvent::Request(id, _) => id,
@ -50,7 +52,7 @@ impl RPCEvent {
} }
} }
impl std::fmt::Display for RPCEvent { impl<T: EthSpec> std::fmt::Display for RPCEvent<T> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self { match self {
RPCEvent::Request(id, req) => write!(f, "RPC Request(id: {}, {})", id, req), RPCEvent::Request(id, req) => write!(f, "RPC Request(id: {}, {})", id, req),
@ -62,16 +64,16 @@ impl std::fmt::Display for RPCEvent {
/// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level /// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level
/// logic. /// logic.
pub struct RPC<TSubstream> { pub struct RPC<TSubstream, TSpec: EthSpec> {
/// Queue of events to processed. /// Queue of events to processed.
events: Vec<NetworkBehaviourAction<RPCEvent, RPCMessage>>, events: Vec<NetworkBehaviourAction<RPCEvent<TSpec>, RPCMessage<TSpec>>>,
/// Pins the generic substream. /// Pins the generic substream.
marker: PhantomData<TSubstream>, marker: PhantomData<TSubstream>,
/// Slog logger for RPC behaviour. /// Slog logger for RPC behaviour.
log: slog::Logger, log: slog::Logger,
} }
impl<TSubstream> RPC<TSubstream> { impl<TSubstream, TSpec: EthSpec> RPC<TSubstream, TSpec> {
pub fn new(log: slog::Logger) -> Self { pub fn new(log: slog::Logger) -> Self {
let log = log.new(o!("service" => "libp2p_rpc")); let log = log.new(o!("service" => "libp2p_rpc"));
RPC { RPC {
@ -84,7 +86,7 @@ impl<TSubstream> RPC<TSubstream> {
/// Submits an RPC request. /// Submits an RPC request.
/// ///
/// The peer must be connected for this to succeed. /// The peer must be connected for this to succeed.
pub fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCEvent) { pub fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCEvent<TSpec>) {
self.events.push(NetworkBehaviourAction::SendEvent { self.events.push(NetworkBehaviourAction::SendEvent {
peer_id, peer_id,
event: rpc_event, event: rpc_event,
@ -92,16 +94,19 @@ impl<TSubstream> RPC<TSubstream> {
} }
} }
impl<TSubstream> NetworkBehaviour for RPC<TSubstream> impl<TSubstream, TSpec> NetworkBehaviour for RPC<TSubstream, TSpec>
where where
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
type ProtocolsHandler = RPCHandler<TSubstream>; type ProtocolsHandler = RPCHandler<TSubstream, TSpec>;
type OutEvent = RPCMessage; type OutEvent = RPCMessage<TSpec>;
fn new_handler(&mut self) -> Self::ProtocolsHandler { fn new_handler(&mut self) -> Self::ProtocolsHandler {
RPCHandler::new( RPCHandler::new(
SubstreamProtocol::new(RPCProtocol), SubstreamProtocol::new(RPCProtocol {
phantom: PhantomData,
}),
Duration::from_secs(30), Duration::from_secs(30),
&self.log, &self.log,
) )
@ -113,15 +118,33 @@ where
} }
fn inject_connected(&mut self, peer_id: PeerId, connected_point: ConnectedPoint) { fn inject_connected(&mut self, peer_id: PeerId, connected_point: ConnectedPoint) {
// TODO: Remove this on proper peer discovery
self.events.push(NetworkBehaviourAction::GenerateEvent(
RPCMessage::PeerConnectedHack(peer_id.clone(), connected_point.clone()),
));
// if initialised the connection, report this upwards to send the HELLO request // if initialised the connection, report this upwards to send the HELLO request
if let ConnectedPoint::Dialer { .. } = connected_point { if let ConnectedPoint::Dialer { .. } = connected_point {
self.events.push(NetworkBehaviourAction::GenerateEvent( self.events.push(NetworkBehaviourAction::GenerateEvent(
RPCMessage::PeerDialed(peer_id), RPCMessage::PeerDialed(peer_id.clone()),
)); ));
} }
// find the peer's meta-data
debug!(self.log, "Requesting new peer's metadata"; "peer_id" => format!("{}",peer_id));
let rpc_event =
RPCEvent::Request(RequestId::from(0usize), RPCRequest::MetaData(PhantomData));
self.events.push(NetworkBehaviourAction::SendEvent {
peer_id,
event: rpc_event,
});
} }
fn inject_disconnected(&mut self, peer_id: &PeerId, _: ConnectedPoint) { fn inject_disconnected(&mut self, peer_id: &PeerId, connected_point: ConnectedPoint) {
// TODO: Remove this on proper peer discovery
self.events.push(NetworkBehaviourAction::GenerateEvent(
RPCMessage::PeerDisconnectedHack(peer_id.clone(), connected_point.clone()),
));
// inform the rpc handler that the peer has disconnected // inform the rpc handler that the peer has disconnected
self.events.push(NetworkBehaviourAction::GenerateEvent( self.events.push(NetworkBehaviourAction::GenerateEvent(
RPCMessage::PeerDisconnected(peer_id.clone()), RPCMessage::PeerDisconnected(peer_id.clone()),
@ -157,8 +180,12 @@ where
} }
/// Messages sent to the user from the RPC protocol. /// Messages sent to the user from the RPC protocol.
pub enum RPCMessage { pub enum RPCMessage<TSpec: EthSpec> {
RPC(PeerId, RPCEvent), RPC(PeerId, RPCEvent<TSpec>),
PeerDialed(PeerId), PeerDialed(PeerId),
PeerDisconnected(PeerId), PeerDisconnected(PeerId),
// TODO: This is a hack to give access to connections to peer manager. Remove this once
// behaviour is re-written
PeerConnectedHack(PeerId, ConnectedPoint),
PeerDisconnectedHack(PeerId, ConnectedPoint),
} }

View File

@ -5,26 +5,26 @@ use crate::rpc::{
codec::{ codec::{
base::{BaseInboundCodec, BaseOutboundCodec}, base::{BaseInboundCodec, BaseOutboundCodec},
ssz::{SSZInboundCodec, SSZOutboundCodec}, ssz::{SSZInboundCodec, SSZOutboundCodec},
ssz_snappy::{SSZSnappyInboundCodec, SSZSnappyOutboundCodec},
InboundCodec, OutboundCodec, InboundCodec, OutboundCodec,
}, },
methods::ResponseTermination, methods::ResponseTermination,
}; };
use futures::{ use futures::future::*;
future::{self, FutureResult}, use futures::{future, sink, stream, Sink, Stream};
sink, stream, Sink, Stream,
};
use libp2p::core::{upgrade, InboundUpgrade, OutboundUpgrade, ProtocolName, UpgradeInfo}; use libp2p::core::{upgrade, InboundUpgrade, OutboundUpgrade, ProtocolName, UpgradeInfo};
use std::io; use std::io;
use std::marker::PhantomData;
use std::time::Duration; use std::time::Duration;
use tokio::codec::Framed; use tokio::codec::Framed;
use tokio::io::{AsyncRead, AsyncWrite}; use tokio::io::{AsyncRead, AsyncWrite};
use tokio::prelude::*;
use tokio::timer::timeout; use tokio::timer::timeout;
use tokio::util::FutureExt; use tokio::util::FutureExt;
use tokio_io_timeout::TimeoutStream; use tokio_io_timeout::TimeoutStream;
use types::EthSpec;
/// The maximum bytes that can be sent across the RPC. /// The maximum bytes that can be sent across the RPC.
const MAX_RPC_SIZE: usize = 4_194_304; // 4M const MAX_RPC_SIZE: usize = 1_048_576; // 1M
/// The protocol prefix the RPC protocol id. /// The protocol prefix the RPC protocol id.
const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req"; const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req";
/// Time allowed for the first byte of a request to arrive before we time out (Time To First Byte). /// Time allowed for the first byte of a request to arrive before we time out (Time To First Byte).
@ -34,43 +34,108 @@ const TTFB_TIMEOUT: u64 = 5;
const REQUEST_TIMEOUT: u64 = 15; const REQUEST_TIMEOUT: u64 = 15;
/// Protocol names to be used. /// Protocol names to be used.
/// The Status protocol name. #[derive(Debug, Clone)]
pub const RPC_STATUS: &str = "status"; pub enum Protocol {
/// The Goodbye protocol name. /// The Status protocol name.
pub const RPC_GOODBYE: &str = "goodbye"; Status,
/// The `BlocksByRange` protocol name. /// The Goodbye protocol name.
pub const RPC_BLOCKS_BY_RANGE: &str = "beacon_blocks_by_range"; Goodbye,
/// The `BlocksByRoot` protocol name. /// The `BlocksByRange` protocol name.
pub const RPC_BLOCKS_BY_ROOT: &str = "beacon_blocks_by_root"; BlocksByRange,
/// The `BlocksByRoot` protocol name.
BlocksByRoot,
/// The `Ping` protocol name.
Ping,
/// The `MetaData` protocol name.
MetaData,
}
/// RPC Versions
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum Version {
/// Version 1 of RPC
V1,
}
/// RPC Encondings supported.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum Encoding {
SSZ,
SSZSnappy,
}
impl std::fmt::Display for Protocol {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let repr = match self {
Protocol::Status => "status",
Protocol::Goodbye => "goodbye",
Protocol::BlocksByRange => "beacon_blocks_by_range",
Protocol::BlocksByRoot => "beacon_blocks_by_root",
Protocol::Ping => "ping",
Protocol::MetaData => "metadata",
};
f.write_str(repr)
}
}
impl std::fmt::Display for Encoding {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let repr = match self {
Encoding::SSZ => "ssz",
Encoding::SSZSnappy => "ssz_snappy",
};
f.write_str(repr)
}
}
impl std::fmt::Display for Version {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let repr = match self {
Version::V1 => "1",
};
f.write_str(repr)
}
}
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct RPCProtocol; pub struct RPCProtocol<TSpec: EthSpec> {
pub phantom: PhantomData<TSpec>,
}
impl UpgradeInfo for RPCProtocol { impl<TSpec: EthSpec> UpgradeInfo for RPCProtocol<TSpec> {
type Info = ProtocolId; type Info = ProtocolId;
type InfoIter = Vec<Self::Info>; type InfoIter = Vec<Self::Info>;
/// The list of supported RPC protocols for Lighthouse.
fn protocol_info(&self) -> Self::InfoIter { fn protocol_info(&self) -> Self::InfoIter {
vec![ vec![
ProtocolId::new(RPC_STATUS, "1", "ssz"), ProtocolId::new(Protocol::Status, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(RPC_GOODBYE, "1", "ssz"), ProtocolId::new(Protocol::Status, Version::V1, Encoding::SSZ),
ProtocolId::new(RPC_BLOCKS_BY_RANGE, "1", "ssz"), ProtocolId::new(Protocol::Goodbye, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(RPC_BLOCKS_BY_ROOT, "1", "ssz"), ProtocolId::new(Protocol::Goodbye, Version::V1, Encoding::SSZ),
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZ),
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZ),
ProtocolId::new(Protocol::Ping, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::Ping, Version::V1, Encoding::SSZ),
ProtocolId::new(Protocol::MetaData, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::MetaData, Version::V1, Encoding::SSZ),
] ]
} }
} }
/// Tracks the types in a protocol id. /// Tracks the types in a protocol id.
#[derive(Clone)] #[derive(Clone, Debug)]
pub struct ProtocolId { pub struct ProtocolId {
/// The rpc message type/name. /// The rpc message type/name.
pub message_name: String, pub message_name: Protocol,
/// The version of the RPC. /// The version of the RPC.
pub version: String, pub version: Version,
/// The encoding of the RPC. /// The encoding of the RPC.
pub encoding: String, pub encoding: Encoding,
/// The protocol id that is formed from the above fields. /// The protocol id that is formed from the above fields.
protocol_id: String, protocol_id: String,
@ -78,16 +143,16 @@ pub struct ProtocolId {
/// An RPC protocol ID. /// An RPC protocol ID.
impl ProtocolId { impl ProtocolId {
pub fn new(message_name: &str, version: &str, encoding: &str) -> Self { pub fn new(message_name: Protocol, version: Version, encoding: Encoding) -> Self {
let protocol_id = format!( let protocol_id = format!(
"{}/{}/{}/{}", "{}/{}/{}/{}",
PROTOCOL_PREFIX, message_name, version, encoding PROTOCOL_PREFIX, message_name, version, encoding
); );
ProtocolId { ProtocolId {
message_name: message_name.into(), message_name,
version: version.into(), version: version,
encoding: encoding.into(), encoding,
protocol_id, protocol_id,
} }
} }
@ -104,27 +169,33 @@ impl ProtocolName for ProtocolId {
// The inbound protocol reads the request, decodes it and returns the stream to the protocol // The inbound protocol reads the request, decodes it and returns the stream to the protocol
// handler to respond to once ready. // handler to respond to once ready.
pub type InboundOutput<TSocket> = (RPCRequest, InboundFramed<TSocket>); pub type InboundOutput<TSocket, TSpec> = (RPCRequest<TSpec>, InboundFramed<TSocket, TSpec>);
pub type InboundFramed<TSocket> = Framed<TimeoutStream<upgrade::Negotiated<TSocket>>, InboundCodec>; pub type InboundFramed<TSocket, TSpec> =
type FnAndThen<TSocket> = fn( Framed<TimeoutStream<upgrade::Negotiated<TSocket>>, InboundCodec<TSpec>>;
(Option<RPCRequest>, InboundFramed<TSocket>), type FnAndThen<TSocket, TSpec> = fn(
) -> FutureResult<InboundOutput<TSocket>, RPCError>; (Option<RPCRequest<TSpec>>, InboundFramed<TSocket, TSpec>),
type FnMapErr<TSocket> = fn(timeout::Error<(RPCError, InboundFramed<TSocket>)>) -> RPCError; ) -> FutureResult<InboundOutput<TSocket, TSpec>, RPCError>;
type FnMapErr<TSocket, TSpec> =
fn(timeout::Error<(RPCError, InboundFramed<TSocket, TSpec>)>) -> RPCError;
impl<TSocket> InboundUpgrade<TSocket> for RPCProtocol impl<TSocket, TSpec> InboundUpgrade<TSocket> for RPCProtocol<TSpec>
where where
TSocket: AsyncRead + AsyncWrite, TSocket: AsyncRead + AsyncWrite,
TSpec: EthSpec,
{ {
type Output = InboundOutput<TSocket>; type Output = InboundOutput<TSocket, TSpec>;
type Error = RPCError; type Error = RPCError;
type Future = future::AndThen< type Future = future::Either<
future::MapErr< FutureResult<InboundOutput<TSocket, TSpec>, RPCError>,
timeout::Timeout<stream::StreamFuture<InboundFramed<TSocket>>>, future::AndThen<
FnMapErr<TSocket>, future::MapErr<
timeout::Timeout<stream::StreamFuture<InboundFramed<TSocket, TSpec>>>,
FnMapErr<TSocket, TSpec>,
>,
FutureResult<InboundOutput<TSocket, TSpec>, RPCError>,
FnAndThen<TSocket, TSpec>,
>, >,
FutureResult<InboundOutput<TSocket>, RPCError>,
FnAndThen<TSocket>,
>; >;
fn upgrade_inbound( fn upgrade_inbound(
@ -132,25 +203,44 @@ where
socket: upgrade::Negotiated<TSocket>, socket: upgrade::Negotiated<TSocket>,
protocol: ProtocolId, protocol: ProtocolId,
) -> Self::Future { ) -> Self::Future {
match protocol.encoding.as_str() { let protocol_name = protocol.message_name.clone();
"ssz" | _ => { let codec = match protocol.encoding {
Encoding::SSZSnappy => {
let ssz_snappy_codec =
BaseInboundCodec::new(SSZSnappyInboundCodec::new(protocol, MAX_RPC_SIZE));
InboundCodec::SSZSnappy(ssz_snappy_codec)
}
Encoding::SSZ => {
let ssz_codec = BaseInboundCodec::new(SSZInboundCodec::new(protocol, MAX_RPC_SIZE)); let ssz_codec = BaseInboundCodec::new(SSZInboundCodec::new(protocol, MAX_RPC_SIZE));
let codec = InboundCodec::SSZ(ssz_codec); InboundCodec::SSZ(ssz_codec)
let mut timed_socket = TimeoutStream::new(socket); }
timed_socket.set_read_timeout(Some(Duration::from_secs(TTFB_TIMEOUT))); };
Framed::new(timed_socket, codec) let mut timed_socket = TimeoutStream::new(socket);
timed_socket.set_read_timeout(Some(Duration::from_secs(TTFB_TIMEOUT)));
let socket = Framed::new(timed_socket, codec);
// MetaData requests should be empty, return the stream
match protocol_name {
Protocol::MetaData => futures::future::Either::A(futures::future::ok((
RPCRequest::MetaData(PhantomData),
socket,
))),
_ => futures::future::Either::B(
socket
.into_future() .into_future()
.timeout(Duration::from_secs(REQUEST_TIMEOUT)) .timeout(Duration::from_secs(REQUEST_TIMEOUT))
.map_err(RPCError::from as FnMapErr<TSocket>) .map_err(RPCError::from as FnMapErr<TSocket, TSpec>)
.and_then({ .and_then({
|(req, stream)| match req { |(req, stream)| match req {
Some(req) => futures::future::ok((req, stream)), Some(request) => futures::future::ok((request, stream)),
None => futures::future::err(RPCError::Custom( None => futures::future::err(RPCError::Custom(
"Stream terminated early".into(), "Stream terminated early".into(),
)), )),
} }
} as FnAndThen<TSocket>) } as FnAndThen<TSocket, TSpec>),
} ),
} }
} }
} }
@ -161,14 +251,16 @@ where
// `OutboundUpgrade` // `OutboundUpgrade`
#[derive(Debug, Clone, PartialEq)] #[derive(Debug, Clone, PartialEq)]
pub enum RPCRequest { pub enum RPCRequest<TSpec: EthSpec> {
Status(StatusMessage), Status(StatusMessage),
Goodbye(GoodbyeReason), Goodbye(GoodbyeReason),
BlocksByRange(BlocksByRangeRequest), BlocksByRange(BlocksByRangeRequest),
BlocksByRoot(BlocksByRootRequest), BlocksByRoot(BlocksByRootRequest),
Ping(Ping),
MetaData(PhantomData<TSpec>),
} }
impl UpgradeInfo for RPCRequest { impl<TSpec: EthSpec> UpgradeInfo for RPCRequest<TSpec> {
type Info = ProtocolId; type Info = ProtocolId;
type InfoIter = Vec<Self::Info>; type InfoIter = Vec<Self::Info>;
@ -179,14 +271,34 @@ impl UpgradeInfo for RPCRequest {
} }
/// Implements the encoding per supported protocol for RPCRequest. /// Implements the encoding per supported protocol for RPCRequest.
impl RPCRequest { impl<TSpec: EthSpec> RPCRequest<TSpec> {
pub fn supported_protocols(&self) -> Vec<ProtocolId> { pub fn supported_protocols(&self) -> Vec<ProtocolId> {
match self { match self {
// add more protocols when versions/encodings are supported // add more protocols when versions/encodings are supported
RPCRequest::Status(_) => vec![ProtocolId::new(RPC_STATUS, "1", "ssz")], RPCRequest::Status(_) => vec![
RPCRequest::Goodbye(_) => vec![ProtocolId::new(RPC_GOODBYE, "1", "ssz")], ProtocolId::new(Protocol::Status, Version::V1, Encoding::SSZSnappy),
RPCRequest::BlocksByRange(_) => vec![ProtocolId::new(RPC_BLOCKS_BY_RANGE, "1", "ssz")], ProtocolId::new(Protocol::Status, Version::V1, Encoding::SSZ),
RPCRequest::BlocksByRoot(_) => vec![ProtocolId::new(RPC_BLOCKS_BY_ROOT, "1", "ssz")], ],
RPCRequest::Goodbye(_) => vec![
ProtocolId::new(Protocol::Goodbye, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::Goodbye, Version::V1, Encoding::SSZ),
],
RPCRequest::BlocksByRange(_) => vec![
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZ),
],
RPCRequest::BlocksByRoot(_) => vec![
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZ),
],
RPCRequest::Ping(_) => vec![
ProtocolId::new(Protocol::Ping, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::Ping, Version::V1, Encoding::SSZ),
],
RPCRequest::MetaData(_) => vec![
ProtocolId::new(Protocol::MetaData, Version::V1, Encoding::SSZSnappy),
ProtocolId::new(Protocol::MetaData, Version::V1, Encoding::SSZ),
],
} }
} }
@ -200,6 +312,8 @@ impl RPCRequest {
RPCRequest::Goodbye(_) => false, RPCRequest::Goodbye(_) => false,
RPCRequest::BlocksByRange(_) => true, RPCRequest::BlocksByRange(_) => true,
RPCRequest::BlocksByRoot(_) => true, RPCRequest::BlocksByRoot(_) => true,
RPCRequest::Ping(_) => true,
RPCRequest::MetaData(_) => true,
} }
} }
@ -211,6 +325,8 @@ impl RPCRequest {
RPCRequest::Goodbye(_) => false, RPCRequest::Goodbye(_) => false,
RPCRequest::BlocksByRange(_) => true, RPCRequest::BlocksByRange(_) => true,
RPCRequest::BlocksByRoot(_) => true, RPCRequest::BlocksByRoot(_) => true,
RPCRequest::Ping(_) => false,
RPCRequest::MetaData(_) => false,
} }
} }
@ -224,6 +340,8 @@ impl RPCRequest {
RPCRequest::BlocksByRoot(_) => ResponseTermination::BlocksByRoot, RPCRequest::BlocksByRoot(_) => ResponseTermination::BlocksByRoot,
RPCRequest::Status(_) => unreachable!(), RPCRequest::Status(_) => unreachable!(),
RPCRequest::Goodbye(_) => unreachable!(), RPCRequest::Goodbye(_) => unreachable!(),
RPCRequest::Ping(_) => unreachable!(),
RPCRequest::MetaData(_) => unreachable!(),
} }
} }
} }
@ -232,28 +350,35 @@ impl RPCRequest {
/* Outbound upgrades */ /* Outbound upgrades */
pub type OutboundFramed<TSocket> = Framed<upgrade::Negotiated<TSocket>, OutboundCodec>; pub type OutboundFramed<TSocket, TSpec> =
Framed<upgrade::Negotiated<TSocket>, OutboundCodec<TSpec>>;
impl<TSocket> OutboundUpgrade<TSocket> for RPCRequest impl<TSocket, TSpec> OutboundUpgrade<TSocket> for RPCRequest<TSpec>
where where
TSpec: EthSpec,
TSocket: AsyncRead + AsyncWrite, TSocket: AsyncRead + AsyncWrite,
{ {
type Output = OutboundFramed<TSocket>; type Output = OutboundFramed<TSocket, TSpec>;
type Error = RPCError; type Error = RPCError;
type Future = sink::Send<OutboundFramed<TSocket>>; type Future = sink::Send<OutboundFramed<TSocket, TSpec>>;
fn upgrade_outbound( fn upgrade_outbound(
self, self,
socket: upgrade::Negotiated<TSocket>, socket: upgrade::Negotiated<TSocket>,
protocol: Self::Info, protocol: Self::Info,
) -> Self::Future { ) -> Self::Future {
match protocol.encoding.as_str() { let codec = match protocol.encoding {
"ssz" | _ => { Encoding::SSZSnappy => {
let ssz_snappy_codec =
BaseOutboundCodec::new(SSZSnappyOutboundCodec::new(protocol, MAX_RPC_SIZE));
OutboundCodec::SSZSnappy(ssz_snappy_codec)
}
Encoding::SSZ => {
let ssz_codec = let ssz_codec =
BaseOutboundCodec::new(SSZOutboundCodec::new(protocol, MAX_RPC_SIZE)); BaseOutboundCodec::new(SSZOutboundCodec::new(protocol, MAX_RPC_SIZE));
let codec = OutboundCodec::SSZ(ssz_codec); OutboundCodec::SSZ(ssz_codec)
Framed::new(socket, codec).send(self)
} }
} };
Framed::new(socket, codec).send(self)
} }
} }
@ -264,6 +389,8 @@ pub enum RPCError {
ReadError(upgrade::ReadOneError), ReadError(upgrade::ReadOneError),
/// Error when decoding the raw buffer from ssz. /// Error when decoding the raw buffer from ssz.
SSZDecodeError(ssz::DecodeError), SSZDecodeError(ssz::DecodeError),
/// Snappy error
SnappyError(snap::Error),
/// Invalid Protocol ID. /// Invalid Protocol ID.
InvalidProtocol(&'static str), InvalidProtocol(&'static str),
/// IO Error. /// IO Error.
@ -311,6 +438,12 @@ impl From<io::Error> for RPCError {
} }
} }
impl From<snap::Error> for RPCError {
fn from(err: snap::Error) -> Self {
RPCError::SnappyError(err)
}
}
// Error trait is required for `ProtocolsHandler` // Error trait is required for `ProtocolsHandler`
impl std::fmt::Display for RPCError { impl std::fmt::Display for RPCError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
@ -321,6 +454,7 @@ impl std::fmt::Display for RPCError {
RPCError::IoError(ref err) => write!(f, "IO Error: {}", err), RPCError::IoError(ref err) => write!(f, "IO Error: {}", err),
RPCError::RPCErrorResponse => write!(f, "RPC Response Error"), RPCError::RPCErrorResponse => write!(f, "RPC Response Error"),
RPCError::StreamTimeout => write!(f, "Stream Timeout"), RPCError::StreamTimeout => write!(f, "Stream Timeout"),
RPCError::SnappyError(ref err) => write!(f, "Snappy error: {}", err),
RPCError::Custom(ref err) => write!(f, "{}", err), RPCError::Custom(ref err) => write!(f, "{}", err),
} }
} }
@ -331,6 +465,7 @@ impl std::error::Error for RPCError {
match *self { match *self {
RPCError::ReadError(ref err) => Some(err), RPCError::ReadError(ref err) => Some(err),
RPCError::SSZDecodeError(_) => None, RPCError::SSZDecodeError(_) => None,
RPCError::SnappyError(ref err) => Some(err),
RPCError::InvalidProtocol(_) => None, RPCError::InvalidProtocol(_) => None,
RPCError::IoError(ref err) => Some(err), RPCError::IoError(ref err) => Some(err),
RPCError::StreamTimeout => None, RPCError::StreamTimeout => None,
@ -340,13 +475,15 @@ impl std::error::Error for RPCError {
} }
} }
impl std::fmt::Display for RPCRequest { impl<TSpec: EthSpec> std::fmt::Display for RPCRequest<TSpec> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self { match self {
RPCRequest::Status(status) => write!(f, "Status Message: {}", status), RPCRequest::Status(status) => write!(f, "Status Message: {}", status),
RPCRequest::Goodbye(reason) => write!(f, "Goodbye: {}", reason), RPCRequest::Goodbye(reason) => write!(f, "Goodbye: {}", reason),
RPCRequest::BlocksByRange(req) => write!(f, "Blocks by range: {}", req), RPCRequest::BlocksByRange(req) => write!(f, "Blocks by range: {}", req),
RPCRequest::BlocksByRoot(req) => write!(f, "Blocks by root: {:?}", req), RPCRequest::BlocksByRoot(req) => write!(f, "Blocks by root: {:?}", req),
RPCRequest::Ping(ping) => write!(f, "Ping: {}", ping.data),
RPCRequest::MetaData(_) => write!(f, "MetaData request"),
} }
} }
} }

View File

@ -1,9 +1,8 @@
use crate::behaviour::{Behaviour, BehaviourEvent, PubsubMessage}; use crate::behaviour::{Behaviour, BehaviourEvent};
use crate::error; use crate::discovery::enr;
use crate::multiaddr::Protocol; use crate::multiaddr::Protocol;
use crate::rpc::RPCEvent; use crate::types::{error, GossipKind};
use crate::NetworkConfig; use crate::{NetworkConfig, NetworkGlobals};
use crate::{NetworkGlobals, Topic, TopicHash};
use futures::prelude::*; use futures::prelude::*;
use futures::Stream; use futures::Stream;
use libp2p::core::{ use libp2p::core::{
@ -15,7 +14,6 @@ use libp2p::core::{
upgrade::{InboundUpgradeExt, OutboundUpgradeExt}, upgrade::{InboundUpgradeExt, OutboundUpgradeExt},
ConnectedPoint, ConnectedPoint,
}; };
use libp2p::gossipsub::MessageId;
use libp2p::{core, noise, secio, swarm::NetworkBehaviour, PeerId, Swarm, Transport}; use libp2p::{core, noise, secio, swarm::NetworkBehaviour, PeerId, Swarm, Transport};
use slog::{crit, debug, error, info, trace, warn}; use slog::{crit, debug, error, info, trace, warn};
use std::fs::File; use std::fs::File;
@ -24,20 +22,21 @@ use std::io::{Error, ErrorKind};
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use tokio::timer::DelayQueue; use tokio::timer::DelayQueue;
use types::{EnrForkId, EthSpec};
type Libp2pStream = Boxed<(PeerId, StreamMuxerBox), Error>; type Libp2pStream = Boxed<(PeerId, StreamMuxerBox), Error>;
type Libp2pBehaviour = Behaviour<Substream<StreamMuxerBox>>; type Libp2pBehaviour<TSpec> = Behaviour<Substream<StreamMuxerBox>, TSpec>;
const NETWORK_KEY_FILENAME: &str = "key"; pub const NETWORK_KEY_FILENAME: &str = "key";
/// The time in milliseconds to wait before banning a peer. This allows for any Goodbye messages to be /// The time in milliseconds to wait before banning a peer. This allows for any Goodbye messages to be
/// flushed and protocols to be negotiated. /// flushed and protocols to be negotiated.
const BAN_PEER_WAIT_TIMEOUT: u64 = 200; const BAN_PEER_WAIT_TIMEOUT: u64 = 200;
/// The configuration and state of the libp2p components for the beacon node. /// The configuration and state of the libp2p components for the beacon node.
pub struct Service { pub struct Service<TSpec: EthSpec> {
/// The libp2p Swarm handler. /// The libp2p Swarm handler.
//TODO: Make this private //TODO: Make this private
pub swarm: Swarm<Libp2pStream, Libp2pBehaviour>, pub swarm: Swarm<Libp2pStream, Libp2pBehaviour<TSpec>>,
/// This node's PeerId. /// This node's PeerId.
pub local_peer_id: PeerId, pub local_peer_id: PeerId,
@ -52,25 +51,36 @@ pub struct Service {
pub log: slog::Logger, pub log: slog::Logger,
} }
impl Service { impl<TSpec: EthSpec> Service<TSpec> {
pub fn new( pub fn new(
config: &NetworkConfig, config: &NetworkConfig,
enr_fork_id: EnrForkId,
log: slog::Logger, log: slog::Logger,
) -> error::Result<(Arc<NetworkGlobals>, Self)> { ) -> error::Result<(Arc<NetworkGlobals<TSpec>>, Self)> {
trace!(log, "Libp2p Service starting"); trace!(log, "Libp2p Service starting");
// initialise the node's ID
let local_keypair = if let Some(hex_bytes) = &config.secret_key_hex { let local_keypair = if let Some(hex_bytes) = &config.secret_key_hex {
keypair_from_hex(hex_bytes)? keypair_from_hex(hex_bytes)?
} else { } else {
load_private_key(config, &log) load_private_key(config, &log)
}; };
// load the private key from CLI flag, disk or generate a new one // Create an ENR or load from disk if appropriate
let local_peer_id = PeerId::from(local_keypair.public()); let enr =
info!(log, "Libp2p Service"; "peer_id" => format!("{:?}", local_peer_id)); enr::build_or_load_enr::<TSpec>(local_keypair.clone(), config, enr_fork_id, &log)?;
let local_peer_id = enr.peer_id();
// set up a collection of variables accessible outside of the network crate // set up a collection of variables accessible outside of the network crate
let network_globals = Arc::new(NetworkGlobals::new(local_peer_id.clone())); let network_globals = Arc::new(NetworkGlobals::new(
enr.clone(),
config.libp2p_port,
config.discovery_port,
&log,
));
info!(log, "Libp2p Service"; "peer_id" => format!("{:?}", enr.peer_id()));
debug!(log, "Attempting to open listening ports"; "address" => format!("{}", config.listen_address), "tcp_port" => config.libp2p_port, "udp_port" => config.discovery_port);
let mut swarm = { let mut swarm = {
// Set up the transport - tcp/ws with noise/secio and mplex/yamux // Set up the transport - tcp/ws with noise/secio and mplex/yamux
@ -128,19 +138,21 @@ impl Service {
if let Protocol::Udp(_) = components[1] { if let Protocol::Udp(_) = components[1] {
continue; continue;
} }
// inform the peer manager that we are currently dialing this peer
network_globals
.peers
.write()
.dialing_peer(&bootnode_enr.peer_id());
dial_addr(multiaddr); dial_addr(multiaddr);
} }
} }
let mut subscribed_topics: Vec<String> = vec![]; let mut subscribed_topics: Vec<GossipKind> = vec![];
for topic in config.topics.clone() { for topic_kind in &config.topics {
let raw_topic: Topic = topic.into(); if swarm.subscribe_kind(topic_kind.clone()) {
let topic_string = raw_topic.no_hash(); subscribed_topics.push(topic_kind.clone());
if swarm.subscribe(raw_topic.clone()) {
trace!(log, "Subscribed to topic"; "topic" => format!("{}", topic_string));
subscribed_topics.push(topic_string.as_str().into());
} else { } else {
warn!(log, "Could not subscribe to topic"; "topic" => format!("{}",topic_string)); warn!(log, "Could not subscribe to topic"; "topic" => format!("{}",topic_kind));
} }
} }
info!(log, "Subscribed to topics"; "topics" => format!("{:?}", subscribed_topics)); info!(log, "Subscribed to topics"; "topics" => format!("{:?}", subscribed_topics));
@ -167,43 +179,16 @@ impl Service {
} }
} }
impl Stream for Service { impl<TSpec: EthSpec> Stream for Service<TSpec> {
type Item = Libp2pEvent; type Item = BehaviourEvent<TSpec>;
type Error = crate::error::Error; type Error = error::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
loop { loop {
match self.swarm.poll() { match self.swarm.poll() {
Ok(Async::Ready(Some(event))) => match event { Ok(Async::Ready(Some(event))) => {
BehaviourEvent::GossipMessage { return Ok(Async::Ready(Some(event)));
id, }
source,
topics,
message,
} => {
trace!(self.log, "Gossipsub message received"; "service" => "Swarm");
return Ok(Async::Ready(Some(Libp2pEvent::PubsubMessage {
id,
source,
topics,
message,
})));
}
BehaviourEvent::RPC(peer_id, event) => {
return Ok(Async::Ready(Some(Libp2pEvent::RPC(peer_id, event))));
}
BehaviourEvent::PeerDialed(peer_id) => {
return Ok(Async::Ready(Some(Libp2pEvent::PeerDialed(peer_id))));
}
BehaviourEvent::PeerDisconnected(peer_id) => {
return Ok(Async::Ready(Some(Libp2pEvent::PeerDisconnected(peer_id))));
}
BehaviourEvent::PeerSubscribed(peer_id, topic) => {
return Ok(Async::Ready(Some(Libp2pEvent::PeerSubscribed(
peer_id, topic,
))));
}
},
Ok(Async::Ready(None)) => unreachable!("Swarm stream shouldn't end"), Ok(Async::Ready(None)) => unreachable!("Swarm stream shouldn't end"),
Ok(Async::NotReady) => break, Ok(Async::NotReady) => break,
_ => break, _ => break,
@ -311,26 +296,6 @@ fn build_transport(local_private_key: Keypair) -> Boxed<(PeerId, StreamMuxerBox)
transport transport
} }
#[derive(Debug)]
/// Events that can be obtained from polling the Libp2p Service.
pub enum Libp2pEvent {
/// An RPC response request has been received on the swarm.
RPC(PeerId, RPCEvent),
/// Initiated the connection to a new peer.
PeerDialed(PeerId),
/// A peer has disconnected.
PeerDisconnected(PeerId),
/// Received pubsub message.
PubsubMessage {
id: MessageId,
source: PeerId,
topics: Vec<TopicHash>,
message: PubsubMessage,
},
/// Subscribed to peer for a topic hash.
PeerSubscribed(PeerId, TopicHash),
}
fn keypair_from_hex(hex_bytes: &str) -> error::Result<Keypair> { fn keypair_from_hex(hex_bytes: &str) -> error::Result<Keypair> {
let hex_bytes = if hex_bytes.starts_with("0x") { let hex_bytes = if hex_bytes.starts_with("0x") {
hex_bytes[2..].to_string() hex_bytes[2..].to_string()

View File

@ -1,71 +0,0 @@
use libp2p::gossipsub::Topic;
use serde_derive::{Deserialize, Serialize};
/// The gossipsub topic names.
// These constants form a topic name of the form /TOPIC_PREFIX/TOPIC/ENCODING_POSTFIX
// For example /eth2/beacon_block/ssz
pub const TOPIC_PREFIX: &str = "eth2";
pub const TOPIC_ENCODING_POSTFIX: &str = "ssz";
pub const BEACON_BLOCK_TOPIC: &str = "beacon_block";
pub const BEACON_ATTESTATION_TOPIC: &str = "beacon_attestation";
pub const VOLUNTARY_EXIT_TOPIC: &str = "voluntary_exit";
pub const PROPOSER_SLASHING_TOPIC: &str = "proposer_slashing";
pub const ATTESTER_SLASHING_TOPIC: &str = "attester_slashing";
pub const SHARD_TOPIC_PREFIX: &str = "shard";
/// Enum that brings these topics into the rust type system.
#[derive(Clone, Debug, Serialize, Deserialize)]
pub enum GossipTopic {
BeaconBlock,
BeaconAttestation,
VoluntaryExit,
ProposerSlashing,
AttesterSlashing,
Shard,
Unknown(String),
}
impl From<&str> for GossipTopic {
fn from(topic: &str) -> GossipTopic {
let topic_parts: Vec<&str> = topic.split('/').collect();
if topic_parts.len() == 4
&& topic_parts[1] == TOPIC_PREFIX
&& topic_parts[3] == TOPIC_ENCODING_POSTFIX
{
match topic_parts[2] {
BEACON_BLOCK_TOPIC => GossipTopic::BeaconBlock,
BEACON_ATTESTATION_TOPIC => GossipTopic::BeaconAttestation,
VOLUNTARY_EXIT_TOPIC => GossipTopic::VoluntaryExit,
PROPOSER_SLASHING_TOPIC => GossipTopic::ProposerSlashing,
ATTESTER_SLASHING_TOPIC => GossipTopic::AttesterSlashing,
unknown_topic => GossipTopic::Unknown(unknown_topic.into()),
}
} else {
GossipTopic::Unknown(topic.into())
}
}
}
impl Into<Topic> for GossipTopic {
fn into(self) -> Topic {
Topic::new(self.into())
}
}
impl Into<String> for GossipTopic {
fn into(self) -> String {
match self {
GossipTopic::BeaconBlock => topic_builder(BEACON_BLOCK_TOPIC),
GossipTopic::BeaconAttestation => topic_builder(BEACON_ATTESTATION_TOPIC),
GossipTopic::VoluntaryExit => topic_builder(VOLUNTARY_EXIT_TOPIC),
GossipTopic::ProposerSlashing => topic_builder(PROPOSER_SLASHING_TOPIC),
GossipTopic::AttesterSlashing => topic_builder(ATTESTER_SLASHING_TOPIC),
GossipTopic::Shard => topic_builder(SHARD_TOPIC_PREFIX),
GossipTopic::Unknown(topic) => topic,
}
}
}
fn topic_builder(topic: &'static str) -> String {
format!("/{}/{}/{}", TOPIC_PREFIX, topic, TOPIC_ENCODING_POSTFIX,)
}

View File

@ -0,0 +1,124 @@
//! A collection of variables that are accessible outside of the network thread itself.
use crate::peer_manager::PeerDB;
use crate::rpc::methods::MetaData;
use crate::types::SyncState;
use crate::{discovery::enr::Eth2Enr, Enr, GossipTopic, Multiaddr, PeerId};
use parking_lot::RwLock;
use std::collections::HashSet;
use std::sync::atomic::{AtomicU16, Ordering};
use types::EthSpec;
pub struct NetworkGlobals<TSpec: EthSpec> {
/// The current local ENR.
pub local_enr: RwLock<Enr>,
/// The current node's meta-data.
pub meta_data: RwLock<MetaData<TSpec>>,
/// The local peer_id.
pub peer_id: RwLock<PeerId>,
/// Listening multiaddrs.
pub listen_multiaddrs: RwLock<Vec<Multiaddr>>,
/// The TCP port that the libp2p service is listening on
pub listen_port_tcp: AtomicU16,
/// The UDP port that the discovery service is listening on
pub listen_port_udp: AtomicU16,
/// The collection of known peers.
pub peers: RwLock<PeerDB<TSpec>>,
/// The current gossipsub topic subscriptions.
pub gossipsub_subscriptions: RwLock<HashSet<GossipTopic>>,
/// The current sync status of the node.
pub sync_state: RwLock<SyncState>,
}
impl<TSpec: EthSpec> NetworkGlobals<TSpec> {
pub fn new(enr: Enr, tcp_port: u16, udp_port: u16, log: &slog::Logger) -> Self {
// set up the local meta data of the node
let meta_data = RwLock::new(MetaData {
seq_number: 0,
attnets: enr
.bitfield::<TSpec>()
.expect("Local ENR must have a bitfield specified"),
});
NetworkGlobals {
local_enr: RwLock::new(enr.clone()),
meta_data,
peer_id: RwLock::new(enr.peer_id()),
listen_multiaddrs: RwLock::new(Vec::new()),
listen_port_tcp: AtomicU16::new(tcp_port),
listen_port_udp: AtomicU16::new(udp_port),
peers: RwLock::new(PeerDB::new(log)),
gossipsub_subscriptions: RwLock::new(HashSet::new()),
sync_state: RwLock::new(SyncState::Stalled),
}
}
/// Returns the local ENR from the underlying Discv5 behaviour that external peers may connect
/// to.
pub fn local_enr(&self) -> Enr {
self.local_enr.read().clone()
}
/// Returns the local libp2p PeerID.
pub fn local_peer_id(&self) -> PeerId {
self.peer_id.read().clone()
}
/// Returns the list of `Multiaddr` that the underlying libp2p instance is listening on.
pub fn listen_multiaddrs(&self) -> Vec<Multiaddr> {
self.listen_multiaddrs.read().clone()
}
/// Returns the libp2p TCP port that this node has been configured to listen on.
pub fn listen_port_tcp(&self) -> u16 {
self.listen_port_tcp.load(Ordering::Relaxed)
}
/// Returns the UDP discovery port that this node has been configured to listen on.
pub fn listen_port_udp(&self) -> u16 {
self.listen_port_udp.load(Ordering::Relaxed)
}
/// Returns the number of libp2p connected peers.
pub fn connected_peers(&self) -> usize {
self.peers.read().connected_peer_ids().count()
}
/// Returns the number of libp2p peers that are either connected or being dialed.
pub fn connected_or_dialing_peers(&self) -> usize {
self.peers.read().connected_or_dialing_peers().count()
}
/// Returns in the node is syncing.
pub fn is_syncing(&self) -> bool {
self.sync_state.read().is_syncing()
}
/// Returns the current sync state of the peer.
pub fn sync_state(&self) -> SyncState {
self.sync_state.read().clone()
}
/// Updates the syncing state of the node.
///
/// If there is a new state, the old state and the new states are returned.
pub fn update_sync_state(&self) -> Option<(SyncState, SyncState)> {
let mut result = None;
// if we are in a range sync, nothing changes. Range sync will update this.
if !self.is_syncing() {
let new_state = self
.peers
.read()
.synced_peers()
.next()
.map(|_| SyncState::Synced)
.unwrap_or_else(|| SyncState::Stalled);
let mut peer_state = self.sync_state.write();
if new_state != *peer_state {
result = Some((peer_state.clone(), new_state.clone()));
}
*peer_state = new_state;
}
result
}
}

View File

@ -0,0 +1,17 @@
pub mod error;
mod globals;
mod pubsub;
mod sync_state;
mod topics;
use types::{BitVector, EthSpec};
#[allow(type_alias_bounds)]
pub type EnrBitfield<T: EthSpec> = BitVector<T::SubnetBitfieldLength>;
pub type Enr = libp2p::discv5::enr::Enr<libp2p::discv5::enr::CombinedKey>;
pub use globals::NetworkGlobals;
pub use pubsub::PubsubMessage;
pub use sync_state::SyncState;
pub use topics::{GossipEncoding, GossipKind, GossipTopic};

View File

@ -0,0 +1,201 @@
//! Handles the encoding and decoding of pubsub messages.
use crate::config::GOSSIP_MAX_SIZE;
use crate::types::{GossipEncoding, GossipKind, GossipTopic};
use crate::TopicHash;
use snap::raw::{decompress_len, Decoder, Encoder};
use ssz::{Decode, Encode};
use std::boxed::Box;
use types::SubnetId;
use types::{
Attestation, AttesterSlashing, EthSpec, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, VoluntaryExit,
};
#[derive(Debug, Clone, PartialEq)]
pub enum PubsubMessage<T: EthSpec> {
/// Gossipsub message providing notification of a new block.
BeaconBlock(Box<SignedBeaconBlock<T>>),
/// Gossipsub message providing notification of a Aggregate attestation and associated proof.
AggregateAndProofAttestation(Box<SignedAggregateAndProof<T>>),
/// Gossipsub message providing notification of a raw un-aggregated attestation with its shard id.
Attestation(Box<(SubnetId, Attestation<T>)>),
/// Gossipsub message providing notification of a voluntary exit.
VoluntaryExit(Box<VoluntaryExit>),
/// Gossipsub message providing notification of a new proposer slashing.
ProposerSlashing(Box<ProposerSlashing>),
/// Gossipsub message providing notification of a new attester slashing.
AttesterSlashing(Box<AttesterSlashing<T>>),
}
impl<T: EthSpec> PubsubMessage<T> {
/// Returns the topics that each pubsub message will be sent across, given a supported
/// gossipsub encoding and fork version.
pub fn topics(&self, encoding: GossipEncoding, fork_version: [u8; 4]) -> Vec<GossipTopic> {
vec![GossipTopic::new(self.kind(), encoding, fork_version)]
}
/// Returns the kind of gossipsub topic associated with the message.
pub fn kind(&self) -> GossipKind {
match self {
PubsubMessage::BeaconBlock(_) => GossipKind::BeaconBlock,
PubsubMessage::AggregateAndProofAttestation(_) => GossipKind::BeaconAggregateAndProof,
PubsubMessage::Attestation(attestation_data) => {
GossipKind::CommitteeIndex(attestation_data.0)
}
PubsubMessage::VoluntaryExit(_) => GossipKind::VoluntaryExit,
PubsubMessage::ProposerSlashing(_) => GossipKind::ProposerSlashing,
PubsubMessage::AttesterSlashing(_) => GossipKind::AttesterSlashing,
}
}
/// This decodes `data` into a `PubsubMessage` given a list of topics.
///
/// The topics are checked
/// in order and as soon as one topic matches the decoded data, we return the data.
/* Note: This is assuming we are not hashing topics. If we choose to hash topics, these will
* need to be modified.
*
* Also note that a message can be associated with many topics. As soon as one of the topics is
* known we match. If none of the topics are known we return an unknown state.
*/
pub fn decode(topics: &[TopicHash], data: &[u8]) -> Result<Self, String> {
let mut unknown_topics = Vec::new();
for topic in topics {
match GossipTopic::decode(topic.as_str()) {
Err(_) => {
unknown_topics.push(topic);
continue;
}
Ok(gossip_topic) => {
let mut decompressed_data: Vec<u8> = Vec::new();
let data = match gossip_topic.encoding() {
// group each part by encoding type
GossipEncoding::SSZSnappy => {
match decompress_len(data) {
Ok(n) if n > GOSSIP_MAX_SIZE => {
return Err("ssz_snappy decoded data > GOSSIP_MAX_SIZE".into());
}
Ok(n) => decompressed_data.resize(n, 0),
Err(e) => {
return Err(format!("{}", e));
}
};
let mut decoder = Decoder::new();
match decoder.decompress(data, &mut decompressed_data) {
Ok(n) => {
decompressed_data.truncate(n);
&decompressed_data
}
Err(e) => return Err(format!("{}", e)),
}
}
GossipEncoding::SSZ => data,
};
// the ssz decoders
match gossip_topic.kind() {
GossipKind::BeaconAggregateAndProof => {
let agg_and_proof = SignedAggregateAndProof::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::AggregateAndProofAttestation(Box::new(
agg_and_proof,
)));
}
GossipKind::CommitteeIndex(subnet_id) => {
let attestation = Attestation::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::Attestation(Box::new((
*subnet_id,
attestation,
))));
}
GossipKind::BeaconBlock => {
let beacon_block = SignedBeaconBlock::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::BeaconBlock(Box::new(beacon_block)));
}
GossipKind::VoluntaryExit => {
let voluntary_exit = VoluntaryExit::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::VoluntaryExit(Box::new(voluntary_exit)));
}
GossipKind::ProposerSlashing => {
let proposer_slashing = ProposerSlashing::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::ProposerSlashing(Box::new(
proposer_slashing,
)));
}
GossipKind::AttesterSlashing => {
let attester_slashing = AttesterSlashing::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::AttesterSlashing(Box::new(
attester_slashing,
)));
}
}
}
}
}
Err(format!("Unknown gossipsub topics: {:?}", unknown_topics))
}
/// Encodes a `PubsubMessage` based on the topic encodings. The first known encoding is used. If
/// no encoding is known, and error is returned.
pub fn encode(&self, encoding: GossipEncoding) -> Result<Vec<u8>, String> {
let data = match &self {
PubsubMessage::BeaconBlock(data) => data.as_ssz_bytes(),
PubsubMessage::AggregateAndProofAttestation(data) => data.as_ssz_bytes(),
PubsubMessage::VoluntaryExit(data) => data.as_ssz_bytes(),
PubsubMessage::ProposerSlashing(data) => data.as_ssz_bytes(),
PubsubMessage::AttesterSlashing(data) => data.as_ssz_bytes(),
PubsubMessage::Attestation(data) => data.1.as_ssz_bytes(),
};
match encoding {
GossipEncoding::SSZ => {
if data.len() > GOSSIP_MAX_SIZE {
return Err("ssz encoded data > GOSSIP_MAX_SIZE".into());
} else {
Ok(data)
}
}
GossipEncoding::SSZSnappy => {
let mut encoder = Encoder::new();
match encoder.compress_vec(&data) {
Ok(compressed) if compressed.len() > GOSSIP_MAX_SIZE => {
Err("ssz_snappy Encoded data > GOSSIP_MAX_SIZE".into())
}
Ok(compressed) => Ok(compressed),
Err(e) => Err(format!("{}", e)),
}
}
}
}
}
impl<T: EthSpec> std::fmt::Display for PubsubMessage<T> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
PubsubMessage::BeaconBlock(block) => write!(
f,
"Beacon Block: slot: {}, proposer_index: {}",
block.message.slot, block.message.proposer_index
),
PubsubMessage::AggregateAndProofAttestation(att) => write!(
f,
"Aggregate and Proof: slot: {}, index: {}, aggregator_index: {}",
att.message.aggregate.data.slot,
att.message.aggregate.data.index,
att.message.aggregator_index,
),
PubsubMessage::Attestation(data) => write!(
f,
"Attestation: subnet_id: {}, attestation_slot: {}, attestation_index: {}",
*data.0, data.1.data.slot, data.1.data.index,
),
PubsubMessage::VoluntaryExit(_data) => write!(f, "Voluntary Exit"),
PubsubMessage::ProposerSlashing(_data) => write!(f, "Proposer Slashing"),
PubsubMessage::AttesterSlashing(_data) => write!(f, "Attester Slashing"),
}
}
}

View File

@ -0,0 +1,66 @@
use serde::{Deserialize, Serialize};
use types::{Hash256, Slot};
/// The current state of the node.
#[derive(Clone, Debug, Serialize, Deserialize)]
pub enum SyncState {
/// The node is performing a long-range (batch) sync over a finalized chain.
/// In this state, parent lookups are disabled.
SyncingFinalized {
start_slot: Slot,
head_slot: Slot,
head_root: Hash256,
},
/// The node is performing a long-range (batch) sync over one or many head chains.
/// In this state parent lookups are disabled.
SyncingHead { start_slot: Slot, head_slot: Slot },
/// The node is up to date with all known peers and is connected to at least one
/// fully synced peer. In this state, parent lookups are enabled.
Synced,
/// No useful peers are connected. Long-range sync's cannot proceed and we have no useful
/// peers to download parents for. More peers need to be connected before we can proceed.
Stalled,
}
impl PartialEq for SyncState {
fn eq(&self, other: &Self) -> bool {
match (self, other) {
(SyncState::SyncingFinalized { .. }, SyncState::SyncingFinalized { .. }) => true,
(SyncState::SyncingHead { .. }, SyncState::SyncingHead { .. }) => true,
(SyncState::Synced, SyncState::Synced) => true,
(SyncState::Stalled, SyncState::Stalled) => true,
_ => false,
}
}
}
impl SyncState {
/// Returns a boolean indicating the node is currently performing a long-range sync.
pub fn is_syncing(&self) -> bool {
match self {
SyncState::SyncingFinalized { .. } => true,
SyncState::SyncingHead { .. } => true,
SyncState::Synced => false,
SyncState::Stalled => false,
}
}
/// Returns true if the node is synced.
pub fn is_synced(&self) -> bool {
match self {
SyncState::Synced => true,
_ => false,
}
}
}
impl std::fmt::Display for SyncState {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
SyncState::SyncingFinalized { .. } => write!(f, "Syncing Finalized Chain"),
SyncState::SyncingHead { .. } => write!(f, "Syncing Head Chain"),
SyncState::Synced { .. } => write!(f, "Synced"),
SyncState::Stalled { .. } => write!(f, "Stalled"),
}
}
}

View File

@ -0,0 +1,205 @@
use libp2p::gossipsub::Topic;
use serde_derive::{Deserialize, Serialize};
use types::SubnetId;
/// The gossipsub topic names.
// These constants form a topic name of the form /TOPIC_PREFIX/TOPIC/ENCODING_POSTFIX
// For example /eth2/beacon_block/ssz
pub const TOPIC_PREFIX: &str = "eth2";
pub const SSZ_ENCODING_POSTFIX: &str = "ssz";
pub const SSZ_SNAPPY_ENCODING_POSTFIX: &str = "ssz_snappy";
pub const BEACON_BLOCK_TOPIC: &str = "beacon_block";
pub const BEACON_AGGREGATE_AND_PROOF_TOPIC: &str = "beacon_aggregate_and_proof";
// for speed and easier string manipulation, committee topic index is split into a prefix and a
// postfix. The topic is committee_index{}_beacon_attestation where {} is an integer.
pub const COMMITEE_INDEX_TOPIC_PREFIX: &str = "committee_index";
pub const COMMITEE_INDEX_TOPIC_POSTFIX: &str = "_beacon_attestation";
pub const VOLUNTARY_EXIT_TOPIC: &str = "voluntary_exit";
pub const PROPOSER_SLASHING_TOPIC: &str = "proposer_slashing";
pub const ATTESTER_SLASHING_TOPIC: &str = "attester_slashing";
/// A gossipsub topic which encapsulates the type of messages that should be sent and received over
/// the pubsub protocol and the way the messages should be encoded.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash)]
pub struct GossipTopic {
/// The encoding of the topic.
encoding: GossipEncoding,
/// The fork digest of the topic,
fork_digest: [u8; 4],
/// The kind of topic.
kind: GossipKind,
}
/// Enum that brings these topics into the rust type system.
// NOTE: There is intentionally no unknown type here. We only allow known gossipsub topics.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash)]
pub enum GossipKind {
/// Topic for publishing beacon blocks.
BeaconBlock,
/// Topic for publishing aggregate attestations and proofs.
BeaconAggregateAndProof,
/// Topic for publishing raw attestations on a particular subnet.
CommitteeIndex(SubnetId),
/// Topic for publishing voluntary exits.
VoluntaryExit,
/// Topic for publishing block proposer slashings.
ProposerSlashing,
/// Topic for publishing attester slashings.
AttesterSlashing,
}
impl std::fmt::Display for GossipKind {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
GossipKind::BeaconBlock => write!(f, "beacon_block"),
GossipKind::BeaconAggregateAndProof => write!(f, "beacon_aggregate_and_proof"),
GossipKind::CommitteeIndex(subnet_id) => write!(f, "committee_index_{}", **subnet_id),
GossipKind::VoluntaryExit => write!(f, "voluntary_exit"),
GossipKind::ProposerSlashing => write!(f, "proposer_slashing"),
GossipKind::AttesterSlashing => write!(f, "attester_slashing"),
}
}
}
/// The known encoding types for gossipsub messages.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash)]
pub enum GossipEncoding {
/// Messages are encoded with SSZ.
SSZ,
/// Messages are encoded with SSZSnappy.
SSZSnappy,
}
impl Default for GossipEncoding {
fn default() -> Self {
GossipEncoding::SSZSnappy
}
}
impl GossipTopic {
pub fn new(kind: GossipKind, encoding: GossipEncoding, fork_digest: [u8; 4]) -> Self {
GossipTopic {
encoding,
kind,
fork_digest,
}
}
/// Returns the encoding type for the gossipsub topic.
pub fn encoding(&self) -> &GossipEncoding {
&self.encoding
}
/// Returns a mutable reference to the fork digest of the gossipsub topic.
pub fn digest(&mut self) -> &mut [u8; 4] {
&mut self.fork_digest
}
/// Returns the kind of message expected on the gossipsub topic.
pub fn kind(&self) -> &GossipKind {
&self.kind
}
pub fn decode(topic: &str) -> Result<Self, String> {
let topic_parts: Vec<&str> = topic.split('/').collect();
if topic_parts.len() == 5 && topic_parts[1] == TOPIC_PREFIX {
let digest_bytes = hex::decode(topic_parts[2])
.map_err(|e| format!("Could not decode fork_digest hex: {}", e))?;
if digest_bytes.len() != 4 {
return Err(format!(
"Invalid gossipsub fork digest size: {}",
digest_bytes.len()
));
}
let mut fork_digest = [0; 4];
fork_digest.copy_from_slice(&digest_bytes);
let encoding = match topic_parts[4] {
SSZ_ENCODING_POSTFIX => GossipEncoding::SSZ,
SSZ_SNAPPY_ENCODING_POSTFIX => GossipEncoding::SSZSnappy,
_ => return Err(format!("Unknown encoding: {}", topic)),
};
let kind = match topic_parts[3] {
BEACON_BLOCK_TOPIC => GossipKind::BeaconBlock,
BEACON_AGGREGATE_AND_PROOF_TOPIC => GossipKind::BeaconAggregateAndProof,
VOLUNTARY_EXIT_TOPIC => GossipKind::VoluntaryExit,
PROPOSER_SLASHING_TOPIC => GossipKind::ProposerSlashing,
ATTESTER_SLASHING_TOPIC => GossipKind::AttesterSlashing,
topic => match committee_topic_index(topic) {
Some(subnet_id) => GossipKind::CommitteeIndex(subnet_id),
None => return Err(format!("Unknown topic: {}", topic)),
},
};
return Ok(GossipTopic {
encoding,
kind,
fork_digest,
});
}
Err(format!("Unknown topic: {}", topic))
}
}
impl Into<Topic> for GossipTopic {
fn into(self) -> Topic {
Topic::new(self.into())
}
}
impl Into<String> for GossipTopic {
fn into(self) -> String {
let encoding = match self.encoding {
GossipEncoding::SSZ => SSZ_ENCODING_POSTFIX,
GossipEncoding::SSZSnappy => SSZ_SNAPPY_ENCODING_POSTFIX,
};
let kind = match self.kind {
GossipKind::BeaconBlock => BEACON_BLOCK_TOPIC.into(),
GossipKind::BeaconAggregateAndProof => BEACON_AGGREGATE_AND_PROOF_TOPIC.into(),
GossipKind::VoluntaryExit => VOLUNTARY_EXIT_TOPIC.into(),
GossipKind::ProposerSlashing => PROPOSER_SLASHING_TOPIC.into(),
GossipKind::AttesterSlashing => ATTESTER_SLASHING_TOPIC.into(),
GossipKind::CommitteeIndex(index) => format!(
"{}{}{}",
COMMITEE_INDEX_TOPIC_PREFIX, *index, COMMITEE_INDEX_TOPIC_POSTFIX
),
};
format!(
"/{}/{}/{}/{}",
TOPIC_PREFIX,
hex::encode(self.fork_digest),
kind,
encoding
)
}
}
impl From<SubnetId> for GossipKind {
fn from(subnet_id: SubnetId) -> Self {
GossipKind::CommitteeIndex(subnet_id)
}
}
// helper functions
// Determines if a string is a committee topic.
fn committee_topic_index(topic: &str) -> Option<SubnetId> {
if topic.starts_with(COMMITEE_INDEX_TOPIC_PREFIX)
&& topic.ends_with(COMMITEE_INDEX_TOPIC_POSTFIX)
{
return Some(SubnetId::new(
u64::from_str_radix(
topic
.trim_start_matches(COMMITEE_INDEX_TOPIC_PREFIX)
.trim_end_matches(COMMITEE_INDEX_TOPIC_POSTFIX),
10,
)
.ok()?,
));
}
None
}

View File

@ -1,11 +1,14 @@
#![cfg(test)] #![cfg(test)]
use enr::Enr; use eth2_libp2p::Enr;
use eth2_libp2p::Multiaddr; use eth2_libp2p::Multiaddr;
use eth2_libp2p::NetworkConfig; use eth2_libp2p::NetworkConfig;
use eth2_libp2p::Service as LibP2PService; use eth2_libp2p::Service as LibP2PService;
use slog::{debug, error, o, Drain}; use slog::{debug, error, o, Drain};
use std::net::{TcpListener, UdpSocket}; use std::net::{TcpListener, UdpSocket};
use std::time::Duration; use std::time::Duration;
use types::{EnrForkId, MinimalEthSpec};
type E = MinimalEthSpec;
use tempdir::TempDir; use tempdir::TempDir;
pub fn build_log(level: slog::Level, enabled: bool) -> slog::Logger { pub fn build_log(level: slog::Level, enabled: bool) -> slog::Logger {
@ -62,6 +65,9 @@ pub fn build_config(
config.libp2p_port = port; // tcp port config.libp2p_port = port; // tcp port
config.discovery_port = port; // udp port config.discovery_port = port; // udp port
config.enr_tcp_port = Some(port);
config.enr_udp_port = Some(port);
config.enr_address = Some("127.0.0.1".parse().unwrap());
config.boot_nodes.append(&mut boot_nodes); config.boot_nodes.append(&mut boot_nodes);
config.secret_key_hex = secret_key; config.secret_key_hex = secret_key;
config.network_dir = path.into_path(); config.network_dir = path.into_path();
@ -72,29 +78,30 @@ pub fn build_config(
} }
pub fn build_libp2p_instance( pub fn build_libp2p_instance(
port: u16,
boot_nodes: Vec<Enr>, boot_nodes: Vec<Enr>,
secret_key: Option<String>, secret_key: Option<String>,
log: slog::Logger, log: slog::Logger,
) -> LibP2PService { ) -> LibP2PService<E> {
let port = unused_port("tcp").unwrap();
let config = build_config(port, boot_nodes, secret_key); let config = build_config(port, boot_nodes, secret_key);
// launch libp2p service // launch libp2p service
LibP2PService::new(&config, log.clone()) LibP2PService::new(&config, EnrForkId::default(), log.clone())
.expect("should build libp2p instance") .expect("should build libp2p instance")
.1 .1
} }
#[allow(dead_code)] #[allow(dead_code)]
pub fn get_enr(node: &LibP2PService) -> Enr { pub fn get_enr(node: &LibP2PService<E>) -> Enr {
node.swarm.discovery().local_enr().clone() let enr = node.swarm.discovery().local_enr().clone();
dbg!(enr.multiaddr());
enr
} }
// Returns `n` libp2p peers in fully connected topology. // Returns `n` libp2p peers in fully connected topology.
#[allow(dead_code)] #[allow(dead_code)]
pub fn build_full_mesh(log: slog::Logger, n: usize, start_port: Option<u16>) -> Vec<LibP2PService> { pub fn build_full_mesh(log: slog::Logger, n: usize) -> Vec<LibP2PService<E>> {
let base_port = start_port.unwrap_or(9000); let mut nodes: Vec<LibP2PService<E>> = (0..n)
let mut nodes: Vec<LibP2PService> = (base_port..base_port + n as u16) .map(|_| build_libp2p_instance(vec![], None, log.clone()))
.map(|p| build_libp2p_instance(p, vec![], None, log.clone()))
.collect(); .collect();
let multiaddrs: Vec<Multiaddr> = nodes let multiaddrs: Vec<Multiaddr> = nodes
.iter() .iter()
@ -117,12 +124,12 @@ pub fn build_full_mesh(log: slog::Logger, n: usize, start_port: Option<u16>) ->
// Constructs a pair of nodes with seperate loggers. The sender dials the receiver. // Constructs a pair of nodes with seperate loggers. The sender dials the receiver.
// This returns a (sender, receiver) pair. // This returns a (sender, receiver) pair.
#[allow(dead_code)] #[allow(dead_code)]
pub fn build_node_pair(log: &slog::Logger, start_port: u16) -> (LibP2PService, LibP2PService) { pub fn build_node_pair(log: &slog::Logger) -> (LibP2PService<E>, LibP2PService<E>) {
let sender_log = log.new(o!("who" => "sender")); let sender_log = log.new(o!("who" => "sender"));
let receiver_log = log.new(o!("who" => "receiver")); let receiver_log = log.new(o!("who" => "receiver"));
let mut sender = build_libp2p_instance(start_port, vec![], None, sender_log); let mut sender = build_libp2p_instance(vec![], None, sender_log);
let receiver = build_libp2p_instance(start_port + 1, vec![], None, receiver_log); let receiver = build_libp2p_instance(vec![], None, receiver_log);
let receiver_multiaddr = receiver.swarm.discovery().local_enr().clone().multiaddr()[1].clone(); let receiver_multiaddr = receiver.swarm.discovery().local_enr().clone().multiaddr()[1].clone();
match libp2p::Swarm::dial_addr(&mut sender.swarm, receiver_multiaddr) { match libp2p::Swarm::dial_addr(&mut sender.swarm, receiver_multiaddr) {
@ -134,10 +141,9 @@ pub fn build_node_pair(log: &slog::Logger, start_port: u16) -> (LibP2PService, L
// Returns `n` peers in a linear topology // Returns `n` peers in a linear topology
#[allow(dead_code)] #[allow(dead_code)]
pub fn build_linear(log: slog::Logger, n: usize, start_port: Option<u16>) -> Vec<LibP2PService> { pub fn build_linear(log: slog::Logger, n: usize) -> Vec<LibP2PService<E>> {
let base_port = start_port.unwrap_or(9000); let mut nodes: Vec<LibP2PService<E>> = (0..n)
let mut nodes: Vec<LibP2PService> = (base_port..base_port + n as u16) .map(|_| build_libp2p_instance(vec![], None, log.clone()))
.map(|p| build_libp2p_instance(p, vec![], None, log.clone()))
.collect(); .collect();
let multiaddrs: Vec<Multiaddr> = nodes let multiaddrs: Vec<Multiaddr> = nodes
.iter() .iter()

View File

@ -1,8 +1,12 @@
#![cfg(test)] #![cfg(test)]
use crate::types::GossipEncoding;
use ::types::{BeaconBlock, EthSpec, MinimalEthSpec, Signature, SignedBeaconBlock};
use eth2_libp2p::*; use eth2_libp2p::*;
use futures::prelude::*; use futures::prelude::*;
use slog::{debug, Level}; use slog::{debug, Level};
type E = MinimalEthSpec;
mod common; mod common;
/* Gossipsub tests */ /* Gossipsub tests */
@ -21,17 +25,27 @@ fn test_gossipsub_forward() {
let log = common::build_log(Level::Info, false); let log = common::build_log(Level::Info, false);
let num_nodes = 20; let num_nodes = 20;
let base_port = common::unused_port("tcp").unwrap(); let mut nodes = common::build_linear(log.clone(), num_nodes);
let mut nodes = common::build_linear(log.clone(), num_nodes, Some(base_port));
let mut received_count = 0; let mut received_count = 0;
let pubsub_message = PubsubMessage::Block(vec![0; 4]); let spec = E::default_spec();
let publishing_topic: String = "/eth2/beacon_block/ssz".into(); let empty_block = BeaconBlock::empty(&spec);
let signed_block = SignedBeaconBlock {
message: empty_block,
signature: Signature::empty_signature(),
};
let pubsub_message = PubsubMessage::BeaconBlock(Box::new(signed_block));
let publishing_topic: String = pubsub_message
.topics(GossipEncoding::default(), [0, 0, 0, 0])
.first()
.unwrap()
.clone()
.into();
let mut subscribed_count = 0; let mut subscribed_count = 0;
tokio::run(futures::future::poll_fn(move || -> Result<_, ()> { tokio::run(futures::future::poll_fn(move || -> Result<_, ()> {
for node in nodes.iter_mut() { for node in nodes.iter_mut() {
loop { loop {
match node.poll().unwrap() { match node.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::PubsubMessage { Async::Ready(Some(BehaviourEvent::PubsubMessage {
topics, topics,
message, message,
source, source,
@ -54,18 +68,13 @@ fn test_gossipsub_forward() {
return Ok(Async::Ready(())); return Ok(Async::Ready(()));
} }
} }
Async::Ready(Some(Libp2pEvent::PeerSubscribed(_, topic))) => { Async::Ready(Some(BehaviourEvent::PeerSubscribed(_, topic))) => {
// Received topics is one of subscribed eth2 topics
assert!(topic.clone().into_string().starts_with("/eth2/"));
// Publish on beacon block topic // Publish on beacon block topic
if topic == TopicHash::from_raw("/eth2/beacon_block/ssz") { if topic == TopicHash::from_raw(publishing_topic.clone()) {
subscribed_count += 1; subscribed_count += 1;
// Every node except the corner nodes are connected to 2 nodes. // Every node except the corner nodes are connected to 2 nodes.
if subscribed_count == (num_nodes * 2) - 2 { if subscribed_count == (num_nodes * 2) - 2 {
node.swarm.publish( node.swarm.publish(vec![pubsub_message.clone()]);
&[Topic::new(topic.into_string())],
pubsub_message.clone(),
);
} }
} }
} }
@ -82,23 +91,33 @@ fn test_gossipsub_forward() {
#[test] #[test]
fn test_gossipsub_full_mesh_publish() { fn test_gossipsub_full_mesh_publish() {
// set up the logging. The level and enabled or not // set up the logging. The level and enabled or not
let log = common::build_log(Level::Info, false); let log = common::build_log(Level::Debug, false);
// Note: This test does not propagate gossipsub messages. // Note: This test does not propagate gossipsub messages.
// Having `num_nodes` > `mesh_n_high` may give inconsistent results // Having `num_nodes` > `mesh_n_high` may give inconsistent results
// as nodes may get pruned out of the mesh before the gossipsub message // as nodes may get pruned out of the mesh before the gossipsub message
// is published to them. // is published to them.
let num_nodes = 12; let num_nodes = 12;
let base_port = common::unused_port("tcp").unwrap(); let mut nodes = common::build_full_mesh(log, num_nodes);
let mut nodes = common::build_full_mesh(log, num_nodes, Some(base_port));
let mut publishing_node = nodes.pop().unwrap(); let mut publishing_node = nodes.pop().unwrap();
let pubsub_message = PubsubMessage::Block(vec![0; 4]); let spec = E::default_spec();
let publishing_topic: String = "/eth2/beacon_block/ssz".into(); let empty_block = BeaconBlock::empty(&spec);
let signed_block = SignedBeaconBlock {
message: empty_block,
signature: Signature::empty_signature(),
};
let pubsub_message = PubsubMessage::BeaconBlock(Box::new(signed_block));
let publishing_topic: String = pubsub_message
.topics(GossipEncoding::default(), [0, 0, 0, 0])
.first()
.unwrap()
.clone()
.into();
let mut subscribed_count = 0; let mut subscribed_count = 0;
let mut received_count = 0; let mut received_count = 0;
tokio::run(futures::future::poll_fn(move || -> Result<_, ()> { tokio::run(futures::future::poll_fn(move || -> Result<_, ()> {
for node in nodes.iter_mut() { for node in nodes.iter_mut() {
while let Async::Ready(Some(Libp2pEvent::PubsubMessage { while let Async::Ready(Some(BehaviourEvent::PubsubMessage {
topics, message, .. topics, message, ..
})) = node.poll().unwrap() })) = node.poll().unwrap()
{ {
@ -116,18 +135,14 @@ fn test_gossipsub_full_mesh_publish() {
} }
} }
} }
while let Async::Ready(Some(Libp2pEvent::PeerSubscribed(_, topic))) = while let Async::Ready(Some(BehaviourEvent::PeerSubscribed(_, topic))) =
publishing_node.poll().unwrap() publishing_node.poll().unwrap()
{ {
// Received topics is one of subscribed eth2 topics
assert!(topic.clone().into_string().starts_with("/eth2/"));
// Publish on beacon block topic // Publish on beacon block topic
if topic == TopicHash::from_raw("/eth2/beacon_block/ssz") { if topic == TopicHash::from_raw(publishing_topic.clone()) {
subscribed_count += 1; subscribed_count += 1;
if subscribed_count == num_nodes - 1 { if subscribed_count == num_nodes - 1 {
publishing_node publishing_node.swarm.publish(vec![pubsub_message.clone()]);
.swarm
.publish(&[Topic::new(topic.into_string())], pubsub_message.clone());
} }
} }
} }

View File

@ -1,6 +1,8 @@
#![cfg(test)] #![cfg(test)]
use crate::behaviour::{Behaviour, BehaviourEvent}; use crate::behaviour::{Behaviour, BehaviourEvent};
use crate::multiaddr::Protocol; use crate::multiaddr::Protocol;
use ::types::{EnrForkId, MinimalEthSpec};
use eth2_libp2p::discovery::build_enr;
use eth2_libp2p::*; use eth2_libp2p::*;
use futures::prelude::*; use futures::prelude::*;
use libp2p::core::identity::Keypair; use libp2p::core::identity::Keypair;
@ -10,16 +12,19 @@ use libp2p::{
secio, PeerId, Swarm, Transport, secio, PeerId, Swarm, Transport,
}; };
use slog::{crit, debug, info, Level}; use slog::{crit, debug, info, Level};
use std::convert::TryInto;
use std::io::{Error, ErrorKind}; use std::io::{Error, ErrorKind};
use std::sync::atomic::{AtomicBool, Ordering::Relaxed}; use std::sync::atomic::{AtomicBool, Ordering::Relaxed};
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use tokio::prelude::*; use tokio::prelude::*;
type TSpec = MinimalEthSpec;
mod common; mod common;
type Libp2pStream = Boxed<(PeerId, StreamMuxerBox), Error>; type Libp2pStream = Boxed<(PeerId, StreamMuxerBox), Error>;
type Libp2pBehaviour = Behaviour<Substream<StreamMuxerBox>>; type Libp2pBehaviour = Behaviour<Substream<StreamMuxerBox>, TSpec>;
/// Build and return a eth2_libp2p Swarm with only secio support. /// Build and return a eth2_libp2p Swarm with only secio support.
fn build_secio_swarm( fn build_secio_swarm(
@ -28,8 +33,14 @@ fn build_secio_swarm(
) -> error::Result<Swarm<Libp2pStream, Libp2pBehaviour>> { ) -> error::Result<Swarm<Libp2pStream, Libp2pBehaviour>> {
let local_keypair = Keypair::generate_secp256k1(); let local_keypair = Keypair::generate_secp256k1();
let local_peer_id = PeerId::from(local_keypair.public()); let local_peer_id = PeerId::from(local_keypair.public());
let enr_key: libp2p::discv5::enr::CombinedKey = local_keypair.clone().try_into().unwrap();
let network_globals = Arc::new(NetworkGlobals::new(local_peer_id.clone())); let enr = build_enr::<TSpec>(&enr_key, config, EnrForkId::default()).unwrap();
let network_globals = Arc::new(NetworkGlobals::new(
enr,
config.libp2p_port,
config.discovery_port,
&log,
));
let mut swarm = { let mut swarm = {
// Set up the transport - tcp/ws with secio and mplex/yamux // Set up the transport - tcp/ws with secio and mplex/yamux
@ -110,13 +121,13 @@ fn build_secio_transport(local_private_key: Keypair) -> Boxed<(PeerId, StreamMux
fn test_secio_noise_fallback() { fn test_secio_noise_fallback() {
// set up the logging. The level and enabled logging or not // set up the logging. The level and enabled logging or not
let log_level = Level::Trace; let log_level = Level::Trace;
let enable_logging = true; let enable_logging = false;
let log = common::build_log(log_level, enable_logging); let log = common::build_log(log_level, enable_logging);
let port = common::unused_port("tcp").unwrap(); let port = common::unused_port("tcp").unwrap();
let noisy_config = common::build_config(port, vec![], None); let noisy_config = common::build_config(port, vec![], None);
let mut noisy_node = Service::new(&noisy_config, log.clone()) let mut noisy_node = Service::new(&noisy_config, EnrForkId::default(), log.clone())
.expect("should build a libp2p instance") .expect("should build a libp2p instance")
.1; .1;

View File

@ -1,16 +1,20 @@
#![cfg(test)] #![cfg(test)]
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::*; use eth2_libp2p::rpc::*;
use eth2_libp2p::{Libp2pEvent, RPCEvent}; use eth2_libp2p::{BehaviourEvent, RPCEvent};
use slog::{warn, Level}; use slog::{warn, Level};
use std::sync::atomic::{AtomicBool, Ordering::Relaxed}; use std::sync::atomic::{AtomicBool, Ordering::Relaxed};
use std::sync::{Arc, Mutex}; use std::sync::{Arc, Mutex};
use std::time::Duration; use std::time::Duration;
use tokio::prelude::*; use tokio::prelude::*;
use types::{Epoch, Hash256, Slot}; use types::{
BeaconBlock, Epoch, EthSpec, Hash256, MinimalEthSpec, Signature, SignedBeaconBlock, Slot,
};
mod common; mod common;
type E = MinimalEthSpec;
#[test] #[test]
// Tests the STATUS RPC message // Tests the STATUS RPC message
fn test_status_rpc() { fn test_status_rpc() {
@ -21,12 +25,11 @@ fn test_status_rpc() {
let log = common::build_log(log_level, enable_logging); let log = common::build_log(log_level, enable_logging);
// get sender/receiver // get sender/receiver
let port = common::unused_port("tcp").unwrap(); let (mut sender, mut receiver) = common::build_node_pair(&log);
let (mut sender, mut receiver) = common::build_node_pair(&log, port);
// Dummy STATUS RPC message // Dummy STATUS RPC message
let rpc_request = RPCRequest::Status(StatusMessage { let rpc_request = RPCRequest::Status(StatusMessage {
fork_version: [0; 4], fork_digest: [0; 4],
finalized_root: Hash256::from_low_u64_be(0), finalized_root: Hash256::from_low_u64_be(0),
finalized_epoch: Epoch::new(1), finalized_epoch: Epoch::new(1),
head_root: Hash256::from_low_u64_be(0), head_root: Hash256::from_low_u64_be(0),
@ -35,7 +38,7 @@ fn test_status_rpc() {
// Dummy STATUS RPC message // Dummy STATUS RPC message
let rpc_response = RPCResponse::Status(StatusMessage { let rpc_response = RPCResponse::Status(StatusMessage {
fork_version: [0; 4], fork_digest: [0; 4],
finalized_root: Hash256::from_low_u64_be(0), finalized_root: Hash256::from_low_u64_be(0),
finalized_epoch: Epoch::new(1), finalized_epoch: Epoch::new(1),
head_root: Hash256::from_low_u64_be(0), head_root: Hash256::from_low_u64_be(0),
@ -50,31 +53,31 @@ fn test_status_rpc() {
let sender_future = future::poll_fn(move || -> Poll<bool, ()> { let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match sender.poll().unwrap() { match sender.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::PeerDialed(peer_id))) => { Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
// Send a STATUS message // Send a STATUS message
warn!(sender_log, "Sending RPC"); warn!(sender_log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone())); .send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
} }
Async::Ready(Some(Libp2pEvent::RPC(_, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
// Should receive the RPC response // Should receive the RPC response
RPCEvent::Response(id, response @ RPCErrorResponse::Success(_)) => { RPCEvent::Response(id, response @ RPCErrorResponse::Success(_)) => {
warn!(sender_log, "Sender Received"); if id == 1 {
assert_eq!(id, 1); warn!(sender_log, "Sender Received");
let response = {
match response {
RPCErrorResponse::Success(r) => r,
_ => unreachable!(),
}
};
assert_eq!(response, sender_response.clone());
let response = { warn!(sender_log, "Sender Completed");
match response { return Ok(Async::Ready(true));
RPCErrorResponse::Success(r) => r, }
_ => unreachable!(),
}
};
assert_eq!(response, sender_response.clone());
warn!(sender_log, "Sender Completed");
return Ok(Async::Ready(true));
} }
_ => panic!("Received invalid RPC message"), e => panic!("Received invalid RPC message {}", e),
}, },
Async::Ready(Some(_)) => (), Async::Ready(Some(_)) => (),
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady), Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
@ -86,20 +89,22 @@ fn test_status_rpc() {
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> { let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match receiver.poll().unwrap() { match receiver.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::RPC(peer_id, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
// Should receive sent RPC request // Should receive sent RPC request
RPCEvent::Request(id, request) => { RPCEvent::Request(id, request) => {
assert_eq!(id, 1); if request == rpc_request {
assert_eq!(rpc_request.clone(), request); // send the response
warn!(log, "Receiver Received");
// send the response receiver.swarm.send_rpc(
warn!(log, "Receiver Received"); peer_id,
receiver.swarm.send_rpc( RPCEvent::Response(
peer_id, id,
RPCEvent::Response(id, RPCErrorResponse::Success(rpc_response.clone())), RPCErrorResponse::Success(rpc_response.clone()),
); ),
);
}
} }
_ => panic!("Received invalid RPC message"), e => panic!("Received invalid RPC message {}", e),
}, },
Async::Ready(Some(_)) => (), Async::Ready(Some(_)) => (),
Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady), Async::Ready(None) | Async::NotReady => return Ok(Async::NotReady),
@ -135,19 +140,23 @@ fn test_blocks_by_range_chunked_rpc() {
let log = common::build_log(log_level, enable_logging); let log = common::build_log(log_level, enable_logging);
// get sender/receiver // get sender/receiver
let port = common::unused_port("tcp").unwrap(); let (mut sender, mut receiver) = common::build_node_pair(&log);
let (mut sender, mut receiver) = common::build_node_pair(&log, port);
// BlocksByRange Request // BlocksByRange Request
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest { let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest {
head_block_root: Hash256::from_low_u64_be(0),
start_slot: 0, start_slot: 0,
count: messages_to_send, count: messages_to_send,
step: 0, step: 0,
}); });
// BlocksByRange Response // BlocksByRange Response
let rpc_response = RPCResponse::BlocksByRange(vec![13, 13, 13]); let spec = E::default_spec();
let empty_block = BeaconBlock::empty(&spec);
let empty_signed = SignedBeaconBlock {
message: empty_block,
signature: Signature::empty_signature(),
};
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed));
let sender_request = rpc_request.clone(); let sender_request = rpc_request.clone();
let sender_log = log.clone(); let sender_log = log.clone();
@ -159,33 +168,37 @@ fn test_blocks_by_range_chunked_rpc() {
let sender_future = future::poll_fn(move || -> Poll<bool, ()> { let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match sender.poll().unwrap() { match sender.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::PeerDialed(peer_id))) => { Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
// Send a BlocksByRange request // Send a BlocksByRange request
warn!(sender_log, "Sender sending RPC request"); warn!(sender_log, "Sender sending RPC request");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone())); .send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
} }
Async::Ready(Some(Libp2pEvent::RPC(_, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
// Should receive the RPC response // Should receive the RPC response
RPCEvent::Response(id, response) => { RPCEvent::Response(id, response) => {
warn!(sender_log, "Sender received a response"); if id == 1 {
assert_eq!(id, 1); warn!(sender_log, "Sender received a response");
match response { match response {
RPCErrorResponse::Success(res) => { RPCErrorResponse::Success(res) => {
assert_eq!(res, sender_response.clone()); assert_eq!(res, sender_response.clone());
*messages_received.lock().unwrap() += 1; *messages_received.lock().unwrap() += 1;
warn!(sender_log, "Chunk received"); warn!(sender_log, "Chunk received");
}
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
) => {
// should be exactly 10 messages before terminating
assert_eq!(
*messages_received.lock().unwrap(),
messages_to_send
);
// end the test
return Ok(Async::Ready(true));
}
_ => panic!("Invalid RPC received"),
} }
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
) => {
// should be exactly 10 messages before terminating
assert_eq!(*messages_received.lock().unwrap(), messages_to_send);
// end the test
return Ok(Async::Ready(true));
}
_ => panic!("Invalid RPC received"),
} }
} }
_ => panic!("Received invalid RPC message"), _ => panic!("Received invalid RPC message"),
@ -200,34 +213,33 @@ fn test_blocks_by_range_chunked_rpc() {
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> { let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match receiver.poll().unwrap() { match receiver.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::RPC(peer_id, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
// Should receive the sent RPC request // Should receive the sent RPC request
RPCEvent::Request(id, request) => { RPCEvent::Request(id, request) => {
assert_eq!(id, 1); if request == rpc_request {
assert_eq!(rpc_request.clone(), request); // send the response
warn!(log, "Receiver got request");
// send the response for _ in 1..=messages_to_send {
warn!(log, "Receiver got request"); receiver.swarm.send_rpc(
peer_id.clone(),
for _ in 1..=messages_to_send { RPCEvent::Response(
id,
RPCErrorResponse::Success(rpc_response.clone()),
),
);
}
// send the stream termination
receiver.swarm.send_rpc( receiver.swarm.send_rpc(
peer_id.clone(), peer_id,
RPCEvent::Response( RPCEvent::Response(
id, id,
RPCErrorResponse::Success(rpc_response.clone()), RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
),
), ),
); );
} }
// send the stream termination
receiver.swarm.send_rpc(
peer_id,
RPCEvent::Response(
id,
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
),
),
);
} }
_ => panic!("Received invalid RPC message"), _ => panic!("Received invalid RPC message"),
}, },
@ -263,19 +275,23 @@ fn test_blocks_by_range_single_empty_rpc() {
let log = common::build_log(log_level, enable_logging); let log = common::build_log(log_level, enable_logging);
// get sender/receiver // get sender/receiver
let port = common::unused_port("tcp").unwrap(); let (mut sender, mut receiver) = common::build_node_pair(&log);
let (mut sender, mut receiver) = common::build_node_pair(&log, port);
// BlocksByRange Request // BlocksByRange Request
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest { let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest {
head_block_root: Hash256::from_low_u64_be(0),
start_slot: 0, start_slot: 0,
count: 10, count: 10,
step: 0, step: 0,
}); });
// BlocksByRange Response // BlocksByRange Response
let rpc_response = RPCResponse::BlocksByRange(vec![]); let spec = E::default_spec();
let empty_block = BeaconBlock::empty(&spec);
let empty_signed = SignedBeaconBlock {
message: empty_block,
signature: Signature::empty_signature(),
};
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed));
let sender_request = rpc_request.clone(); let sender_request = rpc_request.clone();
let sender_log = log.clone(); let sender_log = log.clone();
@ -287,33 +303,34 @@ fn test_blocks_by_range_single_empty_rpc() {
let sender_future = future::poll_fn(move || -> Poll<bool, ()> { let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match sender.poll().unwrap() { match sender.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::PeerDialed(peer_id))) => { Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
// Send a BlocksByRange request // Send a BlocksByRange request
warn!(sender_log, "Sender sending RPC request"); warn!(sender_log, "Sender sending RPC request");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone())); .send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
} }
Async::Ready(Some(Libp2pEvent::RPC(_, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
// Should receive the RPC response // Should receive the RPC response
RPCEvent::Response(id, response) => { RPCEvent::Response(id, response) => {
warn!(sender_log, "Sender received a response"); if id == 1 {
assert_eq!(id, 1); warn!(sender_log, "Sender received a response");
match response { match response {
RPCErrorResponse::Success(res) => { RPCErrorResponse::Success(res) => {
assert_eq!(res, sender_response.clone()); assert_eq!(res, sender_response.clone());
*messages_received.lock().unwrap() += 1; *messages_received.lock().unwrap() += 1;
warn!(sender_log, "Chunk received"); warn!(sender_log, "Chunk received");
}
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
) => {
// should be exactly 1 messages before terminating
assert_eq!(*messages_received.lock().unwrap(), 1);
// end the test
return Ok(Async::Ready(true));
}
_ => panic!("Invalid RPC received"),
} }
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
) => {
// should be exactly 1 messages before terminating
assert_eq!(*messages_received.lock().unwrap(), 1);
// end the test
return Ok(Async::Ready(true));
}
_ => panic!("Invalid RPC received"),
} }
} }
m => panic!("Received invalid RPC message: {}", m), m => panic!("Received invalid RPC message: {}", m),
@ -328,29 +345,31 @@ fn test_blocks_by_range_single_empty_rpc() {
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> { let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match receiver.poll().unwrap() { match receiver.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::RPC(peer_id, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
// Should receive the sent RPC request // Should receive the sent RPC request
RPCEvent::Request(id, request) => { RPCEvent::Request(id, request) => {
assert_eq!(id, 1); if request == rpc_request {
assert_eq!(rpc_request.clone(), request); // send the response
warn!(log, "Receiver got request");
// send the response receiver.swarm.send_rpc(
warn!(log, "Receiver got request"); peer_id.clone(),
RPCEvent::Response(
receiver.swarm.send_rpc( id,
peer_id.clone(), RPCErrorResponse::Success(rpc_response.clone()),
RPCEvent::Response(id, RPCErrorResponse::Success(rpc_response.clone())),
);
// send the stream termination
receiver.swarm.send_rpc(
peer_id,
RPCEvent::Response(
id,
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
), ),
), );
); // send the stream termination
receiver.swarm.send_rpc(
peer_id,
RPCEvent::Response(
id,
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
),
),
);
}
} }
_ => panic!("Received invalid RPC message"), _ => panic!("Received invalid RPC message"),
}, },
@ -378,6 +397,9 @@ fn test_blocks_by_range_single_empty_rpc() {
#[test] #[test]
// Tests a streamed, chunked BlocksByRoot RPC Message // Tests a streamed, chunked BlocksByRoot RPC Message
// The size of the reponse is a full `BeaconBlock`
// which is greater than the Snappy frame size. Hence, this test
// serves to test the snappy framing format as well.
fn test_blocks_by_root_chunked_rpc() { fn test_blocks_by_root_chunked_rpc() {
// set up the logging. The level and enabled logging or not // set up the logging. The level and enabled logging or not
let log_level = Level::Trace; let log_level = Level::Trace;
@ -386,10 +408,10 @@ fn test_blocks_by_root_chunked_rpc() {
let messages_to_send = 3; let messages_to_send = 3;
let log = common::build_log(log_level, enable_logging); let log = common::build_log(log_level, enable_logging);
let spec = E::default_spec();
// get sender/receiver // get sender/receiver
let port = common::unused_port("tcp").unwrap(); let (mut sender, mut receiver) = common::build_node_pair(&log);
let (mut sender, mut receiver) = common::build_node_pair(&log, port);
// BlocksByRoot Request // BlocksByRoot Request
let rpc_request = RPCRequest::BlocksByRoot(BlocksByRootRequest { let rpc_request = RPCRequest::BlocksByRoot(BlocksByRootRequest {
@ -397,7 +419,12 @@ fn test_blocks_by_root_chunked_rpc() {
}); });
// BlocksByRoot Response // BlocksByRoot Response
let rpc_response = RPCResponse::BlocksByRoot(vec![13, 13, 13]); let full_block = BeaconBlock::full(&spec);
let signed_full_block = SignedBeaconBlock {
message: full_block,
signature: Signature::empty_signature(),
};
let rpc_response = RPCResponse::BlocksByRoot(Box::new(signed_full_block));
let sender_request = rpc_request.clone(); let sender_request = rpc_request.clone();
let sender_log = log.clone(); let sender_log = log.clone();
@ -409,14 +436,14 @@ fn test_blocks_by_root_chunked_rpc() {
let sender_future = future::poll_fn(move || -> Poll<bool, ()> { let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match sender.poll().unwrap() { match sender.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::PeerDialed(peer_id))) => { Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
// Send a BlocksByRoot request // Send a BlocksByRoot request
warn!(sender_log, "Sender sending RPC request"); warn!(sender_log, "Sender sending RPC request");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone())); .send_rpc(peer_id, RPCEvent::Request(1, sender_request.clone()));
} }
Async::Ready(Some(Libp2pEvent::RPC(_, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
// Should receive the RPC response // Should receive the RPC response
RPCEvent::Response(id, response) => { RPCEvent::Response(id, response) => {
warn!(sender_log, "Sender received a response"); warn!(sender_log, "Sender received a response");
@ -450,34 +477,33 @@ fn test_blocks_by_root_chunked_rpc() {
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> { let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match receiver.poll().unwrap() { match receiver.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::RPC(peer_id, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(peer_id, event))) => match event {
// Should receive the sent RPC request // Should receive the sent RPC request
RPCEvent::Request(id, request) => { RPCEvent::Request(id, request) => {
assert_eq!(id, 1); if request == rpc_request {
assert_eq!(rpc_request.clone(), request); // send the response
warn!(log, "Receiver got request");
// send the response for _ in 1..=messages_to_send {
warn!(log, "Receiver got request"); receiver.swarm.send_rpc(
peer_id.clone(),
for _ in 1..=messages_to_send { RPCEvent::Response(
id,
RPCErrorResponse::Success(rpc_response.clone()),
),
);
}
// send the stream termination
receiver.swarm.send_rpc( receiver.swarm.send_rpc(
peer_id.clone(), peer_id,
RPCEvent::Response( RPCEvent::Response(
id, id,
RPCErrorResponse::Success(rpc_response.clone()), RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRange,
),
), ),
); );
} }
// send the stream termination
receiver.swarm.send_rpc(
peer_id,
RPCEvent::Response(
id,
RPCErrorResponse::StreamTermination(
ResponseTermination::BlocksByRoot,
),
),
);
} }
_ => panic!("Received invalid RPC message"), _ => panic!("Received invalid RPC message"),
}, },
@ -513,8 +539,7 @@ fn test_goodbye_rpc() {
let log = common::build_log(log_level, enable_logging); let log = common::build_log(log_level, enable_logging);
// get sender/receiver // get sender/receiver
let port = common::unused_port("tcp").unwrap(); let (mut sender, mut receiver) = common::build_node_pair(&log);
let (mut sender, mut receiver) = common::build_node_pair(&log, port);
// Goodbye Request // Goodbye Request
let rpc_request = RPCRequest::Goodbye(GoodbyeReason::ClientShutdown); let rpc_request = RPCRequest::Goodbye(GoodbyeReason::ClientShutdown);
@ -526,7 +551,7 @@ fn test_goodbye_rpc() {
let sender_future = future::poll_fn(move || -> Poll<bool, ()> { let sender_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match sender.poll().unwrap() { match sender.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::PeerDialed(peer_id))) => { Async::Ready(Some(BehaviourEvent::PeerDialed(peer_id))) => {
// Send a Goodbye request // Send a Goodbye request
warn!(sender_log, "Sender sending RPC request"); warn!(sender_log, "Sender sending RPC request");
sender sender
@ -543,13 +568,15 @@ fn test_goodbye_rpc() {
let receiver_future = future::poll_fn(move || -> Poll<bool, ()> { let receiver_future = future::poll_fn(move || -> Poll<bool, ()> {
loop { loop {
match receiver.poll().unwrap() { match receiver.poll().unwrap() {
Async::Ready(Some(Libp2pEvent::RPC(_, event))) => match event { Async::Ready(Some(BehaviourEvent::RPC(_, event))) => match event {
// Should receive the sent RPC request // Should receive the sent RPC request
RPCEvent::Request(id, request) => { RPCEvent::Request(id, request) => {
assert_eq!(id, 0); if request == rpc_request {
assert_eq!(rpc_request.clone(), request); assert_eq!(id, 0);
// receives the goodbye. Nothing left to do assert_eq!(rpc_request.clone(), request);
return Ok(Async::Ready(true)); // receives the goodbye. Nothing left to do
return Ok(Async::Ready(true));
}
} }
_ => panic!("Received invalid RPC message"), _ => panic!("Received invalid RPC message"),
}, },

View File

@ -1,6 +1,6 @@
[package] [package]
name = "genesis" name = "genesis"
version = "0.1.0" version = "0.2.0"
authors = ["Paul Hauner <paul@paulhauner.com>"] authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018" edition = "2018"

View File

@ -1,6 +1,6 @@
[package] [package]
name = "network" name = "network"
version = "0.1.0" version = "0.2.0"
authors = ["Age Manning <Age@AgeManning.com>"] authors = ["Age Manning <Age@AgeManning.com>"]
edition = "2018" edition = "2018"
@ -13,7 +13,10 @@ tempdir = "0.3"
beacon_chain = { path = "../beacon_chain" } beacon_chain = { path = "../beacon_chain" }
store = { path = "../store" } store = { path = "../store" }
eth2-libp2p = { path = "../eth2-libp2p" } eth2-libp2p = { path = "../eth2-libp2p" }
hashmap_delay = { path = "../../eth2/utils/hashmap_delay" }
rest_types = { path = "../../eth2/utils/rest_types" }
types = { path = "../../eth2/types" } types = { path = "../../eth2/types" }
slot_clock = { path = "../../eth2/utils/slot_clock" }
slog = { version = "2.5.2", features = ["max_level_trace"] } slog = { version = "2.5.2", features = ["max_level_trace"] }
hex = "0.3" hex = "0.3"
eth2_ssz = "0.1.2" eth2_ssz = "0.1.2"

View File

@ -0,0 +1,638 @@
//! This service keeps track of which shard subnet the beacon node should be subscribed to at any
//! given time. It schedules subscriptions to shard subnets, requests peer discoveries and
//! determines whether attestations should be aggregated and/or passed to the beacon node.
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::{types::GossipKind, MessageId, NetworkGlobals, PeerId};
use futures::prelude::*;
use hashmap_delay::HashSetDelay;
use rand::seq::SliceRandom;
use rest_types::ValidatorSubscription;
use slog::{crit, debug, error, o, warn};
use slot_clock::SlotClock;
use std::collections::{HashMap, VecDeque};
use std::sync::Arc;
use std::time::{Duration, Instant};
use types::{Attestation, EthSpec, Slot, SubnetId};
/// The minimum number of slots ahead that we attempt to discover peers for a subscription. If the
/// slot is less than this number, skip the peer discovery process.
const MIN_PEER_DISCOVERY_SLOT_LOOK_AHEAD: u64 = 1;
/// The number of slots ahead that we attempt to discover peers for a subscription. If the slot to
/// attest to is greater than this, we queue a discovery request for this many slots prior to
/// subscribing.
const TARGET_PEER_DISCOVERY_SLOT_LOOK_AHEAD: u64 = 6;
/// The time (in slots) before a last seen validator is considered absent and we unsubscribe from the random
/// gossip topics that we subscribed to due to the validator connection.
const LAST_SEEN_VALIDATOR_TIMEOUT: u32 = 150; // 30 mins at a 12s slot time
/// The fraction of a slot that we subscribe to a subnet before the required slot.
///
/// Note: The time is calculated as `time = milliseconds_per_slot / ADVANCE_SUBSCRIPTION_TIME`.
const ADVANCE_SUBSCRIBE_TIME: u32 = 3;
/// The default number of slots before items in hash delay sets used by this class should expire.
const DEFAULT_EXPIRATION_TIMEOUT: u32 = 3; // 36s at 12s slot time
#[derive(Debug, PartialEq)]
pub enum AttServiceMessage {
/// Subscribe to the specified subnet id.
Subscribe(SubnetId),
/// Unsubscribe to the specified subnet id.
Unsubscribe(SubnetId),
/// Add the `SubnetId` to the ENR bitfield.
EnrAdd(SubnetId),
/// Remove the `SubnetId` from the ENR bitfield.
EnrRemove(SubnetId),
/// Discover peers for a particular subnet.
DiscoverPeers(SubnetId),
}
/// A particular subnet at a given slot.
#[derive(PartialEq, Eq, Hash, Clone)]
struct ExactSubnet {
/// The `SubnetId` associated with this subnet.
pub subnet_id: SubnetId,
/// The `Slot` associated with this subnet.
pub slot: Slot,
}
pub struct AttestationService<T: BeaconChainTypes> {
/// Queued events to return to the driving service.
events: VecDeque<AttServiceMessage>,
/// A collection of public network variables.
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
/// A reference to the beacon chain to process received attestations.
beacon_chain: Arc<BeaconChain<T>>,
/// The collection of currently subscribed random subnets mapped to their expiry deadline.
random_subnets: HashSetDelay<SubnetId>,
/// A collection of timeouts for when to start searching for peers for a particular shard.
discover_peers: HashSetDelay<ExactSubnet>,
/// A collection of timeouts for when to subscribe to a shard subnet.
subscriptions: HashSetDelay<ExactSubnet>,
/// A collection of timeouts for when to unsubscribe from a shard subnet.
unsubscriptions: HashSetDelay<ExactSubnet>,
/// A mapping indicating the number of known aggregate validators for a given `ExactSubnet`.
_aggregate_validators_on_subnet: HashMap<ExactSubnet, usize>,
/// A collection of seen validators. These dictate how many random subnets we should be
/// subscribed to. As these time out, we unsubscribe for the required random subnets and update
/// our ENR.
/// This is a set of validator indices.
known_validators: HashSetDelay<u64>,
/// The logger for the attestation service.
log: slog::Logger,
}
impl<T: BeaconChainTypes> AttestationService<T> {
/* Public functions */
pub fn new(
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
log: &slog::Logger,
) -> Self {
let log = log.new(o!("service" => "attestation_service"));
// calculate the random subnet duration from the spec constants
let spec = &beacon_chain.spec;
let slot_duration = beacon_chain.slot_clock.slot_duration();
let random_subnet_duration_millis = spec
.epochs_per_random_subnet_subscription
.saturating_mul(T::EthSpec::slots_per_epoch())
.saturating_mul(slot_duration.as_millis() as u64);
// Panics on overflow. Ensure LAST_SEEN_VALIDATOR_TIMEOUT is not too large.
let last_seen_val_timeout = slot_duration
.checked_mul(LAST_SEEN_VALIDATOR_TIMEOUT)
.expect("LAST_SEEN_VALIDATOR_TIMEOUT must not be ridiculously large");
let default_timeout = slot_duration
.checked_mul(DEFAULT_EXPIRATION_TIMEOUT)
.expect("DEFAULT_EXPIRATION_TIMEOUT must not be ridiculoustly large");
AttestationService {
events: VecDeque::with_capacity(10),
network_globals,
beacon_chain,
random_subnets: HashSetDelay::new(Duration::from_millis(random_subnet_duration_millis)),
discover_peers: HashSetDelay::new(default_timeout),
subscriptions: HashSetDelay::new(default_timeout),
unsubscriptions: HashSetDelay::new(default_timeout),
_aggregate_validators_on_subnet: HashMap::new(),
known_validators: HashSetDelay::new(last_seen_val_timeout),
log,
}
}
/// Processes a list of validator subscriptions.
///
/// This will:
/// - Register new validators as being known.
/// - Subscribe to the required number of random subnets.
/// - Update the local ENR for new random subnets due to seeing new validators.
/// - Search for peers for required subnets.
/// - Request subscriptions for subnets on specific slots when required.
/// - Build the timeouts for each of these events.
///
/// This returns a result simply for the ergonomics of using ?. The result can be
/// safely dropped.
pub fn validator_subscriptions(
&mut self,
subscriptions: Vec<ValidatorSubscription>,
) -> Result<(), ()> {
for subscription in subscriptions {
//NOTE: We assume all subscriptions have been verified before reaching this service
// Registers the validator with the attestation service.
// This will subscribe to long-lived random subnets if required.
self.add_known_validator(subscription.validator_index);
let subnet_id = SubnetId::new(
subscription.attestation_committee_index
% self.beacon_chain.spec.attestation_subnet_count,
);
let exact_subnet = ExactSubnet {
subnet_id,
slot: subscription.slot,
};
// determine if we should run a discovery lookup request and request it if required
if let Err(e) = self.discover_peers_request(exact_subnet.clone()) {
warn!(self.log, "Discovery lookup request error"; "error" => e);
}
// determine if the validator is an aggregator. If so, we subscribe to the subnet and
// if successful add the validator to a mapping of known aggregators for that exact
// subnet.
// NOTE: There is a chance that a fork occurs between now and when the validator needs
// to aggregate attestations. If this happens, the signature will no longer be valid
// and it could be likely the validator no longer needs to aggregate. More
// sophisticated logic should be added using known future forks.
// TODO: Implement
// set the subscription timer to subscribe to the next subnet if required
if let Err(e) = self.subscribe_to_subnet(exact_subnet) {
warn!(self.log, "Subscription to subnet error"; "error" => e);
return Err(());
}
}
Ok(())
}
/// Checks if we have subscribed aggregate validators for the subnet. If not, checks the gossip
/// verification, re-propagates and returns false.
pub fn should_process_attestation(
&mut self,
_message_id: &MessageId,
_peer_id: &PeerId,
_subnet: &SubnetId,
_attestation: &Attestation<T::EthSpec>,
) -> bool {
// TODO: Correctly handle validation aggregator checks
true
}
/* Internal private functions */
/// Checks if there are currently queued discovery requests and the time required to make the
/// request.
///
/// If there is sufficient time and no other request exists, queues a peer discovery request
/// for the required subnet.
fn discover_peers_request(&mut self, exact_subnet: ExactSubnet) -> Result<(), &'static str> {
let current_slot = self
.beacon_chain
.slot_clock
.now()
.ok_or_else(|| "Could not get the current slot")?;
let slot_duration = self.beacon_chain.slot_clock.slot_duration();
// if there is enough time to perform a discovery lookup
if exact_subnet.slot >= current_slot.saturating_add(MIN_PEER_DISCOVERY_SLOT_LOOK_AHEAD) {
// check if a discovery request already exists
if self.discover_peers.get(&exact_subnet).is_some() {
// already a request queued, end
return Ok(());
}
// check current event log to see if there is a discovery event queued
if self
.events
.iter()
.find(|event| event == &&AttServiceMessage::DiscoverPeers(exact_subnet.subnet_id))
.is_some()
{
// already queued a discovery event
return Ok(());
}
// if the slot is more than epoch away, add an event to start looking for peers
if exact_subnet.slot
< current_slot.saturating_add(TARGET_PEER_DISCOVERY_SLOT_LOOK_AHEAD)
{
// then instantly add a discovery request
self.events
.push_back(AttServiceMessage::DiscoverPeers(exact_subnet.subnet_id));
} else {
// Queue the discovery event to be executed for
// TARGET_PEER_DISCOVERY_SLOT_LOOK_AHEAD
let duration_to_discover = {
let duration_to_next_slot = self
.beacon_chain
.slot_clock
.duration_to_next_slot()
.ok_or_else(|| "Unable to determine duration to next slot")?;
// The -1 is done here to exclude the current slot duration, as we will use
// `duration_to_next_slot`.
let slots_until_discover = exact_subnet
.slot
.saturating_sub(current_slot)
.saturating_sub(1u64)
.saturating_sub(TARGET_PEER_DISCOVERY_SLOT_LOOK_AHEAD);
duration_to_next_slot + slot_duration * (slots_until_discover.as_u64() as u32)
};
self.discover_peers
.insert_at(exact_subnet, duration_to_discover);
}
} else {
// TODO: Send the time frame needed to have a peer connected, so that we can
// maintain peers for a least this duration.
// We may want to check the global PeerInfo to see estimated timeouts for each
// peer before they can be removed.
return Err("Not enough time for a discovery search");
}
Ok(())
}
/// Checks the current random subnets and subscriptions to determine if a new subscription for this
/// subnet is required for the given slot.
///
/// If required, adds a subscription event and an associated unsubscription event.
fn subscribe_to_subnet(&mut self, exact_subnet: ExactSubnet) -> Result<(), &'static str> {
// initialise timing variables
let current_slot = self
.beacon_chain
.slot_clock
.now()
.ok_or_else(|| "Could not get the current slot")?;
// Calculate the duration to the subscription event and the duration to the end event.
// There are two main cases. Attempting to subscribe to the current slot and all others.
let (duration_to_subscribe, expected_end_subscription_duration) = {
let duration_to_next_slot = self
.beacon_chain
.slot_clock
.duration_to_next_slot()
.ok_or_else(|| "Unable to determine duration to next slot")?;
if current_slot >= exact_subnet.slot {
(Duration::from_secs(0), duration_to_next_slot)
} else {
let slot_duration = self.beacon_chain.slot_clock.slot_duration();
let advance_subscription_duration = slot_duration
.checked_div(ADVANCE_SUBSCRIBE_TIME)
.expect("ADVANCE_SUBSCRIPTION_TIME cannot be too large");
// calculate the time to subscribe to the subnet
let duration_to_subscribe = self
.beacon_chain
.slot_clock
.duration_to_slot(exact_subnet.slot)
.ok_or_else(|| "Unable to determine duration to subscription slot")?
.checked_sub(advance_subscription_duration)
.unwrap_or_else(|| Duration::from_secs(0));
// the duration until we no longer need this subscription. We assume a single slot is
// sufficient.
let expected_end_subscription_duration = duration_to_subscribe
+ slot_duration
+ std::cmp::min(advance_subscription_duration, duration_to_next_slot);
(duration_to_subscribe, expected_end_subscription_duration)
}
};
// Checks on current subscriptions
// Note: We may be connected to a long-lived random subnet. In this case we still add the
// subscription timeout and check this case when the timeout fires. This is because a
// long-lived random subnet can be unsubscribed at any time when a validator becomes
// in-active. This case is checked on the subscription event (see `handle_subscriptions`).
// Return if we already have a subscription for this subnet_id and slot
if self.subscriptions.contains(&exact_subnet) {
return Ok(());
}
// We are not currently subscribed and have no waiting subscription, create one
self.subscriptions
.insert_at(exact_subnet.clone(), duration_to_subscribe);
// if there is an unsubscription event for the slot prior, we remove it to prevent
// unsubscriptions immediately after the subscription. We also want to minimize
// subscription churn and maintain a consecutive subnet subscriptions.
let to_remove_subnet = ExactSubnet {
subnet_id: exact_subnet.subnet_id,
slot: exact_subnet.slot.saturating_sub(1u64),
};
self.unsubscriptions.remove(&to_remove_subnet);
// add an unsubscription event to remove ourselves from the subnet once completed
self.unsubscriptions
.insert_at(exact_subnet, expected_end_subscription_duration);
Ok(())
}
/// Updates the `known_validators` mapping and subscribes to a set of random subnets if required.
///
/// This also updates the ENR to indicate our long-lived subscription to the subnet
fn add_known_validator(&mut self, validator_index: u64) {
if self.known_validators.get(&validator_index).is_none() {
// New validator has subscribed
// Subscribe to random topics and update the ENR if needed.
let spec = &self.beacon_chain.spec;
if self.random_subnets.len() < spec.attestation_subnet_count as usize {
// Still room for subscriptions
self.subscribe_to_random_subnets(
self.beacon_chain.spec.random_subnets_per_validator as usize,
);
}
}
// add the new validator or update the current timeout for a known validator
self.known_validators.insert(validator_index);
}
/// Subscribe to long-lived random subnets and update the local ENR bitfield.
fn subscribe_to_random_subnets(&mut self, no_subnets_to_subscribe: usize) {
let subnet_count = self.beacon_chain.spec.attestation_subnet_count;
// Build a list of random subnets that we are not currently subscribed to.
let available_subnets = (0..subnet_count)
.map(SubnetId::new)
.filter(|subnet_id| self.random_subnets.get(subnet_id).is_none())
.collect::<Vec<_>>();
let to_subscribe_subnets = {
if available_subnets.len() < no_subnets_to_subscribe {
debug!(self.log, "Reached maximum random subnet subscriptions");
available_subnets
} else {
// select a random sample of available subnets
available_subnets
.choose_multiple(&mut rand::thread_rng(), no_subnets_to_subscribe)
.cloned()
.collect::<Vec<_>>()
}
};
for subnet_id in to_subscribe_subnets {
// remove this subnet from any immediate subscription/un-subscription events
self.subscriptions
.retain(|exact_subnet| exact_subnet.subnet_id != subnet_id);
self.unsubscriptions
.retain(|exact_subnet| exact_subnet.subnet_id != subnet_id);
// insert a new random subnet
self.random_subnets.insert(subnet_id);
// if we are not already subscribed, then subscribe
let topic_kind = &GossipKind::CommitteeIndex(subnet_id);
if let None = self
.network_globals
.gossipsub_subscriptions
.read()
.iter()
.find(|topic| topic.kind() == topic_kind)
{
// not already subscribed to the topic
// send a discovery request and a subscription
self.events
.push_back(AttServiceMessage::DiscoverPeers(subnet_id));
self.events
.push_back(AttServiceMessage::Subscribe(subnet_id));
}
// add the subnet to the ENR bitfield
self.events.push_back(AttServiceMessage::EnrAdd(subnet_id));
}
}
/* A collection of functions that handle the various timeouts */
/// Request a discovery query to find peers for a particular subnet.
fn handle_discover_peers(&mut self, exact_subnet: ExactSubnet) {
debug!(self.log, "Searching for peers for subnet"; "subnet" => *exact_subnet.subnet_id, "target_slot" => exact_subnet.slot);
self.events
.push_back(AttServiceMessage::DiscoverPeers(exact_subnet.subnet_id));
}
/// A queued subscription is ready.
///
/// We add subscriptions events even if we are already subscribed to a random subnet (as these
/// can be unsubscribed at any time by inactive validators). If we are
/// still subscribed at the time the event fires, we don't re-subscribe.
fn handle_subscriptions(&mut self, exact_subnet: ExactSubnet) {
// Check if the subnet currently exists as a long-lasting random subnet
if let Some(expiry) = self.random_subnets.get(&exact_subnet.subnet_id) {
// we are subscribed via a random subnet, if this is to expire during the time we need
// to be subscribed, just extend the expiry
let slot_duration = self.beacon_chain.slot_clock.slot_duration();
let advance_subscription_duration = slot_duration
.checked_div(ADVANCE_SUBSCRIBE_TIME)
.expect("ADVANCE_SUBSCRIPTION_TIME cannot be too large");
// we require the subnet subscription for at least a slot on top of the initial
// subscription time
let expected_end_subscription_duration = slot_duration + advance_subscription_duration;
if expiry < &(Instant::now() + expected_end_subscription_duration) {
self.random_subnets
.update_timeout(&exact_subnet.subnet_id, expected_end_subscription_duration);
}
} else {
// we are also not un-subscribing from a subnet if the next slot requires us to be
// subscribed. Therefore there could be the case that we are already still subscribed
// to the required subnet. In which case we do not issue another subscription request.
let topic_kind = &GossipKind::CommitteeIndex(exact_subnet.subnet_id);
if self
.network_globals
.gossipsub_subscriptions
.read()
.iter()
.find(|topic| topic.kind() == topic_kind)
.is_none()
{
// we are not already subscribed
debug!(self.log, "Subscribing to subnet"; "subnet" => *exact_subnet.subnet_id, "target_slot" => exact_subnet.slot.as_u64());
self.events
.push_back(AttServiceMessage::Subscribe(exact_subnet.subnet_id));
}
}
}
/// A queued unsubscription is ready.
///
/// Unsubscription events are added, even if we are subscribed to long-lived random subnets. If
/// a random subnet is present, we do not unsubscribe from it.
fn handle_unsubscriptions(&mut self, exact_subnet: ExactSubnet) {
// Check if the subnet currently exists as a long-lasting random subnet
if self.random_subnets.contains(&exact_subnet.subnet_id) {
return;
}
debug!(self.log, "Unsubscribing from subnet"; "subnet" => *exact_subnet.subnet_id, "processed_slot" => exact_subnet.slot.as_u64());
// various logic checks
if self.subscriptions.contains(&exact_subnet) {
crit!(self.log, "Unsubscribing from a subnet in subscriptions");
}
self.events
.push_back(AttServiceMessage::Unsubscribe(exact_subnet.subnet_id));
}
/// A random subnet has expired.
///
/// This function selects a new subnet to join, or extends the expiry if there are no more
/// available subnets to choose from.
fn handle_random_subnet_expiry(&mut self, subnet_id: SubnetId) {
let subnet_count = self.beacon_chain.spec.attestation_subnet_count;
if self.random_subnets.len() == (subnet_count - 1) as usize {
// We are at capacity, simply increase the timeout of the current subnet
self.random_subnets.insert(subnet_id);
return;
}
// we are not at capacity, unsubscribe from the current subnet, remove the ENR bitfield bit and choose a new random one
// from the available subnets
// Note: This should not occur during a required subnet as subscriptions update the timeout
// to last as long as they are needed.
debug!(self.log, "Unsubscribing from random subnet"; "subnet_id" => *subnet_id);
self.events
.push_back(AttServiceMessage::Unsubscribe(subnet_id));
self.events
.push_back(AttServiceMessage::EnrRemove(subnet_id));
self.subscribe_to_random_subnets(1);
}
/// A known validator has not sent a subscription in a while. They are considered offline and the
/// beacon node no longer needs to be subscribed to the allocated random subnets.
///
/// We don't keep track of a specific validator to random subnet, rather the ratio of active
/// validators to random subnets. So when a validator goes offline, we can simply remove the
/// allocated amount of random subnets.
fn handle_known_validator_expiry(&mut self) -> Result<(), ()> {
let spec = &self.beacon_chain.spec;
let subnet_count = spec.attestation_subnet_count;
let random_subnets_per_validator = spec.random_subnets_per_validator;
if self.known_validators.len() as u64 * random_subnets_per_validator >= subnet_count {
// have too many validators, ignore
return Ok(());
}
let subscribed_subnets = self.random_subnets.keys_vec();
let to_remove_subnets = subscribed_subnets.choose_multiple(
&mut rand::thread_rng(),
random_subnets_per_validator as usize,
);
let current_slot = self.beacon_chain.slot_clock.now().ok_or_else(|| {
warn!(self.log, "Could not get the current slot");
})?;
for subnet_id in to_remove_subnets {
// If a subscription is queued for two slots in the future, it's associated unsubscription
// will unsubscribe from the expired subnet.
// If there is no subscription for this subnet,slot it is safe to add one, without
// unsubscribing early from a required subnet
let subnet = ExactSubnet {
subnet_id: **subnet_id,
slot: current_slot + 2,
};
if self.subscriptions.get(&subnet).is_none() {
// set an unsubscribe event
let duration_to_next_slot = self
.beacon_chain
.slot_clock
.duration_to_next_slot()
.ok_or_else(|| {
warn!(self.log, "Unable to determine duration to next slot");
})?;
let slot_duration = self.beacon_chain.slot_clock.slot_duration();
// Set the unsubscription timeout
let unsubscription_duration = duration_to_next_slot + slot_duration * 2;
self.unsubscriptions
.insert_at(subnet, unsubscription_duration);
}
// as the long lasting subnet subscription is being removed, remove the subnet_id from
// the ENR bitfield
self.events
.push_back(AttServiceMessage::EnrRemove(**subnet_id));
}
Ok(())
}
}
impl<T: BeaconChainTypes> Stream for AttestationService<T> {
type Item = AttServiceMessage;
type Error = ();
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
// process any peer discovery events
while let Async::Ready(Some(exact_subnet)) =
self.discover_peers.poll().map_err(|e| {
error!(self.log, "Failed to check for peer discovery requests"; "error"=> format!("{}", e));
})?
{
self.handle_discover_peers(exact_subnet);
}
// process any subscription events
while let Async::Ready(Some(exact_subnet)) = self.subscriptions.poll().map_err(|e| {
error!(self.log, "Failed to check for subnet subscription times"; "error"=> format!("{}", e));
})?
{
self.handle_subscriptions(exact_subnet);
}
// process any un-subscription events
while let Async::Ready(Some(exact_subnet)) = self.unsubscriptions.poll().map_err(|e| {
error!(self.log, "Failed to check for subnet unsubscription times"; "error"=> format!("{}", e));
})?
{
self.handle_unsubscriptions(exact_subnet);
}
// process any random subnet expiries
while let Async::Ready(Some(subnet)) = self.random_subnets.poll().map_err(|e| {
error!(self.log, "Failed to check for random subnet cycles"; "error"=> format!("{}", e));
})?
{
self.handle_random_subnet_expiry(subnet);
}
// process any known validator expiries
while let Async::Ready(Some(_validator_index)) = self.known_validators.poll().map_err(|e| {
error!(self.log, "Failed to check for random subnet cycles"; "error"=> format!("{}", e));
})?
{
let _ = self.handle_known_validator_expiry();
}
// process any generated events
if let Some(event) = self.events.pop_front() {
return Ok(Async::Ready(Some(event)));
}
Ok(Async::NotReady)
}
}

View File

@ -1,6 +1,4 @@
// generates error types // generates error types
use eth2_libp2p;
use error_chain::error_chain; use error_chain::error_chain;
error_chain! { error_chain! {

View File

@ -1,12 +1,11 @@
/// This crate provides the network server for Lighthouse. /// This crate provides the network server for Lighthouse.
pub mod error; pub mod error;
pub mod message_handler;
pub mod message_processor;
pub mod persisted_dht;
pub mod service; pub mod service;
pub mod sync;
mod attestation_service;
mod persisted_dht;
mod router;
mod sync;
pub use eth2_libp2p::NetworkConfig; pub use eth2_libp2p::NetworkConfig;
pub use message_processor::MessageProcessor; pub use service::{NetworkMessage, NetworkService};
pub use service::NetworkMessage;
pub use service::Service;

View File

@ -1,367 +0,0 @@
#![allow(clippy::unit_arg)]
use crate::error;
use crate::service::NetworkMessage;
use crate::MessageProcessor;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::{
behaviour::PubsubMessage,
rpc::{RPCError, RPCErrorResponse, RPCRequest, RPCResponse, RequestId, ResponseTermination},
MessageId, PeerId, RPCEvent,
};
use futures::future::Future;
use futures::stream::Stream;
use slog::{debug, o, trace, warn};
use ssz::{Decode, DecodeError};
use std::sync::Arc;
use tokio::sync::mpsc;
use types::{Attestation, AttesterSlashing, ProposerSlashing, SignedBeaconBlock, VoluntaryExit};
/// 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> {
/// 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,
}
/// Types of messages the handler can receive.
#[derive(Debug)]
pub enum HandlerMessage {
/// We have initiated a connection to a new peer.
PeerDialed(PeerId),
/// Peer has disconnected,
PeerDisconnected(PeerId),
/// An RPC response/request has been received.
RPC(PeerId, RPCEvent),
/// A gossip message has been received. The fields are: message id, the peer that sent us this
/// message and the message itself.
PubsubMessage(MessageId, PeerId, PubsubMessage),
}
impl<T: BeaconChainTypes> MessageHandler<T> {
/// Initializes and runs the MessageHandler.
pub fn spawn(
beacon_chain: Arc<BeaconChain<T>>,
network_send: mpsc::UnboundedSender<NetworkMessage>,
executor: &tokio::runtime::TaskExecutor,
log: slog::Logger,
) -> error::Result<mpsc::UnboundedSender<HandlerMessage>> {
let message_handler_log = log.new(o!("service"=> "msg_handler"));
trace!(message_handler_log, "Service starting");
let (handler_send, handler_recv) = mpsc::unbounded_channel();
// 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 {
network_send,
message_processor,
log: message_handler_log,
};
// spawn handler task and move the message handler instance into the spawned thread
executor.spawn(
handler_recv
.for_each(move |msg| Ok(handler.handle_message(msg)))
.map_err(move |_| {
debug!(log, "Network message handler terminated.");
}),
);
Ok(handler_send)
}
/// Handle all messages incoming from the network service.
fn handle_message(&mut self, message: HandlerMessage) {
match message {
// we have initiated a connection to a peer
HandlerMessage::PeerDialed(peer_id) => {
self.message_processor.on_connect(peer_id);
}
// A peer has disconnected
HandlerMessage::PeerDisconnected(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(id, peer_id, gossip) => {
self.handle_gossip(id, peer_id, gossip);
}
}
}
/* RPC - Related functionality */
/// Handle RPC messages
fn handle_rpc_message(&mut self, peer_id: PeerId, rpc_message: RPCEvent) {
match rpc_message {
RPCEvent::Request(id, req) => self.handle_rpc_request(peer_id, id, req),
RPCEvent::Response(id, resp) => self.handle_rpc_response(peer_id, id, resp),
RPCEvent::Error(id, error) => self.handle_rpc_error(peer_id, id, error),
}
}
/// A new RPC request has been received from the network.
fn handle_rpc_request(&mut self, peer_id: PeerId, request_id: RequestId, request: RPCRequest) {
match request {
RPCRequest::Status(status_message) => {
self.message_processor
.on_status_request(peer_id, request_id, status_message)
}
RPCRequest::Goodbye(goodbye_reason) => {
debug!(
self.log, "PeerGoodbye";
"peer" => format!("{:?}", peer_id),
"reason" => format!("{:?}", goodbye_reason),
);
self.message_processor.on_disconnect(peer_id);
}
RPCRequest::BlocksByRange(request) => self
.message_processor
.on_blocks_by_range_request(peer_id, request_id, request),
RPCRequest::BlocksByRoot(request) => self
.message_processor
.on_blocks_by_root_request(peer_id, request_id, request),
}
}
/// An RPC response has been received from the network.
// we match on id and ignore responses past the timeout.
fn handle_rpc_response(
&mut self,
peer_id: PeerId,
request_id: RequestId,
error_response: RPCErrorResponse,
) {
// an error could have occurred.
match error_response {
RPCErrorResponse::InvalidRequest(error) => {
warn!(self.log, "Peer indicated invalid request";"peer_id" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::ServerError(error) => {
warn!(self.log, "Peer internal server error";"peer_id" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::Unknown(error) => {
warn!(self.log, "Unknown peer error";"peer" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::Success(response) => {
match response {
RPCResponse::Status(status_message) => {
self.message_processor
.on_status_response(peer_id, status_message);
}
RPCResponse::BlocksByRange(response) => {
match self.decode_beacon_block(response) {
Ok(beacon_block) => {
self.message_processor.on_blocks_by_range_response(
peer_id,
request_id,
Some(beacon_block),
);
}
Err(e) => {
// TODO: Down-vote Peer
warn!(self.log, "Peer sent invalid BEACON_BLOCKS response";"peer" => format!("{:?}", peer_id), "error" => format!("{:?}", e));
}
}
}
RPCResponse::BlocksByRoot(response) => {
match self.decode_beacon_block(response) {
Ok(beacon_block) => {
self.message_processor.on_blocks_by_root_response(
peer_id,
request_id,
Some(beacon_block),
);
}
Err(e) => {
// TODO: Down-vote Peer
warn!(self.log, "Peer sent invalid BEACON_BLOCKS response";"peer" => format!("{:?}", peer_id), "error" => format!("{:?}", e));
}
}
}
}
}
RPCErrorResponse::StreamTermination(response_type) => {
// have received a stream termination, notify the processing functions
match response_type {
ResponseTermination::BlocksByRange => {
self.message_processor
.on_blocks_by_range_response(peer_id, request_id, None);
}
ResponseTermination::BlocksByRoot => {
self.message_processor
.on_blocks_by_root_response(peer_id, request_id, None);
}
}
}
}
}
/// Handle various RPC errors
fn handle_rpc_error(&mut self, peer_id: PeerId, request_id: RequestId, error: RPCError) {
warn!(self.log, "RPC Error"; "Peer" => format!("{:?}", peer_id), "request_id" => format!("{}", request_id), "Error" => format!("{:?}", error));
self.message_processor.on_rpc_error(peer_id, request_id);
}
/// Handle RPC messages
fn handle_gossip(&mut self, id: MessageId, 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
.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);
}
}
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) => {
// 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) );
}
Err(e) => {
debug!(self.log, "Invalid gossiped exit"; "peer_id" => format!("{}", peer_id), "Error" => format!("{:?}", e));
}
},
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) );
}
Err(e) => {
debug!(self.log, "Invalid gossiped proposer slashing"; "peer_id" => format!("{}", peer_id), "Error" => format!("{:?}", e));
}
}
}
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) );
}
Err(e) => {
debug!(self.log, "Invalid gossiped attester slashing"; "peer_id" => format!("{}", peer_id), "Error" => format!("{:?}", e));
}
}
}
PubsubMessage::Unknown(message) => {
// Received a message from an unknown topic. Ignore for now
debug!(self.log, "Unknown Gossip Message"; "peer_id" => format!("{}", peer_id), "Message" => format!("{:?}", message));
}
}
}
/// Informs the network service that the message should be forwarded to other peers.
fn propagate_message(&mut self, message_id: MessageId, 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
* therefore apply more efficient logic in decoding and verification.
*
* TODO: Apply efficient decoding/verification of these objects
*/
/* Gossipsub Domain Decoding */
// Note: These are not generics as type-specific verification will need to be applied.
fn decode_gossip_block(
&self,
beacon_block: Vec<u8>,
) -> Result<SignedBeaconBlock<T::EthSpec>, DecodeError> {
//TODO: Apply verification before decoding.
SignedBeaconBlock::from_ssz_bytes(&beacon_block)
}
fn decode_gossip_attestation(
&self,
beacon_block: Vec<u8>,
) -> Result<Attestation<T::EthSpec>, DecodeError> {
//TODO: Apply verification before decoding.
Attestation::from_ssz_bytes(&beacon_block)
}
fn decode_gossip_exit(&self, voluntary_exit: Vec<u8>) -> Result<VoluntaryExit, DecodeError> {
//TODO: Apply verification before decoding.
VoluntaryExit::from_ssz_bytes(&voluntary_exit)
}
fn decode_gossip_proposer_slashing(
&self,
proposer_slashing: Vec<u8>,
) -> Result<ProposerSlashing, DecodeError> {
//TODO: Apply verification before decoding.
ProposerSlashing::from_ssz_bytes(&proposer_slashing)
}
fn decode_gossip_attestation_slashing(
&self,
attester_slashing: Vec<u8>,
) -> Result<AttesterSlashing<T::EthSpec>, DecodeError> {
//TODO: Apply verification before decoding.
AttesterSlashing::from_ssz_bytes(&attester_slashing)
}
/* Req/Resp Domain Decoding */
/// Verifies and decodes an ssz-encoded `SignedBeaconBlock`. If `None` is passed, this represents a
/// stream termination.
fn decode_beacon_block(
&self,
beacon_block: Vec<u8>,
) -> Result<SignedBeaconBlock<T::EthSpec>, DecodeError> {
//TODO: Implement faster block verification before decoding entirely
SignedBeaconBlock::from_ssz_bytes(&beacon_block)
}
}

View File

@ -0,0 +1,314 @@
//! This module handles incoming network messages.
//!
//! It routes the messages to appropriate services, such as the Sync
//! and processes those that are
#![allow(clippy::unit_arg)]
pub mod processor;
use crate::error;
use crate::service::NetworkMessage;
use beacon_chain::{AttestationType, BeaconChain, BeaconChainTypes, BlockError};
use eth2_libp2p::{
rpc::{RPCError, RPCErrorResponse, RPCRequest, RPCResponse, RequestId, ResponseTermination},
MessageId, NetworkGlobals, PeerId, PubsubMessage, RPCEvent,
};
use futures::future::Future;
use futures::stream::Stream;
use processor::Processor;
use slog::{debug, o, trace, warn};
use std::sync::Arc;
use tokio::sync::mpsc;
use types::EthSpec;
/// 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 Router<T: BeaconChainTypes> {
/// A channel to the network service to allow for gossip propagation.
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
/// Processes validated and decoded messages from the network. Has direct access to the
/// sync manager.
processor: Processor<T>,
/// The `Router` logger.
log: slog::Logger,
}
/// Types of messages the handler can receive.
#[derive(Debug)]
pub enum RouterMessage<T: EthSpec> {
/// We have initiated a connection to a new peer.
PeerDialed(PeerId),
/// Peer has disconnected,
PeerDisconnected(PeerId),
/// An RPC response/request has been received.
RPC(PeerId, RPCEvent<T>),
/// A gossip message has been received. The fields are: message id, the peer that sent us this
/// message and the message itself.
PubsubMessage(MessageId, PeerId, PubsubMessage<T>),
/// The peer manager has requested we re-status a peer.
StatusPeer(PeerId),
}
impl<T: BeaconChainTypes> Router<T> {
/// Initializes and runs the Router.
pub fn spawn(
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
executor: &tokio::runtime::TaskExecutor,
log: slog::Logger,
) -> error::Result<mpsc::UnboundedSender<RouterMessage<T::EthSpec>>> {
let message_handler_log = log.new(o!("service"=> "router"));
trace!(message_handler_log, "Service starting");
let (handler_send, handler_recv) = mpsc::unbounded_channel();
// Initialise a message instance, which itself spawns the syncing thread.
let processor = Processor::new(
executor,
beacon_chain,
network_globals,
network_send.clone(),
&log,
);
// generate the Message handler
let mut handler = Router {
network_send,
processor,
log: message_handler_log,
};
// spawn handler task and move the message handler instance into the spawned thread
executor.spawn(
handler_recv
.for_each(move |msg| Ok(handler.handle_message(msg)))
.map_err(move |_| {
debug!(log, "Network message handler terminated.");
}),
);
Ok(handler_send)
}
/// Handle all messages incoming from the network service.
fn handle_message(&mut self, message: RouterMessage<T::EthSpec>) {
match message {
// we have initiated a connection to a peer or the peer manager has requested a
// re-status
RouterMessage::PeerDialed(peer_id) | RouterMessage::StatusPeer(peer_id) => {
self.processor.send_status(peer_id);
}
// A peer has disconnected
RouterMessage::PeerDisconnected(peer_id) => {
self.processor.on_disconnect(peer_id);
}
// An RPC message request/response has been received
RouterMessage::RPC(peer_id, rpc_event) => {
self.handle_rpc_message(peer_id, rpc_event);
}
// An RPC message request/response has been received
RouterMessage::PubsubMessage(id, peer_id, gossip) => {
self.handle_gossip(id, peer_id, gossip);
}
}
}
/* RPC - Related functionality */
/// Handle RPC messages
fn handle_rpc_message(&mut self, peer_id: PeerId, rpc_message: RPCEvent<T::EthSpec>) {
match rpc_message {
RPCEvent::Request(id, req) => self.handle_rpc_request(peer_id, id, req),
RPCEvent::Response(id, resp) => self.handle_rpc_response(peer_id, id, resp),
RPCEvent::Error(id, error) => self.handle_rpc_error(peer_id, id, error),
}
}
/// A new RPC request has been received from the network.
fn handle_rpc_request(
&mut self,
peer_id: PeerId,
request_id: RequestId,
request: RPCRequest<T::EthSpec>,
) {
match request {
RPCRequest::Status(status_message) => {
self.processor
.on_status_request(peer_id, request_id, status_message)
}
RPCRequest::Goodbye(goodbye_reason) => {
debug!(
self.log, "PeerGoodbye";
"peer" => format!("{:?}", peer_id),
"reason" => format!("{:?}", goodbye_reason),
);
self.processor.on_disconnect(peer_id);
}
RPCRequest::BlocksByRange(request) => self
.processor
.on_blocks_by_range_request(peer_id, request_id, request),
RPCRequest::BlocksByRoot(request) => self
.processor
.on_blocks_by_root_request(peer_id, request_id, request),
RPCRequest::Ping(_) => unreachable!("Ping MUST be handled in the behaviour"),
RPCRequest::MetaData(_) => unreachable!("MetaData MUST be handled in the behaviour"),
}
}
/// An RPC response has been received from the network.
// we match on id and ignore responses past the timeout.
fn handle_rpc_response(
&mut self,
peer_id: PeerId,
request_id: RequestId,
error_response: RPCErrorResponse<T::EthSpec>,
) {
// an error could have occurred.
match error_response {
RPCErrorResponse::InvalidRequest(error) => {
warn!(self.log, "Peer indicated invalid request";"peer_id" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::ServerError(error) => {
warn!(self.log, "Peer internal server error";"peer_id" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::Unknown(error) => {
warn!(self.log, "Unknown peer error";"peer" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::Success(response) => match response {
RPCResponse::Status(status_message) => {
self.processor.on_status_response(peer_id, status_message);
}
RPCResponse::BlocksByRange(beacon_block) => {
self.processor.on_blocks_by_range_response(
peer_id,
request_id,
Some(beacon_block),
);
}
RPCResponse::BlocksByRoot(beacon_block) => {
self.processor.on_blocks_by_root_response(
peer_id,
request_id,
Some(beacon_block),
);
}
RPCResponse::Pong(_) => {
unreachable!("Ping must be handled in the behaviour");
}
RPCResponse::MetaData(_) => {
unreachable!("Meta data must be handled in the behaviour");
}
},
RPCErrorResponse::StreamTermination(response_type) => {
// have received a stream termination, notify the processing functions
match response_type {
ResponseTermination::BlocksByRange => {
self.processor
.on_blocks_by_range_response(peer_id, request_id, None);
}
ResponseTermination::BlocksByRoot => {
self.processor
.on_blocks_by_root_response(peer_id, request_id, None);
}
}
}
}
}
/// Handle various RPC errors
fn handle_rpc_error(&mut self, peer_id: PeerId, request_id: RequestId, error: RPCError) {
warn!(self.log, "RPC Error"; "Peer" => format!("{:?}", peer_id), "request_id" => format!("{}", request_id), "Error" => format!("{:?}", error));
self.processor.on_rpc_error(peer_id, request_id);
}
/// Handle RPC messages
fn handle_gossip(
&mut self,
id: MessageId,
peer_id: PeerId,
gossip_message: PubsubMessage<T::EthSpec>,
) {
match gossip_message {
// Attestations should never reach the router.
PubsubMessage::AggregateAndProofAttestation(aggregate_and_proof) => {
if self
.processor
.should_forward_aggregate_attestation(&aggregate_and_proof)
{
self.propagate_message(id, peer_id.clone());
}
self.processor.process_attestation_gossip(
peer_id,
aggregate_and_proof.message.aggregate,
AttestationType::Aggregated,
);
}
PubsubMessage::Attestation(subnet_attestation) => {
if self
.processor
.should_forward_attestation(&subnet_attestation.1)
{
self.propagate_message(id, peer_id.clone());
}
self.processor.process_attestation_gossip(
peer_id,
subnet_attestation.1,
AttestationType::Unaggregated { should_store: true },
);
}
PubsubMessage::BeaconBlock(block) => {
match self.processor.should_forward_block(&peer_id, block) {
Ok(verified_block) => {
self.propagate_message(id, peer_id.clone());
self.processor.on_block_gossip(peer_id, verified_block);
}
Err(BlockError::ParentUnknown { .. }) => {} // performing a parent lookup
Err(e) => {
// performing a parent lookup
warn!(self.log, "Could not verify block for gossip";
"error" => format!("{:?}", e));
}
}
}
PubsubMessage::VoluntaryExit(_exit) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle exits
debug!(self.log, "Received a voluntary exit"; "peer_id" => format!("{}", peer_id) );
}
PubsubMessage::ProposerSlashing(_proposer_slashing) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle proposer slashings
debug!(self.log, "Received a proposer slashing"; "peer_id" => format!("{}", peer_id) );
}
PubsubMessage::AttesterSlashing(_attester_slashing) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle attester slashings
debug!(self.log, "Received an attester slashing"; "peer_id" => format!("{}", peer_id) );
}
}
}
/// Informs the network service that the message should be forwarded to other peers.
fn propagate_message(&mut self, message_id: MessageId, 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"
)
});
}
}

View File

@ -1,17 +1,21 @@
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use crate::sync::SyncMessage; use crate::sync::{PeerSyncInfo, SyncMessage};
use beacon_chain::{ use beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BlockProcessingOutcome, AttestationProcessingOutcome, AttestationType, BeaconChain, BeaconChainTypes, BlockError,
BlockProcessingOutcome, GossipVerifiedBlock,
}; };
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::{RPCEvent, RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::rpc::{RPCEvent, RPCRequest, RPCResponse, RequestId};
use eth2_libp2p::PeerId; use eth2_libp2p::{NetworkGlobals, PeerId};
use slog::{debug, error, o, trace, warn}; use slog::{debug, error, o, trace, warn};
use ssz::Encode; use ssz::Encode;
use std::sync::Arc; use std::sync::Arc;
use store::Store; use store::Store;
use tokio::sync::{mpsc, oneshot}; use tokio::sync::{mpsc, oneshot};
use types::{Attestation, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; use types::{
Attestation, ChainSpec, Epoch, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock,
Slot,
};
//TODO: Rate limit requests //TODO: Rate limit requests
@ -19,40 +23,9 @@ use types::{Attestation, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/// Otherwise we queue it. /// Otherwise we queue it.
pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1; pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1;
const SHOULD_FORWARD_GOSSIP_BLOCK: bool = true;
const SHOULD_NOT_FORWARD_GOSSIP_BLOCK: bool = false;
/// Keeps track of syncing information for known connected peers.
#[derive(Clone, Copy, Debug)]
pub struct PeerSyncInfo {
fork_version: [u8; 4],
pub finalized_root: Hash256,
pub finalized_epoch: Epoch,
pub head_root: Hash256,
pub head_slot: Slot,
}
impl From<StatusMessage> for PeerSyncInfo {
fn from(status: StatusMessage) -> PeerSyncInfo {
PeerSyncInfo {
fork_version: status.fork_version,
finalized_root: status.finalized_root,
finalized_epoch: status.finalized_epoch,
head_root: status.head_root,
head_slot: status.head_slot,
}
}
}
impl PeerSyncInfo {
pub fn from_chain<T: BeaconChainTypes>(chain: &Arc<BeaconChain<T>>) -> Option<PeerSyncInfo> {
Some(Self::from(status_message(chain)?))
}
}
/// Processes validated messages from the network. It relays necessary data to the syncing thread /// Processes validated messages from the network. It relays necessary data to the syncing thread
/// and processes blocks from the pubsub network. /// and processes blocks from the pubsub network.
pub struct MessageProcessor<T: BeaconChainTypes> { pub struct Processor<T: BeaconChainTypes> {
/// A reference to the underlying beacon chain. /// A reference to the underlying beacon chain.
chain: Arc<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
/// A channel to the syncing thread. /// A channel to the syncing thread.
@ -60,17 +33,18 @@ pub struct MessageProcessor<T: BeaconChainTypes> {
/// A oneshot channel for destroying the sync thread. /// A oneshot channel for destroying the sync thread.
_sync_exit: oneshot::Sender<()>, _sync_exit: oneshot::Sender<()>,
/// A network context to return and handle RPC requests. /// A network context to return and handle RPC requests.
network: HandlerNetworkContext, network: HandlerNetworkContext<T::EthSpec>,
/// The `RPCHandler` logger. /// The `RPCHandler` logger.
log: slog::Logger, log: slog::Logger,
} }
impl<T: BeaconChainTypes> MessageProcessor<T> { impl<T: BeaconChainTypes> Processor<T> {
/// Instantiate a `MessageProcessor` instance /// Instantiate a `Processor` instance
pub fn new( pub fn new(
executor: &tokio::runtime::TaskExecutor, executor: &tokio::runtime::TaskExecutor,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_send: mpsc::UnboundedSender<NetworkMessage>, network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
log: &slog::Logger, log: &slog::Logger,
) -> Self { ) -> Self {
let sync_logger = log.new(o!("service"=> "sync")); let sync_logger = log.new(o!("service"=> "sync"));
@ -78,12 +52,13 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
// spawn the sync thread // spawn the sync thread
let (sync_send, _sync_exit) = crate::sync::manager::spawn( let (sync_send, _sync_exit) = crate::sync::manager::spawn(
executor, executor,
Arc::downgrade(&beacon_chain), beacon_chain.clone(),
network_globals,
network_send.clone(), network_send.clone(),
sync_logger, sync_logger,
); );
MessageProcessor { Processor {
chain: beacon_chain, chain: beacon_chain,
sync_send, sync_send,
_sync_exit, _sync_exit,
@ -114,16 +89,17 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
self.send_to_sync(SyncMessage::RPCError(peer_id, request_id)); self.send_to_sync(SyncMessage::RPCError(peer_id, request_id));
} }
/// Handle the connection of a new peer.
///
/// Sends a `Status` message to the peer. /// Sends a `Status` message to the peer.
pub fn on_connect(&mut self, peer_id: PeerId) { ///
/// Called when we first connect to a peer, or when the PeerManager determines we need to
/// re-status.
pub fn send_status(&mut self, peer_id: PeerId) {
if let Some(status_message) = status_message(&self.chain) { if let Some(status_message) = status_message(&self.chain) {
debug!( debug!(
self.log, self.log,
"Sending Status Request"; "Sending Status Request";
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
"fork_version" => format!("{:?}", status_message.fork_version), "fork_digest" => format!("{:?}", status_message.fork_digest),
"finalized_root" => format!("{:?}", status_message.finalized_root), "finalized_root" => format!("{:?}", status_message.finalized_root),
"finalized_epoch" => format!("{:?}", status_message.finalized_epoch), "finalized_epoch" => format!("{:?}", status_message.finalized_epoch),
"head_root" => format!("{}", status_message.head_root), "head_root" => format!("{}", status_message.head_root),
@ -147,7 +123,7 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
self.log, self.log,
"Received Status Request"; "Received Status Request";
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
"fork_version" => format!("{:?}", status.fork_version), "fork_digest" => format!("{:?}", status.fork_digest),
"finalized_root" => format!("{:?}", status.finalized_root), "finalized_root" => format!("{:?}", status.finalized_root),
"finalized_epoch" => format!("{:?}", status.finalized_epoch), "finalized_epoch" => format!("{:?}", status.finalized_epoch),
"head_root" => format!("{}", status.head_root), "head_root" => format!("{}", status.head_root),
@ -169,7 +145,16 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
/// Process a `Status` response from a peer. /// Process a `Status` response from a peer.
pub fn on_status_response(&mut self, peer_id: PeerId, status: StatusMessage) { pub fn on_status_response(&mut self, peer_id: PeerId, status: StatusMessage) {
trace!(self.log, "StatusResponse"; "peer" => format!("{:?}", peer_id)); trace!(
self.log,
"Received Status Response";
"peer" => format!("{:?}", peer_id),
"fork_digest" => format!("{:?}", status.fork_digest),
"finalized_root" => format!("{:?}", status.finalized_root),
"finalized_epoch" => format!("{:?}", status.finalized_epoch),
"head_root" => format!("{}", status.head_root),
"head_slot" => format!("{}", status.head_slot),
);
// Process the status message, without sending back another status. // Process the status message, without sending back another status.
self.process_status(peer_id, status); self.process_status(peer_id, status);
@ -193,12 +178,14 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
let start_slot = |epoch: Epoch| epoch.start_slot(T::EthSpec::slots_per_epoch()); let start_slot = |epoch: Epoch| epoch.start_slot(T::EthSpec::slots_per_epoch());
if local.fork_version != remote.fork_version { if local.fork_digest != remote.fork_digest {
// The node is on a different network/fork, disconnect them. // The node is on a different network/fork, disconnect them.
debug!( debug!(
self.log, "Handshake Failure"; self.log, "Handshake Failure";
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
"reason" => "network_id" "reason" => "incompatible forks",
"our_fork" => hex::encode(local.fork_digest),
"their_fork" => hex::encode(remote.fork_digest)
); );
self.network self.network
@ -265,7 +252,7 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
.exists::<SignedBeaconBlock<T::EthSpec>>(&remote.head_root) .exists::<SignedBeaconBlock<T::EthSpec>>(&remote.head_root)
.unwrap_or_else(|_| false) .unwrap_or_else(|_| false)
{ {
trace!( debug!(
self.log, "Peer with known chain found"; self.log, "Peer with known chain found";
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
"remote_head_slot" => remote.head_slot, "remote_head_slot" => remote.head_slot,
@ -303,7 +290,7 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
self.network.send_rpc_response( self.network.send_rpc_response(
peer_id.clone(), peer_id.clone(),
request_id, request_id,
RPCResponse::BlocksByRoot(block.as_ssz_bytes()), RPCResponse::BlocksByRoot(Box::new(block)),
); );
send_block_count += 1; send_block_count += 1;
} else { } else {
@ -389,7 +376,7 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
self.network.send_rpc_response( self.network.send_rpc_response(
peer_id.clone(), peer_id.clone(),
request_id, request_id,
RPCResponse::BlocksByRange(block.as_ssz_bytes()), RPCResponse::BlocksByRange(Box::new(block)),
); );
} }
} else { } else {
@ -436,9 +423,8 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>, beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
) { ) {
let beacon_block = beacon_block.map(Box::new);
trace!( trace!(
self.log, self.log,
"Received BlocksByRange Response"; "Received BlocksByRange Response";
@ -457,9 +443,8 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>, beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
) { ) {
let beacon_block = beacon_block.map(Box::new);
trace!( trace!(
self.log, self.log,
"Received BlocksByRoot Response"; "Received BlocksByRoot Response";
@ -473,6 +458,24 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
}); });
} }
/// Template function to be called on a block to determine if the block should be propagated
/// across the network.
pub fn should_forward_block(
&mut self,
peer_id: &PeerId,
block: Box<SignedBeaconBlock<T::EthSpec>>,
) -> Result<GossipVerifiedBlock<T>, BlockError> {
let result = self.chain.verify_block_for_gossip(*block.clone());
if let Err(BlockError::ParentUnknown(block_hash)) = result {
// if we don't know the parent, start a parent lookup
// TODO: Modify the return to avoid the block clone.
debug!(self.log, "Unknown block received. Starting a parent lookup"; "block_slot" => block.message.slot, "block_hash" => format!("{}", block_hash));
self.send_to_sync(SyncMessage::UnknownBlock(peer_id.clone(), block));
}
result
}
/// Process a gossip message declaring a new block. /// Process a gossip message declaring a new block.
/// ///
/// Attempts to apply to block to the beacon chain. May queue the block for later processing. /// Attempts to apply to block to the beacon chain. May queue the block for later processing.
@ -481,9 +484,10 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
pub fn on_block_gossip( pub fn on_block_gossip(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
block: SignedBeaconBlock<T::EthSpec>, verified_block: GossipVerifiedBlock<T>,
) -> bool { ) -> bool {
match self.chain.process_block(block.clone()) { let block = Box::new(verified_block.block.clone());
match BlockProcessingOutcome::shim(self.chain.process_block(verified_block)) {
Ok(outcome) => match outcome { Ok(outcome) => match outcome {
BlockProcessingOutcome::Processed { .. } => { BlockProcessingOutcome::Processed { .. } => {
trace!(self.log, "Gossipsub block processed"; trace!(self.log, "Gossipsub block processed";
@ -508,24 +512,14 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
"location" => "block gossip" "location" => "block gossip"
), ),
} }
SHOULD_FORWARD_GOSSIP_BLOCK
} }
BlockProcessingOutcome::ParentUnknown { .. } => { BlockProcessingOutcome::ParentUnknown { .. } => {
// Inform the sync manager to find parents for this block // Inform the sync manager to find parents for this block
trace!(self.log, "Block with unknown parent received"; // This should not occur. It should be checked by `should_forward_block`
error!(self.log, "Block with unknown parent attempted to be processed";
"peer_id" => format!("{:?}",peer_id)); "peer_id" => format!("{:?}",peer_id));
self.send_to_sync(SyncMessage::UnknownBlock(peer_id, Box::new(block))); self.send_to_sync(SyncMessage::UnknownBlock(peer_id, block));
SHOULD_FORWARD_GOSSIP_BLOCK
} }
BlockProcessingOutcome::FutureSlot {
present_slot,
block_slot,
} if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot => {
//TODO: Decide the logic here
SHOULD_FORWARD_GOSSIP_BLOCK
}
BlockProcessingOutcome::BlockIsAlreadyKnown => SHOULD_FORWARD_GOSSIP_BLOCK,
other => { other => {
warn!( warn!(
self.log, self.log,
@ -539,7 +533,6 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
"Invalid gossip beacon block ssz"; "Invalid gossip beacon block ssz";
"ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())), "ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())),
); );
SHOULD_NOT_FORWARD_GOSSIP_BLOCK //TODO: Decide if we want to forward these
} }
}, },
Err(_) => { Err(_) => {
@ -549,16 +542,38 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
"Erroneous gossip beacon block ssz"; "Erroneous gossip beacon block ssz";
"ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())), "ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())),
); );
SHOULD_NOT_FORWARD_GOSSIP_BLOCK
} }
} }
// TODO: Update with correct block gossip checking
true
} }
/// Process a gossip message declaring a new attestation. /// Verifies the Aggregate attestation before propagating.
/// pub fn should_forward_aggregate_attestation(
/// Not currently implemented. &self,
pub fn on_attestation_gossip(&mut self, peer_id: PeerId, msg: Attestation<T::EthSpec>) { _aggregate_and_proof: &Box<SignedAggregateAndProof<T::EthSpec>>,
match self.chain.process_attestation(msg.clone()) { ) -> bool {
// TODO: Implement
true
}
/// Verifies the attestation before propagating.
pub fn should_forward_attestation(&self, _aggregate: &Attestation<T::EthSpec>) -> bool {
// TODO: Implement
true
}
/// Process a new attestation received from gossipsub.
pub fn process_attestation_gossip(
&mut self,
peer_id: PeerId,
msg: Attestation<T::EthSpec>,
attestation_type: AttestationType,
) {
match self
.chain
.process_attestation(msg.clone(), attestation_type)
{
Ok(outcome) => match outcome { Ok(outcome) => match outcome {
AttestationProcessingOutcome::Processed => { AttestationProcessingOutcome::Processed => {
debug!( debug!(
@ -572,7 +587,7 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
} }
AttestationProcessingOutcome::UnknownHeadBlock { beacon_block_root } => { AttestationProcessingOutcome::UnknownHeadBlock { beacon_block_root } => {
// TODO: Maintain this attestation and re-process once sync completes // TODO: Maintain this attestation and re-process once sync completes
trace!( debug!(
self.log, self.log,
"Attestation for unknown block"; "Attestation for unknown block";
"peer_id" => format!("{:?}", peer_id), "peer_id" => format!("{:?}", peer_id),
@ -603,7 +618,7 @@ impl<T: BeaconChainTypes> MessageProcessor<T> {
"ssz" => format!("0x{}", hex::encode(msg.as_ssz_bytes())), "ssz" => format!("0x{}", hex::encode(msg.as_ssz_bytes())),
); );
} }
} };
} }
} }
@ -612,9 +627,13 @@ pub(crate) fn status_message<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>, beacon_chain: &BeaconChain<T>,
) -> Option<StatusMessage> { ) -> Option<StatusMessage> {
let head_info = beacon_chain.head_info().ok()?; let head_info = beacon_chain.head_info().ok()?;
let genesis_validators_root = beacon_chain.genesis_validators_root;
let fork_digest =
ChainSpec::compute_fork_digest(head_info.fork.current_version, genesis_validators_root);
Some(StatusMessage { Some(StatusMessage {
fork_version: head_info.fork.current_version, fork_digest,
finalized_root: head_info.finalized_checkpoint.root, finalized_root: head_info.finalized_checkpoint.root,
finalized_epoch: head_info.finalized_checkpoint.epoch, finalized_epoch: head_info.finalized_checkpoint.epoch,
head_root: head_info.block_root, head_root: head_info.block_root,
@ -622,18 +641,19 @@ pub(crate) fn status_message<T: BeaconChainTypes>(
}) })
} }
/// Wraps a Network Channel to employ various RPC related network functionality for the message /// Wraps a Network Channel to employ various RPC related network functionality for the
/// handler. The handler doesn't manage it's own request Id's and can therefore only send /// processor.
/// The Processor doesn't manage it's own request Id's and can therefore only send
/// responses or requests with 0 request Ids. /// responses or requests with 0 request Ids.
pub struct HandlerNetworkContext { pub struct HandlerNetworkContext<T: EthSpec> {
/// The network channel to relay messages to the Network service. /// The network channel to relay messages to the Network service.
network_send: mpsc::UnboundedSender<NetworkMessage>, network_send: mpsc::UnboundedSender<NetworkMessage<T>>,
/// Logger for the `NetworkContext`. /// Logger for the `NetworkContext`.
log: slog::Logger, log: slog::Logger,
} }
impl HandlerNetworkContext { impl<T: EthSpec> HandlerNetworkContext<T> {
pub fn new(network_send: mpsc::UnboundedSender<NetworkMessage>, log: slog::Logger) -> Self { pub fn new(network_send: mpsc::UnboundedSender<NetworkMessage<T>>, log: slog::Logger) -> Self {
Self { network_send, log } Self { network_send, log }
} }
@ -655,7 +675,7 @@ impl HandlerNetworkContext {
}); });
} }
pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest) { pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest<T>) {
// the message handler cannot send requests with ids. Id's are managed by the sync // the message handler cannot send requests with ids. Id's are managed by the sync
// manager. // manager.
let request_id = 0; let request_id = 0;
@ -667,7 +687,7 @@ impl HandlerNetworkContext {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
rpc_response: RPCResponse, rpc_response: RPCResponse<T>,
) { ) {
self.send_rpc_event( self.send_rpc_event(
peer_id, peer_id,
@ -680,12 +700,12 @@ impl HandlerNetworkContext {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
rpc_error_response: RPCErrorResponse, rpc_error_response: RPCErrorResponse<T>,
) { ) {
self.send_rpc_event(peer_id, RPCEvent::Response(request_id, rpc_error_response)); self.send_rpc_event(peer_id, RPCEvent::Response(request_id, rpc_error_response));
} }
fn send_rpc_event(&mut self, peer_id: PeerId, rpc_event: RPCEvent) { fn send_rpc_event(&mut self, peer_id: PeerId, rpc_event: RPCEvent<T>) {
self.network_send self.network_send
.try_send(NetworkMessage::RPC(peer_id, rpc_event)) .try_send(NetworkMessage::RPC(peer_id, rpc_event))
.unwrap_or_else(|_| { .unwrap_or_else(|_| {

View File

@ -1,23 +1,24 @@
use crate::error; use crate::error;
use crate::message_handler::{HandlerMessage, MessageHandler};
use crate::persisted_dht::{load_dht, persist_dht}; use crate::persisted_dht::{load_dht, persist_dht};
use crate::NetworkConfig; use crate::router::{Router, RouterMessage};
use beacon_chain::{BeaconChain, BeaconChainTypes}; use crate::{
use core::marker::PhantomData; attestation_service::{AttServiceMessage, AttestationService},
use eth2_libp2p::Service as LibP2PService; NetworkConfig,
use eth2_libp2p::{
rpc::RPCRequest, Enr, Libp2pEvent, MessageId, Multiaddr, NetworkGlobals, PeerId, Swarm, Topic,
}; };
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::Service as LibP2PService;
use eth2_libp2p::{rpc::RPCRequest, BehaviourEvent, Enr, MessageId, NetworkGlobals, PeerId, Swarm};
use eth2_libp2p::{PubsubMessage, RPCEvent}; use eth2_libp2p::{PubsubMessage, RPCEvent};
use futures::prelude::*; use futures::prelude::*;
use futures::Stream; use futures::Stream;
use rest_types::ValidatorSubscription;
use slog::{debug, error, info, trace}; use slog::{debug, error, info, trace};
use std::collections::HashSet; use std::sync::Arc;
use std::sync::{atomic::Ordering, Arc};
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
use tokio::runtime::TaskExecutor; use tokio::runtime::TaskExecutor;
use tokio::sync::{mpsc, oneshot}; use tokio::sync::{mpsc, oneshot};
use tokio::timer::Delay; use tokio::timer::Delay;
use types::EthSpec;
mod tests; mod tests;
@ -25,110 +26,107 @@ mod tests;
const BAN_PEER_TIMEOUT: u64 = 30; const BAN_PEER_TIMEOUT: u64 = 30;
/// Service that handles communication between internal services and the `eth2_libp2p` network service. /// Service that handles communication between internal services and the `eth2_libp2p` network service.
pub struct Service<T: BeaconChainTypes> { pub struct NetworkService<T: BeaconChainTypes> {
libp2p_port: u16, /// A reference to the underlying beacon chain.
network_globals: Arc<NetworkGlobals>, beacon_chain: Arc<BeaconChain<T>>,
_libp2p_exit: oneshot::Sender<()>, /// The underlying libp2p service that drives all the network interactions.
_network_send: mpsc::UnboundedSender<NetworkMessage>, libp2p: LibP2PService<T::EthSpec>,
_phantom: PhantomData<T>, /// An attestation and subnet manager service.
attestation_service: AttestationService<T>,
/// The receiver channel for lighthouse to communicate with the network service.
network_recv: mpsc::UnboundedReceiver<NetworkMessage<T::EthSpec>>,
/// The sending channel for the network service to send messages to be routed throughout
/// lighthouse.
router_send: mpsc::UnboundedSender<RouterMessage<T::EthSpec>>,
/// A reference to lighthouse's database to persist the DHT.
store: Arc<T::Store>,
/// A collection of global variables, accessible outside of the network service.
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
/// An initial delay to update variables after the libp2p service has started.
initial_delay: Delay,
/// A delay that expires when a new fork takes place.
next_fork_update: Option<Delay>,
/// The logger for the network service.
log: slog::Logger,
/// A probability of propagation.
propagation_percentage: Option<u8>,
} }
impl<T: BeaconChainTypes> Service<T> { impl<T: BeaconChainTypes> NetworkService<T> {
pub fn new( pub fn start(
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
config: &NetworkConfig, config: &NetworkConfig,
executor: &TaskExecutor, executor: &TaskExecutor,
network_log: slog::Logger, network_log: slog::Logger,
) -> error::Result<(Arc<Self>, mpsc::UnboundedSender<NetworkMessage>)> { ) -> error::Result<(
Arc<NetworkGlobals<T::EthSpec>>,
mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
oneshot::Sender<()>,
)> {
// build the network channel // build the network channel
let (network_send, network_recv) = mpsc::unbounded_channel::<NetworkMessage>(); let (network_send, network_recv) = mpsc::unbounded_channel::<NetworkMessage<T::EthSpec>>();
// launch message handler thread // get a reference to the beacon chain store
let store = beacon_chain.store.clone(); let store = beacon_chain.store.clone();
let message_handler_send = MessageHandler::spawn(
beacon_chain,
network_send.clone(),
executor,
network_log.clone(),
)?;
let propagation_percentage = config.propagation_percentage; let propagation_percentage = config.propagation_percentage;
// build the current enr_fork_id for adding to our local ENR
let enr_fork_id = beacon_chain.enr_fork_id();
// keep track of when our fork_id needs to be updated
let next_fork_update = next_fork_delay(&beacon_chain);
// launch libp2p service // launch libp2p service
let (network_globals, mut libp2p_service) = let (network_globals, mut libp2p) =
LibP2PService::new(config, network_log.clone())?; LibP2PService::new(config, enr_fork_id, network_log.clone())?;
for enr in load_dht::<T::Store, T::EthSpec>(store.clone()) { for enr in load_dht::<T::Store, T::EthSpec>(store.clone()) {
libp2p_service.swarm.add_enr(enr); libp2p.swarm.add_enr(enr);
} }
// A delay used to initialise code after the network has started // A delay used to initialise code after the network has started
// This is currently used to obtain the listening addresses from the libp2p service. // This is currently used to obtain the listening addresses from the libp2p service.
let initial_delay = Delay::new(Instant::now() + Duration::from_secs(1)); let initial_delay = Delay::new(Instant::now() + Duration::from_secs(1));
let libp2p_exit = spawn_service::<T>( // launch derived network services
libp2p_service,
network_recv, // router task
message_handler_send, let router_send = Router::spawn(
executor, beacon_chain.clone(),
store,
network_globals.clone(), network_globals.clone(),
initial_delay, network_send.clone(),
executor,
network_log.clone(), network_log.clone(),
propagation_percentage,
)?; )?;
let network_service = Service { // attestation service
libp2p_port: config.libp2p_port, let attestation_service =
network_globals, AttestationService::new(beacon_chain.clone(), network_globals.clone(), &network_log);
_libp2p_exit: libp2p_exit,
_network_send: network_send.clone(), // create the network service and spawn the task
_phantom: PhantomData, let network_service = NetworkService {
beacon_chain,
libp2p,
attestation_service,
network_recv,
router_send,
store,
network_globals: network_globals.clone(),
initial_delay,
next_fork_update,
log: network_log,
propagation_percentage,
}; };
Ok((Arc::new(network_service), network_send)) let network_exit = spawn_service(network_service, &executor)?;
}
/// Returns the local ENR from the underlying Discv5 behaviour that external peers may connect Ok((network_globals, network_send, network_exit))
/// to.
pub fn local_enr(&self) -> Option<Enr> {
self.network_globals.local_enr.read().clone()
}
/// Returns the local libp2p PeerID.
pub fn local_peer_id(&self) -> PeerId {
self.network_globals.peer_id.read().clone()
}
/// Returns the list of `Multiaddr` that the underlying libp2p instance is listening on.
pub fn listen_multiaddrs(&self) -> Vec<Multiaddr> {
self.network_globals.listen_multiaddrs.read().clone()
}
/// Returns the libp2p port that this node has been configured to listen using.
pub fn listen_port(&self) -> u16 {
self.libp2p_port
}
/// Returns the number of libp2p connected peers.
pub fn connected_peers(&self) -> usize {
self.network_globals.connected_peers.load(Ordering::Relaxed)
}
/// Returns the set of `PeerId` that are connected via libp2p.
pub fn connected_peer_set(&self) -> HashSet<PeerId> {
self.network_globals.connected_peer_set.read().clone()
} }
} }
fn spawn_service<T: BeaconChainTypes>( fn spawn_service<T: BeaconChainTypes>(
mut libp2p_service: LibP2PService, mut service: NetworkService<T>,
mut network_recv: mpsc::UnboundedReceiver<NetworkMessage>,
mut message_handler_send: mpsc::UnboundedSender<HandlerMessage>,
executor: &TaskExecutor, executor: &TaskExecutor,
store: Arc<T::Store>,
network_globals: Arc<NetworkGlobals>,
mut initial_delay: Delay,
log: slog::Logger,
propagation_percentage: Option<u8>,
) -> error::Result<tokio::sync::oneshot::Sender<()>> { ) -> error::Result<tokio::sync::oneshot::Sender<()>> {
let (network_exit, mut exit_rx) = tokio::sync::oneshot::channel(); let (network_exit, mut exit_rx) = tokio::sync::oneshot::channel();
@ -136,25 +134,27 @@ fn spawn_service<T: BeaconChainTypes>(
executor.spawn( executor.spawn(
futures::future::poll_fn(move || -> Result<_, ()> { futures::future::poll_fn(move || -> Result<_, ()> {
let log = &service.log;
if !initial_delay.is_elapsed() { // handles any logic which requires an initial delay
if let Ok(Async::Ready(_)) = initial_delay.poll() { if !service.initial_delay.is_elapsed() {
let multi_addrs = Swarm::listeners(&libp2p_service.swarm).cloned().collect(); if let Ok(Async::Ready(_)) = service.initial_delay.poll() {
*network_globals.listen_multiaddrs.write() = multi_addrs; let multi_addrs = Swarm::listeners(&service.libp2p.swarm).cloned().collect();
*service.network_globals.listen_multiaddrs.write() = multi_addrs;
} }
} }
// perform termination tasks when the network is being shutdown // perform termination tasks when the network is being shutdown
if let Ok(Async::Ready(_)) | Err(_) = exit_rx.poll() { if let Ok(Async::Ready(_)) | Err(_) = exit_rx.poll() {
// network thread is terminating // network thread is terminating
let enrs: Vec<Enr> = libp2p_service.swarm.enr_entries().cloned().collect(); let enrs: Vec<Enr> = service.libp2p.swarm.enr_entries().cloned().collect();
debug!( debug!(
log, log,
"Persisting DHT to store"; "Persisting DHT to store";
"Number of peers" => format!("{}", enrs.len()), "Number of peers" => format!("{}", enrs.len()),
); );
match persist_dht::<T::Store, T::EthSpec>(store.clone(), enrs) { match persist_dht::<T::Store, T::EthSpec>(service.store.clone(), enrs) {
Err(e) => error!( Err(e) => error!(
log, log,
"Failed to persist DHT on drop"; "Failed to persist DHT on drop";
@ -173,11 +173,11 @@ fn spawn_service<T: BeaconChainTypes>(
// processes the network channel before processing the libp2p swarm // processes the network channel before processing the libp2p swarm
loop { loop {
// poll the network channel // poll the network channel
match network_recv.poll() { match service.network_recv.poll() {
Ok(Async::Ready(Some(message))) => match message { Ok(Async::Ready(Some(message))) => match message {
NetworkMessage::RPC(peer_id, rpc_event) => { NetworkMessage::RPC(peer_id, rpc_event) => {
trace!(log, "Sending RPC"; "rpc" => format!("{}", rpc_event)); trace!(log, "Sending RPC"; "rpc" => format!("{}", rpc_event));
libp2p_service.swarm.send_rpc(peer_id, rpc_event); service.libp2p.swarm.send_rpc(peer_id, rpc_event);
} }
NetworkMessage::Propagate { NetworkMessage::Propagate {
propagation_source, propagation_source,
@ -186,7 +186,7 @@ fn spawn_service<T: BeaconChainTypes>(
// TODO: Remove this for mainnet // TODO: Remove this for mainnet
// randomly prevents propagation // randomly prevents propagation
let mut should_send = true; let mut should_send = true;
if let Some(percentage) = propagation_percentage { if let Some(percentage) = service.propagation_percentage {
// not exact percentage but close enough // not exact percentage but close enough
let rand = rand::random::<u8>() % 100; let rand = rand::random::<u8>() % 100;
if rand > percentage { if rand > percentage {
@ -201,16 +201,16 @@ fn spawn_service<T: BeaconChainTypes>(
"propagation_peer" => format!("{:?}", propagation_source), "propagation_peer" => format!("{:?}", propagation_source),
"message_id" => message_id.to_string(), "message_id" => message_id.to_string(),
); );
libp2p_service service.libp2p
.swarm .swarm
.propagate_message(&propagation_source, message_id); .propagate_message(&propagation_source, message_id);
} }
} }
NetworkMessage::Publish { topics, message } => { NetworkMessage::Publish { messages } => {
// TODO: Remove this for mainnet // TODO: Remove this for mainnet
// randomly prevents propagation // randomly prevents propagation
let mut should_send = true; let mut should_send = true;
if let Some(percentage) = propagation_percentage { if let Some(percentage) = service.propagation_percentage {
// not exact percentage but close enough // not exact percentage but close enough
let rand = rand::random::<u8>() % 100; let rand = rand::random::<u8>() % 100;
if rand > percentage { if rand > percentage {
@ -219,18 +219,29 @@ fn spawn_service<T: BeaconChainTypes>(
} }
} }
if !should_send { if !should_send {
info!(log, "Random filter did not publish message"); info!(log, "Random filter did not publish messages");
} else { } else {
debug!(log, "Sending pubsub message"; "topics" => format!("{:?}",topics)); let mut topic_kinds = Vec::new();
libp2p_service.swarm.publish(&topics, message); for message in &messages {
if !topic_kinds.contains(&message.kind()) {
topic_kinds.push(message.kind());
}
}
debug!(log, "Sending pubsub messages"; "count" => messages.len(), "topics" => format!("{:?}", topic_kinds));
service.libp2p.swarm.publish(messages);
} }
} }
NetworkMessage::Disconnect { peer_id } => { NetworkMessage::Disconnect { peer_id } => {
libp2p_service.disconnect_and_ban_peer( service.libp2p.disconnect_and_ban_peer(
peer_id, peer_id,
std::time::Duration::from_secs(BAN_PEER_TIMEOUT), std::time::Duration::from_secs(BAN_PEER_TIMEOUT),
); );
} }
NetworkMessage::Subscribe { subscriptions } =>
{
// the result is dropped as it used solely for ergonomics
let _ = service.attestation_service.validator_subscriptions(subscriptions);
}
}, },
Ok(Async::NotReady) => break, Ok(Async::NotReady) => break,
Ok(Async::Ready(None)) => { Ok(Async::Ready(None)) => {
@ -244,45 +255,90 @@ fn spawn_service<T: BeaconChainTypes>(
} }
} }
// process any attestation service events
// NOTE: This must come after the network message processing as that may trigger events in
// the attestation service.
while let Ok(Async::Ready(Some(attestation_service_message))) = service.attestation_service.poll() {
match attestation_service_message {
// TODO: Implement
AttServiceMessage::Subscribe(subnet_id) => {
service.libp2p.swarm.subscribe_to_subnet(subnet_id);
},
AttServiceMessage::Unsubscribe(subnet_id) => {
service.libp2p.swarm.subscribe_to_subnet(subnet_id);
},
AttServiceMessage::EnrAdd(subnet_id) => {
service.libp2p.swarm.update_enr_subnet(subnet_id, true);
},
AttServiceMessage::EnrRemove(subnet_id) => {
service.libp2p.swarm.update_enr_subnet(subnet_id, false);
},
AttServiceMessage::DiscoverPeers(subnet_id) => {
service.libp2p.swarm.peers_request(subnet_id);
},
}
}
let mut peers_to_ban = Vec::new(); let mut peers_to_ban = Vec::new();
// poll the swarm // poll the swarm
loop { loop {
match libp2p_service.poll() { match service.libp2p.poll() {
Ok(Async::Ready(Some(event))) => match event { Ok(Async::Ready(Some(event))) => match event {
Libp2pEvent::RPC(peer_id, rpc_event) => { BehaviourEvent::RPC(peer_id, rpc_event) => {
// trace!(log, "Received RPC"; "rpc" => format!("{}", rpc_event));
// if we received a Goodbye message, drop and ban the peer // if we received a Goodbye message, drop and ban the peer
if let RPCEvent::Request(_, RPCRequest::Goodbye(_)) = rpc_event { if let RPCEvent::Request(_, RPCRequest::Goodbye(_)) = rpc_event {
peers_to_ban.push(peer_id.clone()); peers_to_ban.push(peer_id.clone());
}; };
message_handler_send service.router_send
.try_send(HandlerMessage::RPC(peer_id, rpc_event)) .try_send(RouterMessage::RPC(peer_id, rpc_event))
.map_err(|_| { debug!(log, "Failed to send RPC to handler");} )?; .map_err(|_| { debug!(log, "Failed to send RPC to router");} )?;
} }
Libp2pEvent::PeerDialed(peer_id) => { BehaviourEvent::PeerDialed(peer_id) => {
debug!(log, "Peer Dialed"; "peer_id" => format!("{:?}", peer_id)); debug!(log, "Peer Dialed"; "peer_id" => format!("{}", peer_id));
message_handler_send service.router_send
.try_send(HandlerMessage::PeerDialed(peer_id)) .try_send(RouterMessage::PeerDialed(peer_id))
.map_err(|_| { debug!(log, "Failed to send peer dialed to handler");})?; .map_err(|_| { debug!(log, "Failed to send peer dialed to router");})?;
} }
Libp2pEvent::PeerDisconnected(peer_id) => { BehaviourEvent::PeerDisconnected(peer_id) => {
debug!(log, "Peer Disconnected"; "peer_id" => format!("{:?}", peer_id)); debug!(log, "Peer Disconnected"; "peer_id" => format!("{}", peer_id));
message_handler_send service.router_send
.try_send(HandlerMessage::PeerDisconnected(peer_id)) .try_send(RouterMessage::PeerDisconnected(peer_id))
.map_err(|_| { debug!(log, "Failed to send peer disconnect to handler");})?; .map_err(|_| { debug!(log, "Failed to send peer disconnect to router");})?;
} }
Libp2pEvent::PubsubMessage { BehaviourEvent::StatusPeer(peer_id) => {
service.router_send
.try_send(RouterMessage::StatusPeer(peer_id))
.map_err(|_| { debug!(log, "Failed to send re-status peer to router");})?;
}
BehaviourEvent::PubsubMessage {
id, id,
source, source,
message, message,
.. ..
} => { } => {
message_handler_send
.try_send(HandlerMessage::PubsubMessage(id, source, message)) match message {
.map_err(|_| { debug!(log, "Failed to send pubsub message to handler");})?; // attestation information gets processed in the attestation service
PubsubMessage::Attestation(ref subnet_and_attestation) => {
let subnet = &subnet_and_attestation.0;
let attestation = &subnet_and_attestation.1;
// checks if we have an aggregator for the slot. If so, we process
// the attestation
if service.attestation_service.should_process_attestation(&id, &source, subnet, attestation) {
service.router_send
.try_send(RouterMessage::PubsubMessage(id, source, message))
.map_err(|_| { debug!(log, "Failed to send pubsub message to router");})?;
}
}
_ => {
// all else is sent to the router
service.router_send
.try_send(RouterMessage::PubsubMessage(id, source, message))
.map_err(|_| { debug!(log, "Failed to send pubsub message to router");})?;
}
}
} }
Libp2pEvent::PeerSubscribed(_, _) => {} BehaviourEvent::PeerSubscribed(_, _) => {}
}, },
Ok(Async::Ready(None)) => unreachable!("Stream never ends"), Ok(Async::Ready(None)) => unreachable!("Stream never ends"),
Ok(Async::NotReady) => break, Ok(Async::NotReady) => break,
@ -292,12 +348,22 @@ fn spawn_service<T: BeaconChainTypes>(
// ban and disconnect any peers that sent Goodbye requests // ban and disconnect any peers that sent Goodbye requests
while let Some(peer_id) = peers_to_ban.pop() { while let Some(peer_id) = peers_to_ban.pop() {
libp2p_service.disconnect_and_ban_peer( service.libp2p.disconnect_and_ban_peer(
peer_id.clone(), peer_id.clone(),
std::time::Duration::from_secs(BAN_PEER_TIMEOUT), std::time::Duration::from_secs(BAN_PEER_TIMEOUT),
); );
} }
// if we have just forked, update inform the libp2p layer
if let Some(mut update_fork_delay) = service.next_fork_update.take() {
if !update_fork_delay.is_elapsed() {
if let Ok(Async::Ready(_)) = update_fork_delay.poll() {
service.libp2p.swarm.update_fork_version(service.beacon_chain.enr_fork_id());
service.next_fork_update = next_fork_delay(&service.beacon_chain);
}
}
}
Ok(Async::NotReady) Ok(Async::NotReady)
}) })
@ -306,16 +372,29 @@ fn spawn_service<T: BeaconChainTypes>(
Ok(network_exit) Ok(network_exit)
} }
/// Returns a `Delay` that triggers shortly after the next change in the beacon chain fork version.
/// If there is no scheduled fork, `None` is returned.
fn next_fork_delay<T: BeaconChainTypes>(
beacon_chain: &BeaconChain<T>,
) -> Option<tokio::timer::Delay> {
beacon_chain.duration_to_next_fork().map(|until_fork| {
// Add a short time-out to start within the new fork period.
let delay = Duration::from_millis(200);
tokio::timer::Delay::new(Instant::now() + until_fork + delay)
})
}
/// Types of messages that the network service can receive. /// Types of messages that the network service can receive.
#[derive(Debug)] #[derive(Debug)]
pub enum NetworkMessage { pub enum NetworkMessage<T: EthSpec> {
/// Send an RPC message to the libp2p service. /// Subscribes a list of validators to specific slots for attestation duties.
RPC(PeerId, RPCEvent), Subscribe {
/// Publish a message to gossipsub. subscriptions: Vec<ValidatorSubscription>,
Publish {
topics: Vec<Topic>,
message: PubsubMessage,
}, },
/// Send an RPC message to the libp2p service.
RPC(PeerId, RPCEvent<T>),
/// Publish a list of messages to the gossipsub protocol.
Publish { messages: Vec<PubsubMessage<T>> },
/// Propagate a received gossipsub message. /// Propagate a received gossipsub message.
Propagate { Propagate {
propagation_source: PeerId, propagation_source: PeerId,

View File

@ -2,7 +2,7 @@
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use crate::persisted_dht::load_dht; use crate::persisted_dht::load_dht;
use crate::{NetworkConfig, Service}; use crate::{NetworkConfig, NetworkService};
use beacon_chain::test_utils::BeaconChainHarness; use beacon_chain::test_utils::BeaconChainHarness;
use eth2_libp2p::Enr; use eth2_libp2p::Enr;
use futures::{Future, IntoFuture}; use futures::{Future, IntoFuture};
@ -10,7 +10,6 @@ mod tests {
use sloggers::{null::NullLoggerBuilder, Build}; use sloggers::{null::NullLoggerBuilder, Build};
use std::str::FromStr; use std::str::FromStr;
use std::sync::Arc; use std::sync::Arc;
use store::MemoryStore;
use tokio::runtime::Runtime; use tokio::runtime::Runtime;
use types::{test_utils::generate_deterministic_keypairs, MinimalEthSpec}; use types::{test_utils::generate_deterministic_keypairs, MinimalEthSpec};
@ -44,14 +43,14 @@ mod tests {
.block_on_all( .block_on_all(
// Create a new network service which implicitly gets dropped at the // Create a new network service which implicitly gets dropped at the
// end of the block. // end of the block.
Service::new(beacon_chain.clone(), &config, &executor, log.clone()) NetworkService::start(beacon_chain.clone(), &config, &executor, log.clone())
.into_future() .into_future()
.and_then(move |(_service, _)| Ok(())), .and_then(move |(_globals, _service, _exit)| Ok(())),
) )
.unwrap(); .unwrap();
// Load the persisted dht from the store // Load the persisted dht from the store
let persisted_enrs = load_dht::<MemoryStore<MinimalEthSpec>, MinimalEthSpec>(store); let persisted_enrs = load_dht(store);
assert!( assert!(
persisted_enrs.contains(&enrs[0]), persisted_enrs.contains(&enrs[0]),
"should have persisted the first ENR to store" "should have persisted the first ENR to store"

View File

@ -1,7 +1,7 @@
use crate::message_processor::FUTURE_SLOT_TOLERANCE; use crate::router::processor::FUTURE_SLOT_TOLERANCE;
use crate::sync::manager::SyncMessage; use crate::sync::manager::SyncMessage;
use crate::sync::range_sync::BatchId; use crate::sync::range_sync::{BatchId, ChainId};
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, ChainSegmentResult};
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use slog::{debug, error, trace, warn}; use slog::{debug, error, trace, warn};
use std::sync::{Arc, Weak}; use std::sync::{Arc, Weak};
@ -12,7 +12,7 @@ use types::SignedBeaconBlock;
#[derive(Clone, Debug, PartialEq)] #[derive(Clone, Debug, PartialEq)]
pub enum ProcessId { pub enum ProcessId {
/// Processing Id of a range syncing batch. /// Processing Id of a range syncing batch.
RangeBatchId(BatchId), RangeBatchId(ChainId, BatchId),
/// Processing Id of the parent lookup of a block /// Processing Id of the parent lookup of a block
ParentLookup(PeerId), ParentLookup(PeerId),
} }
@ -25,6 +25,8 @@ pub enum BatchProcessResult {
Success, Success,
/// The batch processing failed. /// The batch processing failed.
Failed, Failed,
/// The batch processing failed but managed to import at least one block.
Partial,
} }
/// Spawns a thread handling the block processing of a request: range syncing or parent lookup. /// Spawns a thread handling the block processing of a request: range syncing or parent lookup.
@ -38,22 +40,28 @@ pub fn spawn_block_processor<T: BeaconChainTypes>(
std::thread::spawn(move || { std::thread::spawn(move || {
match process_id { match process_id {
// this a request from the range sync // this a request from the range sync
ProcessId::RangeBatchId(batch_id) => { ProcessId::RangeBatchId(chain_id, batch_id) => {
debug!(log, "Processing batch"; "id" => *batch_id, "blocks" => downloaded_blocks.len()); debug!(log, "Processing batch"; "id" => *batch_id, "blocks" => downloaded_blocks.len());
let result = match process_blocks(chain, downloaded_blocks.iter(), &log) { let result = match process_blocks(chain, downloaded_blocks.iter(), &log) {
Ok(_) => { (_, Ok(_)) => {
debug!(log, "Batch processed"; "id" => *batch_id ); debug!(log, "Batch processed"; "id" => *batch_id );
BatchProcessResult::Success BatchProcessResult::Success
} }
Err(e) => { (imported_blocks, Err(e)) if imported_blocks > 0 => {
debug!(log, "Batch processing failed but imported some blocks";
"id" => *batch_id, "error" => e, "imported_blocks"=> imported_blocks);
BatchProcessResult::Partial
}
(_, Err(e)) => {
debug!(log, "Batch processing failed"; "id" => *batch_id, "error" => e); debug!(log, "Batch processing failed"; "id" => *batch_id, "error" => e);
BatchProcessResult::Failed BatchProcessResult::Failed
} }
}; };
let msg = SyncMessage::BatchProcessed { let msg = SyncMessage::BatchProcessed {
batch_id: batch_id, chain_id,
downloaded_blocks: downloaded_blocks, batch_id,
downloaded_blocks,
result, result,
}; };
sync_send.try_send(msg).unwrap_or_else(|_| { sync_send.try_send(msg).unwrap_or_else(|_| {
@ -65,11 +73,15 @@ pub fn spawn_block_processor<T: BeaconChainTypes>(
} }
// this a parent lookup request from the sync manager // this a parent lookup request from the sync manager
ProcessId::ParentLookup(peer_id) => { ProcessId::ParentLookup(peer_id) => {
debug!(log, "Processing parent lookup"; "last_peer_id" => format!("{}", peer_id), "blocks" => downloaded_blocks.len()); debug!(
log, "Processing parent lookup";
"last_peer_id" => format!("{}", peer_id),
"blocks" => downloaded_blocks.len()
);
// parent blocks are ordered from highest slot to lowest, so we need to process in // parent blocks are ordered from highest slot to lowest, so we need to process in
// reverse // reverse
match process_blocks(chain, downloaded_blocks.iter().rev(), &log) { match process_blocks(chain, downloaded_blocks.iter().rev(), &log) {
Err(e) => { (_, Err(e)) => {
warn!(log, "Parent lookup failed"; "last_peer_id" => format!("{}", peer_id), "error" => e); warn!(log, "Parent lookup failed"; "last_peer_id" => format!("{}", peer_id), "error" => e);
sync_send sync_send
.try_send(SyncMessage::ParentLookupFailed(peer_id)) .try_send(SyncMessage::ParentLookupFailed(peer_id))
@ -81,7 +93,7 @@ pub fn spawn_block_processor<T: BeaconChainTypes>(
); );
}); });
} }
Ok(_) => { (_, Ok(_)) => {
debug!(log, "Parent lookup processed successfully"); debug!(log, "Parent lookup processed successfully");
} }
} }
@ -99,126 +111,40 @@ fn process_blocks<
chain: Weak<BeaconChain<T>>, chain: Weak<BeaconChain<T>>,
downloaded_blocks: I, downloaded_blocks: I,
log: &slog::Logger, log: &slog::Logger,
) -> Result<(), String> { ) -> (usize, Result<(), String>) {
let mut successful_block_import = false; if let Some(chain) = chain.upgrade() {
for block in downloaded_blocks { let blocks = downloaded_blocks.cloned().collect::<Vec<_>>();
if let Some(chain) = chain.upgrade() { let (imported_blocks, r) = match chain.process_chain_segment(blocks) {
let processing_result = chain.process_block(block.clone()); ChainSegmentResult::Successful { imported_blocks } => {
if imported_blocks == 0 {
if let Ok(outcome) = processing_result { debug!(log, "All blocks already known");
match outcome { } else {
BlockProcessingOutcome::Processed { block_root } => { debug!(
// The block was valid and we processed it successfully. log, "Imported blocks from network";
trace!( "count" => imported_blocks,
log, "Imported block from network"; );
"slot" => block.slot(), // Batch completed successfully with at least one block, run fork choice.
"block_root" => format!("{}", block_root),
);
successful_block_import = true;
}
BlockProcessingOutcome::ParentUnknown { parent, .. } => {
// blocks should be sequential and all parents should exist
// this is a failure if blocks do not have parents
warn!(
log, "Parent block is unknown";
"parent_root" => format!("{}", parent),
"baby_block_slot" => block.slot(),
);
if successful_block_import {
run_fork_choice(chain, log);
}
return Err(format!(
"Block at slot {} has an unknown parent.",
block.slot()
));
}
BlockProcessingOutcome::BlockIsAlreadyKnown => {
// this block is already known to us, move to the next
debug!(
log, "Imported a block that is already known";
"block_slot" => block.slot(),
);
}
BlockProcessingOutcome::FutureSlot {
present_slot,
block_slot,
} => {
if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot {
// The block is too far in the future, drop it.
warn!(
log, "Block is ahead of our slot clock";
"msg" => "block for future slot rejected, check your time",
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
);
if successful_block_import {
run_fork_choice(chain, log);
}
return Err(format!(
"Block at slot {} is too far in the future",
block.slot()
));
} else {
// The block is in the future, but not too far.
debug!(
log, "Block is slightly ahead of our slot clock, ignoring.";
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
);
}
}
BlockProcessingOutcome::WouldRevertFinalizedSlot { .. } => {
debug!(
log, "Finalized or earlier block processed";
"outcome" => format!("{:?}", outcome),
);
// block reached our finalized slot or was earlier, move to the next block
}
BlockProcessingOutcome::GenesisBlock => {
debug!(
log, "Genesis block was processed";
"outcome" => format!("{:?}", outcome),
);
}
_ => {
warn!(
log, "Invalid block received";
"msg" => "peer sent invalid block",
"outcome" => format!("{:?}", outcome),
);
if successful_block_import {
run_fork_choice(chain, log);
}
return Err(format!("Invalid block at slot {}", block.slot()));
}
}
} else {
warn!(
log, "BlockProcessingFailure";
"msg" => "unexpected condition in processing block.",
"outcome" => format!("{:?}", processing_result)
);
if successful_block_import {
run_fork_choice(chain, log); run_fork_choice(chain, log);
} }
return Err(format!(
"Unexpected block processing error: {:?}", (imported_blocks, Ok(()))
processing_result
));
} }
} else { ChainSegmentResult::Failed {
return Ok(()); // terminate early due to dropped beacon chain imported_blocks,
} error,
} => {
let r = handle_failed_chain_segment(error, log);
if imported_blocks > 0 {
run_fork_choice(chain, log);
}
(imported_blocks, r)
}
};
return (imported_blocks, r);
} }
// Batch completed successfully, run fork choice. (0, Ok(()))
if let Some(chain) = chain.upgrade() {
run_fork_choice(chain, log);
}
Ok(())
} }
/// Runs fork-choice on a given chain. This is used during block processing after one successful /// Runs fork-choice on a given chain. This is used during block processing after one successful
@ -238,3 +164,74 @@ fn run_fork_choice<T: BeaconChainTypes>(chain: Arc<BeaconChain<T>>, log: &slog::
), ),
} }
} }
/// Helper function to handle a `BlockError` from `process_chain_segment`
fn handle_failed_chain_segment(error: BlockError, log: &slog::Logger) -> Result<(), String> {
match error {
BlockError::ParentUnknown(parent) => {
// blocks should be sequential and all parents should exist
Err(format!("Block has an unknown parent: {}", parent))
}
BlockError::BlockIsAlreadyKnown => {
// This can happen for many reasons. Head sync's can download multiples and parent
// lookups can download blocks before range sync
Ok(())
}
BlockError::FutureSlot {
present_slot,
block_slot,
} => {
if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot {
// The block is too far in the future, drop it.
warn!(
log, "Block is ahead of our slot clock";
"msg" => "block for future slot rejected, check your time",
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
);
} else {
// The block is in the future, but not too far.
debug!(
log, "Block is slightly ahead of our slot clock, ignoring.";
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
);
}
Err(format!(
"Block with slot {} is higher than the current slot {}",
block_slot, present_slot
))
}
BlockError::WouldRevertFinalizedSlot { .. } => {
debug!( log, "Finalized or earlier block processed";);
Ok(())
}
BlockError::GenesisBlock => {
debug!(log, "Genesis block was processed");
Ok(())
}
BlockError::BeaconChainError(e) => {
warn!(
log, "BlockProcessingFailure";
"msg" => "unexpected condition in processing block.",
"outcome" => format!("{:?}", e)
);
Err(format!("Internal error whilst processing block: {:?}", e))
}
other => {
warn!(
log, "Invalid block received";
"msg" => "peer sent invalid block",
"outcome" => format!("{:?}", other),
);
Err(format!("Peer sent invalid block. Reason: {:?}", other))
}
}
}

View File

@ -35,31 +35,29 @@
use super::block_processor::{spawn_block_processor, BatchProcessResult, ProcessId}; use super::block_processor::{spawn_block_processor, BatchProcessResult, ProcessId};
use super::network_context::SyncNetworkContext; use super::network_context::SyncNetworkContext;
use super::range_sync::{BatchId, RangeSync}; use super::peer_sync_info::{PeerSyncInfo, PeerSyncType};
use crate::message_processor::PeerSyncInfo; use super::range_sync::{BatchId, ChainId, RangeSync};
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{methods::*, RequestId};
use eth2_libp2p::rpc::RequestId; use eth2_libp2p::types::NetworkGlobals;
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::prelude::*; use futures::prelude::*;
use rand::seq::SliceRandom;
use slog::{crit, debug, error, info, trace, warn, Logger}; use slog::{crit, debug, error, info, trace, warn, Logger};
use smallvec::SmallVec; use smallvec::SmallVec;
use std::boxed::Box; use std::boxed::Box;
use std::collections::HashSet;
use std::ops::Sub; use std::ops::Sub;
use std::sync::Weak; use std::sync::Arc;
use tokio::sync::{mpsc, oneshot}; use tokio::sync::{mpsc, oneshot};
use types::{EthSpec, Hash256, SignedBeaconBlock}; use types::{EthSpec, Hash256, SignedBeaconBlock, Slot};
/// The number of slots ahead of us that is allowed before requesting a long-range (batch) Sync /// The number of slots ahead of us that is allowed before requesting a long-range (batch) Sync
/// from a peer. If a peer is within this tolerance (forwards or backwards), it is treated as a /// from a peer. If a peer is within this tolerance (forwards or backwards), it is treated as a
/// fully sync'd peer. /// fully sync'd peer.
const SLOT_IMPORT_TOLERANCE: usize = 20; pub const SLOT_IMPORT_TOLERANCE: usize = 20;
/// How many attempts we try to find a parent of a block before we give up trying . /// How many attempts we try to find a parent of a block before we give up trying .
const PARENT_FAIL_TOLERANCE: usize = 3; const PARENT_FAIL_TOLERANCE: usize = 5;
/// The maximum depth we will search for a parent block. In principle we should have sync'd any /// The maximum depth we will search for a parent block. In principle we should have sync'd any
/// canonical chain to its head once the peer connects. A chain should not appear where it's depth /// canonical chain to its head once the peer connects. A chain should not appear where it's depth
/// is further back than the most recent head slot. /// is further back than the most recent head slot.
@ -100,6 +98,7 @@ pub enum SyncMessage<T: EthSpec> {
/// A batch has been processed by the block processor thread. /// A batch has been processed by the block processor thread.
BatchProcessed { BatchProcessed {
chain_id: ChainId,
batch_id: BatchId, batch_id: BatchId,
downloaded_blocks: Vec<SignedBeaconBlock<T>>, downloaded_blocks: Vec<SignedBeaconBlock<T>>,
result: BatchProcessResult, result: BatchProcessResult,
@ -126,38 +125,22 @@ struct ParentRequests<T: EthSpec> {
pending: Option<RequestId>, pending: Option<RequestId>,
} }
#[derive(PartialEq, Debug, Clone)]
/// The current state of the `ImportManager`.
enum ManagerState {
/// The manager is performing a long-range (batch) sync. In this mode, parent lookups are
/// disabled.
Syncing,
/// The manager is up to date with all known peers and is connected to at least one
/// fully-syncing peer. In this state, parent lookups are enabled.
Regular,
/// No useful peers are connected. Long-range sync's cannot proceed and we have no useful
/// peers to download parents for. More peers need to be connected before we can proceed.
Stalled,
}
/// The primary object for handling and driving all the current syncing logic. It maintains the /// The primary object for handling and driving all the current syncing logic. It maintains the
/// current state of the syncing process, the number of useful peers, downloaded blocks and /// current state of the syncing process, the number of useful peers, downloaded blocks and
/// controls the logic behind both the long-range (batch) sync and the on-going potential parent /// controls the logic behind both the long-range (batch) sync and the on-going potential parent
/// look-up of blocks. /// look-up of blocks.
pub struct SyncManager<T: BeaconChainTypes> { pub struct SyncManager<T: BeaconChainTypes> {
/// A weak reference to the underlying beacon chain. /// A reference to the underlying beacon chain.
chain: Weak<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
/// The current state of the import manager. /// A reference to the network globals and peer-db.
state: ManagerState, network_globals: Arc<NetworkGlobals<T::EthSpec>>,
/// A receiving channel sent by the message processor thread. /// A receiving channel sent by the message processor thread.
input_channel: mpsc::UnboundedReceiver<SyncMessage<T::EthSpec>>, input_channel: mpsc::UnboundedReceiver<SyncMessage<T::EthSpec>>,
/// A network context to contact the network service. /// A network context to contact the network service.
network: SyncNetworkContext, network: SyncNetworkContext<T::EthSpec>,
/// The object handling long-range batch load-balanced syncing. /// The object handling long-range batch load-balanced syncing.
range_sync: RangeSync<T>, range_sync: RangeSync<T>,
@ -169,10 +152,7 @@ pub struct SyncManager<T: BeaconChainTypes> {
/// received or not. /// received or not.
/// ///
/// The flag allows us to determine if the peer returned data or sent us nothing. /// The flag allows us to determine if the peer returned data or sent us nothing.
single_block_lookups: FnvHashMap<RequestId, (Hash256, bool)>, single_block_lookups: FnvHashMap<RequestId, SingleBlockRequest>,
/// The collection of known, connected, fully-sync'd peers.
full_peers: HashSet<PeerId>,
/// The logger for the import manager. /// The logger for the import manager.
log: Logger, log: Logger,
@ -181,13 +161,31 @@ pub struct SyncManager<T: BeaconChainTypes> {
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
} }
/// Object representing a single block lookup request.
struct SingleBlockRequest {
/// The hash of the requested block.
pub hash: Hash256,
/// Whether a block was received from this request, or the peer returned an empty response.
pub block_returned: bool,
}
impl SingleBlockRequest {
pub fn new(hash: Hash256) -> Self {
Self {
hash,
block_returned: false,
}
}
}
/// Spawns a new `SyncManager` thread which has a weak reference to underlying beacon /// Spawns a new `SyncManager` thread which has a weak reference to underlying beacon
/// chain. This allows the chain to be /// chain. This allows the chain to be
/// dropped during the syncing process which will gracefully end the `SyncManager`. /// dropped during the syncing process which will gracefully end the `SyncManager`.
pub fn spawn<T: BeaconChainTypes>( pub fn spawn<T: BeaconChainTypes>(
executor: &tokio::runtime::TaskExecutor, executor: &tokio::runtime::TaskExecutor,
beacon_chain: Weak<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_send: mpsc::UnboundedSender<NetworkMessage>, network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
log: slog::Logger, log: slog::Logger,
) -> ( ) -> (
mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
@ -200,14 +198,18 @@ pub fn spawn<T: BeaconChainTypes>(
// create an instance of the SyncManager // create an instance of the SyncManager
let sync_manager = SyncManager { let sync_manager = SyncManager {
chain: beacon_chain.clone(), range_sync: RangeSync::new(
state: ManagerState::Stalled, beacon_chain.clone(),
input_channel: sync_recv, network_globals.clone(),
sync_send.clone(),
log.clone(),
),
network: SyncNetworkContext::new(network_send, log.clone()), network: SyncNetworkContext::new(network_send, log.clone()),
range_sync: RangeSync::new(beacon_chain, sync_send.clone(), log.clone()), chain: beacon_chain,
network_globals,
input_channel: sync_recv,
parent_queue: SmallVec::new(), parent_queue: SmallVec::new(),
single_block_lookups: FnvHashMap::default(), single_block_lookups: FnvHashMap::default(),
full_peers: HashSet::new(),
log: log.clone(), log: log.clone(),
sync_send: sync_send.clone(), sync_send: sync_send.clone(),
}; };
@ -239,17 +241,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
/// ours that we consider it fully sync'd with respect to our current chain. /// ours that we consider it fully sync'd with respect to our current chain.
fn add_peer(&mut self, peer_id: PeerId, remote: PeerSyncInfo) { fn add_peer(&mut self, peer_id: PeerId, remote: PeerSyncInfo) {
// ensure the beacon chain still exists // ensure the beacon chain still exists
let chain = match self.chain.upgrade() { let local_peer_info = match PeerSyncInfo::from_chain(&self.chain) {
Some(chain) => chain,
None => {
warn!(self.log,
"Beacon chain dropped. Peer not considered for sync";
"peer_id" => format!("{:?}", peer_id));
return;
}
};
let local = match PeerSyncInfo::from_chain(&chain) {
Some(local) => local, Some(local) => local,
None => { None => {
return error!( return error!(
@ -260,36 +252,34 @@ impl<T: BeaconChainTypes> SyncManager<T> {
} }
}; };
// If a peer is within SLOT_IMPORT_TOLERANCE from our head slot, ignore a batch/range sync, match local_peer_info.peer_sync_type(&remote) {
// consider it a fully-sync'd peer. PeerSyncType::FullySynced => {
if remote.head_slot.sub(local.head_slot).as_usize() < SLOT_IMPORT_TOLERANCE { trace!(self.log, "Peer synced to our head found";
trace!(self.log, "Ignoring full sync with peer";
"peer" => format!("{:?}", peer_id),
"peer_head_slot" => remote.head_slot,
"local_head_slot" => local.head_slot,
);
self.add_full_peer(peer_id);
// notify the range sync that a peer has been added
self.range_sync.fully_synced_peer_found();
return;
}
// Check if the peer is significantly behind us. If within `SLOT_IMPORT_TOLERANCE`
// treat them as a fully synced peer. If not, ignore them in the sync process
if local.head_slot.sub(remote.head_slot).as_usize() < SLOT_IMPORT_TOLERANCE {
self.add_full_peer(peer_id.clone());
} else {
debug!(
self.log,
"Out of sync peer connected";
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
); "peer_head_slot" => remote.head_slot,
return; "local_head_slot" => local_peer_info.head_slot,
);
self.synced_peer(&peer_id, remote);
// notify the range sync that a peer has been added
self.range_sync.fully_synced_peer_found();
}
PeerSyncType::Advanced => {
trace!(self.log, "Useful peer for sync found";
"peer" => format!("{:?}", peer_id),
"peer_head_slot" => remote.head_slot,
"local_head_slot" => local_peer_info.head_slot,
"remote_finalized_epoch" => local_peer_info.finalized_epoch,
"local_finalized_epoch" => remote.finalized_epoch,
);
// Add the peer to our RangeSync
self.range_sync
.add_peer(&mut self.network, peer_id.clone(), remote);
self.advanced_peer(&peer_id, remote);
}
PeerSyncType::Behind => {
self.behind_peer(&peer_id, remote);
}
} }
// Add the peer to our RangeSync
self.range_sync.add_peer(&mut self.network, peer_id, remote);
self.update_state();
} }
/// The response to a `BlocksByRoot` request. /// The response to a `BlocksByRoot` request.
@ -310,12 +300,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// check if this is a single block lookup - i.e we were searching for a specific hash // check if this is a single block lookup - i.e we were searching for a specific hash
let mut single_block_hash = None; let mut single_block_hash = None;
if let Some((block_hash, data_received)) = if let Some(block_request) = self.single_block_lookups.get_mut(&request_id) {
self.single_block_lookups.get_mut(&request_id)
{
// update the state of the lookup indicating a block was received from the peer // update the state of the lookup indicating a block was received from the peer
*data_received = true; block_request.block_returned = true;
single_block_hash = Some(block_hash.clone()); single_block_hash = Some(block_request.hash.clone());
} }
if let Some(block_hash) = single_block_hash { if let Some(block_hash) = single_block_hash {
self.single_block_lookup_response(peer_id, block, block_hash); self.single_block_lookup_response(peer_id, block, block_hash);
@ -346,12 +334,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// this is a stream termination // this is a stream termination
// stream termination for a single block lookup, remove the key // stream termination for a single block lookup, remove the key
if let Some((block_hash, data_received)) = if let Some(single_block_request) = self.single_block_lookups.remove(&request_id) {
self.single_block_lookups.remove(&request_id)
{
// the peer didn't respond with a block that it referenced // the peer didn't respond with a block that it referenced
if !data_received { if !single_block_request.block_returned {
warn!(self.log, "Peer didn't respond with a block it referenced"; "referenced_block_hash" => format!("{}", block_hash), "peer_id" => format!("{}", peer_id)); warn!(self.log, "Peer didn't respond with a block it referenced"; "referenced_block_hash" => format!("{}", single_block_request.hash), "peer_id" => format!("{}", peer_id));
self.network.downvote_peer(peer_id); self.network.downvote_peer(peer_id);
} }
return; return;
@ -398,43 +384,41 @@ impl<T: BeaconChainTypes> SyncManager<T> {
} }
// we have the correct block, try and process it // we have the correct block, try and process it
if let Some(chain) = self.chain.upgrade() { match BlockProcessingOutcome::shim(self.chain.process_block(block.clone())) {
match chain.process_block(block.clone()) { Ok(outcome) => {
Ok(outcome) => { match outcome {
match outcome { BlockProcessingOutcome::Processed { block_root } => {
BlockProcessingOutcome::Processed { block_root } => { info!(self.log, "Processed block"; "block" => format!("{}", block_root));
info!(self.log, "Processed block"; "block" => format!("{}", block_root));
match chain.fork_choice() { match self.chain.fork_choice() {
Ok(()) => trace!( Ok(()) => trace!(
self.log, self.log,
"Fork choice success"; "Fork choice success";
"location" => "single block" "location" => "single block"
), ),
Err(e) => error!( Err(e) => error!(
self.log, self.log,
"Fork choice failed"; "Fork choice failed";
"error" => format!("{:?}", e), "error" => format!("{:?}", e),
"location" => "single block" "location" => "single block"
), ),
}
}
BlockProcessingOutcome::ParentUnknown { .. } => {
// We don't know of the blocks parent, begin a parent lookup search
self.add_unknown_block(peer_id, block);
}
BlockProcessingOutcome::BlockIsAlreadyKnown => {
trace!(self.log, "Single block lookup already known");
}
_ => {
warn!(self.log, "Single block lookup failed"; "outcome" => format!("{:?}", outcome));
self.network.downvote_peer(peer_id);
} }
} }
BlockProcessingOutcome::ParentUnknown { .. } => {
// We don't know of the blocks parent, begin a parent lookup search
self.add_unknown_block(peer_id, block);
}
BlockProcessingOutcome::BlockIsAlreadyKnown => {
trace!(self.log, "Single block lookup already known");
}
_ => {
warn!(self.log, "Single block lookup failed"; "outcome" => format!("{:?}", outcome));
self.network.downvote_peer(peer_id);
}
} }
Err(e) => { }
warn!(self.log, "Unexpected block processing error"; "error" => format!("{:?}", e)); Err(e) => {
} warn!(self.log, "Unexpected block processing error"; "error" => format!("{:?}", e));
} }
} }
} }
@ -442,9 +426,24 @@ impl<T: BeaconChainTypes> SyncManager<T> {
/// A block has been sent to us that has an unknown parent. This begins a parent lookup search /// A block has been sent to us that has an unknown parent. This begins a parent lookup search
/// to find the parent or chain of parents that match our current chain. /// to find the parent or chain of parents that match our current chain.
fn add_unknown_block(&mut self, peer_id: PeerId, block: SignedBeaconBlock<T::EthSpec>) { fn add_unknown_block(&mut self, peer_id: PeerId, block: SignedBeaconBlock<T::EthSpec>) {
// If we are not in regular sync mode, ignore this block // If we are not synced or within SLOT_IMPORT_TOLERANCE of the block, ignore
if self.state != ManagerState::Regular { if !self.network_globals.sync_state.read().is_synced() {
return; let head_slot = self
.chain
.head_info()
.map(|info| info.slot)
.unwrap_or_else(|_| Slot::from(0u64));
let unknown_block_slot = block.message.slot;
// if the block is far in the future, ignore it. If its within the slot tolerance of
// our current head, regardless of the syncing state, fetch it.
if (head_slot >= unknown_block_slot
&& head_slot.sub(unknown_block_slot).as_usize() > SLOT_IMPORT_TOLERANCE)
|| (head_slot < unknown_block_slot
&& unknown_block_slot.sub(head_slot).as_usize() > SLOT_IMPORT_TOLERANCE)
{
return;
}
} }
// Make sure this block is not already being searched for // Make sure this block is not already being searched for
@ -473,8 +472,18 @@ impl<T: BeaconChainTypes> SyncManager<T> {
/// A request to search for a block hash has been received. This function begins a BlocksByRoot /// A request to search for a block hash has been received. This function begins a BlocksByRoot
/// request to find the requested block. /// request to find the requested block.
fn search_for_block(&mut self, peer_id: PeerId, block_hash: Hash256) { fn search_for_block(&mut self, peer_id: PeerId, block_hash: Hash256) {
// If we are not in regular sync mode, ignore this block // If we are not synced, ignore this block
if self.state != ManagerState::Regular { if !self.network_globals.sync_state.read().is_synced() {
return;
}
// Do not re-request a block that is already being requested
if self
.single_block_lookups
.values()
.find(|single_block_request| single_block_request.hash == block_hash)
.is_some()
{
return; return;
} }
@ -484,7 +493,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
if let Ok(request_id) = self.network.blocks_by_root_request(peer_id, request) { if let Ok(request_id) = self.network.blocks_by_root_request(peer_id, request) {
self.single_block_lookups self.single_block_lookups
.insert(request_id, (block_hash, false)); .insert(request_id, SingleBlockRequest::new(block_hash));
} }
} }
@ -516,42 +525,57 @@ impl<T: BeaconChainTypes> SyncManager<T> {
fn peer_disconnect(&mut self, peer_id: &PeerId) { fn peer_disconnect(&mut self, peer_id: &PeerId) {
self.range_sync.peer_disconnect(&mut self.network, peer_id); self.range_sync.peer_disconnect(&mut self.network, peer_id);
self.full_peers.remove(peer_id); self.update_sync_state();
self.update_state();
} }
fn add_full_peer(&mut self, peer_id: PeerId) { /// Updates the syncing state of a peer to be synced.
debug!( fn synced_peer(&mut self, peer_id: &PeerId, sync_info: PeerSyncInfo) {
self.log, "Fully synced peer added"; if let Some(peer_info) = self.network_globals.peers.write().peer_info_mut(peer_id) {
"peer" => format!("{:?}", peer_id), if peer_info.sync_status.update_synced(sync_info.into()) {
); debug!(self.log, "Peer transitioned to synced status"; "peer_id" => format!("{}", peer_id));
self.full_peers.insert(peer_id); }
} else {
crit!(self.log, "Status'd peer is unknown"; "peer_id" => format!("{}", peer_id));
}
self.update_sync_state();
} }
/// Updates the syncing state of a peer to be behind.
fn advanced_peer(&mut self, peer_id: &PeerId, sync_info: PeerSyncInfo) {
if let Some(peer_info) = self.network_globals.peers.write().peer_info_mut(peer_id) {
let advanced_slot = sync_info.head_slot;
if peer_info.sync_status.update_ahead(sync_info.into()) {
debug!(self.log, "Peer transitioned to from synced state to ahead"; "peer_id" => format!("{}", peer_id), "head_slot" => advanced_slot);
}
} else {
crit!(self.log, "Status'd peer is unknown"; "peer_id" => format!("{}", peer_id));
}
self.update_sync_state();
}
/// Updates the syncing state of a peer to be behind.
fn behind_peer(&mut self, peer_id: &PeerId, sync_info: PeerSyncInfo) {
if let Some(peer_info) = self.network_globals.peers.write().peer_info_mut(peer_id) {
let behind_slot = sync_info.head_slot;
if peer_info.sync_status.update_behind(sync_info.into()) {
debug!(self.log, "Peer transitioned to from synced state to behind"; "peer_id" => format!("{}", peer_id), "head_slot" => behind_slot);
}
} else {
crit!(self.log, "Status'd peer is unknown"; "peer_id" => format!("{}", peer_id));
}
self.update_sync_state();
}
/// Updates the global sync state and logs any changes.
fn update_sync_state(&mut self) {
if let Some((old_state, new_state)) = self.network_globals.update_sync_state() {
info!(self.log, "Sync state updated"; "old_state" => format!("{}", old_state), "new_state" => format!("{}",new_state));
}
}
/* Processing State Functions */ /* Processing State Functions */
// These functions are called in the main poll function to transition the state of the sync // These functions are called in the main poll function to transition the state of the sync
// manager // manager
/// Updates the syncing state of the `SyncManager`.
fn update_state(&mut self) {
let previous_state = self.state.clone();
self.state = {
if self.range_sync.is_syncing() {
ManagerState::Syncing
} else if !self.full_peers.is_empty() {
ManagerState::Regular
} else {
ManagerState::Stalled
}
};
if self.state != previous_state {
info!(self.log, "Syncing state updated";
"old_state" => format!("{:?}", previous_state),
"new_state" => format!("{:?}", self.state),
);
}
}
/// A new block has been received for a parent lookup query, process it. /// A new block has been received for a parent lookup query, process it.
fn process_parent_request(&mut self, mut parent_request: ParentRequests<T::EthSpec>) { fn process_parent_request(&mut self, mut parent_request: ParentRequests<T::EthSpec>) {
// verify the last added block is the parent of the last requested block // verify the last added block is the parent of the last requested block
@ -598,55 +622,50 @@ impl<T: BeaconChainTypes> SyncManager<T> {
// If the last block in the queue has an unknown parent, we continue the parent // If the last block in the queue has an unknown parent, we continue the parent
// lookup-search. // lookup-search.
if let Some(chain) = self.chain.upgrade() { let newest_block = parent_request
let newest_block = parent_request .downloaded_blocks
.downloaded_blocks .pop()
.pop() .expect("There is always at least one block in the queue");
.expect("There is always at least one block in the queue"); match BlockProcessingOutcome::shim(self.chain.process_block(newest_block.clone())) {
match chain.process_block(newest_block.clone()) { Ok(BlockProcessingOutcome::ParentUnknown { .. }) => {
Ok(BlockProcessingOutcome::ParentUnknown { .. }) => { // need to keep looking for parents
// need to keep looking for parents // add the block back to the queue and continue the search
// add the block back to the queue and continue the search parent_request.downloaded_blocks.push(newest_block);
parent_request.downloaded_blocks.push(newest_block); self.request_parent(parent_request);
self.request_parent(parent_request); return;
return; }
} Ok(BlockProcessingOutcome::Processed { .. })
Ok(BlockProcessingOutcome::Processed { .. }) | Ok(BlockProcessingOutcome::BlockIsAlreadyKnown { .. }) => {
| Ok(BlockProcessingOutcome::BlockIsAlreadyKnown { .. }) => { spawn_block_processor(
spawn_block_processor( Arc::downgrade(&self.chain),
self.chain.clone(), ProcessId::ParentLookup(parent_request.last_submitted_peer.clone()),
ProcessId::ParentLookup(parent_request.last_submitted_peer.clone()), parent_request.downloaded_blocks,
parent_request.downloaded_blocks, self.sync_send.clone(),
self.sync_send.clone(), self.log.clone(),
self.log.clone(), );
); }
} Ok(outcome) => {
Ok(outcome) => { // all else we consider the chain a failure and downvote the peer that sent
// all else we consider the chain a failure and downvote the peer that sent // us the last block
// us the last block warn!(
warn!( self.log, "Invalid parent chain. Downvoting peer";
self.log, "Invalid parent chain. Downvoting peer"; "outcome" => format!("{:?}", outcome),
"outcome" => format!("{:?}", outcome), "last_peer" => format!("{:?}", parent_request.last_submitted_peer),
"last_peer" => format!("{:?}", parent_request.last_submitted_peer), );
); self.network
self.network .downvote_peer(parent_request.last_submitted_peer.clone());
.downvote_peer(parent_request.last_submitted_peer.clone()); return;
return; }
} Err(e) => {
Err(e) => { warn!(
warn!( self.log, "Parent chain processing error. Downvoting peer";
self.log, "Parent chain processing error. Downvoting peer"; "error" => format!("{:?}", e),
"error" => format!("{:?}", e), "last_peer" => format!("{:?}", parent_request.last_submitted_peer),
"last_peer" => format!("{:?}", parent_request.last_submitted_peer), );
); self.network
self.network .downvote_peer(parent_request.last_submitted_peer.clone());
.downvote_peer(parent_request.last_submitted_peer.clone()); return;
return;
}
} }
} else {
// chain doesn't exist, drop the parent queue and return
return;
} }
} }
} }
@ -660,9 +679,16 @@ impl<T: BeaconChainTypes> SyncManager<T> {
if parent_request.failed_attempts >= PARENT_FAIL_TOLERANCE if parent_request.failed_attempts >= PARENT_FAIL_TOLERANCE
|| parent_request.downloaded_blocks.len() >= PARENT_DEPTH_TOLERANCE || parent_request.downloaded_blocks.len() >= PARENT_DEPTH_TOLERANCE
{ {
let error = if parent_request.failed_attempts >= PARENT_FAIL_TOLERANCE {
"too many failed attempts"
} else {
"reached maximum lookup-depth"
};
debug!(self.log, "Parent import failed"; debug!(self.log, "Parent import failed";
"block" => format!("{:?}",parent_request.downloaded_blocks[0].canonical_root()), "block" => format!("{:?}",parent_request.downloaded_blocks[0].canonical_root()),
"ancestors_found" => parent_request.downloaded_blocks.len() "ancestors_found" => parent_request.downloaded_blocks.len(),
"reason" => error
); );
return; // drop the request return; // drop the request
} }
@ -677,14 +703,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
let request = BlocksByRootRequest { let request = BlocksByRootRequest {
block_roots: vec![parent_hash], block_roots: vec![parent_hash],
}; };
// select a random fully synced peer to attempt to download the parent block
let available_peers = self.full_peers.iter().collect::<Vec<_>>(); // We continue to search for the chain of blocks from the same peer. Other peers are not
let peer_id = if let Some(peer_id) = available_peers.choose(&mut rand::thread_rng()) { // guaranteed to have this chain of blocks.
(**peer_id).clone() let peer_id = parent_request.last_submitted_peer.clone();
} else {
// there were no peers to choose from. We drop the lookup request
return;
};
if let Ok(request_id) = self.network.blocks_by_root_request(peer_id, request) { if let Ok(request_id) = self.network.blocks_by_root_request(peer_id, request) {
// if the request was successful add the queue back into self // if the request was successful add the queue back into self
@ -738,12 +760,14 @@ impl<T: BeaconChainTypes> Future for SyncManager<T> {
self.inject_error(peer_id, request_id); self.inject_error(peer_id, request_id);
} }
SyncMessage::BatchProcessed { SyncMessage::BatchProcessed {
chain_id,
batch_id, batch_id,
downloaded_blocks, downloaded_blocks,
result, result,
} => { } => {
self.range_sync.handle_block_process_result( self.range_sync.handle_block_process_result(
&mut self.network, &mut self.network,
chain_id,
batch_id, batch_id,
downloaded_blocks, downloaded_blocks,
result, result,
@ -763,9 +787,6 @@ impl<T: BeaconChainTypes> Future for SyncManager<T> {
} }
} }
// update the state of the manager
self.update_state();
Ok(Async::NotReady) Ok(Async::NotReady)
} }
} }

View File

@ -4,11 +4,8 @@
mod block_processor; mod block_processor;
pub mod manager; pub mod manager;
mod network_context; mod network_context;
mod peer_sync_info;
mod range_sync; mod range_sync;
/// Currently implemented sync methods.
pub enum SyncMethod {
SimpleSync,
}
pub use manager::SyncMessage; pub use manager::SyncMessage;
pub use peer_sync_info::PeerSyncInfo;

View File

@ -1,56 +1,55 @@
//! Provides network functionality for the Syncing thread. This fundamentally wraps a network //! Provides network functionality for the Syncing thread. This fundamentally wraps a network
//! channel and stores a global RPC ID to perform requests. //! channel and stores a global RPC ID to perform requests.
use crate::message_processor::status_message; use crate::router::processor::status_message;
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::{RPCEvent, RPCRequest, RequestId}; use eth2_libp2p::rpc::{RPCEvent, RPCRequest, RequestId};
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use slog::{debug, trace, warn}; use slog::{debug, trace, warn};
use std::sync::Weak; use std::sync::Arc;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::EthSpec;
/// Wraps a Network channel to employ various RPC related network functionality for the Sync manager. This includes management of a global RPC request Id. /// Wraps a Network channel to employ various RPC related network functionality for the Sync manager. This includes management of a global RPC request Id.
pub struct SyncNetworkContext { pub struct SyncNetworkContext<T: EthSpec> {
/// The network channel to relay messages to the Network service. /// The network channel to relay messages to the Network service.
network_send: mpsc::UnboundedSender<NetworkMessage>, network_send: mpsc::UnboundedSender<NetworkMessage<T>>,
request_id: RequestId, request_id: RequestId,
/// Logger for the `SyncNetworkContext`. /// Logger for the `SyncNetworkContext`.
log: slog::Logger, log: slog::Logger,
} }
impl SyncNetworkContext { impl<T: EthSpec> SyncNetworkContext<T> {
pub fn new(network_send: mpsc::UnboundedSender<NetworkMessage>, log: slog::Logger) -> Self { pub fn new(network_send: mpsc::UnboundedSender<NetworkMessage<T>>, log: slog::Logger) -> Self {
Self { Self {
network_send, network_send,
request_id: 0, request_id: 1,
log, log,
} }
} }
pub fn status_peer<T: BeaconChainTypes>( pub fn status_peer<U: BeaconChainTypes>(
&mut self, &mut self,
chain: Weak<BeaconChain<T>>, chain: Arc<BeaconChain<U>>,
peer_id: PeerId, peer_id: PeerId,
) { ) {
if let Some(chain) = chain.upgrade() { if let Some(status_message) = status_message(&chain) {
if let Some(status_message) = status_message(&chain) { debug!(
debug!( self.log,
self.log, "Sending Status Request";
"Sending Status Request"; "peer" => format!("{:?}", peer_id),
"peer" => format!("{:?}", peer_id), "fork_digest" => format!("{:?}", status_message.fork_digest),
"fork_version" => format!("{:?}", status_message.fork_version), "finalized_root" => format!("{:?}", status_message.finalized_root),
"finalized_root" => format!("{:?}", status_message.finalized_root), "finalized_epoch" => format!("{:?}", status_message.finalized_epoch),
"finalized_epoch" => format!("{:?}", status_message.finalized_epoch), "head_root" => format!("{}", status_message.head_root),
"head_root" => format!("{}", status_message.head_root), "head_slot" => format!("{}", status_message.head_slot),
"head_slot" => format!("{}", status_message.head_slot), );
);
let _ = self.send_rpc_request(peer_id, RPCRequest::Status(status_message)); let _ = self.send_rpc_request(peer_id, RPCRequest::Status(status_message));
}
} }
} }
@ -117,7 +116,7 @@ impl SyncNetworkContext {
pub fn send_rpc_request( pub fn send_rpc_request(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
rpc_request: RPCRequest, rpc_request: RPCRequest<T>,
) -> Result<RequestId, &'static str> { ) -> Result<RequestId, &'static str> {
let request_id = self.request_id; let request_id = self.request_id;
self.request_id += 1; self.request_id += 1;
@ -125,7 +124,11 @@ impl SyncNetworkContext {
Ok(request_id) Ok(request_id)
} }
fn send_rpc_event(&mut self, peer_id: PeerId, rpc_event: RPCEvent) -> Result<(), &'static str> { fn send_rpc_event(
&mut self,
peer_id: PeerId,
rpc_event: RPCEvent<T>,
) -> Result<(), &'static str> {
self.network_send self.network_send
.try_send(NetworkMessage::RPC(peer_id, rpc_event)) .try_send(NetworkMessage::RPC(peer_id, rpc_event))
.map_err(|_| { .map_err(|_| {

View File

@ -0,0 +1,114 @@
use super::manager::SLOT_IMPORT_TOLERANCE;
use crate::router::processor::status_message;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::SyncInfo;
use std::ops::Sub;
use std::sync::Arc;
use types::{Epoch, Hash256, Slot};
///
/// Keeps track of syncing information for known connected peers.
#[derive(Clone, Copy, Debug)]
pub struct PeerSyncInfo {
pub fork_digest: [u8; 4],
pub finalized_root: Hash256,
pub finalized_epoch: Epoch,
pub head_root: Hash256,
pub head_slot: Slot,
}
/// The type of peer relative to our current state.
pub enum PeerSyncType {
/// The peer is on our chain and is fully synced with respect to our chain.
FullySynced,
/// The peer has a greater knowledge of the chain that us that warrants a full sync.
Advanced,
/// A peer is behind in the sync and not useful to us for downloading blocks.
Behind,
}
impl From<StatusMessage> for PeerSyncInfo {
fn from(status: StatusMessage) -> PeerSyncInfo {
PeerSyncInfo {
fork_digest: status.fork_digest,
finalized_root: status.finalized_root,
finalized_epoch: status.finalized_epoch,
head_root: status.head_root,
head_slot: status.head_slot,
}
}
}
impl Into<SyncInfo> for PeerSyncInfo {
fn into(self) -> SyncInfo {
SyncInfo {
status_head_slot: self.head_slot,
status_head_root: self.head_root,
status_finalized_epoch: self.finalized_epoch,
status_finalized_root: self.finalized_root,
}
}
}
impl PeerSyncInfo {
/// Derives the peer sync information from a beacon chain.
pub fn from_chain<T: BeaconChainTypes>(chain: &Arc<BeaconChain<T>>) -> Option<PeerSyncInfo> {
Some(Self::from(status_message(chain)?))
}
/// Given another peer's `PeerSyncInfo` this will determine how useful that peer is for us in
/// regards to syncing. This returns the peer sync type that can then be handled by the
/// `SyncManager`.
pub fn peer_sync_type(&self, remote_peer_sync_info: &PeerSyncInfo) -> PeerSyncType {
// check if the peer is fully synced with our current chain
if self.is_fully_synced_peer(remote_peer_sync_info) {
PeerSyncType::FullySynced
}
// if not, check if the peer is ahead of our chain
else if self.is_ahead_peer(remote_peer_sync_info) {
PeerSyncType::Advanced
} else {
// the peer must be behind and not useful
PeerSyncType::Behind
}
}
/// Determines if another peer is fully synced with the current peer.
///
/// A fully synced peer is a peer whose finalized epoch and hash match our own and their
/// head is within SLOT_IMPORT_TOLERANCE of our own.
/// In this case we ignore any batch/range syncing.
fn is_fully_synced_peer(&self, remote: &PeerSyncInfo) -> bool {
// ensure we are on the same chain, with minor differing heads
if remote.finalized_epoch == self.finalized_epoch
&& remote.finalized_root == self.finalized_root
{
// that we are within SLOT_IMPORT_TOLERANCE of our two heads
if (self.head_slot >= remote.head_slot
&& self.head_slot.sub(remote.head_slot).as_usize() <= SLOT_IMPORT_TOLERANCE)
|| (self.head_slot < remote.head_slot)
&& remote.head_slot.sub(self.head_slot).as_usize() <= SLOT_IMPORT_TOLERANCE
{
return true;
}
}
false
}
/// Determines if a peer has more knowledge about the current chain than we do.
///
/// There are two conditions here.
/// 1) The peer could have a head slot that is greater
/// than SLOT_IMPORT_TOLERANCE of our current head.
/// 2) The peer has a greater finalized slot/epoch than our own.
fn is_ahead_peer(&self, remote: &PeerSyncInfo) -> bool {
if remote.head_slot.sub(self.head_slot).as_usize() > SLOT_IMPORT_TOLERANCE
|| self.finalized_epoch < remote.finalized_epoch
{
true
} else {
false
}
}
}

View File

@ -9,7 +9,7 @@ use std::collections::hash_map::Entry;
use std::collections::{HashMap, HashSet}; use std::collections::{HashMap, HashSet};
use std::hash::{Hash, Hasher}; use std::hash::{Hash, Hasher};
use std::ops::Sub; use std::ops::Sub;
use types::{EthSpec, Hash256, SignedBeaconBlock, Slot}; use types::{EthSpec, SignedBeaconBlock, Slot};
#[derive(Copy, Clone, Debug, PartialEq)] #[derive(Copy, Clone, Debug, PartialEq)]
pub struct BatchId(pub u64); pub struct BatchId(pub u64);
@ -41,8 +41,6 @@ pub struct Batch<T: EthSpec> {
pub start_slot: Slot, pub start_slot: Slot,
/// The requested end slot of batch, exclusive. /// The requested end slot of batch, exclusive.
pub end_slot: Slot, pub end_slot: Slot,
/// The hash of the chain root to requested from the peer.
pub head_root: Hash256,
/// The peer that was originally assigned to the batch. /// The peer that was originally assigned to the batch.
pub original_peer: PeerId, pub original_peer: PeerId,
/// The peer that is currently assigned to the batch. /// The peer that is currently assigned to the batch.
@ -61,18 +59,11 @@ pub struct Batch<T: EthSpec> {
impl<T: EthSpec> Eq for Batch<T> {} impl<T: EthSpec> Eq for Batch<T> {}
impl<T: EthSpec> Batch<T> { impl<T: EthSpec> Batch<T> {
pub fn new( pub fn new(id: BatchId, start_slot: Slot, end_slot: Slot, peer_id: PeerId) -> Self {
id: BatchId,
start_slot: Slot,
end_slot: Slot,
head_root: Hash256,
peer_id: PeerId,
) -> Self {
Batch { Batch {
id, id,
start_slot, start_slot,
end_slot, end_slot,
head_root,
original_peer: peer_id.clone(), original_peer: peer_id.clone(),
current_peer: peer_id, current_peer: peer_id,
retries: 0, retries: 0,
@ -84,7 +75,6 @@ impl<T: EthSpec> Batch<T> {
pub fn to_blocks_by_range_request(&self) -> BlocksByRangeRequest { pub fn to_blocks_by_range_request(&self) -> BlocksByRangeRequest {
BlocksByRangeRequest { BlocksByRangeRequest {
head_block_root: self.head_root,
start_slot: self.start_slot.into(), start_slot: self.start_slot.into(),
count: std::cmp::min(BLOCKS_PER_BATCH, self.end_slot.sub(self.start_slot).into()), count: std::cmp::min(BLOCKS_PER_BATCH, self.end_slot.sub(self.start_slot).into()),
step: 1, step: 1,

View File

@ -8,7 +8,7 @@ use eth2_libp2p::PeerId;
use rand::prelude::*; use rand::prelude::*;
use slog::{crit, debug, warn}; use slog::{crit, debug, warn};
use std::collections::HashSet; use std::collections::HashSet;
use std::sync::Weak; use std::sync::Arc;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{Hash256, SignedBeaconBlock, Slot}; use types::{Hash256, SignedBeaconBlock, Slot};
@ -17,7 +17,7 @@ use types::{Hash256, SignedBeaconBlock, Slot};
/// downvote peers with poor bandwidth. This can be set arbitrarily high, in which case the /// downvote peers with poor bandwidth. This can be set arbitrarily high, in which case the
/// responder will fill the response up to the max request size, assuming they have the bandwidth /// responder will fill the response up to the max request size, assuming they have the bandwidth
/// to do so. /// to do so.
pub const BLOCKS_PER_BATCH: u64 = 50; pub const BLOCKS_PER_BATCH: u64 = 64;
/// The number of times to retry a batch before the chain is considered failed and removed. /// The number of times to retry a batch before the chain is considered failed and removed.
const MAX_BATCH_RETRIES: u8 = 5; const MAX_BATCH_RETRIES: u8 = 5;
@ -38,10 +38,16 @@ pub enum ProcessingResult {
RemoveChain, RemoveChain,
} }
/// A chain identifier
pub type ChainId = u64;
/// A chain of blocks that need to be downloaded. Peers who claim to contain the target head /// A chain of blocks that need to be downloaded. Peers who claim to contain the target head
/// root are grouped into the peer pool and queried for batches when downloading the /// root are grouped into the peer pool and queried for batches when downloading the
/// chain. /// chain.
pub struct SyncingChain<T: BeaconChainTypes> { pub struct SyncingChain<T: BeaconChainTypes> {
/// A random id used to identify this chain.
id: ChainId,
/// The original start slot when this chain was initialised. /// The original start slot when this chain was initialised.
pub start_slot: Slot, pub start_slot: Slot,
@ -52,7 +58,7 @@ pub struct SyncingChain<T: BeaconChainTypes> {
pub target_head_root: Hash256, pub target_head_root: Hash256,
/// The batches that are currently awaiting a response from a peer. An RPC request for these /// The batches that are currently awaiting a response from a peer. An RPC request for these
/// have been sent. /// has been sent.
pub pending_batches: PendingBatches<T::EthSpec>, pub pending_batches: PendingBatches<T::EthSpec>,
/// The batches that have been downloaded and are awaiting processing and/or validation. /// The batches that have been downloaded and are awaiting processing and/or validation.
@ -82,7 +88,8 @@ pub struct SyncingChain<T: BeaconChainTypes> {
/// back once batch processing has completed. /// back once batch processing has completed.
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
chain: Weak<BeaconChain<T>>, /// A reference to the underlying beacon chain.
chain: Arc<BeaconChain<T>>,
/// A reference to the sync logger. /// A reference to the sync logger.
log: slog::Logger, log: slog::Logger,
@ -98,18 +105,20 @@ pub enum ChainSyncingState {
impl<T: BeaconChainTypes> SyncingChain<T> { impl<T: BeaconChainTypes> SyncingChain<T> {
pub fn new( pub fn new(
id: u64,
start_slot: Slot, start_slot: Slot,
target_head_slot: Slot, target_head_slot: Slot,
target_head_root: Hash256, target_head_root: Hash256,
peer_id: PeerId, peer_id: PeerId,
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
chain: Weak<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
log: slog::Logger, log: slog::Logger,
) -> Self { ) -> Self {
let mut peer_pool = HashSet::new(); let mut peer_pool = HashSet::new();
peer_pool.insert(peer_id); peer_pool.insert(peer_id);
SyncingChain { SyncingChain {
id,
start_slot, start_slot,
target_head_slot, target_head_slot,
target_head_root, target_head_root,
@ -141,7 +150,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// batch. /// batch.
pub fn on_block_response( pub fn on_block_response(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
request_id: RequestId, request_id: RequestId,
beacon_block: &Option<SignedBeaconBlock<T::EthSpec>>, beacon_block: &Option<SignedBeaconBlock<T::EthSpec>>,
) -> Option<()> { ) -> Option<()> {
@ -161,7 +170,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// failed indicating that further batches are required. /// failed indicating that further batches are required.
fn handle_completed_batch( fn handle_completed_batch(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
batch: Batch<T::EthSpec>, batch: Batch<T::EthSpec>,
) { ) {
// An entire batch of blocks has been received. This functions checks to see if it can be processed, // An entire batch of blocks has been received. This functions checks to see if it can be processed,
@ -241,11 +250,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// Sends a batch to the batch processor. /// Sends a batch to the batch processor.
fn process_batch(&mut self, mut batch: Batch<T::EthSpec>) { fn process_batch(&mut self, mut batch: Batch<T::EthSpec>) {
let downloaded_blocks = std::mem::replace(&mut batch.downloaded_blocks, Vec::new()); let downloaded_blocks = std::mem::replace(&mut batch.downloaded_blocks, Vec::new());
let batch_id = ProcessId::RangeBatchId(batch.id.clone()); let process_id = ProcessId::RangeBatchId(self.id.clone(), batch.id.clone());
self.current_processing_batch = Some(batch); self.current_processing_batch = Some(batch);
spawn_block_processor( spawn_block_processor(
self.chain.clone(), Arc::downgrade(&self.chain.clone()),
batch_id, process_id,
downloaded_blocks, downloaded_blocks,
self.sync_send.clone(), self.sync_send.clone(),
self.log.clone(), self.log.clone(),
@ -256,27 +265,37 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// of the batch processor. /// of the batch processor.
pub fn on_batch_process_result( pub fn on_batch_process_result(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
chain_id: ChainId,
batch_id: BatchId, batch_id: BatchId,
downloaded_blocks: &mut Option<Vec<SignedBeaconBlock<T::EthSpec>>>, downloaded_blocks: &mut Option<Vec<SignedBeaconBlock<T::EthSpec>>>,
result: &BatchProcessResult, result: &BatchProcessResult,
) -> Option<ProcessingResult> { ) -> Option<ProcessingResult> {
if let Some(current_batch) = &self.current_processing_batch { if chain_id != self.id {
if current_batch.id != batch_id { // the result does not belong to this chain
// batch process does not belong to this chain return None;
}
match &self.current_processing_batch {
Some(current_batch) if current_batch.id != batch_id => {
debug!(self.log, "Unexpected batch result";
"chain_id" => self.id, "batch_id" => *batch_id, "expected_batch_id" => *current_batch.id);
return None; return None;
} }
// Continue. This is our processing request None => {
} else { debug!(self.log, "Chain was not expecting a batch result";
// not waiting on a processing result "chain_id" => self.id, "batch_id" => *batch_id);
return None; return None;
}
_ => {
// chain_id and batch_id match, continue
}
} }
// claim the result by consuming the option // claim the result by consuming the option
let downloaded_blocks = downloaded_blocks.take().or_else(|| { let downloaded_blocks = downloaded_blocks.take().or_else(|| {
// if taken by another chain, we are no longer waiting on a result. // if taken by another chain, we are no longer waiting on a result.
self.current_processing_batch = None; self.current_processing_batch = None;
crit!(self.log, "Processed batch taken by another chain"); crit!(self.log, "Processed batch taken by another chain"; "chain_id" => self.id);
None None
})?; })?;
@ -288,6 +307,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
// double check batches are processed in order TODO: Remove for prod // double check batches are processed in order TODO: Remove for prod
if batch.id != self.to_be_processed_id { if batch.id != self.to_be_processed_id {
crit!(self.log, "Batch processed out of order"; crit!(self.log, "Batch processed out of order";
"chain_id" => self.id,
"processed_batch_id" => *batch.id, "processed_batch_id" => *batch.id,
"expected_id" => *self.to_be_processed_id); "expected_id" => *self.to_be_processed_id);
} }
@ -299,40 +319,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
// If the processed batch was not empty, we can validate previous invalidated // If the processed batch was not empty, we can validate previous invalidated
// blocks // blocks
if !batch.downloaded_blocks.is_empty() { if !batch.downloaded_blocks.is_empty() {
// Remove any batches awaiting validation. self.mark_processed_batches_as_valid(network, &batch);
//
// All blocks in processed_batches should be prior batches. As the current
// batch has been processed with blocks in it, all previous batches are valid.
//
// If a previous batch has been validated and it had been re-processed, downvote
// the original peer.
while !self.processed_batches.is_empty() {
let processed_batch = self.processed_batches.remove(0);
if *processed_batch.id >= *batch.id {
crit!(self.log, "A processed batch had a greater id than the current process id";
"processed_id" => *processed_batch.id,
"current_id" => *batch.id);
}
if let Some(prev_hash) = processed_batch.original_hash {
// The validated batch has been re-processed
if prev_hash != processed_batch.hash() {
// The re-downloaded version was different
if processed_batch.current_peer != processed_batch.original_peer {
// A new peer sent the correct batch, the previous peer did not
// downvote the original peer
//
// If the same peer corrected it's mistake, we allow it.... for
// now.
debug!(self.log, "Re-processed batch validated. Downvoting original peer";
"batch_id" => *processed_batch.id,
"original_peer" => format!("{}",processed_batch.original_peer),
"new_peer" => format!("{}", processed_batch.current_peer));
network.downvote_peer(processed_batch.original_peer);
}
}
}
}
} }
// Add the current batch to processed batches to be verified in the future. We are // Add the current batch to processed batches to be verified in the future. We are
@ -360,20 +347,49 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
ProcessingResult::KeepChain ProcessingResult::KeepChain
} }
} }
BatchProcessResult::Failed => { BatchProcessResult::Partial => {
warn!(self.log, "Batch processing failed"; "id" => *batch.id, "peer" => format!("{}", batch.current_peer)); warn!(self.log, "Batch processing failed but at least one block was imported";
// The batch processing failed "chain_id" => self.id, "id" => *batch.id, "peer" => format!("{}", batch.current_peer)
// This could be because this batch is invalid, or a previous invalidated batch );
// is invalid. We need to find out which and downvote the peer that has sent us // At least one block was successfully verified and imported, so we can be sure all
// an invalid batch. // previous batches are valid and we only need to download the current failed
// batch.
self.mark_processed_batches_as_valid(network, &batch);
// check that we have no exceeded the re-process retry counter // check that we have not exceeded the re-process retry counter
if batch.reprocess_retries > INVALID_BATCH_LOOKUP_ATTEMPTS { if batch.reprocess_retries > INVALID_BATCH_LOOKUP_ATTEMPTS {
// if a batch has exceeded the invalid batch lookup attempts limit, it means // if a batch has exceeded the invalid batch lookup attempts limit, it means
// that it is likely all peers in this chain are are sending invalid batches // that it is likely all peers in this chain are are sending invalid batches
// repeatedly and are either malicious or faulty. We drop the chain and // repeatedly and are either malicious or faulty. We drop the chain and
// downvote all peers. // downvote all peers.
warn!(self.log, "Batch failed to download. Dropping chain and downvoting peers"; "id"=> *batch.id); warn!(self.log, "Batch failed to download. Dropping chain and downvoting peers";
"chain_id" => self.id, "id"=> *batch.id);
for peer_id in self.peer_pool.drain() {
network.downvote_peer(peer_id);
}
ProcessingResult::RemoveChain
} else {
// Handle this invalid batch, that is within the re-process retries limit.
self.handle_invalid_batch(network, batch);
ProcessingResult::KeepChain
}
}
BatchProcessResult::Failed => {
warn!(self.log, "Batch processing failed";
"chain_id" => self.id,"id" => *batch.id, "peer" => format!("{}", batch.current_peer));
// The batch processing failed
// This could be because this batch is invalid, or a previous invalidated batch
// is invalid. We need to find out which and downvote the peer that has sent us
// an invalid batch.
// check that we have not exceeded the re-process retry counter
if batch.reprocess_retries > INVALID_BATCH_LOOKUP_ATTEMPTS {
// if a batch has exceeded the invalid batch lookup attempts limit, it means
// that it is likely all peers in this chain are are sending invalid batches
// repeatedly and are either malicious or faulty. We drop the chain and
// downvote all peers.
warn!(self.log, "Batch failed to download. Dropping chain and downvoting peers";
"chain_id" => self.id, "id"=> *batch.id);
for peer_id in self.peer_pool.drain() { for peer_id in self.peer_pool.drain() {
network.downvote_peer(peer_id); network.downvote_peer(peer_id);
} }
@ -389,6 +405,51 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
Some(res) Some(res)
} }
/// Removes any batches awaiting validation.
///
/// All blocks in `processed_batches` should be prior batches. As the `last_batch` has been
/// processed with blocks in it, all previous batches are valid.
///
/// If a previous batch has been validated and it had been re-processed, downvote
/// the original peer.
fn mark_processed_batches_as_valid(
&mut self,
network: &mut SyncNetworkContext<T::EthSpec>,
last_batch: &Batch<T::EthSpec>,
) {
while !self.processed_batches.is_empty() {
let processed_batch = self.processed_batches.remove(0);
if *processed_batch.id >= *last_batch.id {
crit!(self.log, "A processed batch had a greater id than the current process id";
"chain_id" => self.id,
"processed_id" => *processed_batch.id,
"current_id" => *last_batch.id);
}
if let Some(prev_hash) = processed_batch.original_hash {
// The validated batch has been re-processed
if prev_hash != processed_batch.hash() {
// The re-downloaded version was different
if processed_batch.current_peer != processed_batch.original_peer {
// A new peer sent the correct batch, the previous peer did not
// downvote the original peer
//
// If the same peer corrected it's mistake, we allow it.... for
// now.
debug!(
self.log, "Re-processed batch validated. Downvoting original peer";
"chain_id" => self.id,
"batch_id" => *processed_batch.id,
"original_peer" => format!("{}",processed_batch.original_peer),
"new_peer" => format!("{}", processed_batch.current_peer)
);
network.downvote_peer(processed_batch.original_peer);
}
}
}
}
}
/// An invalid batch has been received that could not be processed. /// An invalid batch has been received that could not be processed.
/// ///
/// These events occur when a peer as successfully responded with blocks, but the blocks we /// These events occur when a peer as successfully responded with blocks, but the blocks we
@ -397,7 +458,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
// TODO: Batches could have been partially downloaded due to RPC size-limit restrictions. We // TODO: Batches could have been partially downloaded due to RPC size-limit restrictions. We
// need to add logic for partial batch downloads. Potentially, if another peer returns the same // need to add logic for partial batch downloads. Potentially, if another peer returns the same
// batch, we try a partial download. // batch, we try a partial download.
fn handle_invalid_batch(&mut self, network: &mut SyncNetworkContext, batch: Batch<T::EthSpec>) { fn handle_invalid_batch(
&mut self,
network: &mut SyncNetworkContext<T::EthSpec>,
batch: Batch<T::EthSpec>,
) {
// The current batch could not be processed, indicating either the current or previous // The current batch could not be processed, indicating either the current or previous
// batches are invalid // batches are invalid
@ -427,7 +492,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// ///
/// If the re-downloaded batch is different to the original and can be processed, the original /// If the re-downloaded batch is different to the original and can be processed, the original
/// peer will be downvoted. /// peer will be downvoted.
fn reprocess_batch(&mut self, network: &mut SyncNetworkContext, mut batch: Batch<T::EthSpec>) { fn reprocess_batch(
&mut self,
network: &mut SyncNetworkContext<T::EthSpec>,
mut batch: Batch<T::EthSpec>,
) {
// marks the batch as attempting to be reprocessed by hashing the downloaded blocks // marks the batch as attempting to be reprocessed by hashing the downloaded blocks
batch.original_hash = Some(batch.hash()); batch.original_hash = Some(batch.hash());
@ -449,11 +518,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
batch.current_peer = new_peer.clone(); batch.current_peer = new_peer.clone();
debug!(self.log, "Re-requesting batch"; debug!(self.log, "Re-requesting batch";
"chain_id" => self.id,
"start_slot" => batch.start_slot, "start_slot" => batch.start_slot,
"end_slot" => batch.end_slot, "end_slot" => batch.end_slot,
"id" => *batch.id, "id" => *batch.id,
"peer" => format!("{}", batch.current_peer), "peer" => format!("{}", batch.current_peer),
"head_root"=> format!("{}", batch.head_root),
"retries" => batch.retries, "retries" => batch.retries,
"re-processes" => batch.reprocess_retries); "re-processes" => batch.reprocess_retries);
self.send_batch(network, batch); self.send_batch(network, batch);
@ -467,7 +536,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// This chain has been requested to start syncing. /// This chain has been requested to start syncing.
/// ///
/// This could be new chain, or an old chain that is being resumed. /// This could be new chain, or an old chain that is being resumed.
pub fn start_syncing(&mut self, network: &mut SyncNetworkContext, local_finalized_slot: Slot) { pub fn start_syncing(
&mut self,
network: &mut SyncNetworkContext<T::EthSpec>,
local_finalized_slot: Slot,
) {
// A local finalized slot is provided as other chains may have made // A local finalized slot is provided as other chains may have made
// progress whilst this chain was Stopped or paused. If so, update the `processed_batch_id` to // progress whilst this chain was Stopped or paused. If so, update the `processed_batch_id` to
// accommodate potentially downloaded batches from other chains. Also prune any old batches // accommodate potentially downloaded batches from other chains. Also prune any old batches
@ -479,6 +552,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
if local_finalized_slot > self.current_processed_slot() { if local_finalized_slot > self.current_processed_slot() {
debug!(self.log, "Updating chain's progress"; debug!(self.log, "Updating chain's progress";
"chain_id" => self.id,
"prev_completed_slot" => self.current_processed_slot(), "prev_completed_slot" => self.current_processed_slot(),
"new_completed_slot" => local_finalized_slot.as_u64()); "new_completed_slot" => local_finalized_slot.as_u64());
// Re-index batches // Re-index batches
@ -502,11 +576,12 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// Add a peer to the chain. /// Add a peer to the chain.
/// ///
/// If the chain is active, this starts requesting batches from this peer. /// If the chain is active, this starts requesting batches from this peer.
pub fn add_peer(&mut self, network: &mut SyncNetworkContext, peer_id: PeerId) { pub fn add_peer(&mut self, network: &mut SyncNetworkContext<T::EthSpec>, peer_id: PeerId) {
self.peer_pool.insert(peer_id.clone()); self.peer_pool.insert(peer_id.clone());
// do not request blocks if the chain is not syncing // do not request blocks if the chain is not syncing
if let ChainSyncingState::Stopped = self.state { if let ChainSyncingState::Stopped = self.state {
debug!(self.log, "Peer added to a non-syncing chain"; "peer_id" => format!("{}", peer_id)); debug!(self.log, "Peer added to a non-syncing chain";
"chain_id" => self.id, "peer_id" => format!("{}", peer_id));
return; return;
} }
@ -515,7 +590,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
} }
/// Sends a STATUS message to all peers in the peer pool. /// Sends a STATUS message to all peers in the peer pool.
pub fn status_peers(&self, network: &mut SyncNetworkContext) { pub fn status_peers(&self, network: &mut SyncNetworkContext<T::EthSpec>) {
for peer_id in self.peer_pool.iter() { for peer_id in self.peer_pool.iter() {
network.status_peer(self.chain.clone(), peer_id.clone()); network.status_peer(self.chain.clone(), peer_id.clone());
} }
@ -529,12 +604,13 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// this chain. /// this chain.
pub fn inject_error( pub fn inject_error(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
peer_id: &PeerId, peer_id: &PeerId,
request_id: RequestId, request_id: RequestId,
) -> Option<ProcessingResult> { ) -> Option<ProcessingResult> {
if let Some(batch) = self.pending_batches.remove(request_id) { if let Some(batch) = self.pending_batches.remove(request_id) {
warn!(self.log, "Batch failed. RPC Error"; warn!(self.log, "Batch failed. RPC Error";
"chain_id" => self.id,
"id" => *batch.id, "id" => *batch.id,
"retries" => batch.retries, "retries" => batch.retries,
"peer" => format!("{:?}", peer_id)); "peer" => format!("{:?}", peer_id));
@ -553,7 +629,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// `MAX_BATCH_RETRIES`. /// `MAX_BATCH_RETRIES`.
pub fn failed_batch( pub fn failed_batch(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
mut batch: Batch<T::EthSpec>, mut batch: Batch<T::EthSpec>,
) -> ProcessingResult { ) -> ProcessingResult {
batch.retries += 1; batch.retries += 1;
@ -575,11 +651,11 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
batch.current_peer = new_peer.clone(); batch.current_peer = new_peer.clone();
debug!(self.log, "Re-Requesting batch"; debug!(self.log, "Re-Requesting batch";
"chain_id" => self.id,
"start_slot" => batch.start_slot, "start_slot" => batch.start_slot,
"end_slot" => batch.end_slot, "end_slot" => batch.end_slot,
"id" => *batch.id, "id" => *batch.id,
"peer" => format!("{:?}", batch.current_peer), "peer" => format!("{:?}", batch.current_peer));
"head_root"=> format!("{}", batch.head_root));
self.send_batch(network, batch); self.send_batch(network, batch);
ProcessingResult::KeepChain ProcessingResult::KeepChain
} }
@ -587,7 +663,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// Attempts to request the next required batches from the peer pool if the chain is syncing. It will exhaust the peer /// Attempts to request the next required batches from the peer pool if the chain is syncing. It will exhaust the peer
/// pool and left over batches until the batch buffer is reached or all peers are exhausted. /// pool and left over batches until the batch buffer is reached or all peers are exhausted.
fn request_batches(&mut self, network: &mut SyncNetworkContext) { fn request_batches(&mut self, network: &mut SyncNetworkContext<T::EthSpec>) {
if let ChainSyncingState::Syncing = self.state { if let ChainSyncingState::Syncing = self.state {
while self.send_range_request(network) {} while self.send_range_request(network) {}
} }
@ -595,16 +671,16 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
/// Requests the next required batch from a peer. Returns true, if there was a peer available /// Requests the next required batch from a peer. Returns true, if there was a peer available
/// to send a request and there are batches to request, false otherwise. /// to send a request and there are batches to request, false otherwise.
fn send_range_request(&mut self, network: &mut SyncNetworkContext) -> bool { fn send_range_request(&mut self, network: &mut SyncNetworkContext<T::EthSpec>) -> bool {
// find the next pending batch and request it from the peer // find the next pending batch and request it from the peer
if let Some(peer_id) = self.get_next_peer() { if let Some(peer_id) = self.get_next_peer() {
if let Some(batch) = self.get_next_batch(peer_id) { if let Some(batch) = self.get_next_batch(peer_id) {
debug!(self.log, "Requesting batch"; debug!(self.log, "Requesting batch";
"chain_id" => self.id,
"start_slot" => batch.start_slot, "start_slot" => batch.start_slot,
"end_slot" => batch.end_slot, "end_slot" => batch.end_slot,
"id" => *batch.id, "id" => *batch.id,
"peer" => format!("{}", batch.current_peer), "peer" => format!("{}", batch.current_peer));
"head_root"=> format!("{}", batch.head_root));
// send the batch // send the batch
self.send_batch(network, batch); self.send_batch(network, batch);
return true; return true;
@ -675,13 +751,16 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
batch_id, batch_id,
batch_start_slot, batch_start_slot,
batch_end_slot, batch_end_slot,
self.target_head_root,
peer_id, peer_id,
)) ))
} }
/// Requests the provided batch from the provided peer. /// Requests the provided batch from the provided peer.
fn send_batch(&mut self, network: &mut SyncNetworkContext, batch: Batch<T::EthSpec>) { fn send_batch(
&mut self,
network: &mut SyncNetworkContext<T::EthSpec>,
batch: Batch<T::EthSpec>,
) {
let request = batch.to_blocks_by_range_request(); let request = batch.to_blocks_by_range_request();
if let Ok(request_id) = network.blocks_by_range_request(batch.current_peer.clone(), request) if let Ok(request_id) = network.blocks_by_range_request(batch.current_peer.clone(), request)
{ {

View File

@ -4,52 +4,141 @@
//! with this struct to to simplify the logic of the other layers of sync. //! with this struct to to simplify the logic of the other layers of sync.
use super::chain::{ChainSyncingState, SyncingChain}; use super::chain::{ChainSyncingState, SyncingChain};
use crate::message_processor::PeerSyncInfo;
use crate::sync::manager::SyncMessage; use crate::sync::manager::SyncMessage;
use crate::sync::network_context::SyncNetworkContext; use crate::sync::network_context::SyncNetworkContext;
use crate::sync::PeerSyncInfo;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::PeerId; use eth2_libp2p::{types::SyncState, NetworkGlobals, PeerId};
use slog::{debug, error, warn}; use slog::{debug, error, info};
use std::sync::Weak; use std::sync::Arc;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::EthSpec; use types::EthSpec;
use types::{Hash256, Slot}; use types::{Hash256, Slot};
/// The state of the long range/batch sync. /// The state of the long range/batch sync.
pub enum SyncState { #[derive(Clone)]
pub enum RangeSyncState {
/// A finalized chain is being synced. /// A finalized chain is being synced.
Finalized, Finalized {
/// The start of the finalized chain.
start_slot: Slot,
/// The target head slot of the finalized chain.
head_slot: Slot,
/// The target head root of the finalized chain.
head_root: Hash256,
},
/// There are no finalized chains and we are syncing one more head chains. /// There are no finalized chains and we are syncing one more head chains.
Head, Head {
/// The last finalized checkpoint for all head chains.
start_slot: Slot,
/// The largest known slot to sync to.
head_slot: Slot,
},
/// There are no head or finalized chains and no long range sync is in progress. /// There are no head or finalized chains and no long range sync is in progress.
Idle, Idle,
} }
impl PartialEq for RangeSyncState {
fn eq(&self, other: &Self) -> bool {
match (self, other) {
(RangeSyncState::Finalized { .. }, RangeSyncState::Finalized { .. }) => true,
(RangeSyncState::Head { .. }, RangeSyncState::Head { .. }) => true,
(RangeSyncState::Idle, RangeSyncState::Idle) => true,
_ => false,
}
}
}
impl Into<SyncState> for RangeSyncState {
fn into(self) -> SyncState {
match self {
RangeSyncState::Finalized {
start_slot,
head_slot,
head_root,
} => SyncState::SyncingFinalized {
start_slot,
head_slot,
head_root,
},
RangeSyncState::Head {
start_slot,
head_slot,
} => SyncState::SyncingHead {
start_slot,
head_slot,
},
RangeSyncState::Idle => SyncState::Stalled, // this should never really be used
}
}
}
/// A collection of finalized and head chains currently being processed. /// A collection of finalized and head chains currently being processed.
pub struct ChainCollection<T: BeaconChainTypes> { pub struct ChainCollection<T: BeaconChainTypes> {
/// The beacon chain for processing. /// The beacon chain for processing.
beacon_chain: Weak<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
/// A reference to the global network parameters.
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
/// The set of finalized chains being synced. /// The set of finalized chains being synced.
finalized_chains: Vec<SyncingChain<T>>, finalized_chains: Vec<SyncingChain<T>>,
/// The set of head chains being synced. /// The set of head chains being synced.
head_chains: Vec<SyncingChain<T>>, head_chains: Vec<SyncingChain<T>>,
/// The current sync state of the process. /// The current sync state of the process.
sync_state: SyncState, state: RangeSyncState,
/// Logger for the collection.
log: slog::Logger,
} }
impl<T: BeaconChainTypes> ChainCollection<T> { impl<T: BeaconChainTypes> ChainCollection<T> {
pub fn new(beacon_chain: Weak<BeaconChain<T>>) -> Self { pub fn new(
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
log: slog::Logger,
) -> Self {
ChainCollection { ChainCollection {
sync_state: SyncState::Idle, beacon_chain,
network_globals,
finalized_chains: Vec::new(), finalized_chains: Vec::new(),
head_chains: Vec::new(), head_chains: Vec::new(),
beacon_chain, state: RangeSyncState::Idle,
log,
} }
} }
/// The current syncing state. pub fn state(&self) -> &RangeSyncState {
pub fn sync_state(&self) -> &SyncState { &self.state
&self.sync_state }
/// Updates the global sync state and logs any changes.
pub fn update_sync_state(&mut self) {
// if there is no range sync occurring, the state is either synced or not based on
// connected peers.
if self.state == RangeSyncState::Idle {
// there is no range sync, let the state of peers determine the global node sync state
let new_state = self
.network_globals
.peers
.read()
.synced_peers()
.next()
.map(|_| SyncState::Synced)
.unwrap_or_else(|| SyncState::Stalled);
let mut peer_state = self.network_globals.sync_state.write();
if new_state != *peer_state {
info!(self.log, "Sync state updated"; "old_state" => format!("{}",peer_state), "new_state" => format!("{}",new_state));
}
*peer_state = new_state;
} else {
// The state is based on a range sync state, update it
let mut node_sync_state = self.network_globals.sync_state.write();
let new_state: SyncState = self.state.clone().into();
if *node_sync_state != new_state {
// we are updating the state, inform the user
info!(self.log, "Sync state updated"; "old_state" => format!("{}",node_sync_state), "new_state" => format!("{}",new_state));
}
*node_sync_state = new_state;
}
} }
/// A fully synced peer has joined. /// A fully synced peer has joined.
@ -57,9 +146,11 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
/// We could be awaiting a head sync. If we are in the head syncing state, without any head /// We could be awaiting a head sync. If we are in the head syncing state, without any head
/// chains, then update the state to idle. /// chains, then update the state to idle.
pub fn fully_synced_peer_found(&mut self) { pub fn fully_synced_peer_found(&mut self) {
if let SyncState::Head = self.sync_state { if let RangeSyncState::Head { .. } = self.state {
if self.head_chains.is_empty() { if self.head_chains.is_empty() {
self.sync_state = SyncState::Idle; // Update the global network state to either synced or stalled.
self.state = RangeSyncState::Idle;
self.update_sync_state();
} }
} }
} }
@ -68,8 +159,20 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
/// `SyncState::Head` indicating we are awaiting new peers to connect before we can consider /// `SyncState::Head` indicating we are awaiting new peers to connect before we can consider
/// the state as idle. /// the state as idle.
pub fn set_head_sync(&mut self) { pub fn set_head_sync(&mut self) {
if let SyncState::Idle = self.sync_state { if let RangeSyncState::Idle = self.state {
self.sync_state = SyncState::Head; let current_slot = self
.beacon_chain
.head_info()
.map(|info| info.slot)
.unwrap_or_else(|_| Slot::from(0u64));
// NOTE: This will modify the /node/syncing API to show current slot for all fields
// while we update peers to look for new potentially HEAD chains.
let temp_head_state = RangeSyncState::Head {
start_slot: current_slot,
head_slot: current_slot,
};
self.state = temp_head_state;
} }
} }
@ -103,32 +206,26 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
/// ///
/// This removes any out-dated chains, swaps to any higher priority finalized chains and /// This removes any out-dated chains, swaps to any higher priority finalized chains and
/// updates the state of the collection. /// updates the state of the collection.
pub fn update_finalized(&mut self, network: &mut SyncNetworkContext, log: &slog::Logger) { pub fn update_finalized(&mut self, network: &mut SyncNetworkContext<T::EthSpec>) {
let local_slot = match self.beacon_chain.upgrade() { let local_slot = {
Some(chain) => { let local = match PeerSyncInfo::from_chain(&self.beacon_chain) {
let local = match PeerSyncInfo::from_chain(&chain) { Some(local) => local,
Some(local) => local, None => {
None => { return error!(
return error!( self.log,
log, "Failed to get peer sync info";
"Failed to get peer sync info"; "msg" => "likely due to head lock contention"
"msg" => "likely due to head lock contention" )
) }
} };
};
local local
.finalized_epoch .finalized_epoch
.start_slot(T::EthSpec::slots_per_epoch()) .start_slot(T::EthSpec::slots_per_epoch())
}
None => {
warn!(log, "Beacon chain dropped. Chains not updated");
return;
}
}; };
// Remove any outdated finalized chains // Remove any outdated finalized chains
self.purge_outdated_chains(network, log); self.purge_outdated_chains(network);
// Check if any chains become the new syncing chain // Check if any chains become the new syncing chain
if let Some(index) = self.finalized_syncing_index() { if let Some(index) = self.finalized_syncing_index() {
@ -145,13 +242,20 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
}) })
{ {
// A chain has more peers. Swap the syncing chain // A chain has more peers. Swap the syncing chain
debug!(log, "Switching finalized chains to sync"; "new_target_root" => format!("{}", chain.target_head_root), "new_end_slot" => chain.target_head_slot, "new_start_slot"=> chain.start_slot); debug!(self.log, "Switching finalized chains to sync"; "new_target_root" => format!("{}", chain.target_head_root), "new_end_slot" => chain.target_head_slot, "new_start_slot"=> chain.start_slot);
// update the state to a new finalized state
let state = RangeSyncState::Finalized {
start_slot: chain.start_slot,
head_slot: chain.target_head_slot,
head_root: chain.target_head_root,
};
self.state = state;
// Stop the current chain from syncing // Stop the current chain from syncing
self.finalized_chains[index].stop_syncing(); self.finalized_chains[index].stop_syncing();
// Start the new chain // Start the new chain
self.finalized_chains[new_index].start_syncing(network, local_slot); self.finalized_chains[new_index].start_syncing(network, local_slot);
self.sync_state = SyncState::Finalized;
} }
} else if let Some(chain) = self } else if let Some(chain) = self
.finalized_chains .finalized_chains
@ -159,15 +263,36 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
.max_by_key(|chain| chain.peer_pool.len()) .max_by_key(|chain| chain.peer_pool.len())
{ {
// There is no currently syncing finalization chain, starting the one with the most peers // There is no currently syncing finalization chain, starting the one with the most peers
debug!(log, "New finalized chain started syncing"; "new_target_root" => format!("{}", chain.target_head_root), "new_end_slot" => chain.target_head_slot, "new_start_slot"=> chain.start_slot); debug!(self.log, "New finalized chain started syncing"; "new_target_root" => format!("{}", chain.target_head_root), "new_end_slot" => chain.target_head_slot, "new_start_slot"=> chain.start_slot);
chain.start_syncing(network, local_slot); chain.start_syncing(network, local_slot);
self.sync_state = SyncState::Finalized; let state = RangeSyncState::Finalized {
start_slot: chain.start_slot,
head_slot: chain.target_head_slot,
head_root: chain.target_head_root,
};
self.state = state;
} else { } else {
// There are no finalized chains, update the state. // There are no finalized chains, update the state.
if self.head_chains.is_empty() { if self.head_chains.is_empty() {
self.sync_state = SyncState::Idle; self.state = RangeSyncState::Idle;
} else { } else {
self.sync_state = SyncState::Head; // for the syncing API, we find the minimal start_slot and the maximum
// target_slot of all head chains to report back.
let (min_slot, max_slot) = self.head_chains.iter().fold(
(Slot::from(0u64), Slot::from(0u64)),
|(min, max), chain| {
(
std::cmp::min(min, chain.start_slot),
std::cmp::max(max, chain.target_head_slot),
)
},
);
let head_state = RangeSyncState::Head {
start_slot: min_slot,
head_slot: max_slot,
};
self.state = head_state;
} }
} }
} }
@ -180,16 +305,17 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
target_slot: Slot, target_slot: Slot,
peer_id: PeerId, peer_id: PeerId,
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
log: &slog::Logger,
) { ) {
let chain_id = rand::random();
self.finalized_chains.push(SyncingChain::new( self.finalized_chains.push(SyncingChain::new(
chain_id,
local_finalized_slot, local_finalized_slot,
target_slot, target_slot,
target_head, target_head,
peer_id, peer_id,
sync_send, sync_send,
self.beacon_chain.clone(), self.beacon_chain.clone(),
log.clone(), self.log.clone(),
)); ));
} }
@ -197,13 +323,12 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
pub fn new_head_chain( pub fn new_head_chain(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
remote_finalized_slot: Slot, remote_finalized_slot: Slot,
target_head: Hash256, target_head: Hash256,
target_slot: Slot, target_slot: Slot,
peer_id: PeerId, peer_id: PeerId,
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
log: &slog::Logger,
) { ) {
// remove the peer from any other head chains // remove the peer from any other head chains
@ -212,14 +337,16 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
}); });
self.head_chains.retain(|chain| !chain.peer_pool.is_empty()); self.head_chains.retain(|chain| !chain.peer_pool.is_empty());
let chain_id = rand::random();
let mut new_head_chain = SyncingChain::new( let mut new_head_chain = SyncingChain::new(
chain_id,
remote_finalized_slot, remote_finalized_slot,
target_slot, target_slot,
target_head, target_head,
peer_id, peer_id,
sync_send, sync_send,
self.beacon_chain.clone(), self.beacon_chain.clone(),
log.clone(), self.log.clone(),
); );
// All head chains can sync simultaneously // All head chains can sync simultaneously
new_head_chain.start_syncing(network, remote_finalized_slot); new_head_chain.start_syncing(network, remote_finalized_slot);
@ -277,25 +404,20 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
/// ///
/// This removes chains with no peers, or chains whose start block slot is less than our current /// This removes chains with no peers, or chains whose start block slot is less than our current
/// finalized block slot. /// finalized block slot.
pub fn purge_outdated_chains(&mut self, network: &mut SyncNetworkContext, log: &slog::Logger) { pub fn purge_outdated_chains(&mut self, network: &mut SyncNetworkContext<T::EthSpec>) {
// Remove any chains that have no peers // Remove any chains that have no peers
self.finalized_chains self.finalized_chains
.retain(|chain| !chain.peer_pool.is_empty()); .retain(|chain| !chain.peer_pool.is_empty());
self.head_chains.retain(|chain| !chain.peer_pool.is_empty()); self.head_chains.retain(|chain| !chain.peer_pool.is_empty());
let (beacon_chain, local_info) = match self.beacon_chain.upgrade() { let local_info = match PeerSyncInfo::from_chain(&self.beacon_chain) {
Some(chain) => match PeerSyncInfo::from_chain(&chain) { Some(local) => local,
Some(local) => (chain, local),
None => {
return error!(
log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
},
None => { None => {
return; return error!(
self.log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
} }
}; };
@ -303,6 +425,8 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
.finalized_epoch .finalized_epoch
.start_slot(T::EthSpec::slots_per_epoch()); .start_slot(T::EthSpec::slots_per_epoch());
let beacon_chain = &self.beacon_chain;
let log_ref = &self.log;
// Remove chains that are out-dated and re-status their peers // Remove chains that are out-dated and re-status their peers
self.finalized_chains.retain(|chain| { self.finalized_chains.retain(|chain| {
if chain.target_head_slot <= local_finalized_slot if chain.target_head_slot <= local_finalized_slot
@ -310,7 +434,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
.fork_choice .fork_choice
.contains_block(&chain.target_head_root) .contains_block(&chain.target_head_root)
{ {
debug!(log, "Purging out of finalized chain"; "start_slot" => chain.start_slot, "end_slot" => chain.target_head_slot); debug!(log_ref, "Purging out of finalized chain"; "start_slot" => chain.start_slot, "end_slot" => chain.target_head_slot);
chain.status_peers(network); chain.status_peers(network);
false false
} else { } else {
@ -323,7 +447,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
.fork_choice .fork_choice
.contains_block(&chain.target_head_root) .contains_block(&chain.target_head_root)
{ {
debug!(log, "Purging out of date head chain"; "start_slot" => chain.start_slot, "end_slot" => chain.target_head_slot); debug!(log_ref, "Purging out of date head chain"; "start_slot" => chain.start_slot, "end_slot" => chain.target_head_slot);
chain.status_peers(network); chain.status_peers(network);
false false
} else { } else {
@ -347,12 +471,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
/// finalized chain length, indicates a head chain. /// finalized chain length, indicates a head chain.
/// ///
/// This will re-status the chains peers on removal. The index must exist. /// This will re-status the chains peers on removal. The index must exist.
pub fn remove_chain( pub fn remove_chain(&mut self, network: &mut SyncNetworkContext<T::EthSpec>, index: usize) {
&mut self,
network: &mut SyncNetworkContext,
index: usize,
log: &slog::Logger,
) {
let chain = if index >= self.finalized_chains.len() { let chain = if index >= self.finalized_chains.len() {
let index = index - self.finalized_chains.len(); let index = index - self.finalized_chains.len();
let chain = self.head_chains.swap_remove(index); let chain = self.head_chains.swap_remove(index);
@ -364,10 +483,10 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
chain chain
}; };
debug!(log, "Chain was removed"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64()); debug!(self.log, "Chain was removed"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64());
// update the state // update the state
self.update_finalized(network, log); self.update_finalized(network);
} }
/// Returns the index of finalized chain that is currently syncing. Returns `None` if no /// Returns the index of finalized chain that is currently syncing. Returns `None` if no

View File

@ -8,4 +8,5 @@ mod range;
pub use batch::Batch; pub use batch::Batch;
pub use batch::BatchId; pub use batch::BatchId;
pub use chain::ChainId;
pub use range::RangeSync; pub use range::RangeSync;

View File

@ -39,19 +39,19 @@
//! Each chain is downloaded in batches of blocks. The batched blocks are processed sequentially //! Each chain is downloaded in batches of blocks. The batched blocks are processed sequentially
//! and further batches are requested as current blocks are being processed. //! and further batches are requested as current blocks are being processed.
use super::chain::ProcessingResult; use super::chain::{ChainId, ProcessingResult};
use super::chain_collection::{ChainCollection, SyncState}; use super::chain_collection::{ChainCollection, RangeSyncState};
use super::BatchId; use super::BatchId;
use crate::message_processor::PeerSyncInfo;
use crate::sync::block_processor::BatchProcessResult; use crate::sync::block_processor::BatchProcessResult;
use crate::sync::manager::SyncMessage; use crate::sync::manager::SyncMessage;
use crate::sync::network_context::SyncNetworkContext; use crate::sync::network_context::SyncNetworkContext;
use crate::sync::PeerSyncInfo;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::RequestId; use eth2_libp2p::rpc::RequestId;
use eth2_libp2p::PeerId; use eth2_libp2p::{NetworkGlobals, PeerId};
use slog::{debug, error, trace, warn}; use slog::{debug, error, trace};
use std::collections::HashSet; use std::collections::HashSet;
use std::sync::Weak; use std::sync::Arc;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{EthSpec, SignedBeaconBlock}; use types::{EthSpec, SignedBeaconBlock};
@ -60,7 +60,7 @@ use types::{EthSpec, SignedBeaconBlock};
/// holds the current state of the long range sync. /// holds the current state of the long range sync.
pub struct RangeSync<T: BeaconChainTypes> { pub struct RangeSync<T: BeaconChainTypes> {
/// The beacon chain for processing. /// The beacon chain for processing.
beacon_chain: Weak<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
/// A collection of chains that need to be downloaded. This stores any head or finalized chains /// A collection of chains that need to be downloaded. This stores any head or finalized chains
/// that need to be downloaded. /// that need to be downloaded.
chains: ChainCollection<T>, chains: ChainCollection<T>,
@ -77,13 +77,14 @@ pub struct RangeSync<T: BeaconChainTypes> {
impl<T: BeaconChainTypes> RangeSync<T> { impl<T: BeaconChainTypes> RangeSync<T> {
pub fn new( pub fn new(
beacon_chain: Weak<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>, sync_send: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
log: slog::Logger, log: slog::Logger,
) -> Self { ) -> Self {
RangeSync { RangeSync {
beacon_chain: beacon_chain.clone(), beacon_chain: beacon_chain.clone(),
chains: ChainCollection::new(beacon_chain), chains: ChainCollection::new(beacon_chain, network_globals, log.clone()),
awaiting_head_peers: HashSet::new(), awaiting_head_peers: HashSet::new(),
sync_send, sync_send,
log, log,
@ -109,7 +110,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
/// prioritised by peer-pool size. /// prioritised by peer-pool size.
pub fn add_peer( pub fn add_peer(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
peer_id: PeerId, peer_id: PeerId,
remote: PeerSyncInfo, remote: PeerSyncInfo,
) { ) {
@ -118,21 +119,14 @@ impl<T: BeaconChainTypes> RangeSync<T> {
// determine if we need to run a sync to the nearest finalized state or simply sync to // determine if we need to run a sync to the nearest finalized state or simply sync to
// its current head // its current head
let (chain, local_info) = match self.beacon_chain.upgrade() { let local_info = match PeerSyncInfo::from_chain(&self.beacon_chain) {
Some(chain) => match PeerSyncInfo::from_chain(&chain) { Some(local) => local,
Some(local) => (chain, local),
None => {
return error!(
self.log,
"Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
}
},
None => { None => {
return warn!(self.log, return error!(
"Beacon chain dropped. Peer not considered for sync"; self.log,
"peer_id" => format!("{:?}", peer_id)); "Failed to get peer sync info";
"msg" => "likely due to head lock contention"
)
} }
}; };
@ -148,10 +142,13 @@ impl<T: BeaconChainTypes> RangeSync<T> {
self.remove_peer(network, &peer_id); self.remove_peer(network, &peer_id);
// remove any out-of-date chains // remove any out-of-date chains
self.chains.purge_outdated_chains(network, &self.log); self.chains.purge_outdated_chains(network);
if remote_finalized_slot > local_info.head_slot if remote_finalized_slot > local_info.head_slot
&& !chain.fork_choice.contains_block(&remote.finalized_root) && !self
.beacon_chain
.fork_choice
.contains_block(&remote.finalized_root)
{ {
debug!(self.log, "Finalization sync peer joined"; "peer_id" => format!("{:?}", peer_id)); debug!(self.log, "Finalization sync peer joined"; "peer_id" => format!("{:?}", peer_id));
// Finalized chain search // Finalized chain search
@ -171,7 +168,8 @@ impl<T: BeaconChainTypes> RangeSync<T> {
chain.add_peer(network, peer_id); chain.add_peer(network, peer_id);
// check if the new peer's addition will favour a new syncing chain. // check if the new peer's addition will favour a new syncing chain.
self.chains.update_finalized(network, &self.log); self.chains.update_finalized(network);
self.chains.update_sync_state();
} else { } else {
// there is no finalized chain that matches this peer's last finalized target // there is no finalized chain that matches this peer's last finalized target
// create a new finalized chain // create a new finalized chain
@ -183,9 +181,9 @@ impl<T: BeaconChainTypes> RangeSync<T> {
remote_finalized_slot, remote_finalized_slot,
peer_id, peer_id,
self.sync_send.clone(), self.sync_send.clone(),
&self.log,
); );
self.chains.update_finalized(network, &self.log); self.chains.update_finalized(network);
self.chains.update_sync_state();
} }
} else { } else {
if self.chains.is_finalizing_sync() { if self.chains.is_finalizing_sync() {
@ -216,10 +214,10 @@ impl<T: BeaconChainTypes> RangeSync<T> {
remote.head_slot, remote.head_slot,
peer_id, peer_id,
self.sync_send.clone(), self.sync_send.clone(),
&self.log,
); );
} }
self.chains.update_finalized(network, &self.log); self.chains.update_finalized(network);
self.chains.update_sync_state();
} }
} }
@ -229,7 +227,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
/// This request could complete a chain or simply add to its progress. /// This request could complete a chain or simply add to its progress.
pub fn blocks_by_range_response( pub fn blocks_by_range_response(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>, beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
@ -256,7 +254,8 @@ impl<T: BeaconChainTypes> RangeSync<T> {
pub fn handle_block_process_result( pub fn handle_block_process_result(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
chain_id: ChainId,
batch_id: BatchId, batch_id: BatchId,
downloaded_blocks: Vec<SignedBeaconBlock<T::EthSpec>>, downloaded_blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
result: BatchProcessResult, result: BatchProcessResult,
@ -265,30 +264,38 @@ impl<T: BeaconChainTypes> RangeSync<T> {
let mut downloaded_blocks = Some(downloaded_blocks); let mut downloaded_blocks = Some(downloaded_blocks);
match self.chains.finalized_request(|chain| { match self.chains.finalized_request(|chain| {
chain.on_batch_process_result(network, batch_id, &mut downloaded_blocks, &result) chain.on_batch_process_result(
network,
chain_id,
batch_id,
&mut downloaded_blocks,
&result,
)
}) { }) {
Some((index, ProcessingResult::RemoveChain)) => { Some((index, ProcessingResult::RemoveChain)) => {
let chain = self.chains.remove_finalized_chain(index); let chain = self.chains.remove_finalized_chain(index);
debug!(self.log, "Finalized chain removed"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64()); debug!(self.log, "Finalized chain removed"; "start_slot" => chain.start_slot.as_u64(), "end_slot" => chain.target_head_slot.as_u64());
// update the state of the collection
self.chains.update_finalized(network);
// the chain is complete, re-status it's peers // the chain is complete, re-status it's peers
chain.status_peers(network); chain.status_peers(network);
// update the state of the collection // set the state to a head sync if there are no finalized chains, to inform the manager that we are awaiting a
self.chains.update_finalized(network, &self.log);
// set the state to a head sync, to inform the manager that we are awaiting a
// head chain. // head chain.
self.chains.set_head_sync(); self.chains.set_head_sync();
// Update the global variables
self.chains.update_sync_state();
// if there are no more finalized chains, re-status all known peers awaiting a head // if there are no more finalized chains, re-status all known peers awaiting a head
// sync // sync
match self.chains.sync_state() { match self.chains.state() {
SyncState::Idle | SyncState::Head => { RangeSyncState::Idle | RangeSyncState::Head { .. } => {
for peer_id in self.awaiting_head_peers.drain() { for peer_id in self.awaiting_head_peers.drain() {
network.status_peer(self.beacon_chain.clone(), peer_id); network.status_peer(self.beacon_chain.clone(), peer_id);
} }
} }
SyncState::Finalized => {} // Have more finalized chains to complete RangeSyncState::Finalized { .. } => {} // Have more finalized chains to complete
} }
} }
Some((_, ProcessingResult::KeepChain)) => {} Some((_, ProcessingResult::KeepChain)) => {}
@ -296,6 +303,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
match self.chains.head_request(|chain| { match self.chains.head_request(|chain| {
chain.on_batch_process_result( chain.on_batch_process_result(
network, network,
chain_id,
batch_id, batch_id,
&mut downloaded_blocks, &mut downloaded_blocks,
&result, &result,
@ -308,7 +316,9 @@ impl<T: BeaconChainTypes> RangeSync<T> {
chain.status_peers(network); chain.status_peers(network);
// update the state of the collection // update the state of the collection
self.chains.update_finalized(network, &self.log); self.chains.update_finalized(network);
// update the global state and log any change
self.chains.update_sync_state();
} }
Some((_, ProcessingResult::KeepChain)) => {} Some((_, ProcessingResult::KeepChain)) => {}
None => { None => {
@ -321,18 +331,13 @@ impl<T: BeaconChainTypes> RangeSync<T> {
} }
} }
/// Public method to indicate the current state of the long range sync.
pub fn is_syncing(&self) -> bool {
match self.chains.sync_state() {
SyncState::Finalized => true,
SyncState::Head => true,
SyncState::Idle => false,
}
}
/// A peer has disconnected. This removes the peer from any ongoing chains and mappings. A /// A peer has disconnected. This removes the peer from any ongoing chains and mappings. A
/// disconnected peer could remove a chain /// disconnected peer could remove a chain
pub fn peer_disconnect(&mut self, network: &mut SyncNetworkContext, peer_id: &PeerId) { pub fn peer_disconnect(
&mut self,
network: &mut SyncNetworkContext<T::EthSpec>,
peer_id: &PeerId,
) {
// if the peer is in the awaiting head mapping, remove it // if the peer is in the awaiting head mapping, remove it
self.awaiting_head_peers.remove(&peer_id); self.awaiting_head_peers.remove(&peer_id);
@ -340,13 +345,15 @@ impl<T: BeaconChainTypes> RangeSync<T> {
self.remove_peer(network, peer_id); self.remove_peer(network, peer_id);
// update the state of the collection // update the state of the collection
self.chains.update_finalized(network, &self.log); self.chains.update_finalized(network);
// update the global state and inform the user
self.chains.update_sync_state();
} }
/// When a peer gets removed, both the head and finalized chains need to be searched to check which pool the peer is in. The chain may also have a batch or batches awaiting /// When a peer gets removed, both the head and finalized chains need to be searched to check which pool the peer is in. The chain may also have a batch or batches awaiting
/// for this peer. If so we mark the batch as failed. The batch may then hit it's maximum /// for this peer. If so we mark the batch as failed. The batch may then hit it's maximum
/// retries. In this case, we need to remove the chain and re-status all the peers. /// retries. In this case, we need to remove the chain and re-status all the peers.
fn remove_peer(&mut self, network: &mut SyncNetworkContext, peer_id: &PeerId) { fn remove_peer(&mut self, network: &mut SyncNetworkContext<T::EthSpec>, peer_id: &PeerId) {
if let Some((index, ProcessingResult::RemoveChain)) = if let Some((index, ProcessingResult::RemoveChain)) =
self.chains.head_finalized_request(|chain| { self.chains.head_finalized_request(|chain| {
if chain.peer_pool.remove(peer_id) { if chain.peer_pool.remove(peer_id) {
@ -366,7 +373,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
{ {
// the chain needed to be removed // the chain needed to be removed
debug!(self.log, "Chain being removed due to failed batch"); debug!(self.log, "Chain being removed due to failed batch");
self.chains.remove_chain(network, index, &self.log); self.chains.remove_chain(network, index);
} }
} }
@ -376,7 +383,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
/// been too many failed attempts for the batch, remove the chain. /// been too many failed attempts for the batch, remove the chain.
pub fn inject_error( pub fn inject_error(
&mut self, &mut self,
network: &mut SyncNetworkContext, network: &mut SyncNetworkContext<T::EthSpec>,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
) { ) {
@ -388,7 +395,7 @@ impl<T: BeaconChainTypes> RangeSync<T> {
Some((_, ProcessingResult::KeepChain)) => {} // error handled chain persists Some((_, ProcessingResult::KeepChain)) => {} // error handled chain persists
Some((index, ProcessingResult::RemoveChain)) => { Some((index, ProcessingResult::RemoveChain)) => {
debug!(self.log, "Chain being removed due to RPC error"); debug!(self.log, "Chain being removed due to RPC error");
self.chains.remove_chain(network, index, &self.log) self.chains.remove_chain(network, index)
} }
None => {} // request wasn't in the finalized chains, check the head chains None => {} // request wasn't in the finalized chains, check the head chains
} }

View File

@ -1,12 +1,13 @@
[package] [package]
name = "rest_api" name = "rest_api"
version = "0.1.0" version = "0.2.0"
authors = ["Paul Hauner <paul@paulhauner.com>", "Luke Anderson <luke@sigmaprime.io>"] authors = ["Paul Hauner <paul@paulhauner.com>", "Age Manning <Age@AgeManning.com>", "Luke Anderson <luke@sigmaprime.io>"]
edition = "2018" edition = "2018"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies] [dependencies]
bls = { path = "../../eth2/utils/bls" } bls = { path = "../../eth2/utils/bls" }
rest_types = { path = "../../eth2/utils/rest_types" }
beacon_chain = { path = "../beacon_chain" } beacon_chain = { path = "../beacon_chain" }
network = { path = "../network" } network = { path = "../network" }
eth2-libp2p = { path = "../eth2-libp2p" } eth2-libp2p = { path = "../eth2-libp2p" }
@ -24,7 +25,6 @@ state_processing = { path = "../../eth2/state_processing" }
types = { path = "../../eth2/types" } types = { path = "../../eth2/types" }
http = "0.1" http = "0.1"
hyper = "0.12" hyper = "0.12"
exit-future = "0.1.4"
tokio = "0.1.22" tokio = "0.1.22"
url = "2.1" url = "2.1"
lazy_static = "1.3.0" lazy_static = "1.3.0"
@ -35,6 +35,7 @@ hex = "0.3"
parking_lot = "0.9" parking_lot = "0.9"
futures = "0.1.29" futures = "0.1.29"
operation_pool = { path = "../../eth2/operation_pool" } operation_pool = { path = "../../eth2/operation_pool" }
rayon = "1.3.0"
[dev-dependencies] [dev-dependencies]
remote_beacon_node = { path = "../../eth2/utils/remote_beacon_node" } remote_beacon_node = { path = "../../eth2/utils/remote_beacon_node" }

View File

@ -5,29 +5,17 @@ use crate::{ApiError, ApiResult, BoxFut, UrlQuery};
use beacon_chain::{BeaconChain, BeaconChainTypes, StateSkipConfig}; use beacon_chain::{BeaconChain, BeaconChainTypes, StateSkipConfig};
use futures::{Future, Stream}; use futures::{Future, Stream};
use hyper::{Body, Request}; use hyper::{Body, Request};
use serde::{Deserialize, Serialize}; use rest_types::{
use ssz_derive::{Decode, Encode}; BlockResponse, CanonicalHeadResponse, Committee, HeadBeaconBlock, StateResponse,
ValidatorRequest, ValidatorResponse,
};
use std::sync::Arc; use std::sync::Arc;
use store::Store; use store::Store;
use types::{ use types::{
AttesterSlashing, BeaconState, CommitteeIndex, EthSpec, Hash256, ProposerSlashing, AttesterSlashing, BeaconState, EthSpec, Hash256, ProposerSlashing, PublicKeyBytes,
PublicKeyBytes, RelativeEpoch, SignedBeaconBlock, Slot, Validator, RelativeEpoch, Slot,
}; };
/// Information about the block and state that are at head of the beacon chain.
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
pub struct CanonicalHeadResponse {
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`. /// HTTP handler to return a `BeaconBlock` at a given `root` or `slot`.
pub fn get_head<T: BeaconChainTypes>( pub fn get_head<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
@ -62,15 +50,7 @@ pub fn get_head<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&head) ResponseBuilder::new(&req)?.body(&head)
} }
/// Information about a block that is at the head of a chain. May or may not represent the /// HTTP handler to return a list of head BeaconBlocks.
/// canonical head.
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
pub struct HeadBeaconBlock {
pub beacon_block_root: Hash256,
pub beacon_block_slot: Slot,
}
/// HTTP handler to return a list of head block roots.
pub fn get_heads<T: BeaconChainTypes>( pub fn get_heads<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
@ -87,14 +67,7 @@ pub fn get_heads<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&heads) ResponseBuilder::new(&req)?.body(&heads)
} }
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)] /// HTTP handler to return a `BeaconBlock` at a given `root` or `slot`.
#[serde(bound = "T: EthSpec")]
pub struct BlockResponse<T: EthSpec> {
pub root: Hash256,
pub beacon_block: SignedBeaconBlock<T>,
}
/// HTTP handler to return a `SignedBeaconBlock` at a given `root` or `slot`.
pub fn get_block<T: BeaconChainTypes>( pub fn get_block<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
@ -158,14 +131,6 @@ pub fn get_fork<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&beacon_chain.head()?.beacon_state.fork) ResponseBuilder::new(&req)?.body(&beacon_chain.head()?.beacon_state.fork)
} }
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
pub struct ValidatorResponse {
pub pubkey: PublicKeyBytes,
pub validator_index: Option<usize>,
pub balance: Option<u64>,
pub validator: Option<Validator>,
}
/// HTTP handler to which accepts a query string of a list of validator pubkeys and maps it to a /// HTTP handler to which accepts a query string of a list of validator pubkeys and maps it to a
/// `ValidatorResponse`. /// `ValidatorResponse`.
/// ///
@ -246,13 +211,6 @@ pub fn get_active_validators<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&validators) ResponseBuilder::new(&req)?.body(&validators)
} }
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
pub struct ValidatorRequest {
/// If set to `None`, uses the canonical head state.
pub state_root: Option<Hash256>,
pub pubkeys: Vec<PublicKeyBytes>,
}
/// HTTP handler to which accepts a `ValidatorRequest` and returns a `ValidatorResponse` for /// HTTP handler to which accepts a `ValidatorRequest` and returns a `ValidatorResponse` for
/// each of the given `pubkeys`. When `state_root` is `None`, the canonical head is used. /// each of the given `pubkeys`. When `state_root` is `None`, the canonical head is used.
/// ///
@ -365,13 +323,6 @@ fn validator_response_by_pubkey<E: EthSpec>(
} }
} }
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
pub struct Committee {
pub slot: Slot,
pub index: CommitteeIndex,
pub committee: Vec<usize>,
}
/// HTTP handler /// HTTP handler
pub fn get_committees<T: BeaconChainTypes>( pub fn get_committees<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
@ -405,13 +356,6 @@ pub fn get_committees<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&committees) ResponseBuilder::new(&req)?.body(&committees)
} }
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Encode, Decode)]
#[serde(bound = "T: EthSpec")]
pub struct StateResponse<T: EthSpec> {
pub root: Hash256,
pub beacon_state: BeaconState<T>,
}
/// HTTP handler to return a `BeaconState` at a given `root` or `slot`. /// HTTP handler to return a `BeaconState` at a given `root` or `slot`.
/// ///
/// Will not return a state if the request slot is in the future. Will return states higher than /// Will not return a state if the request slot is in the future. Will return states higher than

View File

@ -4,11 +4,12 @@ use crate::{ApiError, ApiResult, BoxFut, UrlQuery};
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use futures::{Future, Stream}; use futures::{Future, Stream};
use hyper::{Body, Request}; use hyper::{Body, Request};
use rest_types::{IndividualVotesRequest, IndividualVotesResponse};
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use state_processing::per_epoch_processing::{TotalBalances, ValidatorStatus, ValidatorStatuses}; use state_processing::per_epoch_processing::{TotalBalances, ValidatorStatuses};
use std::sync::Arc; use std::sync::Arc;
use types::{Epoch, EthSpec, PublicKeyBytes}; use types::EthSpec;
/// The results of validators voting during an epoch. /// The results of validators voting during an epoch.
/// ///
@ -70,68 +71,6 @@ pub fn get_vote_count<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&report) ResponseBuilder::new(&req)?.body(&report)
} }
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone, Encode, Decode)]
pub struct IndividualVotesRequest {
pub epoch: Epoch,
pub pubkeys: Vec<PublicKeyBytes>,
}
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone, Encode, Decode)]
pub struct IndividualVote {
/// True if the validator has been slashed, ever.
pub is_slashed: bool,
/// True if the validator can withdraw in the current epoch.
pub is_withdrawable_in_current_epoch: bool,
/// True if the validator was active in the state's _current_ epoch.
pub is_active_in_current_epoch: bool,
/// True if the validator was active in the state's _previous_ epoch.
pub is_active_in_previous_epoch: bool,
/// The validator's effective balance in the _current_ epoch.
pub current_epoch_effective_balance_gwei: u64,
/// True if the validator had an attestation included in the _current_ epoch.
pub is_current_epoch_attester: bool,
/// True if the validator's beacon block root attestation for the first slot of the _current_
/// epoch matches the block root known to the state.
pub is_current_epoch_target_attester: bool,
/// True if the validator had an attestation included in the _previous_ epoch.
pub is_previous_epoch_attester: bool,
/// True if the validator's beacon block root attestation for the first slot of the _previous_
/// epoch matches the block root known to the state.
pub is_previous_epoch_target_attester: bool,
/// True if the validator's beacon block root attestation in the _previous_ epoch at the
/// attestation's slot (`attestation_data.slot`) matches the block root known to the state.
pub is_previous_epoch_head_attester: bool,
}
impl Into<IndividualVote> for ValidatorStatus {
fn into(self) -> IndividualVote {
IndividualVote {
is_slashed: self.is_slashed,
is_withdrawable_in_current_epoch: self.is_withdrawable_in_current_epoch,
is_active_in_current_epoch: self.is_active_in_current_epoch,
is_active_in_previous_epoch: self.is_active_in_previous_epoch,
current_epoch_effective_balance_gwei: self.current_epoch_effective_balance,
is_current_epoch_attester: self.is_current_epoch_attester,
is_current_epoch_target_attester: self.is_current_epoch_target_attester,
is_previous_epoch_attester: self.is_previous_epoch_attester,
is_previous_epoch_target_attester: self.is_previous_epoch_target_attester,
is_previous_epoch_head_attester: self.is_previous_epoch_head_attester,
}
}
}
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone, Encode, Decode)]
pub struct IndividualVotesResponse {
/// The epoch which is considered the "current" epoch.
pub epoch: Epoch,
/// The validators public key.
pub pubkey: PublicKeyBytes,
/// The index of the validator in state.validators.
pub validator_index: Option<usize>,
/// Voting statistics for the validator, if they voted in the given epoch.
pub vote: Option<IndividualVote>,
}
pub fn post_individual_votes<T: BeaconChainTypes>( pub fn post_individual_votes<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,

View File

@ -33,11 +33,11 @@ impl ApiError {
impl Into<Response<Body>> for ApiError { impl Into<Response<Body>> for ApiError {
fn into(self) -> Response<Body> { fn into(self) -> Response<Body> {
let status_code = self.status_code(); let (status_code, desc) = self.status_code();
Response::builder() Response::builder()
.status(status_code.0) .status(status_code)
.header("content-type", "text/plain; charset=utf-8") .header("content-type", "text/plain; charset=utf-8")
.body(Body::from(status_code.1)) .body(Body::from(desc))
.expect("Response should always be created.") .expect("Response should always be created.")
} }
} }

View File

@ -1,20 +1,16 @@
use crate::{ApiError, ApiResult}; use crate::{ApiError, ApiResult, NetworkChannel};
use beacon_chain::{BeaconChain, BeaconChainTypes, StateSkipConfig}; use beacon_chain::{BeaconChain, BeaconChainTypes, StateSkipConfig};
use bls::PublicKeyBytes; use bls::PublicKeyBytes;
use eth2_libp2p::GossipTopic;
use eth2_libp2p::PubsubMessage; use eth2_libp2p::PubsubMessage;
use hex; use hex;
use http::header; use http::header;
use hyper::{Body, Request}; use hyper::{Body, Request};
use network::NetworkMessage; use network::NetworkMessage;
use parking_lot::RwLock; use ssz::Decode;
use ssz::{Decode, Encode};
use std::sync::Arc;
use store::{iter::AncestorIter, Store}; use store::{iter::AncestorIter, Store};
use tokio::sync::mpsc;
use types::{ use types::{
Attestation, BeaconState, CommitteeIndex, Epoch, EthSpec, Hash256, RelativeEpoch, Signature, Attestation, BeaconState, ChainSpec, CommitteeIndex, Epoch, EthSpec, Hash256, RelativeEpoch,
SignedBeaconBlock, Slot, SignedAggregateAndProof, SignedBeaconBlock, Slot,
}; };
/// Parse a slot. /// Parse a slot.
@ -49,7 +45,7 @@ pub fn parse_committee_index(string: &str) -> Result<CommitteeIndex, ApiError> {
/// Checks the provided request to ensure that the `content-type` header. /// 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 /// 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. /// explicitly specify `application/json`. If anything else is provided, an error is returned.
pub fn check_content_type_for_json(req: &Request<Body>) -> Result<(), ApiError> { pub fn check_content_type_for_json(req: &Request<Body>) -> Result<(), ApiError> {
match req.headers().get(header::CONTENT_TYPE) { match req.headers().get(header::CONTENT_TYPE) {
Some(h) if h == "application/json" => Ok(()), Some(h) if h == "application/json" => Ok(()),
@ -61,24 +57,26 @@ pub fn check_content_type_for_json(req: &Request<Body>) -> Result<(), ApiError>
} }
} }
/// Parse a signature from a `0x` preixed string. /// Parse an SSZ object from some hex-encoded bytes.
pub fn parse_signature(string: &str) -> Result<Signature, ApiError> { ///
/// E.g., A signature is `"0x0000000000000000000000000000000000000000000000000000000000000000"`
pub fn parse_hex_ssz_bytes<T: Decode>(string: &str) -> Result<T, ApiError> {
const PREFIX: &str = "0x"; const PREFIX: &str = "0x";
if string.starts_with(PREFIX) { if string.starts_with(PREFIX) {
let trimmed = string.trim_start_matches(PREFIX); let trimmed = string.trim_start_matches(PREFIX);
let bytes = hex::decode(trimmed) let bytes = hex::decode(trimmed)
.map_err(|e| ApiError::BadRequest(format!("Unable to parse signature hex: {:?}", e)))?; .map_err(|e| ApiError::BadRequest(format!("Unable to parse SSZ hex: {:?}", e)))?;
Signature::from_ssz_bytes(&bytes) T::from_ssz_bytes(&bytes)
.map_err(|e| ApiError::BadRequest(format!("Unable to parse signature bytes: {:?}", e))) .map_err(|e| ApiError::BadRequest(format!("Unable to parse SSZ bytes: {:?}", e)))
} else { } else {
Err(ApiError::BadRequest( Err(ApiError::BadRequest(
"Signature must have a 0x prefix".to_string(), "Hex bytes must have a 0x prefix".to_string(),
)) ))
} }
} }
/// Parse a root from a `0x` preixed string. /// Parse a root from a `0x` prefixed string.
/// ///
/// E.g., `"0x0000000000000000000000000000000000000000000000000000000000000000"` /// E.g., `"0x0000000000000000000000000000000000000000000000000000000000000000"`
pub fn parse_root(string: &str) -> Result<Hash256, ApiError> { pub fn parse_root(string: &str) -> Result<Hash256, ApiError> {
@ -232,18 +230,14 @@ pub fn implementation_pending_response(_req: Request<Body>) -> ApiResult {
} }
pub fn publish_beacon_block_to_network<T: BeaconChainTypes + 'static>( pub fn publish_beacon_block_to_network<T: BeaconChainTypes + 'static>(
chan: Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>, mut chan: NetworkChannel<T::EthSpec>,
block: SignedBeaconBlock<T::EthSpec>, block: SignedBeaconBlock<T::EthSpec>,
) -> Result<(), ApiError> { ) -> Result<(), ApiError> {
// create the network topic to send on // send the block via SSZ encoding
let topic = GossipTopic::BeaconBlock; let messages = vec![PubsubMessage::BeaconBlock(Box::new(block))];
let message = PubsubMessage::Block(block.as_ssz_bytes());
// Publish the block to the p2p network via gossipsub. // Publish the block to the p2p network via gossipsub.
if let Err(e) = chan.write().try_send(NetworkMessage::Publish { if let Err(e) = chan.try_send(NetworkMessage::Publish { messages }) {
topics: vec![topic.into()],
message,
}) {
return Err(ApiError::ServerError(format!( return Err(ApiError::ServerError(format!(
"Unable to send new block to network: {:?}", "Unable to send new block to network: {:?}",
e e
@ -253,19 +247,50 @@ pub fn publish_beacon_block_to_network<T: BeaconChainTypes + 'static>(
Ok(()) Ok(())
} }
pub fn publish_attestation_to_network<T: BeaconChainTypes + 'static>( /// Publishes a raw un-aggregated attestation to the network.
chan: Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>, pub fn publish_raw_attestations_to_network<T: BeaconChainTypes + 'static>(
attestation: Attestation<T::EthSpec>, mut chan: NetworkChannel<T::EthSpec>,
attestations: Vec<Attestation<T::EthSpec>>,
spec: &ChainSpec,
) -> Result<(), ApiError> { ) -> Result<(), ApiError> {
// create the network topic to send on let messages = attestations
let topic = GossipTopic::BeaconAttestation; .into_iter()
let message = PubsubMessage::Attestation(attestation.as_ssz_bytes()); .map(|attestation| {
// create the gossip message to send to the network
let subnet_id = attestation
.subnet_id(spec)
.map_err(|e| ApiError::ServerError(format!("Unable to get subnet id: {:?}", e)))?;
// Publish the attestation to the p2p network via gossipsub. Ok(PubsubMessage::Attestation(Box::new((
if let Err(e) = chan.write().try_send(NetworkMessage::Publish { subnet_id,
topics: vec![topic.into()], attestation,
message, ))))
}) { })
.collect::<Result<Vec<_>, ApiError>>()?;
// Publish the attestations to the p2p network via gossipsub.
if let Err(e) = chan.try_send(NetworkMessage::Publish { messages }) {
return Err(ApiError::ServerError(format!(
"Unable to send new attestation to network: {:?}",
e
)));
}
Ok(())
}
/// Publishes an aggregated attestation to the network.
pub fn publish_aggregate_attestations_to_network<T: BeaconChainTypes + 'static>(
mut chan: NetworkChannel<T::EthSpec>,
signed_proofs: Vec<SignedAggregateAndProof<T::EthSpec>>,
) -> Result<(), ApiError> {
let messages = signed_proofs
.into_iter()
.map(|signed_proof| PubsubMessage::AggregateAndProofAttestation(Box::new(signed_proof)))
.collect::<Vec<_>>();
// Publish the attestations to the p2p network via gossipsub.
if let Err(e) = chan.try_send(NetworkMessage::Publish { messages }) {
return Err(ApiError::ServerError(format!( return Err(ApiError::ServerError(format!(
"Unable to send new attestation to network: {:?}", "Unable to send new attestation to network: {:?}",
e e

View File

@ -10,6 +10,7 @@ pub mod config;
mod consensus; mod consensus;
mod error; mod error;
mod helpers; mod helpers;
mod lighthouse;
mod metrics; mod metrics;
mod network; mod network;
mod node; mod node;
@ -21,39 +22,32 @@ mod validator;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use client_network::NetworkMessage; use client_network::NetworkMessage;
use client_network::Service as NetworkService;
pub use config::ApiEncodingFormat; pub use config::ApiEncodingFormat;
use error::{ApiError, ApiResult}; use error::{ApiError, ApiResult};
use eth2_config::Eth2Config; use eth2_config::Eth2Config;
use eth2_libp2p::NetworkGlobals;
use hyper::rt::Future; use hyper::rt::Future;
use hyper::server::conn::AddrStream; use hyper::server::conn::AddrStream;
use hyper::service::{make_service_fn, service_fn}; use hyper::service::{make_service_fn, service_fn};
use hyper::{Body, Request, Response, Server}; use hyper::{Body, Request, Response, Server};
use parking_lot::RwLock;
use slog::{info, warn}; use slog::{info, warn};
use std::net::SocketAddr; use std::net::SocketAddr;
use std::ops::Deref; use std::ops::Deref;
use std::path::PathBuf; use std::path::PathBuf;
use std::sync::Arc; use std::sync::Arc;
use tokio::runtime::TaskExecutor; use tokio::runtime::TaskExecutor;
use tokio::sync::mpsc; use tokio::sync::{mpsc, oneshot};
use url_query::UrlQuery; use url_query::UrlQuery;
pub use crate::helpers::parse_pubkey_bytes; pub use crate::helpers::parse_pubkey_bytes;
pub use beacon::{
BlockResponse, CanonicalHeadResponse, Committee, HeadBeaconBlock, StateResponse,
ValidatorRequest, ValidatorResponse,
};
pub use config::Config; pub use config::Config;
pub use consensus::{IndividualVote, IndividualVotesRequest, IndividualVotesResponse};
pub use validator::{ValidatorDutiesRequest, ValidatorDuty};
pub type BoxFut = Box<dyn Future<Item = Response<Body>, Error = ApiError> + Send>; pub type BoxFut = Box<dyn Future<Item = Response<Body>, Error = ApiError> + Send>;
pub type NetworkChannel = Arc<RwLock<mpsc::UnboundedSender<NetworkMessage>>>; pub type NetworkChannel<T> = mpsc::UnboundedSender<NetworkMessage<T>>;
pub struct NetworkInfo<T: BeaconChainTypes> { pub struct NetworkInfo<T: BeaconChainTypes> {
pub network_service: Arc<NetworkService<T>>, pub network_globals: Arc<NetworkGlobals<T::EthSpec>>,
pub network_chan: mpsc::UnboundedSender<NetworkMessage>, pub network_chan: NetworkChannel<T::EthSpec>,
} }
// Allowing more than 7 arguments. // Allowing more than 7 arguments.
@ -67,7 +61,7 @@ pub fn start_server<T: BeaconChainTypes>(
freezer_db_path: PathBuf, freezer_db_path: PathBuf,
eth2_config: Eth2Config, eth2_config: Eth2Config,
log: slog::Logger, log: slog::Logger,
) -> Result<(exit_future::Signal, SocketAddr), hyper::Error> { ) -> Result<(oneshot::Sender<()>, SocketAddr), hyper::Error> {
let inner_log = log.clone(); let inner_log = log.clone();
let eth2_config = Arc::new(eth2_config); let eth2_config = Arc::new(eth2_config);
@ -76,8 +70,8 @@ pub fn start_server<T: BeaconChainTypes>(
let beacon_chain = beacon_chain.clone(); let beacon_chain = beacon_chain.clone();
let log = inner_log.clone(); let log = inner_log.clone();
let eth2_config = eth2_config.clone(); let eth2_config = eth2_config.clone();
let network_service = network_info.network_service.clone(); let network_globals = network_info.network_globals.clone();
let network_channel = Arc::new(RwLock::new(network_info.network_chan.clone())); let network_channel = network_info.network_chan.clone();
let db_path = db_path.clone(); let db_path = db_path.clone();
let freezer_db_path = freezer_db_path.clone(); let freezer_db_path = freezer_db_path.clone();
@ -85,7 +79,7 @@ pub fn start_server<T: BeaconChainTypes>(
router::route( router::route(
req, req,
beacon_chain.clone(), beacon_chain.clone(),
network_service.clone(), network_globals.clone(),
network_channel.clone(), network_channel.clone(),
eth2_config.clone(), eth2_config.clone(),
log.clone(), log.clone(),
@ -105,7 +99,7 @@ pub fn start_server<T: BeaconChainTypes>(
let actual_listen_addr = server.local_addr(); let actual_listen_addr = server.local_addr();
// Build a channel to kill the HTTP server. // Build a channel to kill the HTTP server.
let (exit_signal, exit) = exit_future::signal(); let (exit_signal, exit) = oneshot::channel();
let inner_log = log.clone(); let inner_log = log.clone();
let server_exit = exit.and_then(move |_| { let server_exit = exit.and_then(move |_| {
info!(inner_log, "HTTP service shutdown"); info!(inner_log, "HTTP service shutdown");

View File

@ -0,0 +1,58 @@
//! This contains a collection of lighthouse specific HTTP endpoints.
use crate::response_builder::ResponseBuilder;
use crate::ApiResult;
use eth2_libp2p::{NetworkGlobals, PeerInfo};
use hyper::{Body, Request};
use serde::Serialize;
use std::sync::Arc;
use types::EthSpec;
/// The syncing state of the beacon node.
pub fn syncing<T: EthSpec>(
req: Request<Body>,
network_globals: Arc<NetworkGlobals<T>>,
) -> ApiResult {
ResponseBuilder::new(&req)?.body_no_ssz(&network_globals.sync_state())
}
/// Returns all known peers and corresponding information
pub fn peers<T: EthSpec>(req: Request<Body>, network_globals: Arc<NetworkGlobals<T>>) -> ApiResult {
let peers: Vec<Peer<T>> = network_globals
.peers
.read()
.peers()
.map(|(peer_id, peer_info)| Peer {
peer_id: peer_id.to_string(),
peer_info: peer_info.clone(),
})
.collect();
ResponseBuilder::new(&req)?.body_no_ssz(&peers)
}
/// Returns all known connected peers and their corresponding information
pub fn connected_peers<T: EthSpec>(
req: Request<Body>,
network_globals: Arc<NetworkGlobals<T>>,
) -> ApiResult {
let peers: Vec<Peer<T>> = network_globals
.peers
.read()
.connected_peers()
.map(|(peer_id, peer_info)| Peer {
peer_id: peer_id.to_string(),
peer_info: peer_info.clone(),
})
.collect();
ResponseBuilder::new(&req)?.body_no_ssz(&peers)
}
/// Information returned by `peers` and `connected_peers`.
#[derive(Clone, Debug, Serialize)]
#[serde(bound = "T: EthSpec")]
struct Peer<T: EthSpec> {
/// The Peer's ID
peer_id: String,
/// The PeerInfo associated with the peer.
peer_info: PeerInfo<T>,
}

View File

@ -1,6 +1,6 @@
use crate::error::ApiResult; use crate::error::ApiResult;
use crate::response_builder::ResponseBuilder; use crate::response_builder::ResponseBuilder;
use crate::NetworkService; use crate::NetworkGlobals;
use beacon_chain::BeaconChainTypes; use beacon_chain::BeaconChainTypes;
use eth2_libp2p::{Multiaddr, PeerId}; use eth2_libp2p::{Multiaddr, PeerId};
use hyper::{Body, Request}; use hyper::{Body, Request};
@ -11,7 +11,7 @@ use std::sync::Arc;
/// Returns a list of `Multiaddr`, serialized according to their `serde` impl. /// Returns a list of `Multiaddr`, serialized according to their `serde` impl.
pub fn get_listen_addresses<T: BeaconChainTypes>( pub fn get_listen_addresses<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
) -> ApiResult { ) -> ApiResult {
let multiaddresses: Vec<Multiaddr> = network.listen_multiaddrs(); let multiaddresses: Vec<Multiaddr> = network.listen_multiaddrs();
ResponseBuilder::new(&req)?.body_no_ssz(&multiaddresses) ResponseBuilder::new(&req)?.body_no_ssz(&multiaddresses)
@ -22,9 +22,9 @@ pub fn get_listen_addresses<T: BeaconChainTypes>(
/// Returns the TCP port number in its plain form (which is also valid JSON serialization) /// Returns the TCP port number in its plain form (which is also valid JSON serialization)
pub fn get_listen_port<T: BeaconChainTypes>( pub fn get_listen_port<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
) -> ApiResult { ) -> ApiResult {
ResponseBuilder::new(&req)?.body(&network.listen_port()) ResponseBuilder::new(&req)?.body(&network.listen_port_tcp())
} }
/// HTTP handler to return the Discv5 ENR from the client's libp2p service. /// HTTP handler to return the Discv5 ENR from the client's libp2p service.
@ -32,14 +32,9 @@ pub fn get_listen_port<T: BeaconChainTypes>(
/// ENR is encoded as base64 string. /// ENR is encoded as base64 string.
pub fn get_enr<T: BeaconChainTypes>( pub fn get_enr<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
) -> ApiResult { ) -> ApiResult {
ResponseBuilder::new(&req)?.body_no_ssz( ResponseBuilder::new(&req)?.body_no_ssz(&network.local_enr().to_base64())
&network
.local_enr()
.map(|enr| enr.to_base64())
.unwrap_or_else(|| "".into()),
)
} }
/// HTTP handler to return the `PeerId` from the client's libp2p service. /// HTTP handler to return the `PeerId` from the client's libp2p service.
@ -47,7 +42,7 @@ pub fn get_enr<T: BeaconChainTypes>(
/// PeerId is encoded as base58 string. /// PeerId is encoded as base58 string.
pub fn get_peer_id<T: BeaconChainTypes>( pub fn get_peer_id<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
) -> ApiResult { ) -> ApiResult {
ResponseBuilder::new(&req)?.body_no_ssz(&network.local_peer_id().to_base58()) ResponseBuilder::new(&req)?.body_no_ssz(&network.local_peer_id().to_base58())
} }
@ -55,7 +50,7 @@ pub fn get_peer_id<T: BeaconChainTypes>(
/// HTTP handler 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>( pub fn get_peer_count<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
) -> ApiResult { ) -> ApiResult {
ResponseBuilder::new(&req)?.body(&network.connected_peers()) ResponseBuilder::new(&req)?.body(&network.connected_peers())
} }
@ -65,11 +60,12 @@ pub fn get_peer_count<T: BeaconChainTypes>(
/// Peers are presented as a list of `PeerId::to_string()`. /// Peers are presented as a list of `PeerId::to_string()`.
pub fn get_peer_list<T: BeaconChainTypes>( pub fn get_peer_list<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
network: Arc<NetworkService<T>>, network: Arc<NetworkGlobals<T::EthSpec>>,
) -> ApiResult { ) -> ApiResult {
let connected_peers: Vec<String> = network let connected_peers: Vec<String> = network
.connected_peer_set() .peers
.iter() .read()
.connected_peer_ids()
.map(PeerId::to_string) .map(PeerId::to_string)
.collect(); .collect();
ResponseBuilder::new(&req)?.body_no_ssz(&connected_peers) ResponseBuilder::new(&req)?.body_no_ssz(&connected_peers)

View File

@ -1,9 +1,43 @@
use crate::response_builder::ResponseBuilder; use crate::response_builder::ResponseBuilder;
use crate::ApiResult; use crate::ApiResult;
use eth2_libp2p::{types::SyncState, NetworkGlobals};
use hyper::{Body, Request}; use hyper::{Body, Request};
use rest_types::{SyncingResponse, SyncingStatus};
use std::sync::Arc;
use types::{EthSpec, Slot};
use version; use version;
/// Read the version string from the current Lighthouse build. /// Read the version string from the current Lighthouse build.
pub fn get_version(req: Request<Body>) -> ApiResult { pub fn get_version(req: Request<Body>) -> ApiResult {
ResponseBuilder::new(&req)?.body_no_ssz(&version::version()) ResponseBuilder::new(&req)?.body_no_ssz(&version::version())
} }
pub fn syncing<T: EthSpec>(
req: Request<Body>,
network: Arc<NetworkGlobals<T>>,
current_slot: Slot,
) -> ApiResult {
let (starting_slot, highest_slot) = match network.sync_state() {
SyncState::SyncingFinalized {
start_slot,
head_slot,
..
}
| SyncState::SyncingHead {
start_slot,
head_slot,
} => (start_slot, head_slot),
SyncState::Synced | SyncState::Stalled => (Slot::from(0u64), current_slot),
};
let sync_status = SyncingStatus {
starting_slot,
current_slot,
highest_slot,
};
ResponseBuilder::new(&req)?.body(&SyncingResponse {
is_syncing: network.is_syncing(),
sync_status,
})
}

View File

@ -1,16 +1,17 @@
use crate::{ use crate::{
advanced, beacon, consensus, error::ApiError, helpers, metrics, network, node, spec, validator, advanced, beacon, consensus, error::ApiError, helpers, lighthouse, metrics, network, node,
BoxFut, NetworkChannel, spec, validator, BoxFut, NetworkChannel,
}; };
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use client_network::Service as NetworkService;
use eth2_config::Eth2Config; use eth2_config::Eth2Config;
use eth2_libp2p::NetworkGlobals;
use futures::{Future, IntoFuture}; use futures::{Future, IntoFuture};
use hyper::{Body, Error, Method, Request, Response}; use hyper::{Body, Error, Method, Request, Response};
use slog::debug; use slog::debug;
use std::path::PathBuf; use std::path::PathBuf;
use std::sync::Arc; use std::sync::Arc;
use std::time::Instant; use std::time::Instant;
use types::Slot;
fn into_boxfut<F: IntoFuture + 'static>(item: F) -> BoxFut fn into_boxfut<F: IntoFuture + 'static>(item: F) -> BoxFut
where where
@ -25,8 +26,8 @@ where
pub fn route<T: BeaconChainTypes>( pub fn route<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_service: Arc<NetworkService<T>>, network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_channel: NetworkChannel, network_channel: NetworkChannel<T::EthSpec>,
eth2_config: Arc<Eth2Config>, eth2_config: Arc<Eth2Config>,
local_log: slog::Logger, local_log: slog::Logger,
db_path: PathBuf, db_path: PathBuf,
@ -44,27 +45,37 @@ pub fn route<T: BeaconChainTypes>(
// Methods for Client // Methods for Client
(&Method::GET, "/node/version") => into_boxfut(node::get_version(req)), (&Method::GET, "/node/version") => into_boxfut(node::get_version(req)),
(&Method::GET, "/node/syncing") => { (&Method::GET, "/node/syncing") => {
into_boxfut(helpers::implementation_pending_response(req)) // inform the current slot, or set to 0
let current_slot = beacon_chain
.head_info()
.map(|info| info.slot)
.unwrap_or_else(|_| Slot::from(0u64));
into_boxfut(node::syncing::<T::EthSpec>(
req,
network_globals,
current_slot,
))
} }
// Methods for Network // Methods for Network
(&Method::GET, "/network/enr") => { (&Method::GET, "/network/enr") => {
into_boxfut(network::get_enr::<T>(req, network_service)) into_boxfut(network::get_enr::<T>(req, network_globals))
} }
(&Method::GET, "/network/peer_count") => { (&Method::GET, "/network/peer_count") => {
into_boxfut(network::get_peer_count::<T>(req, network_service)) into_boxfut(network::get_peer_count::<T>(req, network_globals))
} }
(&Method::GET, "/network/peer_id") => { (&Method::GET, "/network/peer_id") => {
into_boxfut(network::get_peer_id::<T>(req, network_service)) into_boxfut(network::get_peer_id::<T>(req, network_globals))
} }
(&Method::GET, "/network/peers") => { (&Method::GET, "/network/peers") => {
into_boxfut(network::get_peer_list::<T>(req, network_service)) into_boxfut(network::get_peer_list::<T>(req, network_globals))
} }
(&Method::GET, "/network/listen_port") => { (&Method::GET, "/network/listen_port") => {
into_boxfut(network::get_listen_port::<T>(req, network_service)) into_boxfut(network::get_listen_port::<T>(req, network_globals))
} }
(&Method::GET, "/network/listen_addresses") => { (&Method::GET, "/network/listen_addresses") => {
into_boxfut(network::get_listen_addresses::<T>(req, network_service)) into_boxfut(network::get_listen_addresses::<T>(req, network_globals))
} }
// Methods for Beacon Node // Methods for Beacon Node
@ -124,6 +135,9 @@ pub fn route<T: BeaconChainTypes>(
drop(timer); drop(timer);
into_boxfut(response) into_boxfut(response)
} }
(&Method::POST, "/validator/subscribe") => {
validator::post_validator_subscriptions::<T>(req, network_channel)
}
(&Method::GET, "/validator/duties/all") => { (&Method::GET, "/validator/duties/all") => {
into_boxfut(validator::get_all_validator_duties::<T>(req, beacon_chain)) into_boxfut(validator::get_all_validator_duties::<T>(req, beacon_chain))
} }
@ -147,10 +161,22 @@ pub fn route<T: BeaconChainTypes>(
drop(timer); drop(timer);
into_boxfut(response) into_boxfut(response)
} }
(&Method::POST, "/validator/attestation") => { (&Method::GET, "/validator/aggregate_attestation") => {
validator::publish_attestation::<T>(req, beacon_chain, network_channel, log) into_boxfut(validator::get_aggregate_attestation::<T>(req, beacon_chain))
}
(&Method::POST, "/validator/attestations") => {
validator::publish_attestations::<T>(req, beacon_chain, network_channel, log)
}
(&Method::POST, "/validator/aggregate_and_proofs") => {
validator::publish_aggregate_and_proofs::<T>(
req,
beacon_chain,
network_channel,
log,
)
} }
// Methods for consensus
(&Method::GET, "/consensus/global_votes") => { (&Method::GET, "/consensus/global_votes") => {
into_boxfut(consensus::get_vote_count::<T>(req, beacon_chain)) into_boxfut(consensus::get_vote_count::<T>(req, beacon_chain))
} }
@ -177,7 +203,6 @@ pub fn route<T: BeaconChainTypes>(
(&Method::GET, "/advanced/operation_pool") => { (&Method::GET, "/advanced/operation_pool") => {
into_boxfut(advanced::get_operation_pool::<T>(req, beacon_chain)) into_boxfut(advanced::get_operation_pool::<T>(req, beacon_chain))
} }
(&Method::GET, "/metrics") => into_boxfut(metrics::get_prometheus::<T>( (&Method::GET, "/metrics") => into_boxfut(metrics::get_prometheus::<T>(
req, req,
beacon_chain, beacon_chain,
@ -185,6 +210,16 @@ pub fn route<T: BeaconChainTypes>(
freezer_db_path, freezer_db_path,
)), )),
// Lighthouse specific
(&Method::GET, "/lighthouse/syncing") => {
into_boxfut(lighthouse::syncing::<T::EthSpec>(req, network_globals))
}
(&Method::GET, "/lighthouse/peers") => {
into_boxfut(lighthouse::peers::<T::EthSpec>(req, network_globals))
}
(&Method::GET, "/lighthouse/connected_peers") => into_boxfut(
lighthouse::connected_peers::<T::EthSpec>(req, network_globals),
),
_ => Box::new(futures::future::err(ApiError::NotFound( _ => Box::new(futures::future::err(ApiError::NotFound(
"Request path and/or method not found.".to_owned(), "Request path and/or method not found.".to_owned(),
))), ))),

View File

@ -1,7 +1,7 @@
use crate::helpers::{parse_committee_index, parse_epoch, parse_signature, parse_slot}; use crate::helpers::{parse_committee_index, parse_epoch, parse_hex_ssz_bytes, parse_slot};
use crate::ApiError; use crate::ApiError;
use hyper::Request; use hyper::Request;
use types::{CommitteeIndex, Epoch, Signature, Slot}; use types::{AttestationData, CommitteeIndex, Epoch, Signature, Slot};
/// Provides handy functions for parsing the query parameters of a URL. /// Provides handy functions for parsing the query parameters of a URL.
@ -106,7 +106,13 @@ impl<'a> UrlQuery<'a> {
/// Returns the value of the first occurrence of the `randao_reveal` key. /// Returns the value of the first occurrence of the `randao_reveal` key.
pub fn randao_reveal(self) -> Result<Signature, ApiError> { pub fn randao_reveal(self) -> Result<Signature, ApiError> {
self.first_of(&["randao_reveal"]) self.first_of(&["randao_reveal"])
.and_then(|(_key, value)| parse_signature(&value)) .and_then(|(_key, value)| parse_hex_ssz_bytes(&value))
}
/// Returns the value of the first occurrence of the `attestation_data` key.
pub fn attestation_data(self) -> Result<AttestationData, ApiError> {
self.first_of(&["attestation_data"])
.and_then(|(_key, value)| parse_hex_ssz_bytes(&value))
} }
} }

View File

@ -1,47 +1,28 @@
use crate::helpers::{ use crate::helpers::{
check_content_type_for_json, publish_attestation_to_network, publish_beacon_block_to_network, check_content_type_for_json, publish_aggregate_attestations_to_network,
publish_beacon_block_to_network, publish_raw_attestations_to_network,
}; };
use crate::response_builder::ResponseBuilder; use crate::response_builder::ResponseBuilder;
use crate::{ApiError, ApiResult, BoxFut, NetworkChannel, UrlQuery}; use crate::{ApiError, ApiResult, BoxFut, NetworkChannel, UrlQuery};
use beacon_chain::{ use beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BlockProcessingOutcome, AttestationProcessingOutcome, AttestationType, BeaconChain, BeaconChainTypes, BlockError,
StateSkipConfig, StateSkipConfig,
}; };
use bls::PublicKeyBytes; use bls::PublicKeyBytes;
use futures::{Future, Stream}; use futures::{Future, Stream};
use hyper::{Body, Request}; use hyper::{Body, Request};
use serde::{Deserialize, Serialize}; use network::NetworkMessage;
use rayon::prelude::*;
use rest_types::{ValidatorDutiesRequest, ValidatorDutyBytes, ValidatorSubscription};
use slog::{error, info, warn, Logger}; use slog::{error, info, warn, Logger};
use ssz_derive::{Decode, Encode};
use std::sync::Arc; use std::sync::Arc;
use types::beacon_state::EthSpec; use types::beacon_state::EthSpec;
use types::{ use types::{
Attestation, BeaconState, CommitteeIndex, Epoch, RelativeEpoch, SignedBeaconBlock, Slot, Attestation, BeaconState, Epoch, RelativeEpoch, SignedAggregateAndProof, SignedBeaconBlock,
Slot,
}; };
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone)] /// HTTP Handler to retrieve the duties for a set of validators during a particular epoch. This
pub struct ValidatorDuty {
/// The validator's BLS public key, uniquely identifying them. _48-bytes, hex encoded with 0x prefix, case insensitive._
pub validator_pubkey: PublicKeyBytes,
/// The validator's index in `state.validators`
pub validator_index: Option<usize>,
/// The slot at which the validator must attest.
pub attestation_slot: Option<Slot>,
/// The index of the committee within `slot` of which the validator is a member.
pub attestation_committee_index: Option<CommitteeIndex>,
/// The position of the validator in the committee.
pub attestation_committee_position: Option<usize>,
/// The slots in which a validator must propose a block (can be empty).
pub block_proposal_slots: Vec<Slot>,
}
#[derive(PartialEq, Debug, Serialize, Deserialize, Clone, Encode, Decode)]
pub struct ValidatorDutiesRequest {
pub epoch: Epoch,
pub pubkeys: Vec<PublicKeyBytes>,
}
/// HTTP Handler to retrieve a the duties for a set of validators during a particular epoch. This
/// method allows for collecting bulk sets of validator duties without risking exceeding the max /// method allows for collecting bulk sets of validator duties without risking exceeding the max
/// URL length with query pairs. /// URL length with query pairs.
pub fn post_validator_duties<T: BeaconChainTypes>( pub fn post_validator_duties<T: BeaconChainTypes>(
@ -74,6 +55,42 @@ pub fn post_validator_duties<T: BeaconChainTypes>(
Box::new(future) Box::new(future)
} }
/// HTTP Handler to retrieve subscriptions for a set of validators. This allows the node to
/// organise peer discovery and topic subscription for known validators.
pub fn post_validator_subscriptions<T: BeaconChainTypes>(
req: Request<Body>,
mut network_chan: NetworkChannel<T::EthSpec>,
) -> BoxFut {
try_future!(check_content_type_for_json(&req));
let response_builder = ResponseBuilder::new(&req);
let body = req.into_body();
Box::new(
body.concat2()
.map_err(|e| ApiError::ServerError(format!("Unable to get request body: {:?}", e)))
.and_then(|chunks| {
serde_json::from_slice(&chunks).map_err(|e| {
ApiError::BadRequest(format!(
"Unable to parse JSON into ValidatorSubscriptions: {:?}",
e
))
})
})
.and_then(move |subscriptions: Vec<ValidatorSubscription>| {
network_chan
.try_send(NetworkMessage::Subscribe { subscriptions })
.map_err(|e| {
ApiError::ServerError(format!(
"Unable to subscriptions to the network: {:?}",
e
))
})?;
Ok(())
})
.and_then(|_| response_builder?.body_no_ssz(&())),
)
}
/// HTTP Handler to retrieve all validator duties for the given epoch. /// HTTP Handler to retrieve all validator duties for the given epoch.
pub fn get_all_validator_duties<T: BeaconChainTypes>( pub fn get_all_validator_duties<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
@ -154,7 +171,7 @@ fn return_validator_duties<T: BeaconChainTypes>(
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
epoch: Epoch, epoch: Epoch,
validator_pubkeys: Vec<PublicKeyBytes>, validator_pubkeys: Vec<PublicKeyBytes>,
) -> Result<Vec<ValidatorDuty>, ApiError> { ) -> Result<Vec<ValidatorDutyBytes>, ApiError> {
let mut state = get_state_for_epoch(&beacon_chain, epoch, StateSkipConfig::WithoutStateRoots)?; let mut state = get_state_for_epoch(&beacon_chain, epoch, StateSkipConfig::WithoutStateRoots)?;
let relative_epoch = RelativeEpoch::from_epoch(state.current_epoch(), epoch) let relative_epoch = RelativeEpoch::from_epoch(state.current_epoch(), epoch)
@ -209,28 +226,39 @@ fn return_validator_duties<T: BeaconChainTypes>(
)) ))
})?; })?;
// Obtain the aggregator modulo
let aggregator_modulo = duties.map(|d| {
std::cmp::max(
1,
d.committee_len as u64
/ &beacon_chain.spec.target_aggregators_per_committee,
)
});
let block_proposal_slots = validator_proposers let block_proposal_slots = validator_proposers
.iter() .iter()
.filter(|(i, _slot)| validator_index == *i) .filter(|(i, _slot)| validator_index == *i)
.map(|(_i, slot)| *slot) .map(|(_i, slot)| *slot)
.collect(); .collect();
Ok(ValidatorDuty { Ok(ValidatorDutyBytes {
validator_pubkey, validator_pubkey,
validator_index: Some(validator_index), validator_index: Some(validator_index as u64),
attestation_slot: duties.map(|d| d.slot), attestation_slot: duties.map(|d| d.slot),
attestation_committee_index: duties.map(|d| d.index), attestation_committee_index: duties.map(|d| d.index),
attestation_committee_position: duties.map(|d| d.committee_position), attestation_committee_position: duties.map(|d| d.committee_position),
block_proposal_slots, block_proposal_slots,
aggregator_modulo,
}) })
} else { } else {
Ok(ValidatorDuty { Ok(ValidatorDutyBytes {
validator_pubkey, validator_pubkey,
validator_index: None, validator_index: None,
attestation_slot: None, attestation_slot: None,
attestation_committee_index: None, attestation_committee_index: None,
attestation_committee_position: None, attestation_committee_position: None,
block_proposal_slots: vec![], block_proposal_slots: vec![],
aggregator_modulo: None,
}) })
} }
}) })
@ -270,7 +298,7 @@ pub fn get_new_beacon_block<T: BeaconChainTypes>(
pub fn publish_beacon_block<T: BeaconChainTypes>( pub fn publish_beacon_block<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_chan: NetworkChannel, network_chan: NetworkChannel<T::EthSpec>,
log: Logger, log: Logger,
) -> BoxFut { ) -> BoxFut {
try_future!(check_content_type_for_json(&req)); try_future!(check_content_type_for_json(&req));
@ -288,7 +316,7 @@ pub fn publish_beacon_block<T: BeaconChainTypes>(
.and_then(move |block: SignedBeaconBlock<T::EthSpec>| { .and_then(move |block: SignedBeaconBlock<T::EthSpec>| {
let slot = block.slot(); let slot = block.slot();
match beacon_chain.process_block(block.clone()) { match beacon_chain.process_block(block.clone()) {
Ok(BlockProcessingOutcome::Processed { block_root }) => { Ok(block_root) => {
// Block was processed, publish via gossipsub // Block was processed, publish via gossipsub
info!( info!(
log, log,
@ -331,19 +359,7 @@ pub fn publish_beacon_block<T: BeaconChainTypes>(
Ok(()) Ok(())
} }
Ok(outcome) => { Err(BlockError::BeaconChainError(e)) => {
warn!(
log,
"Invalid block from local validator";
"outcome" => format!("{:?}", outcome)
);
Err(ApiError::ProcessingError(format!(
"The SignedBeaconBlock could not be processed and has not been published: {:?}",
outcome
)))
}
Err(e) => {
error!( error!(
log, log,
"Error whilst processing block"; "Error whilst processing block";
@ -355,6 +371,18 @@ pub fn publish_beacon_block<T: BeaconChainTypes>(
e e
))) )))
} }
Err(other) => {
warn!(
log,
"Invalid block from local validator";
"outcome" => format!("{:?}", other)
);
Err(ApiError::ProcessingError(format!(
"The SignedBeaconBlock could not be processed and has not been published: {:?}",
other
)))
}
} }
}) })
.and_then(|_| response_builder?.body_no_ssz(&())) .and_then(|_| response_builder?.body_no_ssz(&()))
@ -378,11 +406,32 @@ pub fn get_new_attestation<T: BeaconChainTypes>(
ResponseBuilder::new(&req)?.body(&attestation) ResponseBuilder::new(&req)?.body(&attestation)
} }
/// HTTP Handler to publish an Attestation, which has been signed by a validator. /// HTTP Handler to retrieve the aggregate attestation for a slot
pub fn publish_attestation<T: BeaconChainTypes>( pub fn get_aggregate_attestation<T: BeaconChainTypes>(
req: Request<Body>, req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,
network_chan: NetworkChannel, ) -> ApiResult {
let query = UrlQuery::from_request(&req)?;
let attestation_data = query.attestation_data()?;
match beacon_chain.get_aggregated_attestation(&attestation_data) {
Ok(Some(attestation)) => ResponseBuilder::new(&req)?.body(&attestation),
Ok(None) => Err(ApiError::NotFound(
"No matching aggregate attestation is known".into(),
)),
Err(e) => Err(ApiError::ServerError(format!(
"Unable to obtain attestation: {:?}",
e
))),
}
}
/// HTTP Handler to publish a list of Attestations, which have been signed by a number of validators.
pub fn publish_attestations<T: BeaconChainTypes>(
req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>,
network_chan: NetworkChannel<T::EthSpec>,
log: Logger, log: Logger,
) -> BoxFut { ) -> BoxFut {
try_future!(check_content_type_for_json(&req)); try_future!(check_content_type_for_json(&req));
@ -396,13 +445,30 @@ pub fn publish_attestation<T: BeaconChainTypes>(
.and_then(|chunks| { .and_then(|chunks| {
serde_json::from_slice(&chunks.as_slice()).map_err(|e| { serde_json::from_slice(&chunks.as_slice()).map_err(|e| {
ApiError::BadRequest(format!( ApiError::BadRequest(format!(
"Unable to deserialize JSON into a SignedBeaconBlock: {:?}", "Unable to deserialize JSON into a list of attestations: {:?}",
e e
)) ))
}) })
}) })
.and_then(move |attestation: Attestation<T::EthSpec>| { .and_then(move |attestations: Vec<Attestation<T::EthSpec>>| {
match beacon_chain.process_attestation(attestation.clone()) { // Note: This is a new attestation from a validator. We want to process this and
// inform the validator whether the attestation was valid. In doing so, we store
// this un-aggregated raw attestation in the op_pool by default. This is
// sub-optimal as if we have no validators needing to aggregate, these don't need
// to be stored in the op-pool. This is minimal however as the op_pool gets pruned
// every slot
attestations.par_iter().try_for_each(|attestation| {
// In accordance with the naive aggregation strategy, the validator client should
// only publish attestations to this endpoint with a single signature.
if attestation.aggregation_bits.num_set_bits() != 1 {
return Err(ApiError::BadRequest(format!("Attestation should have exactly one aggregation bit set")))
}
// TODO: we only need to store these attestations if we're aggregating for the
// given subnet.
let attestation_type = AttestationType::Unaggregated { should_store: true };
match beacon_chain.process_attestation(attestation.clone(), attestation_type) {
Ok(AttestationProcessingOutcome::Processed) => { Ok(AttestationProcessingOutcome::Processed) => {
// Block was processed, publish via gossipsub // Block was processed, publish via gossipsub
info!( info!(
@ -413,7 +479,7 @@ pub fn publish_attestation<T: BeaconChainTypes>(
"index" => attestation.data.index, "index" => attestation.data.index,
"slot" => attestation.data.slot, "slot" => attestation.data.slot,
); );
publish_attestation_to_network::<T>(network_chan, attestation) Ok(())
} }
Ok(outcome) => { Ok(outcome) => {
warn!( warn!(
@ -423,7 +489,7 @@ pub fn publish_attestation<T: BeaconChainTypes>(
); );
Err(ApiError::ProcessingError(format!( Err(ApiError::ProcessingError(format!(
"The Attestation could not be processed and has not been published: {:?}", "An Attestation could not be processed and has not been published: {:?}",
outcome outcome
))) )))
} }
@ -440,6 +506,123 @@ pub fn publish_attestation<T: BeaconChainTypes>(
))) )))
} }
} }
})?;
Ok((attestations, beacon_chain))
})
.and_then(|(attestations, beacon_chain)| {
publish_raw_attestations_to_network::<T>(network_chan, attestations, &beacon_chain.spec)
})
.and_then(|_| response_builder?.body_no_ssz(&())),
)
}
/// HTTP Handler to publish an Attestation, which has been signed by a validator.
pub fn publish_aggregate_and_proofs<T: BeaconChainTypes>(
req: Request<Body>,
beacon_chain: Arc<BeaconChain<T>>,
network_chan: NetworkChannel<T::EthSpec>,
log: Logger,
) -> BoxFut {
try_future!(check_content_type_for_json(&req));
let response_builder = ResponseBuilder::new(&req);
Box::new(
req.into_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 list of SignedAggregateAndProof: {:?}",
e
))
})
})
.and_then(move |signed_proofs: Vec<SignedAggregateAndProof<T::EthSpec>>| {
// Verify the signatures for the aggregate and proof and if valid process the
// aggregate
// TODO: Double check speed and logic consistency of handling current fork vs
// validator fork for signatures.
// TODO: More efficient way of getting a fork?
let fork = &beacon_chain.head()?.beacon_state.fork;
// TODO: Update to shift this task to dedicated task using await
signed_proofs.par_iter().try_for_each(|signed_proof| {
let agg_proof = &signed_proof.message;
let validator_pubkey = &beacon_chain.validator_pubkey(agg_proof.aggregator_index as usize)?.ok_or_else(|| {
warn!(
log,
"Unknown validator from local validator client";
);
ApiError::ProcessingError(format!("The validator is known"))
})?;
/*
* TODO: checking that `signed_proof.is_valid()` is not sufficient. It
* is also necessary to check that the validator is actually designated as an
* aggregator for this attestation.
*
* I (Paul H) will pick this up in a future PR.
*/
if signed_proof.is_valid(validator_pubkey, fork, beacon_chain.genesis_validators_root, &beacon_chain.spec) {
let attestation = &agg_proof.aggregate;
match beacon_chain.process_attestation(attestation.clone(), AttestationType::Aggregated) {
Ok(AttestationProcessingOutcome::Processed) => {
// Block was processed, publish via gossipsub
info!(
log,
"Attestation from local validator";
"target" => attestation.data.source.epoch,
"source" => attestation.data.source.epoch,
"index" => attestation.data.index,
"slot" => attestation.data.slot,
);
Ok(())
}
Ok(outcome) => {
warn!(
log,
"Invalid attestation from local validator";
"outcome" => format!("{:?}", outcome)
);
Err(ApiError::ProcessingError(format!(
"The Attestation could not be processed and has not been published: {:?}",
outcome
)))
}
Err(e) => {
error!(
log,
"Error whilst processing attestation";
"error" => format!("{:?}", e)
);
Err(ApiError::ServerError(format!(
"Error while processing attestation: {:?}",
e
)))
}
}
} else {
error!(
log,
"Invalid AggregateAndProof Signature"
);
Err(ApiError::ServerError(format!(
"Invalid AggregateAndProof Signature"
)))
}
})?;
Ok(signed_proofs)
})
.and_then(move |signed_proofs| {
publish_aggregate_attestations_to_network::<T>(network_chan, signed_proofs)
}) })
.and_then(|_| response_builder?.body_no_ssz(&())), .and_then(|_| response_builder?.body_no_ssz(&())),
) )

View File

@ -6,9 +6,9 @@ use node_test_rig::{
testing_client_config, ClientConfig, ClientGenesis, LocalBeaconNode, testing_client_config, ClientConfig, ClientGenesis, LocalBeaconNode,
}; };
use remote_beacon_node::{ use remote_beacon_node::{
Committee, HeadBeaconBlock, PersistedOperationPool, PublishStatus, ValidatorDuty, Committee, HeadBeaconBlock, PersistedOperationPool, PublishStatus, ValidatorResponse,
ValidatorResponse,
}; };
use rest_types::ValidatorDutyBytes;
use std::convert::TryInto; use std::convert::TryInto;
use std::sync::Arc; use std::sync::Arc;
use types::{ use types::{
@ -17,7 +17,8 @@ use types::{
generate_deterministic_keypair, AttesterSlashingTestTask, ProposerSlashingTestTask, generate_deterministic_keypair, AttesterSlashingTestTask, ProposerSlashingTestTask,
}, },
BeaconBlock, BeaconState, ChainSpec, Domain, Epoch, EthSpec, MinimalEthSpec, PublicKey, BeaconBlock, BeaconState, ChainSpec, Domain, Epoch, EthSpec, MinimalEthSpec, PublicKey,
RelativeEpoch, Signature, SignedBeaconBlock, SignedRoot, Slot, Validator, RelativeEpoch, Signature, SignedAggregateAndProof, SignedBeaconBlock, SignedRoot, Slot,
Validator,
}; };
use version; use version;
@ -89,6 +90,7 @@ fn validator_produce_attestation() {
.client .client
.beacon_chain() .beacon_chain()
.expect("client should have beacon chain"); .expect("client should have beacon chain");
let genesis_validators_root = beacon_chain.genesis_validators_root;
let state = beacon_chain.head().expect("should get head").beacon_state; let state = beacon_chain.head().expect("should get head").beacon_state;
let validator_index = 0; let validator_index = 0;
@ -130,21 +132,56 @@ fn validator_produce_attestation() {
.expect("should fetch duties from http api"); .expect("should fetch duties from http api");
let duties = &duties[0]; let duties = &duties[0];
// Try publishing the attestation without a signature, ensure it is flagged as invalid. // Try publishing the attestation without a signature or a committee bit set, ensure it is
// raises an error.
let publish_result = env.runtime().block_on(
remote_node
.http
.validator()
.publish_attestations(vec![attestation.clone()]),
);
assert!(
publish_result.is_err(),
"the unsigned published attestation should return error"
);
// Set the aggregation bit.
attestation
.aggregation_bits
.set(
duties
.attestation_committee_position
.expect("should have committee position"),
true,
)
.expect("should set attestation bit");
// Try publishing with an aggreagation bit set, but an invalid signature.
let publish_status = env let publish_status = env
.runtime() .runtime()
.block_on( .block_on(
remote_node remote_node
.http .http
.validator() .validator()
.publish_attestation(attestation.clone()), .publish_attestations(vec![attestation.clone()]),
) )
.expect("should publish attestation"); .expect("should publish attestation with invalid signature");
assert!( assert!(
!publish_status.is_valid(), !publish_status.is_valid(),
"the unsigned published attestation should not be valid" "the unsigned published attestation should not be valid"
); );
// Un-set the aggregation bit, so signing doesn't error.
attestation
.aggregation_bits
.set(
duties
.attestation_committee_position
.expect("should have committee position"),
false,
)
.expect("should un-set attestation bit");
attestation attestation
.sign( .sign(
&keypair.sk, &keypair.sk,
@ -164,13 +201,49 @@ fn validator_produce_attestation() {
remote_node remote_node
.http .http
.validator() .validator()
.publish_attestation(attestation), .publish_attestations(vec![attestation.clone()]),
) )
.expect("should publish attestation"); .expect("should publish attestation");
assert!( assert!(
publish_status.is_valid(), publish_status.is_valid(),
"the signed published attestation should be valid" "the signed published attestation should be valid"
); );
// Try obtaining an aggregated attestation with a matching attestation data to the previous
// one.
let aggregated_attestation = env
.runtime()
.block_on(
remote_node
.http
.validator()
.produce_aggregate_attestation(&attestation.data),
)
.expect("should fetch aggregated attestation from http api");
let signed_aggregate_and_proof = SignedAggregateAndProof::from_aggregate(
validator_index as u64,
aggregated_attestation,
&keypair.sk,
&state.fork,
genesis_validators_root,
spec,
);
// Publish the signed aggregate.
let publish_status = env
.runtime()
.block_on(
remote_node
.http
.validator()
.publish_aggregate_and_proof(vec![signed_aggregate_and_proof]),
)
.expect("should publish aggregate and proof");
assert!(
publish_status.is_valid(),
"the signed aggregate and proof should be valid"
);
} }
#[test] #[test]
@ -226,7 +299,7 @@ fn validator_duties() {
} }
fn check_duties<T: BeaconChainTypes>( fn check_duties<T: BeaconChainTypes>(
duties: Vec<ValidatorDuty>, duties: Vec<ValidatorDutyBytes>,
epoch: Epoch, epoch: Epoch,
validators: Vec<PublicKey>, validators: Vec<PublicKey>,
beacon_chain: Arc<BeaconChain<T>>, beacon_chain: Arc<BeaconChain<T>>,

View File

@ -53,11 +53,18 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.default_value("9000") .default_value("9000")
.takes_value(true), .takes_value(true),
) )
.arg(
Arg::with_name("discovery-port")
.long("discovery-port")
.value_name("PORT")
.help("The UDP port that discovery will listen on. Defaults to `port`")
.takes_value(true),
)
.arg( .arg(
Arg::with_name("maxpeers") Arg::with_name("maxpeers")
.long("maxpeers") .long("maxpeers")
.help("The maximum number of peers.") .help("The maximum number of peers.")
.default_value("10") .default_value("50")
.takes_value(true), .takes_value(true),
) )
.arg( .arg(
@ -69,28 +76,43 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.takes_value(true), .takes_value(true),
) )
.arg( .arg(
Arg::with_name("discovery-port") Arg::with_name("enr-udp-port")
.long("disc-port") .long("enr-udp-port")
.value_name("PORT") .value_name("PORT")
.help("The discovery UDP port.") .help("The UDP port of the local ENR. Set this only if you are sure other nodes can connect to your local node on this port.")
.default_value("9000")
.takes_value(true), .takes_value(true),
) )
.arg( .arg(
Arg::with_name("discovery-address") Arg::with_name("enr-tcp-port")
.long("discovery-address") .long("enr-tcp-port")
.value_name("PORT")
.help("The TCP port of the local ENR. Set this only if you are sure other nodes can connect to your local node on this port.\
The --port flag is used if this is not set.")
.takes_value(true),
)
.arg(
Arg::with_name("enr-address")
.long("enr-address")
.value_name("ADDRESS") .value_name("ADDRESS")
.help("The IP address to broadcast to other peers on how to reach this node. \ .help("The IP address to broadcast to other peers on how to reach this node. \
Default will load previous values from disk failing this it is set to 127.0.0.1 \ Set this only if you are sure other nodes can connect to your local node on this address. \
and will be updated when connecting to other nodes on the network.") Discovery will automatically find your external address,if possible.
")
.takes_value(true), .takes_value(true),
) )
.arg( .arg(
Arg::with_name("topics") Arg::with_name("enr-match")
.long("topics") .short("e")
.value_name("STRING") .long("enr-match")
.help("One or more comma-delimited gossipsub topic strings to subscribe to. Default \ .help("Sets the local ENR IP address and port to match those set for lighthouse. \
is determined automatically.") Specifically, the IP address will be the value of --listen-address and the UDP port will be --discovery-port.")
)
.arg(
Arg::with_name("disable-enr-auto-update")
.short("x")
.long("disable-enr-auto-update")
.help("Discovery automatically updates the nodes local ENR with an external IP address and port as seen by other peers on the network. \
This disables this feature, fixing the ENR's IP/PORT to those specified on boot.")
.takes_value(true), .takes_value(true),
) )
.arg( .arg(

View File

@ -3,7 +3,7 @@ use clap::ArgMatches;
use client::{config::DEFAULT_DATADIR, ClientConfig, ClientGenesis}; use client::{config::DEFAULT_DATADIR, ClientConfig, ClientGenesis};
use eth2_libp2p::{Enr, Multiaddr}; use eth2_libp2p::{Enr, Multiaddr};
use eth2_testnet_config::Eth2TestnetConfig; use eth2_testnet_config::Eth2TestnetConfig;
use slog::{crit, warn, Logger}; use slog::{crit, info, Logger};
use ssz::Encode; use ssz::Encode;
use std::fs; use std::fs;
use std::fs::File; use std::fs::File;
@ -64,6 +64,12 @@ pub fn get_config<E: EthSpec>(
fs::create_dir_all(&client_config.data_dir) fs::create_dir_all(&client_config.data_dir)
.map_err(|e| format!("Failed to create data dir: {}", e))?; .map_err(|e| format!("Failed to create data dir: {}", e))?;
// logs the chosen data directory
let mut log_dir = client_config.data_dir.clone();
// remove /beacon from the end
log_dir.pop();
info!(log, "Data directory initialised"; "datadir" => format!("{}",log_dir.into_os_string().into_string().expect("Datadir should be a valid os string")));
// Load the client config, if it exists . // Load the client config, if it exists .
let config_file_path = client_config.data_dir.join(CLIENT_CONFIG_FILENAME); let config_file_path = client_config.data_dir.join(CLIENT_CONFIG_FILENAME);
let config_file_existed = config_file_path.exists(); let config_file_existed = config_file_path.exists();
@ -108,6 +114,13 @@ pub fn get_config<E: EthSpec>(
client_config.network.discovery_port = port; client_config.network.discovery_port = port;
} }
if let Some(port_str) = cli_args.value_of("discovery-port") {
let port = port_str
.parse::<u16>()
.map_err(|_| format!("Invalid port: {}", port_str))?;
client_config.network.discovery_port = port;
}
if let Some(boot_enr_str) = cli_args.value_of("boot-nodes") { if let Some(boot_enr_str) = cli_args.value_of("boot-nodes") {
client_config.network.boot_nodes = boot_enr_str client_config.network.boot_nodes = boot_enr_str
.split(',') .split(',')
@ -126,22 +139,45 @@ pub fn get_config<E: EthSpec>(
.collect::<Result<Vec<Multiaddr>, _>>()?; .collect::<Result<Vec<Multiaddr>, _>>()?;
} }
if let Some(topics_str) = cli_args.value_of("topics") { if let Some(enr_address_str) = cli_args.value_of("enr-address") {
client_config.network.topics = topics_str.split(',').map(|s| s.into()).collect(); client_config.network.enr_address = Some(
} enr_address_str
if let Some(discovery_address_str) = cli_args.value_of("discovery-address") {
client_config.network.discovery_address = Some(
discovery_address_str
.parse() .parse()
.map_err(|_| format!("Invalid discovery address: {:?}", discovery_address_str))?, .map_err(|_| format!("Invalid discovery address: {:?}", enr_address_str))?,
) )
} }
if let Some(disc_port_str) = cli_args.value_of("disc-port") { if let Some(enr_udp_port_str) = cli_args.value_of("enr-udp-port") {
client_config.network.discovery_port = disc_port_str client_config.network.enr_udp_port = Some(
.parse::<u16>() enr_udp_port_str
.map_err(|_| format!("Invalid discovery port: {}", disc_port_str))?; .parse::<u16>()
.map_err(|_| format!("Invalid discovery port: {}", enr_udp_port_str))?,
);
}
if let Some(enr_tcp_port_str) = cli_args.value_of("enr-tcp-port") {
client_config.network.enr_tcp_port = Some(
enr_tcp_port_str
.parse::<u16>()
.map_err(|_| format!("Invalid ENR TCP port: {}", enr_tcp_port_str))?,
);
}
if cli_args.is_present("enr-match") {
// set the enr address to localhost if the address is 0.0.0.0
if client_config.network.listen_address
== "0.0.0.0".parse::<IpAddr>().expect("valid ip addr")
{
client_config.network.enr_address =
Some("127.0.0.1".parse::<IpAddr>().expect("valid ip addr"));
} else {
client_config.network.enr_address = Some(client_config.network.listen_address);
}
client_config.network.enr_udp_port = Some(client_config.network.discovery_port);
}
if cli_args.is_present("disable_enr_auto_update") {
client_config.network.discv5_config.enr_update = false;
} }
if let Some(p2p_priv_key) = cli_args.value_of("p2p-priv-key") { if let Some(p2p_priv_key) = cli_args.value_of("p2p-priv-key") {
@ -271,8 +307,8 @@ pub fn get_config<E: EthSpec>(
* Discovery address is set to localhost by default. * Discovery address is set to localhost by default.
*/ */
if cli_args.is_present("zero-ports") { if cli_args.is_present("zero-ports") {
if client_config.network.discovery_address == Some(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0))) { if client_config.network.enr_address == Some(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0))) {
client_config.network.discovery_address = None client_config.network.enr_address = None
} }
client_config.network.libp2p_port = client_config.network.libp2p_port =
unused_port("tcp").map_err(|e| format!("Failed to get port for libp2p: {}", e))?; unused_port("tcp").map_err(|e| format!("Failed to get port for libp2p: {}", e))?;
@ -282,16 +318,6 @@ pub fn get_config<E: EthSpec>(
client_config.websocket_server.port = 0; client_config.websocket_server.port = 0;
} }
// ENR IP needs to be explicit for node to be discoverable
if client_config.network.discovery_address == Some(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0))) {
warn!(
log,
"Discovery address cannot be 0.0.0.0, Setting to to 127.0.0.1"
);
client_config.network.discovery_address =
Some("127.0.0.1".parse().expect("Valid IP address"))
}
/* /*
* Load the eth2 testnet dir to obtain some additional config values. * Load the eth2 testnet dir to obtain some additional config values.
*/ */

View File

@ -69,7 +69,7 @@ impl<E: EthSpec> ProductionBeaconNode<E> {
/// Client behaviour is defined by the given `client_config`. /// Client behaviour is defined by the given `client_config`.
pub fn new( pub fn new(
context: RuntimeContext<E>, context: RuntimeContext<E>,
client_config: ClientConfig, mut client_config: ClientConfig,
) -> impl Future<Item = Self, Error = String> { ) -> impl Future<Item = Self, Error = String> {
let http_eth2_config = context.eth2_config().clone(); let http_eth2_config = context.eth2_config().clone();
let spec = context.eth2_config().spec.clone(); let spec = context.eth2_config().spec.clone();
@ -121,7 +121,7 @@ impl<E: EthSpec> ProductionBeaconNode<E> {
.system_time_slot_clock()? .system_time_slot_clock()?
.websocket_event_handler(client_config.websocket_server.clone())? .websocket_event_handler(client_config.websocket_server.clone())?
.build_beacon_chain()? .build_beacon_chain()?
.libp2p_network(&client_config.network)? .network(&mut client_config.network)?
.notifier()?; .notifier()?;
let builder = if client_config.rest_api.enabled { let builder = if client_config.rest_api.enabled {

View File

@ -1,6 +1,6 @@
[package] [package]
name = "store" name = "store"
version = "0.1.0" version = "0.2.0"
authors = ["Paul Hauner <paul@paulhauner.com>"] authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018" edition = "2018"

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