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:
parent
350b6f19de
commit
17a2c778e3
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -7312,6 +7312,7 @@ dependencies = [
|
||||
"futures",
|
||||
"hex",
|
||||
"hyper",
|
||||
"itertools 0.10.1",
|
||||
"lazy_static",
|
||||
"libc",
|
||||
"libsecp256k1",
|
||||
|
@ -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(
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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));
|
||||
|
@ -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"
|
||||
|
@ -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()))
|
||||
|
294
beacon_node/http_api/src/sync_committees.rs
Normal file
294
beacon_node/http_api/src/sync_committees.rs
Normal 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(())
|
||||
}
|
||||
}
|
28
beacon_node/http_api/src/version.rs
Normal file
28
beacon_node/http_api/src/version.rs
Normal 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))
|
||||
}
|
142
beacon_node/http_api/tests/common.rs
Normal file
142
beacon_node/http_api/tests/common.rs
Normal 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,
|
||||
}
|
||||
}
|
305
beacon_node/http_api/tests/fork_tests.rs
Normal file
305
beacon_node/http_api/tests/fork_tests.rs
Normal 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)
|
||||
);
|
||||
}
|
||||
}
|
6
beacon_node/http_api/tests/main.rs
Normal file
6
beacon_node/http_api/tests/main.rs
Normal 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;
|
@ -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
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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)
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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))
|
||||
}
|
||||
|
@ -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>,
|
||||
|
@ -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"
|
@ -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"]
|
||||
|
@ -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 {
|
||||
|
@ -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.
|
||||
|
@ -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)]
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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 {
|
||||
|
83
consensus/types/src/sync_duty.rs
Normal file
83
consensus/types/src/sync_duty.rs
Normal 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,
|
||||
)
|
||||
}
|
||||
}
|
@ -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")
|
||||
|
@ -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))
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
||||
|
@ -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 "
|
||||
|
@ -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" }
|
||||
|
@ -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>,
|
||||
|
@ -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
|
||||
|
599
validator_client/src/duties_service/sync.rs
Normal file
599
validator_client/src/duties_service/sync.rs
Normal 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,
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
@ -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(())
|
||||
}
|
||||
}
|
@ -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()),
|
||||
|
@ -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",
|
||||
|
@ -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>(
|
||||
|
537
validator_client/src/sync_committee_service.rs
Normal file
537
validator_client/src/sync_committee_service.rs
Normal 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,
|
||||
})
|
||||
}
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user