Altair validator client and HTTP API (#2404)

## Proposed Changes

* Implement the validator client and HTTP API changes necessary to support Altair


Co-authored-by: realbigsean <seananderson33@gmail.com>
Co-authored-by: Michael Sproul <michael@sigmaprime.io>
This commit is contained in:
Michael Sproul 2021-08-06 00:47:31 +00:00
parent 350b6f19de
commit 17a2c778e3
44 changed files with 3144 additions and 705 deletions

1
Cargo.lock generated
View File

@ -7312,6 +7312,7 @@ dependencies = [
"futures",
"hex",
"hyper",
"itertools 0.10.1",
"lazy_static",
"libc",
"libsecp256k1",

View File

@ -43,7 +43,7 @@ use crate::validator_pubkey_cache::ValidatorPubkeyCache;
use crate::BeaconForkChoiceStore;
use crate::BeaconSnapshot;
use crate::{metrics, BeaconChainError};
use eth2::types::{EventKind, SseBlock, SseChainReorg, SseFinalizedCheckpoint, SseHead};
use eth2::types::{EventKind, SseBlock, SseChainReorg, SseFinalizedCheckpoint, SseHead, SyncDuty};
use fork_choice::ForkChoice;
use futures::channel::mpsc::Sender;
use itertools::process_results;
@ -1081,6 +1081,29 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(pubkey_cache.get_index(pubkey))
}
/// Return the validator indices of all public keys fetched from an iterator.
///
/// If any public key doesn't belong to a known validator then an error will be returned.
/// We could consider relaxing this by returning `Vec<Option<usize>>` in future.
pub fn validator_indices<'a>(
&self,
validator_pubkeys: impl Iterator<Item = &'a PublicKeyBytes>,
) -> Result<Vec<u64>, Error> {
let pubkey_cache = self
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(Error::ValidatorPubkeyCacheLockTimeout)?;
validator_pubkeys
.map(|pubkey| {
pubkey_cache
.get_index(pubkey)
.map(|id| id as u64)
.ok_or(Error::ValidatorPubkeyUnknown(*pubkey))
})
.collect()
}
/// Returns the validator pubkey (if any) for the given validator index.
///
/// ## Notes
@ -1214,6 +1237,16 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.get_by_slot_and_root(slot, attestation_data_root)
}
/// Return an aggregated `SyncCommitteeContribution` matching the given `root`.
pub fn get_aggregated_sync_committee_contribution(
&self,
sync_contribution_data: &SyncContributionData,
) -> Option<SyncCommitteeContribution<T::EthSpec>> {
self.naive_sync_aggregation_pool
.read()
.get(sync_contribution_data)
}
/// Produce an unaggregated `Attestation` that is valid for the given `slot` and `index`.
///
/// The produced `Attestation` will not be valid until it has been signed by exactly one
@ -1882,6 +1915,19 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(())
}
/// Attempt to obtain sync committee duties from the head.
pub fn sync_committee_duties_from_head(
&self,
epoch: Epoch,
validator_indices: &[u64],
) -> Result<Vec<Option<SyncDuty>>, Error> {
self.with_head(move |head| {
head.beacon_state
.get_sync_committee_duties(epoch, validator_indices, &self.spec)
.map_err(Error::SyncDutiesError)
})
}
/// Attempt to verify and import a chain of blocks to `self`.
///
/// The provided blocks _must_ each reference the previous block via `block.parent_root` (i.e.,
@ -2624,6 +2670,22 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let proposer_index = state.get_beacon_proposer_index(state.slot(), &self.spec)? as u64;
let voluntary_exits = self.op_pool.get_voluntary_exits(&state, &self.spec).into();
// Closure to fetch a sync aggregate in cases where it is required.
let get_sync_aggregate = || -> Result<SyncAggregate<_>, BlockProductionError> {
Ok(self
.op_pool
.get_sync_aggregate(&state)
.map_err(BlockProductionError::OpPoolError)?
.unwrap_or_else(|| {
warn!(
self.log,
"Producing block with no sync contributions";
"slot" => state.slot(),
);
SyncAggregate::new()
}))
};
let inner_block = match state {
BeaconState::Base(_) => BeaconBlock::Base(BeaconBlockBase {
slot,
@ -2641,24 +2703,26 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
voluntary_exits,
},
}),
BeaconState::Altair(_) => BeaconBlock::Altair(BeaconBlockAltair {
slot,
proposer_index,
parent_root,
state_root: Hash256::zero(),
body: BeaconBlockBodyAltair {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings: proposer_slashings.into(),
attester_slashings: attester_slashings.into(),
attestations,
deposits,
voluntary_exits,
// FIXME(altair): put a sync aggregate from the pool here (once implemented)
sync_aggregate: SyncAggregate::new(),
},
}),
BeaconState::Altair(_) => {
let sync_aggregate = get_sync_aggregate()?;
BeaconBlock::Altair(BeaconBlockAltair {
slot,
proposer_index,
parent_root,
state_root: Hash256::zero(),
body: BeaconBlockBodyAltair {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings: proposer_slashings.into(),
attester_slashings: attester_slashings.into(),
attestations,
deposits,
voluntary_exits,
sync_aggregate,
},
})
}
};
let block = SignedBeaconBlock::from_block(

View File

@ -87,6 +87,7 @@ pub enum BeaconChainError {
DuplicateValidatorPublicKey,
ValidatorPubkeyCacheFileError(String),
ValidatorIndexUnknown(usize),
ValidatorPubkeyUnknown(PublicKeyBytes),
OpPoolError(OpPoolError),
NaiveAggregationError(NaiveAggregationError),
ObservedAttestationsError(ObservedAttestationsError),
@ -120,6 +121,7 @@ pub enum BeaconChainError {
state_epoch: Epoch,
shuffling_epoch: Epoch,
},
SyncDutiesError(BeaconStateError),
InconsistentForwardsIter {
request_slot: Slot,
slot: Slot,

View File

@ -151,7 +151,7 @@ pub fn test_spec<E: EthSpec>() -> ChainSpec {
pub struct BeaconChainHarness<T: BeaconChainTypes> {
pub validator_keypairs: Vec<Keypair>,
pub chain: BeaconChain<T>,
pub chain: Arc<BeaconChain<T>>,
pub spec: ChainSpec,
pub data_dir: TempDir,
pub shutdown_receiver: Receiver<ShutdownReason>,
@ -229,6 +229,29 @@ impl<E: EthSpec> BeaconChainHarness<EphemeralHarnessType<E>> {
target_aggregators_per_committee: u64,
store_config: StoreConfig,
chain_config: ChainConfig,
) -> Self {
Self::new_with_mutator(
eth_spec_instance,
spec,
validator_keypairs,
target_aggregators_per_committee,
store_config,
chain_config,
|x| x,
)
}
/// Apply a function to beacon chain builder before building.
pub fn new_with_mutator(
eth_spec_instance: E,
spec: Option<ChainSpec>,
validator_keypairs: Vec<Keypair>,
target_aggregators_per_committee: u64,
store_config: StoreConfig,
chain_config: ChainConfig,
mutator: impl FnOnce(
BeaconChainBuilder<EphemeralHarnessType<E>>,
) -> BeaconChainBuilder<EphemeralHarnessType<E>>,
) -> Self {
let data_dir = tempdir().expect("should create temporary data_dir");
let mut spec = spec.unwrap_or_else(test_spec::<E>);
@ -240,7 +263,7 @@ impl<E: EthSpec> BeaconChainHarness<EphemeralHarnessType<E>> {
let log = test_logger();
let store = HotColdDB::open_ephemeral(store_config, spec.clone(), log.clone()).unwrap();
let chain = BeaconChainBuilder::new(eth_spec_instance)
let builder = BeaconChainBuilder::new(eth_spec_instance)
.logger(log.clone())
.custom_spec(spec.clone())
.store(Arc::new(store))
@ -260,13 +283,13 @@ impl<E: EthSpec> BeaconChainHarness<EphemeralHarnessType<E>> {
log.clone(),
1,
)))
.monitor_validators(true, vec![], log)
.build()
.expect("should build");
.monitor_validators(true, vec![], log);
let chain = mutator(builder).build().expect("should build");
Self {
spec: chain.spec.clone(),
chain,
chain: Arc::new(chain),
validator_keypairs,
data_dir,
shutdown_receiver,
@ -311,7 +334,7 @@ impl<E: EthSpec> BeaconChainHarness<DiskHarnessType<E>> {
Self {
spec: chain.spec.clone(),
chain,
chain: Arc::new(chain),
validator_keypairs,
data_dir,
shutdown_receiver,
@ -353,7 +376,7 @@ impl<E: EthSpec> BeaconChainHarness<DiskHarnessType<E>> {
Self {
spec: chain.spec.clone(),
chain,
chain: Arc::new(chain),
validator_keypairs,
data_dir,
shutdown_receiver,

View File

@ -4,7 +4,7 @@
extern crate lazy_static;
use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
test_logger, AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
};
use beacon_chain::{BeaconSnapshot, BlockError, ChainConfig, ChainSegmentResult};
use slasher::{Config as SlasherConfig, Slasher};
@ -830,17 +830,25 @@ fn block_gossip_verification() {
#[test]
fn verify_block_for_gossip_slashing_detection() {
let mut harness = get_harness(VALIDATOR_COUNT);
let slasher_dir = tempdir().unwrap();
let slasher = Arc::new(
Slasher::open(
SlasherConfig::new(slasher_dir.path().into()).for_testing(),
harness.logger().clone(),
test_logger(),
)
.unwrap(),
);
harness.chain.slasher = Some(slasher.clone());
let harness = BeaconChainHarness::new_with_mutator(
MainnetEthSpec,
None,
KEYPAIRS.to_vec(),
1 << 32,
StoreConfig::default(),
ChainConfig::default(),
|builder| builder.slasher(slasher.clone()),
);
harness.advance_slot();
let state = harness.get_current_state();
let (block1, _) = harness.make_block(state.clone(), Slot::new(1));

View File

@ -3,6 +3,7 @@ name = "http_api"
version = "0.1.0"
authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018"
autotests = false # using a single test binary compiles faster
[dependencies]
warp = { git = "https://github.com/paulhauner/warp ", branch = "cors-wildcard" }
@ -35,3 +36,7 @@ store = { path = "../store" }
environment = { path = "../../lighthouse/environment" }
tree_hash = "0.1.1"
sensitive_url = { path = "../../common/sensitive_url" }
[[test]]
name = "bn_http_api_tests"
path = "tests/main.rs"

View File

@ -10,7 +10,9 @@ mod block_id;
mod metrics;
mod proposer_duties;
mod state_id;
mod sync_committees;
mod validator_inclusion;
mod version;
use beacon_chain::{
attestation_verification::SignatureVerifiedAttestation,
@ -20,7 +22,7 @@ use beacon_chain::{
WhenSlotSkipped,
};
use block_id::BlockId;
use eth2::types::{self as api_types, ValidatorId};
use eth2::types::{self as api_types, EndpointVersion, ValidatorId};
use eth2_libp2p::{types::SyncState, EnrExt, NetworkGlobals, PeerId, PubsubMessage};
use lighthouse_version::version_with_platform;
use network::NetworkMessage;
@ -37,10 +39,12 @@ use std::sync::Arc;
use tokio::sync::mpsc::UnboundedSender;
use tokio_stream::{wrappers::BroadcastStream, StreamExt};
use types::{
Attestation, AttesterSlashing, CommitteeCache, ConfigAndPreset, Epoch, EthSpec,
ProposerSlashing, RelativeEpoch, SignedAggregateAndProof, SignedBeaconBlock,
SignedVoluntaryExit, Slot,
Attestation, AttesterSlashing, BeaconStateError, CommitteeCache, ConfigAndPreset, Epoch,
EthSpec, ProposerSlashing, RelativeEpoch, SignedAggregateAndProof, SignedBeaconBlock,
SignedContributionAndProof, SignedVoluntaryExit, Slot, SyncCommitteeMessage,
SyncContributionData,
};
use version::{fork_versioned_response, unsupported_version_rejection, V1};
use warp::http::StatusCode;
use warp::sse::Event;
use warp::Reply;
@ -48,7 +52,6 @@ use warp::{http::Response, Filter};
use warp_utils::task::{blocking_json_task, blocking_task};
const API_PREFIX: &str = "eth";
const API_VERSION: &str = "v1";
/// If the node is within this many epochs from the head, we declare it to be synced regardless of
/// the network sync state.
@ -152,7 +155,7 @@ pub fn prometheus_metrics() -> warp::filters::log::Log<impl Fn(warp::filters::lo
// a block hash).
let path = {
let equals = |s: &'static str| -> Option<&'static str> {
if info.path() == format!("/{}/{}/{}", API_PREFIX, API_VERSION, s) {
if info.path() == format!("/{}/{}", API_PREFIX, s) {
Some(s)
} else {
None
@ -160,30 +163,30 @@ pub fn prometheus_metrics() -> warp::filters::log::Log<impl Fn(warp::filters::lo
};
let starts_with = |s: &'static str| -> Option<&'static str> {
if info
.path()
.starts_with(&format!("/{}/{}/{}", API_PREFIX, API_VERSION, s))
{
if info.path().starts_with(&format!("/{}/{}", API_PREFIX, s)) {
Some(s)
} else {
None
}
};
equals("beacon/blocks")
.or_else(|| starts_with("validator/duties/attester"))
.or_else(|| starts_with("validator/duties/proposer"))
.or_else(|| starts_with("validator/attestation_data"))
.or_else(|| starts_with("validator/blocks"))
.or_else(|| starts_with("validator/aggregate_attestation"))
.or_else(|| starts_with("validator/aggregate_and_proofs"))
.or_else(|| starts_with("validator/beacon_committee_subscriptions"))
.or_else(|| starts_with("beacon/"))
.or_else(|| starts_with("config/"))
.or_else(|| starts_with("debug/"))
.or_else(|| starts_with("events/"))
.or_else(|| starts_with("node/"))
.or_else(|| starts_with("validator/"))
// First line covers `POST /v1/beacon/blocks` only
equals("v1/beacon/blocks")
.or_else(|| starts_with("v1/validator/duties/attester"))
.or_else(|| starts_with("v1/validator/duties/proposer"))
.or_else(|| starts_with("v1/validator/attestation_data"))
.or_else(|| starts_with("v1/validator/blocks"))
.or_else(|| starts_with("v2/validator/blocks"))
.or_else(|| starts_with("v1/validator/aggregate_attestation"))
.or_else(|| starts_with("v1/validator/aggregate_and_proofs"))
.or_else(|| starts_with("v1/validator/beacon_committee_subscriptions"))
.or_else(|| starts_with("v1/beacon/"))
.or_else(|| starts_with("v2/beacon/"))
.or_else(|| starts_with("v1/config/"))
.or_else(|| starts_with("v1/debug/"))
.or_else(|| starts_with("v1/events/"))
.or_else(|| starts_with("v1/node/"))
.or_else(|| starts_with("v1/validator/"))
.unwrap_or("other")
};
@ -239,7 +242,29 @@ pub fn serve<T: BeaconChainTypes>(
));
}
let eth1_v1 = warp::path(API_PREFIX).and(warp::path(API_VERSION));
// Create a filter that extracts the endpoint version.
let any_version = warp::path(API_PREFIX).and(warp::path::param::<EndpointVersion>().or_else(
|_| async move {
Err(warp_utils::reject::custom_bad_request(
"Invalid version identifier".to_string(),
))
},
));
// Filter that enforces a single endpoint version and then discards the `EndpointVersion`.
let single_version = |reqd: EndpointVersion| {
any_version
.and_then(move |version| async move {
if version == reqd {
Ok(())
} else {
Err(unsupported_version_rejection(version))
}
})
.untuple_one()
};
let eth1_v1 = single_version(V1);
// Create a `warp` filter that provides access to the network globals.
let inner_network_globals = ctx.network_globals.clone();
@ -659,6 +684,61 @@ pub fn serve<T: BeaconChainTypes>(
},
);
// GET beacon/states/{state_id}/sync_committees?epoch
let get_beacon_state_sync_committees = beacon_states_path
.clone()
.and(warp::path("sync_committees"))
.and(warp::query::<api_types::SyncCommitteesQuery>())
.and(warp::path::end())
.and_then(
|state_id: StateId,
chain: Arc<BeaconChain<T>>,
query: api_types::SyncCommitteesQuery| {
blocking_json_task(move || {
let sync_committee = state_id.map_state(&chain, |state| {
let current_epoch = state.current_epoch();
let epoch = query.epoch.unwrap_or(current_epoch);
state
.get_built_sync_committee(epoch, &chain.spec)
.map(|committee| committee.clone())
.map_err(|e| match e {
BeaconStateError::SyncCommitteeNotKnown { .. } => {
warp_utils::reject::custom_bad_request(format!(
"state at epoch {} has no sync committee for epoch {}",
current_epoch, epoch
))
}
BeaconStateError::IncorrectStateVariant => {
warp_utils::reject::custom_bad_request(format!(
"state at epoch {} is not activated for Altair",
current_epoch,
))
}
e => warp_utils::reject::beacon_state_error(e),
})
})?;
let validators = chain
.validator_indices(sync_committee.pubkeys.iter())
.map_err(warp_utils::reject::beacon_chain_error)?;
let validator_aggregates = validators
.chunks_exact(T::EthSpec::sync_subcommittee_size())
.map(|indices| api_types::SyncSubcommittee {
indices: indices.to_vec(),
})
.collect();
let response = api_types::SyncCommitteeByValidatorIndices {
validators,
validator_aggregates,
};
Ok(api_types::GenericResponse::from(response))
})
},
);
// GET beacon/headers
//
// Note: this endpoint only returns information about blocks in the canonical chain. Given that
@ -875,23 +955,32 @@ pub fn serve<T: BeaconChainTypes>(
},
);
let beacon_blocks_path = eth1_v1
let block_id_or_err = warp::path::param::<BlockId>().or_else(|_| async {
Err(warp_utils::reject::custom_bad_request(
"Invalid block ID".to_string(),
))
});
let beacon_blocks_path_v1 = eth1_v1
.and(warp::path("beacon"))
.and(warp::path("blocks"))
.and(warp::path::param::<BlockId>().or_else(|_| async {
Err(warp_utils::reject::custom_bad_request(
"Invalid block ID".to_string(),
))
}))
.and(block_id_or_err)
.and(chain_filter.clone());
let beacon_blocks_path_any = any_version
.and(warp::path("beacon"))
.and(warp::path("blocks"))
.and(block_id_or_err)
.and(chain_filter.clone());
// GET beacon/blocks/{block_id}
let get_beacon_block = beacon_blocks_path
let get_beacon_block = beacon_blocks_path_any
.clone()
.and(warp::path::end())
.and(warp::header::optional::<api_types::Accept>("accept"))
.and_then(
|block_id: BlockId,
|endpoint_version: EndpointVersion,
block_id: BlockId,
chain: Arc<BeaconChain<T>>,
accept_header: Option<api_types::Accept>| {
blocking_task(move || {
@ -907,17 +996,18 @@ pub fn serve<T: BeaconChainTypes>(
e
))
}),
_ => Ok(
warp::reply::json(&api_types::GenericResponseRef::from(&block))
.into_response(),
),
_ => {
let fork_name = block.fork_name(&chain.spec).ok();
fork_versioned_response(endpoint_version, fork_name, block)
.map(|res| warp::reply::json(&res).into_response())
}
}
})
},
);
// GET beacon/blocks/{block_id}/root
let get_beacon_block_root = beacon_blocks_path
let get_beacon_block_root = beacon_blocks_path_v1
.clone()
.and(warp::path("root"))
.and(warp::path::end())
@ -931,7 +1021,7 @@ pub fn serve<T: BeaconChainTypes>(
});
// GET beacon/blocks/{block_id}/attestations
let get_beacon_block_attestations = beacon_blocks_path
let get_beacon_block_attestations = beacon_blocks_path_v1
.clone()
.and(warp::path("attestations"))
.and(warp::path::end())
@ -1250,6 +1340,28 @@ pub fn serve<T: BeaconChainTypes>(
})
});
// POST beacon/pool/sync_committees
let post_beacon_pool_sync_committees = beacon_pool_path
.clone()
.and(warp::path("sync_committees"))
.and(warp::path::end())
.and(warp::body::json())
.and(network_tx_filter.clone())
.and(log_filter.clone())
.and_then(
|chain: Arc<BeaconChain<T>>,
signatures: Vec<SyncCommitteeMessage>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
log: Logger| {
blocking_json_task(move || {
sync_committees::process_sync_committee_signatures(
signatures, network_tx, &chain, log,
)?;
Ok(api_types::GenericResponse::from(()))
})
},
);
/*
* config/fork_schedule
*/
@ -1307,7 +1419,7 @@ pub fn serve<T: BeaconChainTypes>(
*/
// GET debug/beacon/states/{state_id}
let get_debug_beacon_states = eth1_v1
let get_debug_beacon_states = any_version
.and(warp::path("debug"))
.and(warp::path("beacon"))
.and(warp::path("states"))
@ -1320,7 +1432,8 @@ pub fn serve<T: BeaconChainTypes>(
.and(warp::header::optional::<api_types::Accept>("accept"))
.and(chain_filter.clone())
.and_then(
|state_id: StateId,
|endpoint_version: EndpointVersion,
state_id: StateId,
accept_header: Option<api_types::Accept>,
chain: Arc<BeaconChain<T>>| {
blocking_task(move || match accept_header {
@ -1338,10 +1451,9 @@ pub fn serve<T: BeaconChainTypes>(
})
}
_ => state_id.map_state(&chain, |state| {
Ok(
warp::reply::json(&api_types::GenericResponseRef::from(&state))
.into_response(),
)
let fork_name = state.fork_name(&chain.spec).ok();
let res = fork_versioned_response(endpoint_version, fork_name, &state)?;
Ok(warp::reply::json(&res).into_response())
}),
})
},
@ -1659,7 +1771,7 @@ pub fn serve<T: BeaconChainTypes>(
});
// GET validator/blocks/{slot}
let get_validator_blocks = eth1_v1
let get_validator_blocks = any_version
.and(warp::path("validator"))
.and(warp::path("blocks"))
.and(warp::path::param::<Slot>().or_else(|_| async {
@ -1672,7 +1784,10 @@ pub fn serve<T: BeaconChainTypes>(
.and(warp::query::<api_types::ValidatorBlocksQuery>())
.and(chain_filter.clone())
.and_then(
|slot: Slot, query: api_types::ValidatorBlocksQuery, chain: Arc<BeaconChain<T>>| {
|endpoint_version: EndpointVersion,
slot: Slot,
query: api_types::ValidatorBlocksQuery,
chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let randao_reveal = (&query.randao_reveal).try_into().map_err(|e| {
warp_utils::reject::custom_bad_request(format!(
@ -1681,11 +1796,11 @@ pub fn serve<T: BeaconChainTypes>(
))
})?;
chain
let (block, _) = chain
.produce_block(randao_reveal, slot, query.graffiti.map(Into::into))
.map(|block_and_state| block_and_state.0)
.map(api_types::GenericResponse::from)
.map_err(warp_utils::reject::block_production_error)
.map_err(warp_utils::reject::block_production_error)?;
let fork_name = block.to_ref().fork_name(&chain.spec).ok();
fork_versioned_response(endpoint_version, fork_name, block)
})
},
);
@ -1770,12 +1885,57 @@ pub fn serve<T: BeaconChainTypes>(
},
);
// POST validator/duties/sync
let post_validator_duties_sync = eth1_v1
.and(warp::path("validator"))
.and(warp::path("duties"))
.and(warp::path("sync"))
.and(warp::path::param::<Epoch>().or_else(|_| async {
Err(warp_utils::reject::custom_bad_request(
"Invalid epoch".to_string(),
))
}))
.and(warp::path::end())
.and(not_while_syncing_filter.clone())
.and(warp::body::json())
.and(chain_filter.clone())
.and_then(
|epoch: Epoch, indices: api_types::ValidatorIndexData, chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
sync_committees::sync_committee_duties(epoch, &indices.0, &chain)
})
},
);
// GET validator/sync_committee_contribution
let get_validator_sync_committee_contribution = eth1_v1
.and(warp::path("validator"))
.and(warp::path("sync_committee_contribution"))
.and(warp::path::end())
.and(warp::query::<SyncContributionData>())
.and(not_while_syncing_filter.clone())
.and(chain_filter.clone())
.and_then(
|sync_committee_data: SyncContributionData, chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
chain
.get_aggregated_sync_committee_contribution(&sync_committee_data)
.map(api_types::GenericResponse::from)
.ok_or_else(|| {
warp_utils::reject::custom_not_found(
"no matching sync contribution found".to_string(),
)
})
})
},
);
// POST validator/aggregate_and_proofs
let post_validator_aggregate_and_proofs = eth1_v1
.and(warp::path("validator"))
.and(warp::path("aggregate_and_proofs"))
.and(warp::path::end())
.and(not_while_syncing_filter)
.and(not_while_syncing_filter.clone())
.and(chain_filter.clone())
.and(warp::body::json())
.and(network_tx_filter.clone())
@ -1871,13 +2031,39 @@ pub fn serve<T: BeaconChainTypes>(
},
);
let post_validator_contribution_and_proofs = eth1_v1
.and(warp::path("validator"))
.and(warp::path("contribution_and_proofs"))
.and(warp::path::end())
.and(not_while_syncing_filter)
.and(chain_filter.clone())
.and(warp::body::json())
.and(network_tx_filter.clone())
.and(log_filter.clone())
.and_then(
|chain: Arc<BeaconChain<T>>,
contributions: Vec<SignedContributionAndProof<T::EthSpec>>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
log: Logger| {
blocking_json_task(move || {
sync_committees::process_signed_contribution_and_proofs(
contributions,
network_tx,
&chain,
log,
)?;
Ok(api_types::GenericResponse::from(()))
})
},
);
// POST validator/beacon_committee_subscriptions
let post_validator_beacon_committee_subscriptions = eth1_v1
.and(warp::path("validator"))
.and(warp::path("beacon_committee_subscriptions"))
.and(warp::path::end())
.and(warp::body::json())
.and(network_tx_filter)
.and(network_tx_filter.clone())
.and(chain_filter.clone())
.and_then(
|subscriptions: Vec<api_types::BeaconCommitteeSubscription>,
@ -1911,6 +2097,38 @@ pub fn serve<T: BeaconChainTypes>(
},
);
// POST validator/sync_committee_subscriptions
let post_validator_sync_committee_subscriptions = eth1_v1
.and(warp::path("validator"))
.and(warp::path("sync_committee_subscriptions"))
.and(warp::path::end())
.and(warp::body::json())
.and(network_tx_filter)
.and(chain_filter.clone())
.and_then(
|subscriptions: Vec<types::SyncCommitteeSubscription>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
for subscription in subscriptions {
chain
.validator_monitor
.write()
.auto_register_local_validator(subscription.validator_index);
publish_network_message(
&network_tx,
NetworkMessage::SyncCommitteeSubscribe {
subscriptions: vec![subscription],
},
)?;
}
Ok(())
})
},
);
// POST lighthouse/liveness
let post_lighthouse_liveness = warp::path("lighthouse")
.and(warp::path("liveness"))
@ -2248,6 +2466,7 @@ pub fn serve<T: BeaconChainTypes>(
.or(get_beacon_state_validators.boxed())
.or(get_beacon_state_validators_id.boxed())
.or(get_beacon_state_committees.boxed())
.or(get_beacon_state_sync_committees.boxed())
.or(get_beacon_headers.boxed())
.or(get_beacon_headers_block_id.boxed())
.or(get_beacon_block.boxed())
@ -2273,6 +2492,7 @@ pub fn serve<T: BeaconChainTypes>(
.or(get_validator_blocks.boxed())
.or(get_validator_attestation_data.boxed())
.or(get_validator_aggregate_attestation.boxed())
.or(get_validator_sync_committee_contribution.boxed())
.or(get_lighthouse_health.boxed())
.or(get_lighthouse_syncing.boxed())
.or(get_lighthouse_peers.boxed())
@ -2294,10 +2514,14 @@ pub fn serve<T: BeaconChainTypes>(
.or(post_beacon_pool_attester_slashings.boxed())
.or(post_beacon_pool_proposer_slashings.boxed())
.or(post_beacon_pool_voluntary_exits.boxed())
.or(post_beacon_pool_sync_committees.boxed())
.or(post_validator_duties_attester.boxed())
.or(post_validator_duties_sync.boxed())
.or(post_validator_aggregate_and_proofs.boxed())
.or(post_lighthouse_liveness.boxed())
.or(post_validator_beacon_committee_subscriptions.boxed()),
.or(post_validator_contribution_and_proofs.boxed())
.or(post_validator_beacon_committee_subscriptions.boxed())
.or(post_validator_sync_committee_subscriptions.boxed())
.or(post_lighthouse_liveness.boxed()),
))
.recover(warp_utils::reject::handle_rejection)
.with(slog_logging(log.clone()))

View File

@ -0,0 +1,294 @@
//! Handlers for sync committee endpoints.
use crate::publish_pubsub_message;
use beacon_chain::sync_committee_verification::{
Error as SyncVerificationError, VerifiedSyncCommitteeMessage,
};
use beacon_chain::{
BeaconChain, BeaconChainError, BeaconChainTypes, StateSkipConfig,
MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
use eth2::types::{self as api_types};
use eth2_libp2p::PubsubMessage;
use network::NetworkMessage;
use slog::{error, warn, Logger};
use slot_clock::SlotClock;
use std::cmp::max;
use std::collections::HashMap;
use tokio::sync::mpsc::UnboundedSender;
use types::{
slot_data::SlotData, BeaconStateError, Epoch, EthSpec, SignedContributionAndProof,
SyncCommitteeMessage, SyncDuty, SyncSubnetId,
};
/// The struct that is returned to the requesting HTTP client.
type SyncDuties = api_types::GenericResponse<Vec<SyncDuty>>;
/// Handles a request from the HTTP API for sync committee duties.
pub fn sync_committee_duties<T: BeaconChainTypes>(
request_epoch: Epoch,
request_indices: &[u64],
chain: &BeaconChain<T>,
) -> Result<SyncDuties, warp::reject::Rejection> {
let altair_fork_epoch = if let Some(altair_fork_epoch) = chain.spec.altair_fork_epoch {
altair_fork_epoch
} else {
// Empty response for networks with Altair disabled.
return Ok(convert_to_response(vec![]));
};
// Try using the head's sync committees to satisfy the request. This should be sufficient for
// the vast majority of requests. Rather than checking if we think the request will succeed in a
// way prone to data races, we attempt the request immediately and check the error code.
match chain.sync_committee_duties_from_head(request_epoch, request_indices) {
Ok(duties) => return Ok(convert_to_response(duties)),
Err(BeaconChainError::SyncDutiesError(BeaconStateError::SyncCommitteeNotKnown {
..
}))
| Err(BeaconChainError::SyncDutiesError(BeaconStateError::IncorrectStateVariant)) => (),
Err(e) => return Err(warp_utils::reject::beacon_chain_error(e)),
}
let duties = duties_from_state_load(request_epoch, request_indices, altair_fork_epoch, chain)
.map_err(|e| match e {
BeaconChainError::SyncDutiesError(BeaconStateError::SyncCommitteeNotKnown {
current_epoch,
..
}) => warp_utils::reject::custom_bad_request(format!(
"invalid epoch: {}, current epoch: {}",
request_epoch, current_epoch
)),
e => warp_utils::reject::beacon_chain_error(e),
})?;
Ok(convert_to_response(duties))
}
/// Slow path for duties: load a state and use it to compute the duties.
fn duties_from_state_load<T: BeaconChainTypes>(
request_epoch: Epoch,
request_indices: &[u64],
altair_fork_epoch: Epoch,
chain: &BeaconChain<T>,
) -> Result<Vec<Option<SyncDuty>>, BeaconChainError> {
// Determine what the current epoch would be if we fast-forward our system clock by
// `MAXIMUM_GOSSIP_CLOCK_DISPARITY`.
//
// Most of the time, `tolerant_current_epoch` will be equal to `current_epoch`. However, during
// the last `MAXIMUM_GOSSIP_CLOCK_DISPARITY` duration of the epoch `tolerant_current_epoch`
// will equal `current_epoch + 1`
let current_epoch = chain.epoch()?;
let tolerant_current_epoch = chain
.slot_clock
.now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY)
.ok_or(BeaconChainError::UnableToReadSlot)?
.epoch(T::EthSpec::slots_per_epoch());
let max_sync_committee_period = tolerant_current_epoch.sync_committee_period(&chain.spec)? + 1;
let sync_committee_period = request_epoch.sync_committee_period(&chain.spec)?;
if tolerant_current_epoch < altair_fork_epoch {
// Empty response if the epoch is pre-Altair.
Ok(vec![])
} else if sync_committee_period <= max_sync_committee_period {
// Load the state at the start of the *previous* sync committee period.
// This is sufficient for historical duties, and efficient in the case where the head
// is lagging the current epoch and we need duties for the next period (because we only
// have to transition the head to start of the current period).
//
// We also need to ensure that the load slot is after the Altair fork.
let load_slot = max(
chain.spec.epochs_per_sync_committee_period * sync_committee_period.saturating_sub(1),
altair_fork_epoch,
)
.start_slot(T::EthSpec::slots_per_epoch());
let state = chain.state_at_slot(load_slot, StateSkipConfig::WithoutStateRoots)?;
state
.get_sync_committee_duties(request_epoch, request_indices, &chain.spec)
.map_err(BeaconChainError::SyncDutiesError)
} else {
Err(BeaconChainError::SyncDutiesError(
BeaconStateError::SyncCommitteeNotKnown {
current_epoch,
epoch: request_epoch,
},
))
}
}
fn convert_to_response(duties: Vec<Option<SyncDuty>>) -> SyncDuties {
api_types::GenericResponse::from(duties.into_iter().flatten().collect::<Vec<_>>())
}
/// Receive sync committee duties, storing them in the pools & broadcasting them.
pub fn process_sync_committee_signatures<T: BeaconChainTypes>(
sync_committee_signatures: Vec<SyncCommitteeMessage>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
chain: &BeaconChain<T>,
log: Logger,
) -> Result<(), warp::reject::Rejection> {
let mut failures = vec![];
for (i, sync_committee_signature) in sync_committee_signatures.iter().enumerate() {
let subnet_positions = match get_subnet_positions_for_sync_committee_message(
sync_committee_signature,
chain,
) {
Ok(positions) => positions,
Err(e) => {
error!(
log,
"Unable to compute subnet positions for sync message";
"error" => ?e,
"slot" => sync_committee_signature.slot,
);
failures.push(api_types::Failure::new(i, format!("Verification: {:?}", e)));
continue;
}
};
// Verify and publish on all relevant subnets.
//
// The number of assigned subnets on any practical network should be ~1, so the apparent
// inefficiency of verifying multiple times is not a real inefficiency.
let mut verified_for_pool = None;
for subnet_id in subnet_positions.keys().copied() {
match VerifiedSyncCommitteeMessage::verify(
sync_committee_signature.clone(),
subnet_id,
chain,
) {
Ok(verified) => {
publish_pubsub_message(
&network_tx,
PubsubMessage::SyncCommitteeMessage(Box::new((
subnet_id,
verified.sync_message().clone(),
))),
)?;
verified_for_pool = Some(verified);
}
Err(e) => {
error!(
log,
"Failure verifying sync committee signature for gossip";
"error" => ?e,
"request_index" => i,
"slot" => sync_committee_signature.slot,
"validator_index" => sync_committee_signature.validator_index,
);
failures.push(api_types::Failure::new(i, format!("Verification: {:?}", e)));
}
}
}
if let Some(verified) = verified_for_pool {
if let Err(e) = chain.add_to_naive_sync_aggregation_pool(verified) {
error!(
log,
"Unable to add sync committee signature to pool";
"error" => ?e,
"slot" => sync_committee_signature.slot,
"validator_index" => sync_committee_signature.validator_index,
);
}
}
}
if failures.is_empty() {
Ok(())
} else {
Err(warp_utils::reject::indexed_bad_request(
"error processing sync committee signatures".to_string(),
failures,
))
}
}
/// Get the set of all subnet assignments for a `SyncCommitteeMessage`.
pub fn get_subnet_positions_for_sync_committee_message<T: BeaconChainTypes>(
sync_message: &SyncCommitteeMessage,
chain: &BeaconChain<T>,
) -> Result<HashMap<SyncSubnetId, Vec<usize>>, SyncVerificationError> {
let pubkey = chain
.validator_pubkey_bytes(sync_message.validator_index as usize)?
.ok_or(SyncVerificationError::UnknownValidatorIndex(
sync_message.validator_index as usize,
))?;
let sync_committee = chain.sync_committee_at_next_slot(sync_message.get_slot())?;
Ok(sync_committee.subcommittee_positions_for_public_key(&pubkey)?)
}
/// Receive signed contributions and proofs, storing them in the op pool and broadcasting.
pub fn process_signed_contribution_and_proofs<T: BeaconChainTypes>(
signed_contribution_and_proofs: Vec<SignedContributionAndProof<T::EthSpec>>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
chain: &BeaconChain<T>,
log: Logger,
) -> Result<(), warp::reject::Rejection> {
let mut verified_contributions = Vec::with_capacity(signed_contribution_and_proofs.len());
let mut failures = vec![];
// Verify contributions & broadcast to the network.
for (index, contribution) in signed_contribution_and_proofs.into_iter().enumerate() {
let aggregator_index = contribution.message.aggregator_index;
let subcommittee_index = contribution.message.contribution.subcommittee_index;
let contribution_slot = contribution.message.contribution.slot;
match chain.verify_sync_contribution_for_gossip(contribution) {
Ok(verified_contribution) => {
publish_pubsub_message(
&network_tx,
PubsubMessage::SignedContributionAndProof(Box::new(
verified_contribution.aggregate().clone(),
)),
)?;
// FIXME(altair): notify validator monitor
verified_contributions.push((index, verified_contribution));
}
// If we already know the contribution, don't broadcast it or attempt to
// further verify it. Return success.
Err(SyncVerificationError::SyncContributionAlreadyKnown(_)) => continue,
Err(e) => {
error!(
log,
"Failure verifying signed contribution and proof";
"error" => ?e,
"request_index" => index,
"aggregator_index" => aggregator_index,
"subcommittee_index" => subcommittee_index,
"contribution_slot" => contribution_slot,
);
failures.push(api_types::Failure::new(
index,
format!("Verification: {:?}", e),
));
}
}
}
// Add to the block inclusion pool.
for (index, verified_contribution) in verified_contributions {
if let Err(e) = chain.add_contribution_to_block_inclusion_pool(verified_contribution) {
warn!(
log,
"Could not add verified sync contribution to the inclusion pool";
"error" => ?e,
"request_index" => index,
);
failures.push(api_types::Failure::new(index, format!("Op pool: {:?}", e)));
}
}
if !failures.is_empty() {
Err(warp_utils::reject::indexed_bad_request(
"error processing contribution and proofs".to_string(),
failures,
))
} else {
Ok(())
}
}

View File

@ -0,0 +1,28 @@
use crate::api_types::{EndpointVersion, ForkVersionedResponse};
use serde::Serialize;
use types::ForkName;
pub const V1: EndpointVersion = EndpointVersion(1);
pub const V2: EndpointVersion = EndpointVersion(2);
pub fn fork_versioned_response<T: Serialize>(
endpoint_version: EndpointVersion,
fork_name: Option<ForkName>,
data: T,
) -> Result<ForkVersionedResponse<T>, warp::reject::Rejection> {
let fork_name = if endpoint_version == V1 {
None
} else if endpoint_version == V2 {
fork_name
} else {
return Err(unsupported_version_rejection(endpoint_version));
};
Ok(ForkVersionedResponse {
version: fork_name,
data,
})
}
pub fn unsupported_version_rejection(version: EndpointVersion) -> warp::reject::Rejection {
warp_utils::reject::custom_bad_request(format!("Unsupported endpoint version: {}", version))
}

View File

@ -0,0 +1,142 @@
use beacon_chain::{
test_utils::{BeaconChainHarness, EphemeralHarnessType},
BeaconChain, BeaconChainTypes,
};
use eth2::{BeaconNodeHttpClient, Timeouts};
use eth2_libp2p::{
discv5::enr::{CombinedKey, EnrBuilder},
rpc::methods::{MetaData, MetaDataV2},
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield, SyncState},
Enr, NetworkGlobals, PeerId,
};
use http_api::{Config, Context};
use network::NetworkMessage;
use sensitive_url::SensitiveUrl;
use slog::Logger;
use std::future::Future;
use std::net::{Ipv4Addr, SocketAddr};
use std::sync::Arc;
use std::time::Duration;
use tokio::sync::{mpsc, oneshot};
use types::{test_utils::generate_deterministic_keypairs, ChainSpec, EthSpec};
pub const TCP_PORT: u16 = 42;
pub const UDP_PORT: u16 = 42;
pub const SEQ_NUMBER: u64 = 0;
pub const EXTERNAL_ADDR: &str = "/ip4/0.0.0.0/tcp/9000";
/// HTTP API tester that allows interaction with the underlying beacon chain harness.
pub struct InteractiveTester<E: EthSpec> {
pub harness: BeaconChainHarness<EphemeralHarnessType<E>>,
pub client: BeaconNodeHttpClient,
pub network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
_server_shutdown: oneshot::Sender<()>,
}
/// The result of calling `create_api_server`.
///
/// Glue-type between `tests::ApiTester` and `InteractiveTester`.
pub struct ApiServer<E: EthSpec, SFut: Future<Output = ()>> {
pub server: SFut,
pub listening_socket: SocketAddr,
pub shutdown_tx: oneshot::Sender<()>,
pub network_rx: tokio::sync::mpsc::UnboundedReceiver<NetworkMessage<E>>,
pub local_enr: Enr,
pub external_peer_id: PeerId,
}
impl<E: EthSpec> InteractiveTester<E> {
pub fn new(spec: Option<ChainSpec>, validator_count: usize) -> Self {
let harness = BeaconChainHarness::new(
E::default(),
spec,
generate_deterministic_keypairs(validator_count),
);
let ApiServer {
server,
listening_socket,
shutdown_tx: _server_shutdown,
network_rx,
..
} = create_api_server(harness.chain.clone(), harness.logger().clone());
tokio::spawn(server);
let client = BeaconNodeHttpClient::new(
SensitiveUrl::parse(&format!(
"http://{}:{}",
listening_socket.ip(),
listening_socket.port()
))
.unwrap(),
Timeouts::set_all(Duration::from_secs(1)),
);
Self {
harness,
client,
network_rx,
_server_shutdown,
}
}
}
pub fn create_api_server<T: BeaconChainTypes>(
chain: Arc<BeaconChain<T>>,
log: Logger,
) -> ApiServer<T::EthSpec, impl Future<Output = ()>> {
let (network_tx, network_rx) = mpsc::unbounded_channel();
// Default metadata
let meta_data = MetaData::V2(MetaDataV2 {
seq_number: SEQ_NUMBER,
attnets: EnrAttestationBitfield::<T::EthSpec>::default(),
syncnets: EnrSyncCommitteeBitfield::<T::EthSpec>::default(),
});
let enr_key = CombinedKey::generate_secp256k1();
let enr = EnrBuilder::new("v4").build(&enr_key).unwrap();
let network_globals =
NetworkGlobals::new(enr.clone(), TCP_PORT, UDP_PORT, meta_data, vec![], &log);
let peer_id = PeerId::random();
network_globals
.peers
.write()
.connect_ingoing(&peer_id, EXTERNAL_ADDR.parse().unwrap(), None);
*network_globals.sync_state.write() = SyncState::Synced;
let eth1_service = eth1::Service::new(eth1::Config::default(), log.clone(), chain.spec.clone());
let context = Arc::new(Context {
config: Config {
enabled: true,
listen_addr: Ipv4Addr::new(127, 0, 0, 1),
listen_port: 0,
allow_origin: None,
serve_legacy_spec: true,
},
chain: Some(chain.clone()),
network_tx: Some(network_tx),
network_globals: Some(Arc::new(network_globals)),
eth1_service: Some(eth1_service),
log,
});
let ctx = context.clone();
let (shutdown_tx, shutdown_rx) = oneshot::channel();
let server_shutdown = async {
// It's not really interesting why this triggered, just that it happened.
let _ = shutdown_rx.await;
};
let (listening_socket, server) = http_api::serve(ctx, server_shutdown).unwrap();
ApiServer {
server,
listening_socket,
shutdown_tx,
network_rx,
local_enr: enr,
external_peer_id: peer_id,
}
}

View File

@ -0,0 +1,305 @@
//! Tests for API behaviour across fork boundaries.
use crate::common::*;
use beacon_chain::{test_utils::RelativeSyncCommittee, StateSkipConfig};
use eth2::types::{StateId, SyncSubcommittee};
use types::{ChainSpec, Epoch, EthSpec, MinimalEthSpec, Slot};
type E = MinimalEthSpec;
fn altair_spec(altair_fork_epoch: Epoch) -> ChainSpec {
let mut spec = E::default_spec();
spec.altair_fork_epoch = Some(altair_fork_epoch);
spec
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn sync_committee_duties_across_fork() {
let validator_count = E::sync_committee_size();
let fork_epoch = Epoch::new(8);
let spec = altair_spec(fork_epoch);
let tester = InteractiveTester::<E>::new(Some(spec.clone()), validator_count);
let harness = &tester.harness;
let client = &tester.client;
let all_validators = harness.get_all_validators();
let all_validators_u64 = all_validators.iter().map(|x| *x as u64).collect::<Vec<_>>();
assert_eq!(harness.get_current_slot(), 0);
// Prior to the fork the endpoint should return an empty vec.
let early_duties = client
.post_validator_duties_sync(fork_epoch - 1, &all_validators_u64)
.await
.unwrap()
.data;
assert!(early_duties.is_empty());
// If there's a skip slot at the fork slot, the endpoint should return duties, even
// though the head state hasn't transitioned yet.
let fork_slot = fork_epoch.start_slot(E::slots_per_epoch());
let (genesis_state, genesis_state_root) = harness.get_current_state_and_root();
let (_, state) = harness
.add_attested_block_at_slot(
fork_slot - 1,
genesis_state,
genesis_state_root,
&all_validators,
)
.unwrap();
harness.advance_slot();
assert_eq!(harness.get_current_slot(), fork_slot);
let sync_duties = client
.post_validator_duties_sync(fork_epoch, &all_validators_u64)
.await
.unwrap()
.data;
assert_eq!(sync_duties.len(), E::sync_committee_size());
// After applying a block at the fork slot the duties should remain unchanged.
let state_root = state.canonical_root();
harness
.add_attested_block_at_slot(fork_slot, state, state_root, &all_validators)
.unwrap();
assert_eq!(
client
.post_validator_duties_sync(fork_epoch, &all_validators_u64)
.await
.unwrap()
.data,
sync_duties
);
// Sync duties should also be available for the next period.
let current_period = fork_epoch.sync_committee_period(&spec).unwrap();
let next_period_epoch = spec.epochs_per_sync_committee_period * (current_period + 1);
let next_period_duties = client
.post_validator_duties_sync(next_period_epoch, &all_validators_u64)
.await
.unwrap()
.data;
assert_eq!(next_period_duties.len(), E::sync_committee_size());
// Sync duties should *not* be available for the period after the next period.
// We expect a 400 (bad request) response.
let next_next_period_epoch = spec.epochs_per_sync_committee_period * (current_period + 2);
assert_eq!(
client
.post_validator_duties_sync(next_next_period_epoch, &all_validators_u64)
.await
.unwrap_err()
.status()
.unwrap(),
400
);
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn attestations_across_fork_with_skip_slots() {
let validator_count = E::sync_committee_size();
let fork_epoch = Epoch::new(8);
let spec = altair_spec(fork_epoch);
let tester = InteractiveTester::<E>::new(Some(spec.clone()), validator_count);
let harness = &tester.harness;
let client = &tester.client;
let all_validators = harness.get_all_validators();
let fork_slot = fork_epoch.start_slot(E::slots_per_epoch());
let fork_state = harness
.chain
.state_at_slot(fork_slot, StateSkipConfig::WithStateRoots)
.unwrap();
harness.set_current_slot(fork_slot);
let attestations = harness.make_attestations(
&all_validators,
&fork_state,
fork_state.canonical_root(),
(*fork_state.get_block_root(fork_slot - 1).unwrap()).into(),
fork_slot,
);
let unaggregated_attestations = attestations
.iter()
.flat_map(|(atts, _)| atts.iter().map(|(att, _)| att.clone()))
.collect::<Vec<_>>();
assert!(!unaggregated_attestations.is_empty());
client
.post_beacon_pool_attestations(&unaggregated_attestations)
.await
.unwrap();
let signed_aggregates = attestations
.into_iter()
.filter_map(|(_, op_aggregate)| op_aggregate)
.collect::<Vec<_>>();
assert!(!signed_aggregates.is_empty());
client
.post_validator_aggregate_and_proof(&signed_aggregates)
.await
.unwrap();
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn sync_contributions_across_fork_with_skip_slots() {
let validator_count = E::sync_committee_size();
let fork_epoch = Epoch::new(8);
let spec = altair_spec(fork_epoch);
let tester = InteractiveTester::<E>::new(Some(spec.clone()), validator_count);
let harness = &tester.harness;
let client = &tester.client;
let fork_slot = fork_epoch.start_slot(E::slots_per_epoch());
let fork_state = harness
.chain
.state_at_slot(fork_slot, StateSkipConfig::WithStateRoots)
.unwrap();
harness.set_current_slot(fork_slot);
let sync_messages = harness.make_sync_contributions(
&fork_state,
*fork_state.get_block_root(fork_slot - 1).unwrap(),
fork_slot,
RelativeSyncCommittee::Current,
);
let sync_committee_messages = sync_messages
.iter()
.flat_map(|(messages, _)| messages.iter().map(|(message, _subnet)| message.clone()))
.collect::<Vec<_>>();
assert!(!sync_committee_messages.is_empty());
client
.post_beacon_pool_sync_committee_signatures(&sync_committee_messages)
.await
.unwrap();
let signed_contributions = sync_messages
.into_iter()
.filter_map(|(_, op_aggregate)| op_aggregate)
.collect::<Vec<_>>();
assert!(!signed_contributions.is_empty());
client
.post_validator_contribution_and_proofs(&signed_contributions)
.await
.unwrap();
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn sync_committee_indices_across_fork() {
let validator_count = E::sync_committee_size();
let fork_epoch = Epoch::new(8);
let spec = altair_spec(fork_epoch);
let tester = InteractiveTester::<E>::new(Some(spec.clone()), validator_count);
let harness = &tester.harness;
let client = &tester.client;
let all_validators = harness.get_all_validators();
// Flatten subcommittees into a single vec.
let flatten = |subcommittees: &[SyncSubcommittee]| -> Vec<u64> {
subcommittees
.iter()
.flat_map(|sub| sub.indices.iter().copied())
.collect()
};
// Prior to the fork the `sync_committees` endpoint should return a 400 error.
assert_eq!(
client
.get_beacon_states_sync_committees(StateId::Slot(Slot::new(0)), None)
.await
.unwrap_err()
.status()
.unwrap(),
400
);
assert_eq!(
client
.get_beacon_states_sync_committees(StateId::Head, Some(Epoch::new(0)))
.await
.unwrap_err()
.status()
.unwrap(),
400
);
// If there's a skip slot at the fork slot, the endpoint will return a 400 until a block is
// applied.
let fork_slot = fork_epoch.start_slot(E::slots_per_epoch());
let (genesis_state, genesis_state_root) = harness.get_current_state_and_root();
let (_, state) = harness
.add_attested_block_at_slot(
fork_slot - 1,
genesis_state,
genesis_state_root,
&all_validators,
)
.unwrap();
harness.advance_slot();
assert_eq!(harness.get_current_slot(), fork_slot);
// Using the head state must fail.
assert_eq!(
client
.get_beacon_states_sync_committees(StateId::Head, Some(fork_epoch))
.await
.unwrap_err()
.status()
.unwrap(),
400
);
// In theory we could do a state advance and make this work, but to keep things simple I've
// avoided doing that for now.
assert_eq!(
client
.get_beacon_states_sync_committees(StateId::Slot(fork_slot), None)
.await
.unwrap_err()
.status()
.unwrap(),
400
);
// Once the head is updated it should be useable for requests, including in the next sync
// committee period.
let state_root = state.canonical_root();
harness
.add_attested_block_at_slot(fork_slot + 1, state, state_root, &all_validators)
.unwrap();
let current_period = fork_epoch.sync_committee_period(&spec).unwrap();
let next_period_epoch = spec.epochs_per_sync_committee_period * (current_period + 1);
assert!(next_period_epoch > fork_epoch);
for epoch in [
None,
Some(fork_epoch),
Some(fork_epoch + 1),
Some(next_period_epoch),
Some(next_period_epoch + 1),
] {
let committee = client
.get_beacon_states_sync_committees(StateId::Head, epoch)
.await
.unwrap()
.data;
assert_eq!(committee.validators.len(), E::sync_committee_size());
assert_eq!(
committee.validators,
flatten(&committee.validator_aggregates)
);
}
}

View File

@ -0,0 +1,6 @@
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
#![recursion_limit = "256"]
pub mod common;
pub mod fork_tests;
pub mod tests;

View File

@ -1,6 +1,4 @@
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
#![recursion_limit = "256"]
use crate::common::{create_api_server, ApiServer};
use beacon_chain::{
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
BeaconChain, StateSkipConfig, WhenSlotSkipped, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
@ -9,21 +7,14 @@ use environment::null_logger;
use eth2::Error;
use eth2::StatusCode;
use eth2::{types::*, BeaconNodeHttpClient, Timeouts};
use eth2_libp2p::discv5::enr::{CombinedKey, EnrBuilder};
use eth2_libp2p::{
rpc::methods::{MetaData, MetaDataV2},
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield, SyncState},
Enr, EnrExt, NetworkGlobals, PeerId,
};
use eth2_libp2p::{Enr, EnrExt, PeerId};
use futures::stream::{Stream, StreamExt};
use futures::FutureExt;
use http_api::{Config, Context};
use network::NetworkMessage;
use sensitive_url::SensitiveUrl;
use slot_clock::SlotClock;
use state_processing::per_slot_processing;
use std::convert::TryInto;
use std::net::Ipv4Addr;
use std::sync::Arc;
use tokio::sync::{mpsc, oneshot};
use tokio::time::Duration;
@ -41,9 +32,6 @@ const VALIDATOR_COUNT: usize = SLOTS_PER_EPOCH as usize;
const CHAIN_LENGTH: u64 = SLOTS_PER_EPOCH * 5 - 1; // Make `next_block` an epoch transition
const JUSTIFIED_EPOCH: u64 = 4;
const FINALIZED_EPOCH: u64 = 3;
const TCP_PORT: u16 = 42;
const UDP_PORT: u16 = 42;
const SEQ_NUMBER: u64 = 0;
const EXTERNAL_ADDR: &str = "/ip4/0.0.0.0/tcp/9000";
/// Skipping the slots around the epoch boundary allows us to check that we're obtaining states
@ -74,9 +62,13 @@ struct ApiTester {
impl ApiTester {
pub fn new() -> Self {
let mut harness = BeaconChainHarness::new(
// This allows for testing voluntary exits without building out a massive chain.
let mut spec = E::default_spec();
spec.shard_committee_period = 2;
let harness = BeaconChainHarness::new(
MainnetEthSpec,
None,
Some(spec),
generate_deterministic_keypairs(VALIDATOR_COUNT),
);
@ -134,13 +126,7 @@ impl ApiTester {
let proposer_slashing = harness.make_proposer_slashing(2);
let voluntary_exit = harness.make_voluntary_exit(3, harness.chain.epoch().unwrap());
// Changing this *after* the chain has been initialized is a bit cheeky, but it shouldn't
// cause issue.
//
// This allows for testing voluntary exits without building out a massive chain.
harness.chain.spec.shard_committee_period = 2;
let chain = Arc::new(harness.chain);
let chain = harness.chain.clone();
assert_eq!(
chain.head_info().unwrap().finalized_checkpoint.epoch,
@ -157,56 +143,18 @@ impl ApiTester {
"precondition: justification"
);
let (network_tx, network_rx) = mpsc::unbounded_channel();
let log = null_logger().unwrap();
// Default metadata
let meta_data = MetaData::V2(MetaDataV2 {
seq_number: SEQ_NUMBER,
attnets: EnrAttestationBitfield::<MainnetEthSpec>::default(),
syncnets: EnrSyncCommitteeBitfield::<MainnetEthSpec>::default(),
});
let enr_key = CombinedKey::generate_secp256k1();
let enr = EnrBuilder::new("v4").build(&enr_key).unwrap();
let enr_clone = enr.clone();
let network_globals = NetworkGlobals::new(enr, TCP_PORT, UDP_PORT, meta_data, vec![], &log);
let ApiServer {
server,
listening_socket,
shutdown_tx,
network_rx,
local_enr,
external_peer_id,
} = create_api_server(chain.clone(), log);
let peer_id = PeerId::random();
network_globals.peers.write().connect_ingoing(
&peer_id,
EXTERNAL_ADDR.parse().unwrap(),
None,
);
*network_globals.sync_state.write() = SyncState::Synced;
let eth1_service =
eth1::Service::new(eth1::Config::default(), log.clone(), chain.spec.clone());
let context = Arc::new(Context {
config: Config {
enabled: true,
listen_addr: Ipv4Addr::new(127, 0, 0, 1),
listen_port: 0,
allow_origin: None,
serve_legacy_spec: true,
},
chain: Some(chain.clone()),
network_tx: Some(network_tx),
network_globals: Some(Arc::new(network_globals)),
eth1_service: Some(eth1_service),
log,
});
let ctx = context.clone();
let (shutdown_tx, shutdown_rx) = oneshot::channel();
let server_shutdown = async {
// It's not really interesting why this triggered, just that it happened.
let _ = shutdown_rx.await;
};
let (listening_socket, server) = http_api::serve(ctx, server_shutdown).unwrap();
tokio::spawn(async { server.await });
tokio::spawn(server);
let client = BeaconNodeHttpClient::new(
SensitiveUrl::parse(&format!(
@ -230,8 +178,8 @@ impl ApiTester {
_server_shutdown: shutdown_tx,
validator_keypairs: harness.validator_keypairs,
network_rx,
local_enr: enr_clone,
external_peer_id: peer_id,
local_enr,
external_peer_id,
}
}
@ -271,58 +219,20 @@ impl ApiTester {
let proposer_slashing = harness.make_proposer_slashing(2);
let voluntary_exit = harness.make_voluntary_exit(3, harness.chain.epoch().unwrap());
let chain = Arc::new(harness.chain);
let (network_tx, network_rx) = mpsc::unbounded_channel();
let chain = harness.chain.clone();
let log = null_logger().unwrap();
// Default metadata
let meta_data = MetaData::V2(MetaDataV2 {
seq_number: SEQ_NUMBER,
attnets: EnrAttestationBitfield::<MainnetEthSpec>::default(),
syncnets: EnrSyncCommitteeBitfield::<MainnetEthSpec>::default(),
});
let enr_key = CombinedKey::generate_secp256k1();
let enr = EnrBuilder::new("v4").build(&enr_key).unwrap();
let enr_clone = enr.clone();
let network_globals = NetworkGlobals::new(enr, TCP_PORT, UDP_PORT, meta_data, vec![], &log);
let ApiServer {
server,
listening_socket,
shutdown_tx,
network_rx,
local_enr,
external_peer_id,
} = create_api_server(chain.clone(), log);
let peer_id = PeerId::random();
network_globals.peers.write().connect_ingoing(
&peer_id,
EXTERNAL_ADDR.parse().unwrap(),
None,
);
*network_globals.sync_state.write() = SyncState::Synced;
let eth1_service =
eth1::Service::new(eth1::Config::default(), log.clone(), chain.spec.clone());
let context = Arc::new(Context {
config: Config {
enabled: true,
listen_addr: Ipv4Addr::new(127, 0, 0, 1),
listen_port: 0,
allow_origin: None,
serve_legacy_spec: true,
},
chain: Some(chain.clone()),
network_tx: Some(network_tx),
network_globals: Some(Arc::new(network_globals)),
eth1_service: Some(eth1_service),
log,
});
let ctx = context.clone();
let (shutdown_tx, shutdown_rx) = oneshot::channel();
let server_shutdown = async {
// It's not really interesting why this triggered, just that it happened.
let _ = shutdown_rx.await;
};
let (listening_socket, server) = http_api::serve(ctx, server_shutdown).unwrap();
tokio::spawn(async { server.await });
tokio::spawn(server);
let client = BeaconNodeHttpClient::new(
SensitiveUrl::parse(&format!(
@ -346,8 +256,8 @@ impl ApiTester {
_server_shutdown: shutdown_tx,
validator_keypairs: harness.validator_keypairs,
network_rx,
local_enr: enr_clone,
external_peer_id: peer_id,
local_enr,
external_peer_id,
}
}
@ -1011,13 +921,18 @@ impl ApiTester {
}
}
let json_result = self
.client
.get_beacon_blocks(block_id)
.await
.unwrap()
.map(|res| res.data);
assert_eq!(json_result, expected, "{:?}", block_id);
let json_result = self.client.get_beacon_blocks(block_id).await.unwrap();
if let (Some(json), Some(expected)) = (&json_result, &expected) {
assert_eq!(json.data, *expected, "{:?}", block_id);
assert_eq!(
json.version,
Some(expected.fork_name(&self.chain.spec).unwrap())
);
} else {
assert_eq!(json_result, None);
assert_eq!(expected, None);
}
let ssz_result = self
.client
@ -1025,6 +940,16 @@ impl ApiTester {
.await
.unwrap();
assert_eq!(ssz_result, expected, "{:?}", block_id);
// Check that the legacy v1 API still works but doesn't return a version field.
let v1_result = self.client.get_beacon_blocks_v1(block_id).await.unwrap();
if let (Some(v1_result), Some(expected)) = (&v1_result, &expected) {
assert_eq!(v1_result.version, None);
assert_eq!(v1_result.data, *expected);
} else {
assert_eq!(v1_result, None);
assert_eq!(expected, None);
}
}
self
@ -1443,23 +1368,44 @@ impl ApiTester {
pub async fn test_get_debug_beacon_states(self) -> Self {
for state_id in self.interesting_state_ids() {
let result_json = self.client.get_debug_beacon_states(state_id).await.unwrap();
let mut expected = self.get_state(state_id);
expected.as_mut().map(|state| state.drop_all_caches());
if let (Some(json), Some(expected)) = (&result_json, &expected) {
assert_eq!(json.data, *expected, "{:?}", state_id);
assert_eq!(
json.version,
Some(expected.fork_name(&self.chain.spec).unwrap())
);
} else {
assert_eq!(result_json, None);
assert_eq!(expected, None);
}
// Check SSZ API.
let result_ssz = self
.client
.get_debug_beacon_states_ssz(state_id, &self.chain.spec)
.await
.unwrap();
let result_json = self
.client
.get_debug_beacon_states(state_id)
.await
.unwrap()
.map(|res| res.data);
let mut expected = self.get_state(state_id);
expected.as_mut().map(|state| state.drop_all_caches());
assert_eq!(result_ssz, expected, "{:?}", state_id);
assert_eq!(result_json, expected, "{:?}", state_id);
// Check legacy v1 API.
let result_v1 = self
.client
.get_debug_beacon_states_v1(state_id)
.await
.unwrap();
if let (Some(json), Some(expected)) = (&result_v1, &expected) {
assert_eq!(json.version, None);
assert_eq!(json.data, *expected, "{:?}", state_id);
} else {
assert_eq!(result_v1, None);
assert_eq!(expected, None);
}
}
self

View File

@ -23,8 +23,8 @@ use std::time::Duration;
use tokio::runtime::Runtime;
use tokio::sync::mpsc;
use types::{
test_utils::generate_deterministic_keypairs, Attestation, AttesterSlashing, MainnetEthSpec,
ProposerSlashing, SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
test_utils::generate_deterministic_keypairs, Attestation, AttesterSlashing, EthSpec,
MainnetEthSpec, ProposerSlashing, SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
};
type E = MainnetEthSpec;
@ -71,9 +71,13 @@ impl Drop for TestRig {
impl TestRig {
pub fn new(chain_length: u64) -> Self {
let mut harness = BeaconChainHarness::new(
// This allows for testing voluntary exits without building out a massive chain.
let mut spec = E::default_spec();
spec.shard_committee_period = 2;
let harness = BeaconChainHarness::new(
MainnetEthSpec,
None,
Some(spec),
generate_deterministic_keypairs(VALIDATOR_COUNT),
);
@ -151,13 +155,7 @@ impl TestRig {
let proposer_slashing = harness.make_proposer_slashing(2);
let voluntary_exit = harness.make_voluntary_exit(3, harness.chain.epoch().unwrap());
// Changing this *after* the chain has been initialized is a bit cheeky, but it shouldn't
// cause issue.
//
// This allows for testing voluntary exits without building out a massive chain.
harness.chain.spec.shard_committee_period = 2;
let chain = Arc::new(harness.chain);
let chain = harness.chain;
let (network_tx, _network_rx) = mpsc::unbounded_channel();

View File

@ -27,8 +27,8 @@ impl<T: BeaconChainTypes> Worker<T> {
/// Creates a log if there is an internal error.
fn send_sync_message(&self, message: SyncMessage<T::EthSpec>) {
self.sync_tx.send(message).unwrap_or_else(|e| {
debug!(self.log, "Could not send message to the sync service, likely shutdown";
"error" => %e)
debug!(self.log, "Could not send message to the sync service";
"error" => %e)
});
}

View File

@ -35,15 +35,13 @@ mod tests {
fn test_dht_persistence() {
let log = get_logger(false);
let beacon_chain = Arc::new(
BeaconChainHarness::new_with_store_config(
MinimalEthSpec,
None,
generate_deterministic_keypairs(8),
StoreConfig::default(),
)
.chain,
);
let beacon_chain = BeaconChainHarness::new_with_store_config(
MinimalEthSpec,
None,
generate_deterministic_keypairs(8),
StoreConfig::default(),
)
.chain;
let store = beacon_chain.store.clone();

View File

@ -40,13 +40,22 @@ pub async fn run<T: EthSpec>(config: BootNodeConfig<T>, log: slog::Logger) {
};
// construct the discv5 server
let mut discv5 = Discv5::new(config.local_enr, config.local_key, discv5_config).unwrap();
let mut discv5 =
Discv5::new(config.local_enr.clone(), config.local_key, discv5_config).unwrap();
// If there are any bootnodes add them to the routing table
for enr in config.boot_nodes {
info!(log, "Adding bootnode"; "address" => format!("{:?}", enr.udp_socket()), "peer_id" => enr.peer_id().to_string(), "node_id" => enr.node_id().to_string());
if let Err(e) = discv5.add_enr(enr) {
slog::warn!(log, "Failed adding ENR"; "error" => e.to_string());
info!(
log,
"Adding bootnode";
"address" => ?enr.udp_socket(),
"peer_id" => enr.peer_id().to_string(),
"node_id" => enr.node_id().to_string()
);
if enr != config.local_enr {
if let Err(e) = discv5.add_enr(enr) {
slog::warn!(log, "Failed adding ENR"; "error" => e.to_string());
}
}
}

View File

@ -12,7 +12,7 @@ pub mod lighthouse;
pub mod lighthouse_vc;
pub mod types;
use self::types::*;
use self::types::{Error as ResponseError, *};
use eth2_libp2p::PeerId;
use futures::Stream;
use futures_util::StreamExt;
@ -26,6 +26,9 @@ use std::fmt;
use std::iter::Iterator;
use std::time::Duration;
pub const V1: EndpointVersion = EndpointVersion(1);
pub const V2: EndpointVersion = EndpointVersion(2);
#[derive(Debug)]
pub enum Error {
/// The `reqwest` client raised an error.
@ -86,6 +89,7 @@ pub struct Timeouts {
pub liveness: Duration,
pub proposal: Duration,
pub proposer_duties: Duration,
pub sync_duties: Duration,
}
impl Timeouts {
@ -96,6 +100,7 @@ impl Timeouts {
liveness: timeout,
proposal: timeout,
proposer_duties: timeout,
sync_duties: timeout,
}
}
}
@ -142,14 +147,14 @@ impl BeaconNodeHttpClient {
}
}
/// Return the path with the standard `/eth1/v1` prefix applied.
fn eth_path(&self) -> Result<Url, Error> {
/// Return the path with the standard `/eth/vX` prefix applied.
fn eth_path(&self, version: EndpointVersion) -> Result<Url, Error> {
let mut path = self.server.full.clone();
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("eth")
.push("v1");
.push(&version.to_string());
Ok(path)
}
@ -315,7 +320,7 @@ impl BeaconNodeHttpClient {
///
/// May return a `404` if beacon chain genesis has not yet occurred.
pub async fn get_beacon_genesis(&self) -> Result<GenericResponse<GenesisData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -332,7 +337,7 @@ impl BeaconNodeHttpClient {
&self,
state_id: StateId,
) -> Result<Option<GenericResponse<RootData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -351,7 +356,7 @@ impl BeaconNodeHttpClient {
&self,
state_id: StateId,
) -> Result<Option<GenericResponse<Fork>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -370,7 +375,7 @@ impl BeaconNodeHttpClient {
&self,
state_id: StateId,
) -> Result<Option<GenericResponse<FinalityCheckpointsData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -390,7 +395,7 @@ impl BeaconNodeHttpClient {
state_id: StateId,
ids: Option<&[ValidatorId]>,
) -> Result<Option<GenericResponse<Vec<ValidatorBalanceData>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -420,7 +425,7 @@ impl BeaconNodeHttpClient {
ids: Option<&[ValidatorId]>,
statuses: Option<&[ValidatorStatus]>,
) -> Result<Option<GenericResponse<Vec<ValidatorData>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -460,7 +465,7 @@ impl BeaconNodeHttpClient {
index: Option<u64>,
epoch: Option<Epoch>,
) -> Result<Option<GenericResponse<Vec<CommitteeData>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -487,6 +492,29 @@ impl BeaconNodeHttpClient {
self.get_opt(path).await
}
/// `GET beacon/states/{state_id}/sync_committees?epoch`
pub async fn get_beacon_states_sync_committees(
&self,
state_id: StateId,
epoch: Option<Epoch>,
) -> Result<GenericResponse<SyncCommitteeByValidatorIndices>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("states")
.push(&state_id.to_string())
.push("sync_committees");
if let Some(epoch) = epoch {
path.query_pairs_mut()
.append_pair("epoch", &epoch.to_string());
}
self.get(path).await
}
/// `GET beacon/states/{state_id}/validators/{validator_id}`
///
/// Returns `Ok(None)` on a 404 error.
@ -495,7 +523,7 @@ impl BeaconNodeHttpClient {
state_id: StateId,
validator_id: &ValidatorId,
) -> Result<Option<GenericResponse<ValidatorData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -516,7 +544,7 @@ impl BeaconNodeHttpClient {
slot: Option<Slot>,
parent_root: Option<Hash256>,
) -> Result<Option<GenericResponse<Vec<BlockHeaderData>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -543,7 +571,7 @@ impl BeaconNodeHttpClient {
&self,
block_id: BlockId,
) -> Result<Option<GenericResponse<BlockHeaderData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -561,7 +589,7 @@ impl BeaconNodeHttpClient {
&self,
block: &SignedBeaconBlock<T>,
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -574,14 +602,32 @@ impl BeaconNodeHttpClient {
Ok(())
}
/// `GET beacon/blocks`
/// `GET v2/beacon/blocks`
///
/// Returns `Ok(None)` on a 404 error.
pub async fn get_beacon_blocks<T: EthSpec>(
&self,
block_id: BlockId,
) -> Result<Option<GenericResponse<SignedBeaconBlock<T>>>, Error> {
let mut path = self.eth_path()?;
) -> Result<Option<ForkVersionedResponse<SignedBeaconBlock<T>>>, Error> {
let mut path = self.eth_path(V2)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("blocks")
.push(&block_id.to_string());
self.get_opt(path).await
}
/// `GET v1/beacon/blocks` (LEGACY)
///
/// Returns `Ok(None)` on a 404 error.
pub async fn get_beacon_blocks_v1<T: EthSpec>(
&self,
block_id: BlockId,
) -> Result<Option<ForkVersionedResponse<SignedBeaconBlock<T>>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -600,7 +646,7 @@ impl BeaconNodeHttpClient {
block_id: BlockId,
spec: &ChainSpec,
) -> Result<Option<SignedBeaconBlock<T>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V2)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -621,7 +667,7 @@ impl BeaconNodeHttpClient {
&self,
block_id: BlockId,
) -> Result<Option<GenericResponse<RootData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -640,7 +686,7 @@ impl BeaconNodeHttpClient {
&self,
block_id: BlockId,
) -> Result<Option<GenericResponse<Vec<Attestation<T>>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -657,7 +703,7 @@ impl BeaconNodeHttpClient {
&self,
attestations: &[Attestation<T>],
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -665,15 +711,8 @@ impl BeaconNodeHttpClient {
.push("pool")
.push("attestations");
let response = self
.client
.post(path)
.timeout(self.timeouts.attestation)
.json(attestations)
.send()
.await
.map_err(Error::Reqwest)?;
ok_or_indexed_error(response).await?;
self.post_with_timeout(path, &attestations, self.timeouts.attestation)
.await?;
Ok(())
}
@ -684,7 +723,7 @@ impl BeaconNodeHttpClient {
slot: Option<Slot>,
committee_index: Option<u64>,
) -> Result<GenericResponse<Vec<Attestation<T>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -710,7 +749,7 @@ impl BeaconNodeHttpClient {
&self,
slashing: &AttesterSlashing<T>,
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -727,7 +766,7 @@ impl BeaconNodeHttpClient {
pub async fn get_beacon_pool_attester_slashings<T: EthSpec>(
&self,
) -> Result<GenericResponse<Vec<AttesterSlashing<T>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -743,7 +782,7 @@ impl BeaconNodeHttpClient {
&self,
slashing: &ProposerSlashing,
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -760,7 +799,7 @@ impl BeaconNodeHttpClient {
pub async fn get_beacon_pool_proposer_slashings(
&self,
) -> Result<GenericResponse<Vec<ProposerSlashing>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -776,7 +815,7 @@ impl BeaconNodeHttpClient {
&self,
exit: &SignedVoluntaryExit,
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -793,7 +832,7 @@ impl BeaconNodeHttpClient {
pub async fn get_beacon_pool_voluntary_exits(
&self,
) -> Result<GenericResponse<Vec<SignedVoluntaryExit>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -804,9 +843,44 @@ impl BeaconNodeHttpClient {
self.get(path).await
}
/// `POST beacon/pool/sync_committees`
pub async fn post_beacon_pool_sync_committee_signatures(
&self,
signatures: &[SyncCommitteeMessage],
) -> Result<(), Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("pool")
.push("sync_committees");
self.post(path, &signatures).await?;
Ok(())
}
/// `POST validator/contribution_and_proofs`
pub async fn post_validator_contribution_and_proofs<T: EthSpec>(
&self,
signed_contributions: &[SignedContributionAndProof<T>],
) -> Result<(), Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("validator")
.push("contribution_and_proofs");
self.post(path, &signed_contributions).await?;
Ok(())
}
/// `GET config/fork_schedule`
pub async fn get_config_fork_schedule(&self) -> Result<GenericResponse<Vec<Fork>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -818,7 +892,7 @@ impl BeaconNodeHttpClient {
/// `GET config/spec`
pub async fn get_config_spec(&self) -> Result<GenericResponse<ConfigAndPreset>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -832,7 +906,7 @@ impl BeaconNodeHttpClient {
pub async fn get_config_deposit_contract(
&self,
) -> Result<GenericResponse<DepositContractData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -844,7 +918,7 @@ impl BeaconNodeHttpClient {
/// `GET node/version`
pub async fn get_node_version(&self) -> Result<GenericResponse<VersionData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -856,7 +930,7 @@ impl BeaconNodeHttpClient {
/// `GET node/identity`
pub async fn get_node_identity(&self) -> Result<GenericResponse<IdentityData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -868,7 +942,7 @@ impl BeaconNodeHttpClient {
/// `GET node/syncing`
pub async fn get_node_syncing(&self) -> Result<GenericResponse<SyncingData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -880,7 +954,7 @@ impl BeaconNodeHttpClient {
/// `GET node/health`
pub async fn get_node_health(&self) -> Result<StatusCode, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -906,7 +980,7 @@ impl BeaconNodeHttpClient {
&self,
peer_id: PeerId,
) -> Result<GenericResponse<PeerData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -923,7 +997,7 @@ impl BeaconNodeHttpClient {
states: Option<&[PeerState]>,
directions: Option<&[PeerDirection]>,
) -> Result<PeersData, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -953,7 +1027,7 @@ impl BeaconNodeHttpClient {
/// `GET node/peer_count`
pub async fn get_node_peer_count(&self) -> Result<GenericResponse<PeerCount>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -963,12 +1037,29 @@ impl BeaconNodeHttpClient {
self.get(path).await
}
/// `GET debug/beacon/states/{state_id}`
/// `GET v2/debug/beacon/states/{state_id}`
pub async fn get_debug_beacon_states<T: EthSpec>(
&self,
state_id: StateId,
) -> Result<Option<GenericResponse<BeaconState<T>>>, Error> {
let mut path = self.eth_path()?;
) -> Result<Option<ForkVersionedResponse<BeaconState<T>>>, Error> {
let mut path = self.eth_path(V2)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("debug")
.push("beacon")
.push("states")
.push(&state_id.to_string());
self.get_opt(path).await
}
/// `GET v1/debug/beacon/states/{state_id}` (LEGACY)
pub async fn get_debug_beacon_states_v1<T: EthSpec>(
&self,
state_id: StateId,
) -> Result<Option<ForkVersionedResponse<BeaconState<T>>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -987,7 +1078,7 @@ impl BeaconNodeHttpClient {
state_id: StateId,
spec: &ChainSpec,
) -> Result<Option<BeaconState<T>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1006,7 +1097,7 @@ impl BeaconNodeHttpClient {
pub async fn get_debug_beacon_heads(
&self,
) -> Result<GenericResponse<Vec<ChainHeadData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1022,7 +1113,7 @@ impl BeaconNodeHttpClient {
&self,
epoch: Epoch,
) -> Result<DutiesResponse<Vec<ProposerData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1035,14 +1126,14 @@ impl BeaconNodeHttpClient {
.await
}
/// `GET validator/blocks/{slot}`
/// `GET v2/validator/blocks/{slot}`
pub async fn get_validator_blocks<T: EthSpec>(
&self,
slot: Slot,
randao_reveal: &SignatureBytes,
graffiti: Option<&Graffiti>,
) -> Result<GenericResponse<BeaconBlock<T>>, Error> {
let mut path = self.eth_path()?;
) -> Result<ForkVersionedResponse<BeaconBlock<T>>, Error> {
let mut path = self.eth_path(V2)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1067,7 +1158,7 @@ impl BeaconNodeHttpClient {
slot: Slot,
committee_index: CommitteeIndex,
) -> Result<GenericResponse<AttestationData>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1087,7 +1178,7 @@ impl BeaconNodeHttpClient {
slot: Slot,
attestation_data_root: Hash256,
) -> Result<Option<GenericResponse<Attestation<T>>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1105,6 +1196,32 @@ impl BeaconNodeHttpClient {
.await
}
/// `GET validator/sync_committee_contribution`
pub async fn get_validator_sync_committee_contribution<T: EthSpec>(
&self,
sync_committee_data: &SyncContributionData,
) -> Result<Option<GenericResponse<SyncCommitteeContribution<T>>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("validator")
.push("sync_committee_contribution");
path.query_pairs_mut()
.append_pair("slot", &sync_committee_data.slot.to_string())
.append_pair(
"beacon_block_root",
&format!("{:?}", sync_committee_data.beacon_block_root),
)
.append_pair(
"subcommittee_index",
&sync_committee_data.subcommittee_index.to_string(),
);
self.get_opt(path).await
}
/// `POST lighthouse/liveness`
pub async fn post_lighthouse_liveness(
&self,
@ -1135,7 +1252,7 @@ impl BeaconNodeHttpClient {
epoch: Epoch,
indices: &[u64],
) -> Result<DutiesResponse<Vec<AttesterData>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1153,22 +1270,15 @@ impl BeaconNodeHttpClient {
&self,
aggregates: &[SignedAggregateAndProof<T>],
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("validator")
.push("aggregate_and_proofs");
let response = self
.client
.post(path)
.timeout(self.timeouts.attestation)
.json(aggregates)
.send()
.await
.map_err(Error::Reqwest)?;
ok_or_indexed_error(response).await?;
self.post_with_timeout(path, &aggregates, self.timeouts.attestation)
.await?;
Ok(())
}
@ -1178,7 +1288,7 @@ impl BeaconNodeHttpClient {
&self,
subscriptions: &[BeaconCommitteeSubscription],
) -> Result<(), Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
@ -1190,12 +1300,29 @@ impl BeaconNodeHttpClient {
Ok(())
}
/// `POST validator/sync_committee_subscriptions`
pub async fn post_validator_sync_committee_subscriptions(
&self,
subscriptions: &[SyncCommitteeSubscription],
) -> Result<(), Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("validator")
.push("sync_committee_subscriptions");
self.post(path, &subscriptions).await?;
Ok(())
}
/// `GET events?topics`
pub async fn get_events<T: EthSpec>(
&self,
topic: &[EventTopic],
) -> Result<impl Stream<Item = Result<EventKind<T>, Error>>, Error> {
let mut path = self.eth_path()?;
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("events");
@ -1219,6 +1346,25 @@ impl BeaconNodeHttpClient {
Err(e) => Err(Error::Reqwest(e)),
}))
}
/// `POST validator/duties/sync/{epoch}`
pub async fn post_validator_duties_sync(
&self,
epoch: Epoch,
indices: &[u64],
) -> Result<GenericResponse<Vec<SyncDuty>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("validator")
.push("duties")
.push("sync")
.push(&epoch.to_string());
self.post_with_timeout_and_response(path, &indices, self.timeouts.sync_duties)
.await
}
}
/// Returns `Ok(response)` if the response is a `200 OK` response. Otherwise, creates an
@ -1229,21 +1375,10 @@ async fn ok_or_error(response: Response) -> Result<Response, Error> {
if status == StatusCode::OK {
Ok(response)
} else if let Ok(message) = response.json().await {
Err(Error::ServerMessage(message))
} else {
Err(Error::StatusCode(status))
}
}
/// Returns `Ok(response)` if the response is a `200 OK` response. Otherwise, creates an
/// appropriate indexed error message.
async fn ok_or_indexed_error(response: Response) -> Result<Response, Error> {
let status = response.status();
if status == StatusCode::OK {
Ok(response)
} else if let Ok(message) = response.json().await {
Err(Error::ServerIndexedMessage(message))
match message {
ResponseError::Message(message) => Err(Error::ServerMessage(message)),
ResponseError::Indexed(indexed) => Err(Error::ServerIndexedMessage(indexed)),
}
} else {
Err(Error::StatusCode(status))
}

View File

@ -10,6 +10,14 @@ use std::fmt;
use std::str::{from_utf8, FromStr};
pub use types::*;
/// An API error serializable to JSON.
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
#[serde(untagged)]
pub enum Error {
Indexed(IndexedErrorMessage),
Message(ErrorMessage),
}
/// An API error serializable to JSON.
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct ErrorMessage {
@ -43,6 +51,30 @@ impl Failure {
}
}
/// The version of a single API endpoint, e.g. the `v1` in `/eth/v1/beacon/blocks`.
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct EndpointVersion(pub u64);
impl FromStr for EndpointVersion {
type Err = ();
fn from_str(s: &str) -> Result<Self, Self::Err> {
if let Some(version_str) = s.strip_prefix('v') {
u64::from_str(version_str)
.map(EndpointVersion)
.map_err(|_| ())
} else {
Err(())
}
}
}
impl std::fmt::Display for EndpointVersion {
fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> {
write!(fmt, "v{}", self.0)
}
}
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct GenesisData {
#[serde(with = "serde_utils::quoted_u64")]
@ -179,6 +211,14 @@ impl<'a, T: Serialize> From<&'a T> for GenericResponseRef<'a, T> {
}
}
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
// #[serde(bound = "T: Serialize + serde::de::DeserializeOwned")]
pub struct ForkVersionedResponse<T> {
#[serde(skip_serializing_if = "Option::is_none")]
pub version: Option<ForkName>,
pub data: T,
}
#[derive(Debug, Clone, Copy, PartialEq, Serialize, Deserialize)]
pub struct RootData {
pub root: Hash256,
@ -378,6 +418,11 @@ pub struct CommitteesQuery {
pub epoch: Option<Epoch>,
}
#[derive(Serialize, Deserialize)]
pub struct SyncCommitteesQuery {
pub epoch: Option<Epoch>,
}
#[derive(Serialize, Deserialize)]
pub struct AttestationPoolQuery {
pub slot: Option<Slot>,
@ -399,6 +444,20 @@ pub struct CommitteeData {
pub validators: Vec<u64>,
}
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct SyncCommitteeByValidatorIndices {
#[serde(with = "serde_utils::quoted_u64_vec")]
pub validators: Vec<u64>,
pub validator_aggregates: Vec<SyncSubcommittee>,
}
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
#[serde(transparent)]
pub struct SyncSubcommittee {
#[serde(with = "serde_utils::quoted_u64_vec")]
pub indices: Vec<u64>,
}
#[derive(Serialize, Deserialize)]
pub struct HeadersQuery {
pub slot: Option<Slot>,

View File

@ -1,27 +0,0 @@
[package]
name = "rest_types"
version = "0.2.0"
authors = ["Sigma Prime <contact@sigmaprime.io>"]
edition = "2018"
[dependencies]
types = { path = "../../consensus/types" }
eth2_ssz_derive = "0.1.0"
eth2_ssz = "0.1.2"
eth2_hashing = "0.1.0"
tree_hash = "0.1.0"
state_processing = { path = "../../consensus/state_processing" }
bls = { path = "../../crypto/bls" }
serde = { version = "1.0.110", features = ["derive"] }
rayon = "1.3.0"
hyper = "0.14.4"
tokio = { version = "1.1.0", features = ["sync"] }
environment = { path = "../../lighthouse/environment" }
store = { path = "../../beacon_node/store" }
beacon_chain = { path = "../../beacon_node/beacon_chain" }
serde_json = "1.0.52"
serde_yaml = "0.8.11"
[target.'cfg(target_os = "linux")'.dependencies]
psutil = "3.1.0"
procinfo = "0.4.2"

View File

@ -50,6 +50,7 @@ superstruct = "0.2.0"
serde_json = "1.0.58"
criterion = "0.3.3"
beacon_chain = { path = "../../beacon_node/beacon_chain" }
eth2_interop_keypairs = { path = "../../common/eth2_interop_keypairs" }
[features]
default = ["sqlite", "legacy-arith"]

View File

@ -149,6 +149,27 @@ impl<T: EthSpec> BeaconBlock<T> {
}
impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
/// Returns the name of the fork pertaining to `self`.
///
/// Will return an `Err` if `self` has been instantiated to a variant conflicting with the fork
/// dictated by `self.slot()`.
pub fn fork_name(&self, spec: &ChainSpec) -> Result<ForkName, InconsistentFork> {
let fork_at_slot = spec.fork_name_at_slot::<T>(self.slot());
let object_fork = match self {
BeaconBlockRef::Base { .. } => ForkName::Base,
BeaconBlockRef::Altair { .. } => ForkName::Altair,
};
if fork_at_slot == object_fork {
Ok(object_fork)
} else {
Err(InconsistentFork {
fork_at_slot,
object_fork,
})
}
}
/// Convenience accessor for the `body` as a `BeaconBlockBodyRef`.
pub fn body(&self) -> BeaconBlockBodyRef<'a, T> {
match self {

View File

@ -837,6 +837,32 @@ impl<T: EthSpec> BeaconState<T> {
})
}
/// Get the sync committee duties for a list of validator indices.
///
/// Will return a `SyncCommitteeNotKnown` error if the `epoch` is out of bounds with respect
/// to the current or next sync committee periods.
pub fn get_sync_committee_duties(
&self,
epoch: Epoch,
validator_indices: &[u64],
spec: &ChainSpec,
) -> Result<Vec<Option<SyncDuty>>, Error> {
let sync_committee = self.get_built_sync_committee(epoch, spec)?;
validator_indices
.iter()
.map(|&validator_index| {
let pubkey = self.get_validator(validator_index as usize)?.pubkey;
Ok(SyncDuty::from_sync_committee(
validator_index,
pubkey,
sync_committee,
))
})
.collect()
}
/// Get the canonical root of the `latest_block_header`, filling in its state root if necessary.
///
/// It needs filling in on all slots where there isn't a skip.

View File

@ -685,6 +685,8 @@ where
#[cfg(test)]
mod tests {
use super::*;
use itertools::Itertools;
use safe_arith::SafeArith;
#[test]
fn test_mainnet_spec_can_be_constructed() {
@ -745,6 +747,33 @@ mod tests {
}
}
}
// Test that `next_fork_epoch` is consistent with the other functions.
#[test]
fn next_fork_epoch_consistency() {
type E = MainnetEthSpec;
let spec = ChainSpec::mainnet();
let mut last_fork_slot = Slot::new(0);
for (_, fork) in ForkName::list_all().into_iter().tuple_windows() {
if let Some(fork_epoch) = spec.fork_epoch(fork) {
last_fork_slot = fork_epoch.start_slot(E::slots_per_epoch());
// Fork is activated at non-zero epoch: check that `next_fork_epoch` returns
// the correct result.
if let Ok(prior_slot) = last_fork_slot.safe_sub(1) {
let (next_fork, next_fork_epoch) =
spec.next_fork_epoch::<E>(prior_slot).unwrap();
assert_eq!(fork, next_fork);
assert_eq!(spec.fork_epoch(fork).unwrap(), next_fork_epoch);
}
} else {
// Fork is not activated, check that `next_fork_epoch` returns `None`.
assert_eq!(spec.next_fork_epoch::<E>(last_fork_slot), None);
}
}
}
}
#[cfg(test)]

View File

@ -1,6 +1,12 @@
use crate::{ChainSpec, Epoch};
use serde_derive::{Deserialize, Serialize};
use std::convert::TryFrom;
use std::fmt::{self, Display, Formatter};
use std::str::FromStr;
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)]
#[serde(try_from = "String")]
#[serde(into = "String")]
pub enum ForkName {
Base,
Altair,
@ -48,7 +54,7 @@ impl ForkName {
}
}
impl std::str::FromStr for ForkName {
impl FromStr for ForkName {
type Err = ();
fn from_str(fork_name: &str) -> Result<Self, ()> {
@ -60,6 +66,29 @@ impl std::str::FromStr for ForkName {
}
}
impl Display for ForkName {
fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), fmt::Error> {
match self {
ForkName::Base => "phase0".fmt(f),
ForkName::Altair => "altair".fmt(f),
}
}
}
impl From<ForkName> for String {
fn from(fork: ForkName) -> String {
fork.to_string()
}
}
impl TryFrom<String> for ForkName {
type Error = String;
fn try_from(s: String) -> Result<Self, Self::Error> {
Self::from_str(&s).map_err(|()| format!("Invalid fork name: {}", s))
}
}
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct InconsistentFork {
pub fork_at_slot: ForkName,

View File

@ -56,6 +56,7 @@ pub mod signed_contribution_and_proof;
pub mod signed_voluntary_exit;
pub mod signing_data;
pub mod sync_committee_subscription;
pub mod sync_duty;
pub mod validator;
pub mod validator_subscription;
pub mod voluntary_exit;
@ -137,9 +138,10 @@ pub use crate::subnet_id::SubnetId;
pub use crate::sync_aggregate::SyncAggregate;
pub use crate::sync_aggregator_selection_data::SyncAggregatorSelectionData;
pub use crate::sync_committee::SyncCommittee;
pub use crate::sync_committee_contribution::SyncCommitteeContribution;
pub use crate::sync_committee_contribution::{SyncCommitteeContribution, SyncContributionData};
pub use crate::sync_committee_message::SyncCommitteeMessage;
pub use crate::sync_committee_subscription::SyncCommitteeSubscription;
pub use crate::sync_duty::SyncDuty;
pub use crate::sync_selection_proof::SyncSelectionProof;
pub use crate::sync_subnet_id::SyncSubnetId;
pub use crate::validator::Validator;

View File

@ -71,20 +71,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
/// Will return an `Err` if `self` has been instantiated to a variant conflicting with the fork
/// dictated by `self.slot()`.
pub fn fork_name(&self, spec: &ChainSpec) -> Result<ForkName, InconsistentFork> {
let fork_at_slot = spec.fork_name_at_slot::<E>(self.slot());
let object_fork = match self {
SignedBeaconBlock::Base { .. } => ForkName::Base,
SignedBeaconBlock::Altair { .. } => ForkName::Altair,
};
if fork_at_slot == object_fork {
Ok(object_fork)
} else {
Err(InconsistentFork {
fork_at_slot,
object_fork,
})
}
self.message().fork_name(spec)
}
/// SSZ decode.

View File

@ -77,9 +77,9 @@ impl SignedRoot for Hash256 {}
/// This is not in the spec, but useful for determining uniqueness of sync committee contributions
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct SyncContributionData {
slot: Slot,
beacon_block_root: Hash256,
subcommittee_index: u64,
pub slot: Slot,
pub beacon_block_root: Hash256,
pub subcommittee_index: u64,
}
impl SyncContributionData {

View File

@ -0,0 +1,83 @@
use crate::{EthSpec, SyncCommittee, SyncSubnetId};
use bls::PublicKeyBytes;
use safe_arith::ArithError;
use serde_derive::{Deserialize, Serialize};
use std::collections::HashSet;
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)]
pub struct SyncDuty {
pub pubkey: PublicKeyBytes,
#[serde(with = "serde_utils::quoted_u64")]
pub validator_index: u64,
#[serde(with = "serde_utils::quoted_u64_vec")]
pub validator_sync_committee_indices: Vec<u64>,
}
impl SyncDuty {
/// Create a new `SyncDuty` from the list of validator indices in a sync committee.
pub fn from_sync_committee_indices(
validator_index: u64,
pubkey: PublicKeyBytes,
sync_committee_indices: &[usize],
) -> Option<Self> {
// Positions of the `validator_index` within the committee.
let validator_sync_committee_indices = sync_committee_indices
.iter()
.enumerate()
.filter_map(|(i, &v)| {
if validator_index == v as u64 {
Some(i as u64)
} else {
None
}
})
.collect();
Self::new(validator_index, pubkey, validator_sync_committee_indices)
}
/// Create a new `SyncDuty` from a `SyncCommittee`, which contains the pubkeys but not the
/// indices.
pub fn from_sync_committee<T: EthSpec>(
validator_index: u64,
pubkey: PublicKeyBytes,
sync_committee: &SyncCommittee<T>,
) -> Option<Self> {
let validator_sync_committee_indices = sync_committee
.pubkeys
.iter()
.enumerate()
.filter_map(|(i, committee_pubkey)| {
if &pubkey == committee_pubkey {
Some(i as u64)
} else {
None
}
})
.collect();
Self::new(validator_index, pubkey, validator_sync_committee_indices)
}
/// Create a duty if the `validator_sync_committee_indices` is non-empty.
fn new(
validator_index: u64,
pubkey: PublicKeyBytes,
validator_sync_committee_indices: Vec<u64>,
) -> Option<Self> {
if !validator_sync_committee_indices.is_empty() {
Some(SyncDuty {
pubkey,
validator_index,
validator_sync_committee_indices,
})
} else {
None
}
}
/// Get the set of subnet IDs for this duty.
pub fn subnet_ids<E: EthSpec>(&self) -> Result<HashSet<SyncSubnetId>, ArithError> {
SyncSubnetId::compute_subnets_for_sync_committee::<E>(
&self.validator_sync_committee_indices,
)
}
}

View File

@ -41,7 +41,8 @@ fn main() {
.takes_value(true)
.required(true)
.possible_values(&["minimal", "mainnet"])
.default_value("mainnet"),
.default_value("mainnet")
.global(true),
)
.arg(
Arg::with_name("testnet-dir")

View File

@ -20,4 +20,5 @@ exec lighthouse \
--enr-udp-port $2 \
--enr-tcp-port $2 \
--port $2 \
--http-port $3
--http-port $3 \
--target-peers $((NODE_COUNT - 1))

View File

@ -9,6 +9,8 @@
# $VALIDATOR_COUNT and $NODE_COUNT variables.
#
set -o nounset -o errexit -o pipefail
source ./vars.env
lcli \
@ -20,10 +22,9 @@ lcli \
NOW=`date +%s`
GENESIS_TIME=`expr $NOW + $GENESIS_DELAY`
lcli \
--spec mainnet \
new-testnet \
--spec $SPEC_PRESET \
--deposit-contract-address $DEPOSIT_CONTRACT_ADDRESS \
--testnet-dir $TESTNET_DIR \
--min-genesis-active-validator-count $GENESIS_VALIDATOR_COUNT \
@ -50,8 +51,8 @@ echo Validators generated with keystore passwords at $DATADIR.
echo "Building genesis state... (this might take a while)"
lcli \
--spec mainnet \
interop-genesis \
--spec $SPEC_PRESET \
--genesis-time $GENESIS_TIME \
--testnet-dir $TESTNET_DIR \
$GENESIS_VALIDATOR_COUNT

View File

@ -29,6 +29,9 @@ NETWORK_ID=4242
# Hard fork configuration
ALTAIR_FORK_EPOCH=18446744073709551615
# Spec version (mainnet or minimal)
SPEC_PRESET=mainnet
# Seconds per Eth2 slot
SECONDS_PER_SLOT=3

View File

@ -35,5 +35,8 @@ SECONDS_PER_SLOT=3
# Seconds per Eth1 block
SECONDS_PER_ETH1_BLOCK=1
# Spec preset
SPEC_PRESET=mainnet
# Enable doppelganger detection
VC_ARGS=" --enable-doppelganger-protection "

View File

@ -64,6 +64,7 @@ scrypt = { version = "0.5.0", default-features = false }
lighthouse_metrics = { path = "../common/lighthouse_metrics" }
lazy_static = "1.4.0"
fallback = { path = "../common/fallback" }
itertools = "0.10.0"
monitoring_api = { path = "../common/monitoring_api" }
sensitive_url = { path = "../common/sensitive_url" }
task_executor = { path = "../common/task_executor" }

View File

@ -18,7 +18,7 @@ use types::{
};
/// Builds an `AttestationService`.
pub struct AttestationServiceBuilder<T, E: EthSpec> {
pub struct AttestationServiceBuilder<T: SlotClock + 'static, E: EthSpec> {
duties_service: Option<Arc<DutiesService<T, E>>>,
validator_store: Option<Arc<ValidatorStore<T, E>>>,
slot_clock: Option<T>,

View File

@ -6,6 +6,8 @@
//! The `DutiesService` is also responsible for sending events to the `BlockService` which trigger
//! block production.
mod sync;
use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced};
use crate::{
block_service::BlockServiceNotification,
@ -20,6 +22,8 @@ use slog::{debug, error, info, warn, Logger};
use slot_clock::SlotClock;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use sync::poll_sync_committee_duties;
use sync::SyncDutiesMap;
use tokio::{sync::mpsc::Sender, time::sleep};
use types::{ChainSpec, Epoch, EthSpec, Hash256, PublicKeyBytes, SelectionProof, Slot};
@ -40,6 +44,14 @@ pub enum Error {
FailedToDownloadAttesters(String),
FailedToProduceSelectionProof(ValidatorStoreError),
InvalidModulo(ArithError),
Arith(ArithError),
SyncDutiesNotFound(u64),
}
impl From<ArithError> for Error {
fn from(e: ArithError) -> Self {
Self::Arith(e)
}
}
/// Neatly joins the server-generated `AttesterData` with the locally-generated `selection_proof`.
@ -94,6 +106,8 @@ pub struct DutiesService<T, E: EthSpec> {
/// Maps an epoch to all *local* proposers in this epoch. Notably, this does not contain
/// proposals for any validators which are not registered locally.
pub proposers: RwLock<ProposerMap>,
/// Map from validator index to sync committee duties.
pub sync_duties: SyncDutiesMap,
/// Provides the canonical list of locally-managed validators.
pub validator_store: Arc<ValidatorStore<T, E>>,
/// Tracks the current slot.
@ -302,6 +316,37 @@ pub fn start_update_service<T: SlotClock + 'static, E: EthSpec>(
},
"duties_service_attesters",
);
// Spawn the task which keeps track of local sync committee duties.
let duties_service = core_duties_service.clone();
let log = core_duties_service.context.log().clone();
core_duties_service.context.executor.spawn(
async move {
loop {
if let Err(e) = poll_sync_committee_duties(&duties_service).await {
error!(
log,
"Failed to poll sync committee duties";
"error" => ?e
);
}
// Wait until the next slot before polling again.
// This doesn't mean that the beacon node will get polled every slot
// as the sync duties service will return early if it deems it already has
// enough information.
if let Some(duration) = duties_service.slot_clock.duration_to_next_slot() {
sleep(duration).await;
} else {
// Just sleep for one slot if we are unable to read the system clock, this gives
// us an opportunity for the clock to eventually come good.
sleep(duties_service.slot_clock.slot_duration()).await;
continue;
}
}
},
"duties_service_sync_committee",
);
}
/// Iterate through all the voting pubkeys in the `ValidatorStore` and attempt to learn any unknown

View File

@ -0,0 +1,599 @@
use crate::{
doppelganger_service::DoppelgangerStatus,
duties_service::{DutiesService, Error},
};
use itertools::Itertools;
use parking_lot::{MappedRwLockReadGuard, RwLock, RwLockReadGuard, RwLockWriteGuard};
use slog::{crit, debug, info, warn};
use slot_clock::SlotClock;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use types::{
ChainSpec, Epoch, EthSpec, PublicKeyBytes, Slot, SyncDuty, SyncSelectionProof, SyncSubnetId,
};
/// Number of epochs in advance to compute selection proofs.
pub const AGGREGATION_PRE_COMPUTE_EPOCHS: u64 = 2;
/// Top-level data-structure containing sync duty information.
///
/// This data is structured as a series of nested `HashMap`s wrapped in `RwLock`s. Fine-grained
/// locking is used to provide maximum concurrency for the different services reading and writing.
///
/// Deadlocks are prevented by:
///
/// 1. Hierarchical locking. It is impossible to lock an inner lock (e.g. `validators`) without
/// first locking its parent.
/// 2. One-at-a-time locking. For the innermost locks on the aggregator duties, all of the functions
/// in this file take care to only lock one validator at a time. We never hold a lock while
/// trying to obtain another one (hence no lock ordering issues).
pub struct SyncDutiesMap {
/// Map from sync committee period to duties for members of that sync committee.
committees: RwLock<HashMap<u64, CommitteeDuties>>,
}
/// Duties for a single sync committee period.
#[derive(Default)]
pub struct CommitteeDuties {
/// Map from validator index to validator duties.
///
/// A `None` value indicates that the validator index is known *not* to be a member of the sync
/// committee, while a `Some` indicates a known member. An absent value indicates that the
/// validator index was not part of the set of local validators when the duties were fetched.
/// This allows us to track changes to the set of local validators.
validators: RwLock<HashMap<u64, Option<ValidatorDuties>>>,
}
/// Duties for a single validator.
pub struct ValidatorDuties {
/// The sync duty: including validator sync committee indices & pubkey.
duty: SyncDuty,
/// The aggregator duties: cached selection proofs for upcoming epochs.
aggregation_duties: AggregatorDuties,
}
/// Aggregator duties for a single validator.
pub struct AggregatorDuties {
/// The epoch up to which aggregation proofs have already been computed (inclusive).
pre_compute_epoch: RwLock<Option<Epoch>>,
/// Map from slot & subnet ID to proof that this validator is an aggregator.
///
/// The slot is the slot at which the signed contribution and proof should be broadcast,
/// which is 1 less than the slot for which the `duty` was computed.
proofs: RwLock<HashMap<(Slot, SyncSubnetId), SyncSelectionProof>>,
}
/// Duties for multiple validators, for a single slot.
///
/// This type is returned to the sync service.
pub struct SlotDuties {
/// List of duties for all sync committee members at this slot.
///
/// Note: this is intentionally NOT split by subnet so that we only sign
/// one `SyncCommitteeMessage` per validator (recall a validator may be part of multiple
/// subnets).
pub duties: Vec<SyncDuty>,
/// Map from subnet ID to validator index and selection proof of each aggregator.
pub aggregators: HashMap<SyncSubnetId, Vec<(u64, PublicKeyBytes, SyncSelectionProof)>>,
}
impl Default for SyncDutiesMap {
fn default() -> Self {
Self {
committees: RwLock::new(HashMap::new()),
}
}
}
impl SyncDutiesMap {
/// Check if duties are already known for all of the given validators for `committee_period`.
fn all_duties_known(&self, committee_period: u64, validator_indices: &[u64]) -> bool {
self.committees
.read()
.get(&committee_period)
.map_or(false, |committee_duties| {
let validator_duties = committee_duties.validators.read();
validator_indices
.iter()
.all(|index| validator_duties.contains_key(index))
})
}
/// Prepare for pre-computation of selection proofs for `committee_period`.
///
/// Return the epoch up to which proofs should be pre-computed, as well as a vec of
/// `(previous_pre_compute_epoch, sync_duty)` pairs for all validators which need to have proofs
/// computed. See `fill_in_aggregation_proofs` for the actual calculation.
fn prepare_for_aggregator_pre_compute(
&self,
committee_period: u64,
current_epoch: Epoch,
spec: &ChainSpec,
) -> (Epoch, Vec<(Epoch, SyncDuty)>) {
let default_start_epoch =
std::cmp::max(current_epoch, first_epoch_of_period(committee_period, spec));
let pre_compute_epoch = std::cmp::min(
current_epoch + AGGREGATION_PRE_COMPUTE_EPOCHS,
last_epoch_of_period(committee_period, spec),
);
let pre_compute_duties = self.committees.read().get(&committee_period).map_or_else(
Vec::new,
|committee_duties| {
let validator_duties = committee_duties.validators.read();
validator_duties
.values()
.filter_map(|maybe_duty| {
let duty = maybe_duty.as_ref()?;
let old_pre_compute_epoch = duty
.aggregation_duties
.pre_compute_epoch
.write()
.replace(pre_compute_epoch);
match old_pre_compute_epoch {
// No proofs pre-computed previously, compute all from the start of
// the period or the current epoch (whichever is later).
None => Some((default_start_epoch, duty.duty.clone())),
// Proofs computed up to `prev`, start from the subsequent epoch.
Some(prev) if prev < pre_compute_epoch => {
Some((prev + 1, duty.duty.clone()))
}
// Proofs already known, no need to compute.
_ => None,
}
})
.collect()
},
);
(pre_compute_epoch, pre_compute_duties)
}
fn get_or_create_committee_duties<'a, 'b>(
&'a self,
committee_period: u64,
validator_indices: impl IntoIterator<Item = &'b u64>,
) -> MappedRwLockReadGuard<'a, CommitteeDuties> {
let mut committees_writer = self.committees.write();
committees_writer
.entry(committee_period)
.or_insert_with(CommitteeDuties::default)
.init(validator_indices);
// Return shared reference
RwLockReadGuard::map(
RwLockWriteGuard::downgrade(committees_writer),
|committees_reader| &committees_reader[&committee_period],
)
}
/// Get duties for all validators for the given `wall_clock_slot`.
///
/// This is the entry-point for the sync committee service.
pub fn get_duties_for_slot<E: EthSpec>(
&self,
wall_clock_slot: Slot,
spec: &ChainSpec,
) -> Option<SlotDuties> {
// Sync duties lag their assigned slot by 1
let duty_slot = wall_clock_slot + 1;
let sync_committee_period = duty_slot
.epoch(E::slots_per_epoch())
.sync_committee_period(spec)
.ok()?;
let committees_reader = self.committees.read();
let committee_duties = committees_reader.get(&sync_committee_period)?;
let mut duties = vec![];
let mut aggregators = HashMap::new();
committee_duties
.validators
.read()
.values()
// Filter out non-members & failed subnet IDs.
.filter_map(|opt_duties| {
let duty = opt_duties.as_ref()?;
let subnet_ids = duty.duty.subnet_ids::<E>().ok()?;
Some((duty, subnet_ids))
})
// Add duties for members to the vec of all duties, and aggregators to the
// aggregators map.
.for_each(|(validator_duty, subnet_ids)| {
duties.push(validator_duty.duty.clone());
let proofs = validator_duty.aggregation_duties.proofs.read();
for subnet_id in subnet_ids {
if let Some(proof) = proofs.get(&(wall_clock_slot, subnet_id)) {
aggregators.entry(subnet_id).or_insert_with(Vec::new).push((
validator_duty.duty.validator_index,
validator_duty.duty.pubkey,
proof.clone(),
));
}
}
});
Some(SlotDuties {
duties,
aggregators,
})
}
/// Prune duties for past sync committee periods from the map.
fn prune(&self, current_sync_committee_period: u64) {
self.committees
.write()
.retain(|period, _| *period >= current_sync_committee_period)
}
}
impl CommitteeDuties {
fn init<'b>(&mut self, validator_indices: impl IntoIterator<Item = &'b u64>) {
validator_indices.into_iter().for_each(|validator_index| {
self.validators
.get_mut()
.entry(*validator_index)
.or_insert(None);
})
}
}
impl ValidatorDuties {
fn new(duty: SyncDuty) -> Self {
Self {
duty,
aggregation_duties: AggregatorDuties {
pre_compute_epoch: RwLock::new(None),
proofs: RwLock::new(HashMap::new()),
},
}
}
}
/// Number of epochs to wait from the start of the period before actually fetching duties.
fn epoch_offset(spec: &ChainSpec) -> u64 {
spec.epochs_per_sync_committee_period.as_u64() / 2
}
fn first_epoch_of_period(sync_committee_period: u64, spec: &ChainSpec) -> Epoch {
spec.epochs_per_sync_committee_period * sync_committee_period
}
fn last_epoch_of_period(sync_committee_period: u64, spec: &ChainSpec) -> Epoch {
first_epoch_of_period(sync_committee_period + 1, spec) - 1
}
pub async fn poll_sync_committee_duties<T: SlotClock + 'static, E: EthSpec>(
duties_service: &Arc<DutiesService<T, E>>,
) -> Result<(), Error> {
let sync_duties = &duties_service.sync_duties;
let spec = &duties_service.spec;
let current_epoch = duties_service
.slot_clock
.now()
.ok_or(Error::UnableToReadSlotClock)?
.epoch(E::slots_per_epoch());
// If the Altair fork is yet to be activated, do not attempt to poll for duties.
if spec
.altair_fork_epoch
.map_or(true, |altair_epoch| current_epoch < altair_epoch)
{
return Ok(());
}
let current_sync_committee_period = current_epoch.sync_committee_period(spec)?;
let next_sync_committee_period = current_sync_committee_period + 1;
// Collect *all* pubkeys, even those undergoing doppelganger protection.
//
// Sync committee messages are not slashable and are currently excluded from doppelganger
// protection.
let local_pubkeys: HashSet<_> = duties_service
.validator_store
.voting_pubkeys(DoppelgangerStatus::ignored);
let local_indices = {
let mut local_indices = Vec::with_capacity(local_pubkeys.len());
let vals_ref = duties_service.validator_store.initialized_validators();
let vals = vals_ref.read();
for &pubkey in &local_pubkeys {
if let Some(validator_index) = vals.get_index(&pubkey) {
local_indices.push(validator_index)
}
}
local_indices
};
// If duties aren't known for the current period, poll for them.
if !sync_duties.all_duties_known(current_sync_committee_period, &local_indices) {
poll_sync_committee_duties_for_period(
duties_service,
&local_indices,
current_sync_committee_period,
)
.await?;
// Prune previous duties (we avoid doing this too often as it locks the whole map).
sync_duties.prune(current_sync_committee_period);
}
// Pre-compute aggregator selection proofs for the current period.
let (current_pre_compute_epoch, new_pre_compute_duties) = sync_duties
.prepare_for_aggregator_pre_compute(current_sync_committee_period, current_epoch, spec);
if !new_pre_compute_duties.is_empty() {
let sub_duties_service = duties_service.clone();
duties_service.context.executor.spawn_blocking(
move || {
fill_in_aggregation_proofs(
sub_duties_service,
&new_pre_compute_duties,
current_sync_committee_period,
current_epoch,
current_pre_compute_epoch,
)
},
"duties_service_sync_selection_proofs",
);
}
// If we're past the point in the current period where we should determine duties for the next
// period and they are not yet known, then poll.
if current_epoch.as_u64() % spec.epochs_per_sync_committee_period.as_u64() >= epoch_offset(spec)
&& !sync_duties.all_duties_known(next_sync_committee_period, &local_indices)
{
poll_sync_committee_duties_for_period(
duties_service,
&local_indices,
next_sync_committee_period,
)
.await?;
// Prune (this is the main code path for updating duties, so we should almost always hit
// this prune).
sync_duties.prune(current_sync_committee_period);
}
// Pre-compute aggregator selection proofs for the next period.
if (current_epoch + AGGREGATION_PRE_COMPUTE_EPOCHS).sync_committee_period(spec)?
== next_sync_committee_period
{
let (pre_compute_epoch, new_pre_compute_duties) = sync_duties
.prepare_for_aggregator_pre_compute(next_sync_committee_period, current_epoch, spec);
if !new_pre_compute_duties.is_empty() {
let sub_duties_service = duties_service.clone();
duties_service.context.executor.spawn_blocking(
move || {
fill_in_aggregation_proofs(
sub_duties_service,
&new_pre_compute_duties,
next_sync_committee_period,
current_epoch,
pre_compute_epoch,
)
},
"duties_service_sync_selection_proofs",
);
}
}
Ok(())
}
pub async fn poll_sync_committee_duties_for_period<T: SlotClock + 'static, E: EthSpec>(
duties_service: &Arc<DutiesService<T, E>>,
local_indices: &[u64],
sync_committee_period: u64,
) -> Result<(), Error> {
let spec = &duties_service.spec;
let log = duties_service.context.log();
debug!(
log,
"Fetching sync committee duties";
"sync_committee_period" => sync_committee_period,
"num_validators" => local_indices.len(),
);
let period_start_epoch = spec.epochs_per_sync_committee_period * sync_committee_period;
let duties_response = duties_service
.beacon_nodes
.first_success(duties_service.require_synced, |beacon_node| async move {
beacon_node
.post_validator_duties_sync(period_start_epoch, local_indices)
.await
})
.await;
let duties = match duties_response {
Ok(res) => res.data,
Err(e) => {
warn!(
log,
"Failed to download sync committee duties";
"sync_committee_period" => sync_committee_period,
"error" => %e,
);
return Ok(());
}
};
debug!(log, "Fetched sync duties from BN"; "count" => duties.len());
// Add duties to map.
let committee_duties = duties_service
.sync_duties
.get_or_create_committee_duties(sync_committee_period, local_indices);
let mut validator_writer = committee_duties.validators.write();
for duty in duties {
let validator_duties = validator_writer
.get_mut(&duty.validator_index)
.ok_or(Error::SyncDutiesNotFound(duty.validator_index))?;
let updated = validator_duties.as_ref().map_or(true, |existing_duties| {
let updated_due_to_reorg = existing_duties.duty.validator_sync_committee_indices
!= duty.validator_sync_committee_indices;
if updated_due_to_reorg {
warn!(
log,
"Sync committee duties changed";
"message" => "this could be due to a really long re-org, or a bug"
);
}
updated_due_to_reorg
});
if updated {
info!(
log,
"Validator in sync committee";
"validator_index" => duty.validator_index,
"sync_committee_period" => sync_committee_period,
);
*validator_duties = Some(ValidatorDuties::new(duty));
}
}
Ok(())
}
pub fn fill_in_aggregation_proofs<T: SlotClock + 'static, E: EthSpec>(
duties_service: Arc<DutiesService<T, E>>,
pre_compute_duties: &[(Epoch, SyncDuty)],
sync_committee_period: u64,
current_epoch: Epoch,
pre_compute_epoch: Epoch,
) {
let log = duties_service.context.log();
debug!(
log,
"Calculating sync selection proofs";
"period" => sync_committee_period,
"current_epoch" => current_epoch,
"pre_compute_epoch" => pre_compute_epoch
);
// Generate selection proofs for each validator at each slot, one epoch at a time.
for epoch in (current_epoch.as_u64()..=pre_compute_epoch.as_u64()).map(Epoch::new) {
// Generate proofs.
let validator_proofs: Vec<(u64, Vec<_>)> = pre_compute_duties
.iter()
.filter_map(|(validator_start_epoch, duty)| {
// Proofs are already known at this epoch for this validator.
if epoch < *validator_start_epoch {
return None;
}
let subnet_ids = duty
.subnet_ids::<E>()
.map_err(|e| {
crit!(
log,
"Arithmetic error computing subnet IDs";
"error" => ?e,
);
})
.ok()?;
let proofs = epoch
.slot_iter(E::slots_per_epoch())
.cartesian_product(&subnet_ids)
.filter_map(|(duty_slot, &subnet_id)| {
// Construct proof for prior slot.
let slot = duty_slot - 1;
let proof = duties_service
.validator_store
.produce_sync_selection_proof(&duty.pubkey, slot, subnet_id)
.map_err(|_| {
warn!(
log,
"Pubkey missing when signing selection proof";
"pubkey" => ?duty.pubkey,
"slot" => slot,
);
})
.ok()?;
let is_aggregator = proof
.is_aggregator::<E>()
.map_err(|e| {
warn!(
log,
"Error determining is_aggregator";
"pubkey" => ?duty.pubkey,
"slot" => slot,
"error" => ?e,
);
})
.ok()?;
if is_aggregator {
debug!(
log,
"Validator is sync aggregator";
"validator_index" => duty.validator_index,
"slot" => slot,
"subnet_id" => %subnet_id,
);
Some(((slot, subnet_id), proof))
} else {
None
}
})
.collect();
Some((duty.validator_index, proofs))
})
.collect();
// Add to global storage (we add regularly so the proofs can be used ASAP).
let sync_map = duties_service.sync_duties.committees.read();
let committee_duties = if let Some(duties) = sync_map.get(&sync_committee_period) {
duties
} else {
debug!(
log,
"Missing sync duties";
"period" => sync_committee_period,
);
continue;
};
let validators = committee_duties.validators.read();
let num_validators_updated = validator_proofs.len();
for (validator_index, proofs) in validator_proofs {
if let Some(Some(duty)) = validators.get(&validator_index) {
duty.aggregation_duties.proofs.write().extend(proofs);
} else {
debug!(
log,
"Missing sync duty to update";
"validator_index" => validator_index,
"period" => sync_committee_period,
);
}
}
if num_validators_updated > 0 {
debug!(
log,
"Finished computing sync selection proofs";
"epoch" => epoch,
"updated_validators" => num_validators_updated,
);
}
}
}

View File

@ -1,222 +0,0 @@
use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced};
use crate::http_metrics::metrics;
use environment::RuntimeContext;
use eth2::types::StateId;
use parking_lot::RwLock;
use slog::{debug, trace};
use slog::{error, Logger};
use slot_clock::SlotClock;
use std::ops::Deref;
use std::sync::Arc;
use tokio::time::{sleep, Duration};
use types::{EthSpec, Fork};
/// Delay this period of time after the slot starts. This allows the node to process the new slot.
const TIME_DELAY_FROM_SLOT: Duration = Duration::from_millis(80);
/// Builds a `ForkService`.
pub struct ForkServiceBuilder<T, E: EthSpec> {
fork: Option<Fork>,
slot_clock: Option<T>,
beacon_nodes: Option<Arc<BeaconNodeFallback<T, E>>>,
log: Option<Logger>,
}
impl<T: SlotClock + 'static, E: EthSpec> ForkServiceBuilder<T, E> {
pub fn new() -> Self {
Self {
fork: None,
slot_clock: None,
beacon_nodes: None,
log: None,
}
}
pub fn fork(mut self, fork: Fork) -> Self {
self.fork = Some(fork);
self
}
pub fn slot_clock(mut self, slot_clock: T) -> Self {
self.slot_clock = Some(slot_clock);
self
}
pub fn beacon_nodes(mut self, beacon_nodes: Arc<BeaconNodeFallback<T, E>>) -> Self {
self.beacon_nodes = Some(beacon_nodes);
self
}
pub fn log(mut self, log: Logger) -> Self {
self.log = Some(log);
self
}
pub fn build(self) -> Result<ForkService<T, E>, String> {
Ok(ForkService {
inner: Arc::new(Inner {
fork: RwLock::new(self.fork.ok_or("Cannot build ForkService without fork")?),
slot_clock: self
.slot_clock
.ok_or("Cannot build ForkService without slot_clock")?,
beacon_nodes: self
.beacon_nodes
.ok_or("Cannot build ForkService without beacon_node")?,
log: self
.log
.ok_or("Cannot build ForkService without logger")?
.clone(),
}),
})
}
}
#[cfg(test)]
#[allow(dead_code)]
impl<E: EthSpec> ForkServiceBuilder<slot_clock::TestingSlotClock, E> {
pub fn testing_only(spec: types::ChainSpec, log: Logger) -> Self {
use crate::beacon_node_fallback::CandidateBeaconNode;
let slot_clock = slot_clock::TestingSlotClock::new(
types::Slot::new(0),
std::time::Duration::from_secs(42),
std::time::Duration::from_secs(42),
);
let candidates = vec![CandidateBeaconNode::new(eth2::BeaconNodeHttpClient::new(
sensitive_url::SensitiveUrl::parse("http://127.0.0.1").unwrap(),
eth2::Timeouts::set_all(Duration::from_secs(12)),
))];
let mut beacon_nodes = BeaconNodeFallback::new(candidates, spec, log.clone());
beacon_nodes.set_slot_clock(slot_clock);
Self {
fork: Some(types::Fork::default()),
slot_clock: Some(slot_clock::TestingSlotClock::new(
types::Slot::new(0),
std::time::Duration::from_secs(42),
std::time::Duration::from_secs(42),
)),
beacon_nodes: Some(Arc::new(beacon_nodes)),
log: Some(log),
}
}
}
/// Helper to minimise `Arc` usage.
pub struct Inner<T, E: EthSpec> {
fork: RwLock<Fork>,
beacon_nodes: Arc<BeaconNodeFallback<T, E>>,
log: Logger,
slot_clock: T,
}
/// Attempts to download the `Fork` struct from the beacon node at the start of each epoch.
pub struct ForkService<T, E: EthSpec> {
inner: Arc<Inner<T, E>>,
}
impl<T, E: EthSpec> Clone for ForkService<T, E> {
fn clone(&self) -> Self {
Self {
inner: self.inner.clone(),
}
}
}
impl<T, E: EthSpec> Deref for ForkService<T, E> {
type Target = Inner<T, E>;
fn deref(&self) -> &Self::Target {
self.inner.deref()
}
}
impl<T: SlotClock + 'static, E: EthSpec> ForkService<T, E> {
/// Returns the last fork downloaded from the beacon node, if any.
pub fn fork(&self) -> Fork {
*self.fork.read()
}
/// Returns the slot clock.
pub fn slot_clock(&self) -> T {
self.slot_clock.clone()
}
/// Starts the service that periodically polls for the `Fork`.
pub fn start_update_service(self, context: &RuntimeContext<E>) -> Result<(), String> {
// Run an immediate update before starting the updater service.
context
.executor
.spawn_ignoring_error(self.clone().do_update(), "fork service update");
let executor = context.executor.clone();
let log = context.log().clone();
let spec = E::default_spec();
let interval_fut = async move {
loop {
// Run this poll before the wait, this should hopefully download the fork before the
// other services need them.
self.clone().do_update().await.ok();
if let Some(duration_to_next_epoch) =
self.slot_clock.duration_to_next_epoch(E::slots_per_epoch())
{
sleep(duration_to_next_epoch + TIME_DELAY_FROM_SLOT).await;
} else {
error!(log, "Failed to read slot clock");
// If we can't read the slot clock, just wait another slot.
sleep(Duration::from_secs(spec.seconds_per_slot)).await;
continue;
}
}
};
executor.spawn(interval_fut, "fork_service");
Ok(())
}
/// Attempts to download the `Fork` from the server.
async fn do_update(self) -> Result<(), ()> {
let _timer =
metrics::start_timer_vec(&metrics::FORK_SERVICE_TIMES, &[metrics::FULL_UPDATE]);
let log = &self.log;
let fork = self
.inner
.beacon_nodes
.first_success(RequireSynced::No, |beacon_node| async move {
beacon_node
.get_beacon_states_fork(StateId::Head)
.await
.map_err(|e| {
trace!(
log,
"Fork update failed";
"error" => format!("Error retrieving fork: {:?}", e)
)
})?
.ok_or_else(|| {
trace!(
log,
"Fork update failed";
"error" => "The beacon head fork is unknown"
)
})
.map(|result| result.data)
})
.await
.map_err(|_| ())?;
if *(self.fork.read()) != fork {
*(self.fork.write()) = fork;
}
debug!(self.log, "Fork update success");
// Returning an error will stop the interval. This is not desired, a single failure
// should not stop all future attempts.
Ok(())
}
}

View File

@ -4,7 +4,7 @@
use crate::doppelganger_service::DoppelgangerService;
use crate::{
http_api::{ApiSecret, Config as HttpConfig, Context},
Config, ForkServiceBuilder, InitializedValidators, ValidatorDefinitions, ValidatorStore,
Config, InitializedValidators, ValidatorDefinitions, ValidatorStore,
};
use account_utils::{
eth2_wallet::WalletBuilder, mnemonic_from_phrase, random_mnemonic, random_password,
@ -17,10 +17,11 @@ use eth2_keystore::KeystoreBuilder;
use parking_lot::RwLock;
use sensitive_url::SensitiveUrl;
use slashing_protection::{SlashingDatabase, SLASHING_PROTECTION_FILENAME};
use slot_clock::TestingSlotClock;
use slot_clock::{SlotClock, TestingSlotClock};
use std::marker::PhantomData;
use std::net::Ipv4Addr;
use std::sync::Arc;
use std::time::Duration;
use tempfile::{tempdir, TempDir};
use tokio::runtime::Runtime;
use tokio::sync::oneshot;
@ -73,20 +74,19 @@ impl ApiTester {
let spec = E::default_spec();
let fork_service = ForkServiceBuilder::testing_only(spec.clone(), log.clone())
.build()
.unwrap();
let slashing_db_path = config.validator_dir.join(SLASHING_PROTECTION_FILENAME);
let slashing_protection = SlashingDatabase::open_or_create(&slashing_db_path).unwrap();
let validator_store: ValidatorStore<TestingSlotClock, E> = ValidatorStore::new(
let slot_clock =
TestingSlotClock::new(Slot::new(0), Duration::from_secs(0), Duration::from_secs(1));
let validator_store = ValidatorStore::<_, E>::new(
initialized_validators,
slashing_protection,
Hash256::repeat_byte(42),
spec,
fork_service.clone(),
Some(Arc::new(DoppelgangerService::new(log.clone()))),
slot_clock,
log.clone(),
);
@ -96,7 +96,7 @@ impl ApiTester {
let initialized_validators = validator_store.initialized_validators();
let context: Arc<Context<TestingSlotClock, E>> = Arc::new(Context {
let context = Arc::new(Context {
runtime,
api_secret,
validator_dir: Some(validator_dir.path().into()),

View File

@ -67,16 +67,26 @@ lazy_static::lazy_static! {
"Total count of attempted SelectionProof signings",
&["status"]
);
pub static ref SIGNED_SYNC_COMMITTEE_MESSAGES_TOTAL: Result<IntCounterVec> = try_create_int_counter_vec(
"vc_signed_sync_committee_messages_total",
"Total count of attempted SyncCommitteeMessage signings",
&["status"]
);
pub static ref SIGNED_SYNC_COMMITTEE_CONTRIBUTIONS_TOTAL: Result<IntCounterVec> = try_create_int_counter_vec(
"vc_signed_sync_committee_contributions_total",
"Total count of attempted ContributionAndProof signings",
&["status"]
);
pub static ref SIGNED_SYNC_SELECTION_PROOFS_TOTAL: Result<IntCounterVec> = try_create_int_counter_vec(
"vc_signed_sync_selection_proofs_total",
"Total count of attempted SyncSelectionProof signings",
&["status"]
);
pub static ref DUTIES_SERVICE_TIMES: Result<HistogramVec> = try_create_histogram_vec(
"vc_duties_service_task_times_seconds",
"Duration to perform duties service tasks",
&["task"]
);
pub static ref FORK_SERVICE_TIMES: Result<HistogramVec> = try_create_histogram_vec(
"vc_fork_service_task_times_seconds",
"Duration to perform fork service tasks",
&["task"]
);
pub static ref ATTESTATION_SERVICE_TIMES: Result<HistogramVec> = try_create_histogram_vec(
"vc_attestation_service_task_times_seconds",
"Duration to perform attestation service tasks",

View File

@ -5,12 +5,12 @@ mod check_synced;
mod cli;
mod config;
mod duties_service;
mod fork_service;
mod graffiti_file;
mod http_metrics;
mod initialized_validators;
mod key_cache;
mod notifier;
mod sync_committee_service;
mod validator_store;
mod doppelganger_service;
@ -31,9 +31,7 @@ use block_service::{BlockService, BlockServiceBuilder};
use clap::ArgMatches;
use duties_service::DutiesService;
use environment::RuntimeContext;
use eth2::types::StateId;
use eth2::{reqwest::ClientBuilder, BeaconNodeHttpClient, StatusCode, Timeouts};
use fork_service::{ForkService, ForkServiceBuilder};
use http_api::ApiSecret;
use initialized_validators::InitializedValidators;
use notifier::spawn_notifier;
@ -46,11 +44,12 @@ use std::marker::PhantomData;
use std::net::SocketAddr;
use std::sync::Arc;
use std::time::{SystemTime, UNIX_EPOCH};
use sync_committee_service::SyncCommitteeService;
use tokio::{
sync::mpsc,
time::{sleep, Duration},
};
use types::{EthSpec, Fork, Hash256};
use types::{EthSpec, Hash256};
use validator_store::ValidatorStore;
/// The interval between attempts to contact the beacon node during startup.
@ -66,6 +65,7 @@ const HTTP_ATTESTER_DUTIES_TIMEOUT_QUOTIENT: u32 = 4;
const HTTP_LIVENESS_TIMEOUT_QUOTIENT: u32 = 4;
const HTTP_PROPOSAL_TIMEOUT_QUOTIENT: u32 = 2;
const HTTP_PROPOSER_DUTIES_TIMEOUT_QUOTIENT: u32 = 4;
const HTTP_SYNC_DUTIES_TIMEOUT_QUOTIENT: u32 = 4;
const DOPPELGANGER_SERVICE_NAME: &str = "doppelganger";
@ -73,9 +73,9 @@ const DOPPELGANGER_SERVICE_NAME: &str = "doppelganger";
pub struct ProductionValidatorClient<T: EthSpec> {
context: RuntimeContext<T>,
duties_service: Arc<DutiesService<SystemTimeSlotClock, T>>,
fork_service: ForkService<SystemTimeSlotClock, T>,
block_service: BlockService<SystemTimeSlotClock, T>,
attestation_service: AttestationService<SystemTimeSlotClock, T>,
sync_committee_service: SyncCommitteeService<SystemTimeSlotClock, T>,
doppelganger_service: Option<Arc<DoppelgangerService>>,
validator_store: Arc<ValidatorStore<SystemTimeSlotClock, T>>,
http_api_listen_addr: Option<SocketAddr>,
@ -263,6 +263,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
liveness: slot_duration / HTTP_LIVENESS_TIMEOUT_QUOTIENT,
proposal: slot_duration / HTTP_PROPOSAL_TIMEOUT_QUOTIENT,
proposer_duties: slot_duration / HTTP_PROPOSER_DUTIES_TIMEOUT_QUOTIENT,
sync_duties: slot_duration / HTTP_SYNC_DUTIES_TIMEOUT_QUOTIENT,
}
} else {
Timeouts::set_all(slot_duration)
@ -293,7 +294,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
BeaconNodeFallback::new(candidates, context.eth2_config.spec.clone(), log.clone());
// Perform some potentially long-running initialization tasks.
let (genesis_time, genesis_validators_root, fork) = tokio::select! {
let (genesis_time, genesis_validators_root) = tokio::select! {
tuple = init_from_beacon_node(&beacon_nodes, &context) => tuple?,
() = context.executor.exit() => return Err("Shutting down".to_string())
};
@ -313,13 +314,6 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
let beacon_nodes = Arc::new(beacon_nodes);
start_fallback_updater_service(context.clone(), beacon_nodes.clone())?;
let fork_service = ForkServiceBuilder::new()
.fork(fork)
.slot_clock(slot_clock.clone())
.beacon_nodes(beacon_nodes.clone())
.log(log.clone())
.build()?;
let doppelganger_service = if config.enable_doppelganger_protection {
Some(Arc::new(DoppelgangerService::new(
context
@ -331,16 +325,15 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
None
};
let validator_store: Arc<ValidatorStore<SystemTimeSlotClock, T>> =
Arc::new(ValidatorStore::new(
validators,
slashing_protection,
genesis_validators_root,
context.eth2_config.spec.clone(),
fork_service.clone(),
doppelganger_service.clone(),
log.clone(),
));
let validator_store = Arc::new(ValidatorStore::new(
validators,
slashing_protection,
genesis_validators_root,
context.eth2_config.spec.clone(),
doppelganger_service.clone(),
slot_clock.clone(),
log.clone(),
));
info!(
log,
@ -359,6 +352,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
let duties_service = Arc::new(DutiesService {
attesters: <_>::default(),
proposers: <_>::default(),
sync_duties: <_>::default(),
slot_clock: slot_clock.clone(),
beacon_nodes: beacon_nodes.clone(),
validator_store: validator_store.clone(),
@ -394,6 +388,14 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
.runtime_context(context.service_context("attestation".into()))
.build()?;
let sync_committee_service = SyncCommitteeService::new(
duties_service.clone(),
validator_store.clone(),
slot_clock,
beacon_nodes.clone(),
context.service_context("sync_committee".into()),
);
// Wait until genesis has occured.
//
// It seems most sensible to move this into the `start_service` function, but I'm caution
@ -406,9 +408,9 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
Ok(Self {
context,
duties_service,
fork_service,
block_service,
attestation_service,
sync_committee_service,
doppelganger_service,
validator_store,
config,
@ -427,11 +429,6 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
duties_service::start_update_service(self.duties_service.clone(), block_service_tx);
self.fork_service
.clone()
.start_update_service(&self.context)
.map_err(|e| format!("Unable to start fork service: {}", e))?;
self.block_service
.clone()
.start_update_service(block_service_rx)
@ -442,6 +439,11 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
.start_update_service(&self.context.eth2_config.spec)
.map_err(|e| format!("Unable to start attestation service: {}", e))?;
self.sync_committee_service
.clone()
.start_update_service(&self.context.eth2_config.spec)
.map_err(|e| format!("Unable to start sync committee service: {}", e))?;
if let Some(doppelganger_service) = self.doppelganger_service.clone() {
DoppelgangerService::start_update_service(
doppelganger_service,
@ -461,7 +463,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
let api_secret = ApiSecret::create_or_open(&self.config.validator_dir)?;
self.http_api_listen_addr = if self.config.http_api.enabled {
let ctx: Arc<http_api::Context<SystemTimeSlotClock, T>> = Arc::new(http_api::Context {
let ctx = Arc::new(http_api::Context {
runtime: self.context.executor.runtime(),
api_secret,
validator_store: Some(self.validator_store.clone()),
@ -495,7 +497,7 @@ impl<T: EthSpec> ProductionValidatorClient<T> {
async fn init_from_beacon_node<E: EthSpec>(
beacon_nodes: &BeaconNodeFallback<SystemTimeSlotClock, E>,
context: &RuntimeContext<E>,
) -> Result<(u64, Hash256, Fork), String> {
) -> Result<(u64, Hash256), String> {
loop {
beacon_nodes.update_unready_candidates().await;
let num_available = beacon_nodes.num_available().await;
@ -554,33 +556,7 @@ async fn init_from_beacon_node<E: EthSpec>(
sleep(RETRY_DELAY).await;
};
let fork = loop {
match beacon_nodes
.first_success(RequireSynced::No, |node| async move {
node.get_beacon_states_fork(StateId::Head).await
})
.await
{
Ok(Some(fork)) => break fork.data,
Ok(None) => {
info!(
context.log(),
"Failed to get fork, state not found";
);
}
Err(errors) => {
error!(
context.log(),
"Failed to get fork";
"error" => %errors
);
}
}
sleep(RETRY_DELAY).await;
};
Ok((genesis.genesis_time, genesis.genesis_validators_root, fork))
Ok((genesis.genesis_time, genesis.genesis_validators_root))
}
async fn wait_for_genesis<E: EthSpec>(

View File

@ -0,0 +1,537 @@
use crate::beacon_node_fallback::{BeaconNodeFallback, RequireSynced};
use crate::{duties_service::DutiesService, validator_store::ValidatorStore};
use environment::RuntimeContext;
use eth2::types::BlockId;
use futures::future::FutureExt;
use slog::{crit, debug, error, info, trace, warn};
use slot_clock::SlotClock;
use std::collections::HashMap;
use std::ops::Deref;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use tokio::time::{sleep, sleep_until, Duration, Instant};
use types::{
ChainSpec, EthSpec, Hash256, PublicKeyBytes, Slot, SyncCommitteeSubscription,
SyncContributionData, SyncDuty, SyncSelectionProof, SyncSubnetId,
};
pub const SUBSCRIPTION_LOOKAHEAD_EPOCHS: u64 = 4;
pub struct SyncCommitteeService<T: SlotClock + 'static, E: EthSpec> {
inner: Arc<Inner<T, E>>,
}
impl<T: SlotClock + 'static, E: EthSpec> Clone for SyncCommitteeService<T, E> {
fn clone(&self) -> Self {
Self {
inner: self.inner.clone(),
}
}
}
impl<T: SlotClock + 'static, E: EthSpec> Deref for SyncCommitteeService<T, E> {
type Target = Inner<T, E>;
fn deref(&self) -> &Self::Target {
self.inner.deref()
}
}
pub struct Inner<T: SlotClock + 'static, E: EthSpec> {
duties_service: Arc<DutiesService<T, E>>,
validator_store: Arc<ValidatorStore<T, E>>,
slot_clock: T,
beacon_nodes: Arc<BeaconNodeFallback<T, E>>,
context: RuntimeContext<E>,
/// Boolean to track whether the service has posted subscriptions to the BN at least once.
///
/// This acts as a latch that fires once upon start-up, and then never again.
first_subscription_done: AtomicBool,
}
impl<T: SlotClock + 'static, E: EthSpec> SyncCommitteeService<T, E> {
pub fn new(
duties_service: Arc<DutiesService<T, E>>,
validator_store: Arc<ValidatorStore<T, E>>,
slot_clock: T,
beacon_nodes: Arc<BeaconNodeFallback<T, E>>,
context: RuntimeContext<E>,
) -> Self {
Self {
inner: Arc::new(Inner {
duties_service,
validator_store,
slot_clock,
beacon_nodes,
context,
first_subscription_done: AtomicBool::new(false),
}),
}
}
/// Check if the Altair fork has been activated and therefore sync duties should be performed.
///
/// Slot clock errors are mapped to `false`.
fn altair_fork_activated(&self) -> bool {
self.duties_service
.spec
.altair_fork_epoch
.and_then(|fork_epoch| {
let current_epoch = self.slot_clock.now()?.epoch(E::slots_per_epoch());
Some(current_epoch >= fork_epoch)
})
.unwrap_or(false)
}
pub fn start_update_service(self, spec: &ChainSpec) -> Result<(), String> {
let log = self.context.log().clone();
let slot_duration = Duration::from_secs(spec.seconds_per_slot);
let duration_to_next_slot = self
.slot_clock
.duration_to_next_slot()
.ok_or("Unable to determine duration to next slot")?;
info!(
log,
"Sync committee service started";
"next_update_millis" => duration_to_next_slot.as_millis()
);
let executor = self.context.executor.clone();
let interval_fut = async move {
loop {
if let Some(duration_to_next_slot) = self.slot_clock.duration_to_next_slot() {
// Wait for contribution broadcast interval 1/3 of the way through the slot.
let log = self.context.log();
sleep(duration_to_next_slot + slot_duration / 3).await;
// Do nothing if the Altair fork has not yet occurred.
if !self.altair_fork_activated() {
continue;
}
if let Err(e) = self.spawn_contribution_tasks(slot_duration).await {
crit!(
log,
"Failed to spawn sync contribution tasks";
"error" => e
)
} else {
trace!(
log,
"Spawned sync contribution tasks";
)
}
// Do subscriptions for future slots/epochs.
self.spawn_subscription_tasks();
} else {
error!(log, "Failed to read slot clock");
// If we can't read the slot clock, just wait another slot.
sleep(slot_duration).await;
}
}
};
executor.spawn(interval_fut, "sync_committee_service");
Ok(())
}
async fn spawn_contribution_tasks(&self, slot_duration: Duration) -> Result<(), String> {
let log = self.context.log().clone();
let slot = self.slot_clock.now().ok_or("Failed to read slot clock")?;
let duration_to_next_slot = self
.slot_clock
.duration_to_next_slot()
.ok_or("Unable to determine duration to next slot")?;
// If a validator needs to publish a sync aggregate, they must do so at 2/3
// through the slot. This delay triggers at this time
let aggregate_production_instant = Instant::now()
+ duration_to_next_slot
.checked_sub(slot_duration / 3)
.unwrap_or_else(|| Duration::from_secs(0));
let slot_duties = self
.duties_service
.sync_duties
.get_duties_for_slot::<E>(slot, &self.duties_service.spec)
.ok_or_else(|| format!("Error fetching duties for slot {}", slot))?;
if slot_duties.duties.is_empty() {
debug!(
log,
"No local validators in current sync committee";
"slot" => slot,
);
return Ok(());
}
// Fetch block root for `SyncCommitteeContribution`.
let block_root = self
.beacon_nodes
.first_success(RequireSynced::Yes, |beacon_node| async move {
beacon_node.get_beacon_blocks_root(BlockId::Head).await
})
.await
.map_err(|e| e.to_string())?
.ok_or_else(|| format!("No block root found for slot {}", slot))?
.data
.root;
// Spawn one task to publish all of the sync committee signatures.
let validator_duties = slot_duties.duties;
self.inner.context.executor.spawn(
self.clone()
.publish_sync_committee_signatures(slot, block_root, validator_duties)
.map(|_| ()),
"sync_committee_signature_publish",
);
let aggregators = slot_duties.aggregators;
self.inner.context.executor.spawn(
self.clone()
.publish_sync_committee_aggregates(
slot,
block_root,
aggregators,
aggregate_production_instant,
)
.map(|_| ()),
"sync_committee_aggregate_publish",
);
Ok(())
}
/// Publish sync committee signatures.
async fn publish_sync_committee_signatures(
self,
slot: Slot,
beacon_block_root: Hash256,
validator_duties: Vec<SyncDuty>,
) -> Result<(), ()> {
let log = self.context.log().clone();
let committee_signatures = validator_duties
.iter()
.filter_map(|duty| {
self.validator_store
.produce_sync_committee_signature(
slot,
beacon_block_root,
duty.validator_index,
&duty.pubkey,
)
.map_err(|e| {
crit!(
log,
"Failed to sign sync committee signature";
"validator_index" => duty.validator_index,
"slot" => slot,
"error" => ?e,
);
})
.ok()
})
.collect::<Vec<_>>();
let signatures_slice = &committee_signatures;
self.beacon_nodes
.first_success(RequireSynced::No, |beacon_node| async move {
beacon_node
.post_beacon_pool_sync_committee_signatures(signatures_slice)
.await
})
.await
.map_err(|e| {
error!(
log,
"Unable to publish sync committee messages";
"slot" => slot,
"error" => %e,
);
})?;
info!(
log,
"Successfully published sync committee messages";
"count" => committee_signatures.len(),
"head_block" => ?beacon_block_root,
"slot" => slot,
);
Ok(())
}
async fn publish_sync_committee_aggregates(
self,
slot: Slot,
beacon_block_root: Hash256,
aggregators: HashMap<SyncSubnetId, Vec<(u64, PublicKeyBytes, SyncSelectionProof)>>,
aggregate_instant: Instant,
) {
for (subnet_id, subnet_aggregators) in aggregators {
let service = self.clone();
self.inner.context.executor.spawn(
service
.publish_sync_committee_aggregate_for_subnet(
slot,
beacon_block_root,
subnet_id,
subnet_aggregators,
aggregate_instant,
)
.map(|_| ()),
"sync_committee_aggregate_publish_subnet",
);
}
}
async fn publish_sync_committee_aggregate_for_subnet(
self,
slot: Slot,
beacon_block_root: Hash256,
subnet_id: SyncSubnetId,
subnet_aggregators: Vec<(u64, PublicKeyBytes, SyncSelectionProof)>,
aggregate_instant: Instant,
) -> Result<(), ()> {
sleep_until(aggregate_instant).await;
let log = self.context.log();
let contribution = self
.beacon_nodes
.first_success(RequireSynced::No, |beacon_node| async move {
let sync_contribution_data = SyncContributionData {
slot,
beacon_block_root,
subcommittee_index: subnet_id.into(),
};
beacon_node
.get_validator_sync_committee_contribution::<E>(&sync_contribution_data)
.await
})
.await
.map_err(|e| {
crit!(
log,
"Failed to produce sync contribution";
"slot" => slot,
"beacon_block_root" => ?beacon_block_root,
"error" => %e,
)
})?
.ok_or_else(|| {
crit!(
log,
"No aggregate contribution found";
"slot" => slot,
"beacon_block_root" => ?beacon_block_root,
);
})?
.data;
// Make `SignedContributionAndProof`s
let signed_contributions = subnet_aggregators
.into_iter()
.filter_map(|(aggregator_index, aggregator_pk, selection_proof)| {
self.validator_store
.produce_signed_contribution_and_proof(
aggregator_index,
&aggregator_pk,
contribution.clone(),
selection_proof,
)
.map_err(|e| {
crit!(
log,
"Unable to sign sync committee contribution";
"slot" => slot,
"error" => ?e,
);
})
.ok()
})
.collect::<Vec<_>>();
// Publish to the beacon node.
let signed_contributions_slice = &signed_contributions;
self.beacon_nodes
.first_success(RequireSynced::No, |beacon_node| async move {
beacon_node
.post_validator_contribution_and_proofs(signed_contributions_slice)
.await
})
.await
.map_err(|e| {
error!(
log,
"Unable to publish signed contributions and proofs";
"slot" => slot,
"error" => %e,
);
})?;
info!(
log,
"Successfully published sync contributions";
"subnet" => %subnet_id,
"beacon_block_root" => %beacon_block_root,
"num_signers" => contribution.aggregation_bits.num_set_bits(),
"slot" => slot,
);
Ok(())
}
fn spawn_subscription_tasks(&self) {
let service = self.clone();
let log = self.context.log().clone();
self.inner.context.executor.spawn(
async move {
service.publish_subscriptions().await.unwrap_or_else(|e| {
error!(
log,
"Error publishing subscriptions";
"error" => ?e,
)
});
},
"sync_committee_subscription_publish",
);
}
async fn publish_subscriptions(self) -> Result<(), String> {
let log = self.context.log().clone();
let spec = &self.duties_service.spec;
let slot = self.slot_clock.now().ok_or("Failed to read slot clock")?;
let mut duty_slots = vec![];
let mut all_succeeded = true;
// At the start of every epoch during the current period, re-post the subscriptions
// to the beacon node. This covers the case where the BN has forgotten the subscriptions
// due to a restart, or where the VC has switched to a fallback BN.
let current_period = sync_period_of_slot::<E>(slot, spec)?;
if !self.first_subscription_done.load(Ordering::Relaxed)
|| slot.as_u64() % E::slots_per_epoch() == 0
{
duty_slots.push((slot, current_period));
}
// Near the end of the current period, push subscriptions for the next period to the
// beacon node. We aggressively push every slot in the lead-up, as this is the main way
// that we want to ensure that the BN is subscribed (well in advance).
let lookahead_slot = slot + SUBSCRIPTION_LOOKAHEAD_EPOCHS * E::slots_per_epoch();
let lookahead_period = sync_period_of_slot::<E>(lookahead_slot, spec)?;
if lookahead_period > current_period {
duty_slots.push((lookahead_slot, lookahead_period));
}
if duty_slots.is_empty() {
return Ok(());
}
// Collect subscriptions.
let mut subscriptions = vec![];
for (duty_slot, sync_committee_period) in duty_slots {
debug!(
log,
"Fetching subscription duties";
"duty_slot" => duty_slot,
"current_slot" => slot,
);
match self
.duties_service
.sync_duties
.get_duties_for_slot::<E>(duty_slot, spec)
{
Some(duties) => subscriptions.extend(subscriptions_from_sync_duties(
duties.duties,
sync_committee_period,
spec,
)),
None => {
warn!(
log,
"Missing duties for subscription";
"slot" => duty_slot,
);
all_succeeded = false;
}
}
}
// Post subscriptions to BN.
debug!(
log,
"Posting sync subscriptions to BN";
"count" => subscriptions.len(),
);
let subscriptions_slice = &subscriptions;
for subscription in subscriptions_slice {
debug!(
log,
"Subscription";
"validator_index" => subscription.validator_index,
"validator_sync_committee_indices" => ?subscription.sync_committee_indices,
"until_epoch" => subscription.until_epoch,
);
}
if let Err(e) = self
.beacon_nodes
.first_success(RequireSynced::No, |beacon_node| async move {
beacon_node
.post_validator_sync_committee_subscriptions(subscriptions_slice)
.await
})
.await
{
error!(
log,
"Unable to post sync committee subscriptions";
"slot" => slot,
"error" => %e,
);
all_succeeded = false;
}
// Disable first-subscription latch once all duties have succeeded once.
if all_succeeded {
self.first_subscription_done.store(true, Ordering::Relaxed);
}
Ok(())
}
}
fn sync_period_of_slot<E: EthSpec>(slot: Slot, spec: &ChainSpec) -> Result<u64, String> {
slot.epoch(E::slots_per_epoch())
.sync_committee_period(spec)
.map_err(|e| format!("Error computing sync period: {:?}", e))
}
fn subscriptions_from_sync_duties(
duties: Vec<SyncDuty>,
sync_committee_period: u64,
spec: &ChainSpec,
) -> impl Iterator<Item = SyncCommitteeSubscription> {
let until_epoch = spec.epochs_per_sync_committee_period * (sync_committee_period + 1);
duties
.into_iter()
.map(move |duty| SyncCommitteeSubscription {
validator_index: duty.validator_index,
sync_committee_indices: duty.validator_sync_committee_indices,
until_epoch,
})
}

View File

@ -1,5 +1,5 @@
use crate::{
doppelganger_service::DoppelgangerService, fork_service::ForkService, http_metrics::metrics,
doppelganger_service::DoppelgangerService, http_metrics::metrics,
initialized_validators::InitializedValidators,
};
use account_utils::{validator_definitions::ValidatorDefinition, ZeroizeString};
@ -8,12 +8,15 @@ use slashing_protection::{NotSafe, Safe, SlashingDatabase};
use slog::{crit, error, info, warn, Logger};
use slot_clock::SlotClock;
use std::iter::FromIterator;
use std::marker::PhantomData;
use std::path::Path;
use std::sync::Arc;
use types::{
attestation::Error as AttestationError, graffiti::GraffitiString, Attestation, BeaconBlock,
ChainSpec, Domain, Epoch, EthSpec, Fork, Graffiti, Hash256, Keypair, PublicKeyBytes,
SelectionProof, Signature, SignedAggregateAndProof, SignedBeaconBlock, SignedRoot, Slot,
SelectionProof, Signature, SignedAggregateAndProof, SignedBeaconBlock,
SignedContributionAndProof, SignedRoot, Slot, SyncCommitteeContribution, SyncCommitteeMessage,
SyncSelectionProof, SyncSubnetId,
};
use validator_dir::ValidatorDir;
@ -69,8 +72,8 @@ pub struct ValidatorStore<T, E: EthSpec> {
spec: Arc<ChainSpec>,
log: Logger,
doppelganger_service: Option<Arc<DoppelgangerService>>,
fork_service: ForkService<T, E>,
slot_clock: T,
_phantom: PhantomData<E>,
}
impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
@ -79,8 +82,8 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
slashing_protection: SlashingDatabase,
genesis_validators_root: Hash256,
spec: ChainSpec,
fork_service: ForkService<T, E>,
doppelganger_service: Option<Arc<DoppelgangerService>>,
slot_clock: T,
log: Logger,
) -> Self {
Self {
@ -89,10 +92,10 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
slashing_protection_last_prune: Arc::new(Mutex::new(Epoch::new(0))),
genesis_validators_root,
spec: Arc::new(spec),
log: log.clone(),
log,
doppelganger_service,
slot_clock: fork_service.slot_clock(),
fork_service,
slot_clock,
_phantom: PhantomData,
}
}
@ -253,8 +256,8 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
self.validators.read().num_enabled()
}
fn fork(&self) -> Fork {
self.fork_service.fork()
fn fork(&self, epoch: Epoch) -> Fork {
self.spec.fork_at_epoch(epoch)
}
/// Runs `func`, providing it access to the `Keypair` corresponding to `validator_pubkey`.
@ -301,7 +304,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
let domain = self.spec.get_domain(
epoch,
Domain::Randao,
&self.fork(),
&self.fork(epoch),
self.genesis_validators_root,
);
let message = epoch.signing_root(domain);
@ -334,7 +337,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
}
// Check for slashing conditions.
let fork = self.fork();
let fork = self.fork(block.epoch());
let domain = self.spec.get_domain(
block.epoch(),
Domain::BeaconProposer,
@ -403,7 +406,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
}
// Checking for slashing conditions.
let fork = self.fork();
let fork = self.fork(attestation.data.target.epoch);
let domain = self.spec.get_domain(
attestation.data.target.epoch,
@ -486,8 +489,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
aggregate: Attestation<E>,
selection_proof: SelectionProof,
) -> Result<SignedAggregateAndProof<E>, Error> {
// Take the fork early to avoid lock interleaving.
let fork = self.fork();
let fork = self.fork(aggregate.data.target.epoch);
let proof = self.with_validator_keypair(validator_pubkey, move |keypair| {
SignedAggregateAndProof::from_aggregate(
@ -513,9 +515,6 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
validator_pubkey: PublicKeyBytes,
slot: Slot,
) -> Result<SelectionProof, Error> {
// Take the fork early to avoid lock interleaving.
let fork = self.fork();
// Bypass the `with_validator_keypair` function.
//
// This is because we don't care about doppelganger protection when it comes to selection
@ -531,7 +530,7 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
let proof = SelectionProof::new::<E>(
slot,
&keypair.sk,
&fork,
&self.fork(slot.epoch(E::slots_per_epoch())),
self.genesis_validators_root,
&self.spec,
);
@ -541,6 +540,93 @@ impl<T: SlotClock + 'static, E: EthSpec> ValidatorStore<T, E> {
Ok(proof)
}
/// Produce a `SyncSelectionProof` for `slot` signed by the secret key of `validator_pubkey`.
pub fn produce_sync_selection_proof(
&self,
validator_pubkey: &PublicKeyBytes,
slot: Slot,
subnet_id: SyncSubnetId,
) -> Result<SyncSelectionProof, Error> {
// Bypass `with_validator_keypair`: sync committee messages are not slashable.
let validators = self.validators.read();
let voting_keypair = validators
.voting_keypair(validator_pubkey)
.ok_or(Error::UnknownPubkey(*validator_pubkey))?;
metrics::inc_counter_vec(
&metrics::SIGNED_SYNC_SELECTION_PROOFS_TOTAL,
&[metrics::SUCCESS],
);
Ok(SyncSelectionProof::new::<E>(
slot,
subnet_id.into(),
&voting_keypair.sk,
&self.fork(slot.epoch(E::slots_per_epoch())),
self.genesis_validators_root,
&self.spec,
))
}
pub fn produce_sync_committee_signature(
&self,
slot: Slot,
beacon_block_root: Hash256,
validator_index: u64,
validator_pubkey: &PublicKeyBytes,
) -> Result<SyncCommitteeMessage, Error> {
// Bypass `with_validator_keypair`: sync committee messages are not slashable.
let validators = self.validators.read();
let voting_keypair = validators
.voting_keypair(validator_pubkey)
.ok_or(Error::UnknownPubkey(*validator_pubkey))?;
metrics::inc_counter_vec(
&metrics::SIGNED_SYNC_COMMITTEE_MESSAGES_TOTAL,
&[metrics::SUCCESS],
);
Ok(SyncCommitteeMessage::new::<E>(
slot,
beacon_block_root,
validator_index,
&voting_keypair.sk,
&self.fork(slot.epoch(E::slots_per_epoch())),
self.genesis_validators_root,
&self.spec,
))
}
pub fn produce_signed_contribution_and_proof(
&self,
aggregator_index: u64,
aggregator_pubkey: &PublicKeyBytes,
contribution: SyncCommitteeContribution<E>,
selection_proof: SyncSelectionProof,
) -> Result<SignedContributionAndProof<E>, Error> {
// Bypass `with_validator_keypair`: sync committee messages are not slashable.
let validators = self.validators.read();
let voting_keypair = validators
.voting_keypair(aggregator_pubkey)
.ok_or(Error::UnknownPubkey(*aggregator_pubkey))?;
let fork = self.fork(contribution.slot.epoch(E::slots_per_epoch()));
metrics::inc_counter_vec(
&metrics::SIGNED_SYNC_COMMITTEE_CONTRIBUTIONS_TOTAL,
&[metrics::SUCCESS],
);
Ok(SignedContributionAndProof::from_aggregate(
aggregator_index,
contribution,
Some(selection_proof),
&voting_keypair.sk,
&fork,
self.genesis_validators_root,
&self.spec,
))
}
/// Prune the slashing protection database so that it remains performant.
///
/// This function will only do actual pruning periodically, so it should usually be