API for LightClientBootstrap, LightClientFinalityUpdate, LightClientOptimisticUpdate and light client events (#3954)

* rebase and add comment

* conditional test

* test

* optimistic chould be working now

* finality should be working now

* try again

* try again

* clippy fix

* add lc bootstrap beacon api

* add lc optimistic/finality update to events

* fmt

* That error isn't occuring on my computer but I think this should fix it

* Add missing test file

* Update light client types to comply with Altair light client spec.

* Fix test compilation

* Support deserializing light client structures for the Bellatrix fork

* Move `get_light_client_bootstrap` logic to `BeaconChain`. `LightClientBootstrap` API to return `ForkVersionedResponse`.

* Misc fixes.
- log cleanup
- move http_api config mutation to `config::get_config` for consistency
- fix light client API responses

* Add light client bootstrap API test and fix existing ones.

* Fix test for `light-client-server` http api config.

* Appease clippy

* Efficiency improvement when retrieving beacon state.

---------

Co-authored-by: Jimmy Chen <jchen.tc@gmail.com>
This commit is contained in:
GeemoCandama 2023-11-28 00:14:29 -06:00 committed by GitHub
parent 44c1817c2b
commit 8a599ec7dc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 632 additions and 111 deletions

View File

@ -6446,6 +6446,39 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub fn data_availability_boundary(&self) -> Option<Epoch> { pub fn data_availability_boundary(&self) -> Option<Epoch> {
self.data_availability_checker.data_availability_boundary() self.data_availability_checker.data_availability_boundary()
} }
/// Gets the `LightClientBootstrap` object for a requested block root.
///
/// Returns `None` when the state or block is not found in the database.
#[allow(clippy::type_complexity)]
pub fn get_light_client_bootstrap(
&self,
block_root: &Hash256,
) -> Result<Option<(LightClientBootstrap<T::EthSpec>, ForkName)>, Error> {
let Some((state_root, slot)) = self
.get_blinded_block(block_root)?
.map(|block| (block.state_root(), block.slot()))
else {
return Ok(None);
};
let Some(mut state) = self.get_state(&state_root, Some(slot))? else {
return Ok(None);
};
let fork_name = state
.fork_name(&self.spec)
.map_err(Error::InconsistentFork)?;
match fork_name {
ForkName::Altair | ForkName::Merge => {
LightClientBootstrap::from_beacon_state(&mut state)
.map(|bootstrap| Some((bootstrap, fork_name)))
.map_err(Error::LightClientError)
}
ForkName::Base | ForkName::Capella | ForkName::Deneb => Err(Error::UnsupportedFork),
}
}
} }
impl<T: BeaconChainTypes> Drop for BeaconChain<T> { impl<T: BeaconChainTypes> Drop for BeaconChain<T> {

View File

@ -221,6 +221,8 @@ pub enum BeaconChainError {
ProposerHeadForkChoiceError(fork_choice::Error<proto_array::Error>), ProposerHeadForkChoiceError(fork_choice::Error<proto_array::Error>),
UnableToPublish, UnableToPublish,
AvailabilityCheckError(AvailabilityCheckError), AvailabilityCheckError(AvailabilityCheckError),
LightClientError(LightClientError),
UnsupportedFork,
} }
easy_from_to!(SlotProcessingError, BeaconChainError); easy_from_to!(SlotProcessingError, BeaconChainError);

View File

@ -17,6 +17,8 @@ pub struct ServerSentEventHandler<T: EthSpec> {
contribution_tx: Sender<EventKind<T>>, contribution_tx: Sender<EventKind<T>>,
payload_attributes_tx: Sender<EventKind<T>>, payload_attributes_tx: Sender<EventKind<T>>,
late_head: Sender<EventKind<T>>, late_head: Sender<EventKind<T>>,
light_client_finality_update_tx: Sender<EventKind<T>>,
light_client_optimistic_update_tx: Sender<EventKind<T>>,
block_reward_tx: Sender<EventKind<T>>, block_reward_tx: Sender<EventKind<T>>,
log: Logger, log: Logger,
} }
@ -40,6 +42,8 @@ impl<T: EthSpec> ServerSentEventHandler<T> {
let (contribution_tx, _) = broadcast::channel(capacity); let (contribution_tx, _) = broadcast::channel(capacity);
let (payload_attributes_tx, _) = broadcast::channel(capacity); let (payload_attributes_tx, _) = broadcast::channel(capacity);
let (late_head, _) = broadcast::channel(capacity); let (late_head, _) = broadcast::channel(capacity);
let (light_client_finality_update_tx, _) = broadcast::channel(capacity);
let (light_client_optimistic_update_tx, _) = broadcast::channel(capacity);
let (block_reward_tx, _) = broadcast::channel(capacity); let (block_reward_tx, _) = broadcast::channel(capacity);
Self { Self {
@ -53,6 +57,8 @@ impl<T: EthSpec> ServerSentEventHandler<T> {
contribution_tx, contribution_tx,
payload_attributes_tx, payload_attributes_tx,
late_head, late_head,
light_client_finality_update_tx,
light_client_optimistic_update_tx,
block_reward_tx, block_reward_tx,
log, log,
} }
@ -108,6 +114,14 @@ impl<T: EthSpec> ServerSentEventHandler<T> {
.late_head .late_head
.send(kind) .send(kind)
.map(|count| log_count("late head", count)), .map(|count| log_count("late head", count)),
EventKind::LightClientFinalityUpdate(_) => self
.light_client_finality_update_tx
.send(kind)
.map(|count| log_count("light client finality update", count)),
EventKind::LightClientOptimisticUpdate(_) => self
.light_client_optimistic_update_tx
.send(kind)
.map(|count| log_count("light client optimistic update", count)),
EventKind::BlockReward(_) => self EventKind::BlockReward(_) => self
.block_reward_tx .block_reward_tx
.send(kind) .send(kind)
@ -158,6 +172,14 @@ impl<T: EthSpec> ServerSentEventHandler<T> {
self.late_head.subscribe() self.late_head.subscribe()
} }
pub fn subscribe_light_client_finality_update(&self) -> Receiver<EventKind<T>> {
self.light_client_finality_update_tx.subscribe()
}
pub fn subscribe_light_client_optimistic_update(&self) -> Receiver<EventKind<T>> {
self.light_client_optimistic_update_tx.subscribe()
}
pub fn subscribe_block_reward(&self) -> Receiver<EventKind<T>> { pub fn subscribe_block_reward(&self) -> Receiver<EventKind<T>> {
self.block_reward_tx.subscribe() self.block_reward_tx.subscribe()
} }

View File

@ -67,7 +67,7 @@ impl<T: BeaconChainTypes> VerifiedLightClientFinalityUpdate<T> {
chain: &BeaconChain<T>, chain: &BeaconChain<T>,
seen_timestamp: Duration, seen_timestamp: Duration,
) -> Result<Self, Error> { ) -> Result<Self, Error> {
let gossiped_finality_slot = light_client_finality_update.finalized_header.slot; let gossiped_finality_slot = light_client_finality_update.finalized_header.beacon.slot;
let one_third_slot_duration = Duration::new(chain.spec.seconds_per_slot / 3, 0); let one_third_slot_duration = Duration::new(chain.spec.seconds_per_slot / 3, 0);
let signature_slot = light_client_finality_update.signature_slot; let signature_slot = light_client_finality_update.signature_slot;
let start_time = chain.slot_clock.start_of(signature_slot); let start_time = chain.slot_clock.start_of(signature_slot);
@ -88,7 +88,7 @@ impl<T: BeaconChainTypes> VerifiedLightClientFinalityUpdate<T> {
.get_blinded_block(&finalized_block_root)? .get_blinded_block(&finalized_block_root)?
.ok_or(Error::FailedConstructingUpdate)?; .ok_or(Error::FailedConstructingUpdate)?;
let latest_seen_finality_update_slot = match latest_seen_finality_update.as_ref() { let latest_seen_finality_update_slot = match latest_seen_finality_update.as_ref() {
Some(update) => update.finalized_header.slot, Some(update) => update.finalized_header.beacon.slot,
None => Slot::new(0), None => Slot::new(0),
}; };

View File

@ -71,7 +71,7 @@ impl<T: BeaconChainTypes> VerifiedLightClientOptimisticUpdate<T> {
chain: &BeaconChain<T>, chain: &BeaconChain<T>,
seen_timestamp: Duration, seen_timestamp: Duration,
) -> Result<Self, Error> { ) -> Result<Self, Error> {
let gossiped_optimistic_slot = light_client_optimistic_update.attested_header.slot; let gossiped_optimistic_slot = light_client_optimistic_update.attested_header.beacon.slot;
let one_third_slot_duration = Duration::new(chain.spec.seconds_per_slot / 3, 0); let one_third_slot_duration = Duration::new(chain.spec.seconds_per_slot / 3, 0);
let signature_slot = light_client_optimistic_update.signature_slot; let signature_slot = light_client_optimistic_update.signature_slot;
let start_time = chain.slot_clock.start_of(signature_slot); let start_time = chain.slot_clock.start_of(signature_slot);
@ -88,7 +88,7 @@ impl<T: BeaconChainTypes> VerifiedLightClientOptimisticUpdate<T> {
.get_state(&attested_block.state_root(), Some(attested_block.slot()))? .get_state(&attested_block.state_root(), Some(attested_block.slot()))?
.ok_or(Error::FailedConstructingUpdate)?; .ok_or(Error::FailedConstructingUpdate)?;
let latest_seen_optimistic_update_slot = match latest_seen_optimistic_update.as_ref() { let latest_seen_optimistic_update_slot = match latest_seen_optimistic_update.as_ref() {
Some(update) => update.attested_header.slot, Some(update) => update.attested_header.beacon.slot,
None => Slot::new(0), None => Slot::new(0),
}; };
@ -114,6 +114,7 @@ impl<T: BeaconChainTypes> VerifiedLightClientOptimisticUpdate<T> {
// otherwise queue // otherwise queue
let canonical_root = light_client_optimistic_update let canonical_root = light_client_optimistic_update
.attested_header .attested_header
.beacon
.canonical_root(); .canonical_root();
if canonical_root != head_block.message().parent_root() { if canonical_root != head_block.message().parent_root() {

View File

@ -77,9 +77,10 @@ use tokio_stream::{
use types::{ use types::{
Attestation, AttestationData, AttestationShufflingId, AttesterSlashing, BeaconStateError, Attestation, AttestationData, AttestationShufflingId, AttesterSlashing, BeaconStateError,
BlindedPayload, CommitteeCache, ConfigAndPreset, Epoch, EthSpec, ForkName, BlindedPayload, CommitteeCache, ConfigAndPreset, Epoch, EthSpec, ForkName,
ProposerPreparationData, ProposerSlashing, RelativeEpoch, SignedAggregateAndProof, ForkVersionedResponse, Hash256, ProposerPreparationData, ProposerSlashing, RelativeEpoch,
SignedBlsToExecutionChange, SignedContributionAndProof, SignedValidatorRegistrationData, SignedAggregateAndProof, SignedBlsToExecutionChange, SignedContributionAndProof,
SignedVoluntaryExit, Slot, SyncCommitteeMessage, SyncContributionData, SignedValidatorRegistrationData, SignedVoluntaryExit, Slot, SyncCommitteeMessage,
SyncContributionData,
}; };
use validator::pubkey_to_validator_index; use validator::pubkey_to_validator_index;
use version::{ use version::{
@ -143,6 +144,7 @@ pub struct Config {
pub enable_beacon_processor: bool, pub enable_beacon_processor: bool,
#[serde(with = "eth2::types::serde_status_code")] #[serde(with = "eth2::types::serde_status_code")]
pub duplicate_block_status_code: StatusCode, pub duplicate_block_status_code: StatusCode,
pub enable_light_client_server: bool,
} }
impl Default for Config { impl Default for Config {
@ -159,6 +161,7 @@ impl Default for Config {
sse_capacity_multiplier: 1, sse_capacity_multiplier: 1,
enable_beacon_processor: true, enable_beacon_processor: true,
duplicate_block_status_code: StatusCode::ACCEPTED, duplicate_block_status_code: StatusCode::ACCEPTED,
enable_light_client_server: false,
} }
} }
} }
@ -279,6 +282,18 @@ pub fn prometheus_metrics() -> warp::filters::log::Log<impl Fn(warp::filters::lo
}) })
} }
fn enable(is_enabled: bool) -> impl Filter<Extract = (), Error = warp::Rejection> + Clone {
warp::any()
.and_then(move || async move {
if is_enabled {
Ok(())
} else {
Err(warp::reject::not_found())
}
})
.untuple_one()
}
/// Creates a server that will serve requests using information from `ctx`. /// Creates a server that will serve requests using information from `ctx`.
/// ///
/// The server will shut down gracefully when the `shutdown` future resolves. /// The server will shut down gracefully when the `shutdown` future resolves.
@ -2379,6 +2394,164 @@ pub fn serve<T: BeaconChainTypes>(
}, },
); );
/*
* beacon/light_client
*/
let beacon_light_client_path = eth_v1
.and(warp::path("beacon"))
.and(warp::path("light_client"))
.and(chain_filter.clone());
// GET beacon/light_client/bootstrap/{block_root}
let get_beacon_light_client_bootstrap = beacon_light_client_path
.clone()
.and(task_spawner_filter.clone())
.and(warp::path("bootstrap"))
.and(warp::path::param::<Hash256>().or_else(|_| async {
Err(warp_utils::reject::custom_bad_request(
"Invalid block root value".to_string(),
))
}))
.and(warp::path::end())
.and(warp::header::optional::<api_types::Accept>("accept"))
.then(
|chain: Arc<BeaconChain<T>>,
task_spawner: TaskSpawner<T::EthSpec>,
block_root: Hash256,
accept_header: Option<api_types::Accept>| {
task_spawner.blocking_response_task(Priority::P1, move || {
let (bootstrap, fork_name) = match chain.get_light_client_bootstrap(&block_root)
{
Ok(Some(res)) => res,
Ok(None) => {
return Err(warp_utils::reject::custom_not_found(
"Light client bootstrap unavailable".to_string(),
));
}
Err(e) => {
return Err(warp_utils::reject::custom_server_error(format!(
"Unable to obtain LightClientBootstrap instance: {e:?}"
)));
}
};
match accept_header {
Some(api_types::Accept::Ssz) => Response::builder()
.status(200)
.header("Content-Type", "application/octet-stream")
.body(bootstrap.as_ssz_bytes().into())
.map_err(|e| {
warp_utils::reject::custom_server_error(format!(
"failed to create response: {}",
e
))
}),
_ => Ok(warp::reply::json(&ForkVersionedResponse {
version: Some(fork_name),
data: bootstrap,
})
.into_response()),
}
.map(|resp| add_consensus_version_header(resp, fork_name))
})
},
);
// GET beacon/light_client/optimistic_update
let get_beacon_light_client_optimistic_update = beacon_light_client_path
.clone()
.and(task_spawner_filter.clone())
.and(warp::path("optimistic_update"))
.and(warp::path::end())
.and(warp::header::optional::<api_types::Accept>("accept"))
.then(
|chain: Arc<BeaconChain<T>>,
task_spawner: TaskSpawner<T::EthSpec>,
accept_header: Option<api_types::Accept>| {
task_spawner.blocking_response_task(Priority::P1, move || {
let update = chain
.latest_seen_optimistic_update
.lock()
.clone()
.ok_or_else(|| {
warp_utils::reject::custom_not_found(
"No LightClientOptimisticUpdate is available".to_string(),
)
})?;
let fork_name = chain
.spec
.fork_name_at_slot::<T::EthSpec>(update.signature_slot);
match accept_header {
Some(api_types::Accept::Ssz) => Response::builder()
.status(200)
.header("Content-Type", "application/octet-stream")
.body(update.as_ssz_bytes().into())
.map_err(|e| {
warp_utils::reject::custom_server_error(format!(
"failed to create response: {}",
e
))
}),
_ => Ok(warp::reply::json(&ForkVersionedResponse {
version: Some(fork_name),
data: update,
})
.into_response()),
}
.map(|resp| add_consensus_version_header(resp, fork_name))
})
},
);
// GET beacon/light_client/finality_update
let get_beacon_light_client_finality_update = beacon_light_client_path
.clone()
.and(task_spawner_filter.clone())
.and(warp::path("finality_update"))
.and(warp::path::end())
.and(warp::header::optional::<api_types::Accept>("accept"))
.then(
|chain: Arc<BeaconChain<T>>,
task_spawner: TaskSpawner<T::EthSpec>,
accept_header: Option<api_types::Accept>| {
task_spawner.blocking_response_task(Priority::P1, move || {
let update = chain
.latest_seen_finality_update
.lock()
.clone()
.ok_or_else(|| {
warp_utils::reject::custom_not_found(
"No LightClientFinalityUpdate is available".to_string(),
)
})?;
let fork_name = chain
.spec
.fork_name_at_slot::<T::EthSpec>(update.signature_slot);
match accept_header {
Some(api_types::Accept::Ssz) => Response::builder()
.status(200)
.header("Content-Type", "application/octet-stream")
.body(update.as_ssz_bytes().into())
.map_err(|e| {
warp_utils::reject::custom_server_error(format!(
"failed to create response: {}",
e
))
}),
_ => Ok(warp::reply::json(&ForkVersionedResponse {
version: Some(fork_name),
data: update,
})
.into_response()),
}
.map(|resp| add_consensus_version_header(resp, fork_name))
})
},
);
/* /*
* beacon/rewards * beacon/rewards
*/ */
@ -4339,6 +4512,12 @@ pub fn serve<T: BeaconChainTypes>(
api_types::EventTopic::LateHead => { api_types::EventTopic::LateHead => {
event_handler.subscribe_late_head() event_handler.subscribe_late_head()
} }
api_types::EventTopic::LightClientFinalityUpdate => {
event_handler.subscribe_light_client_finality_update()
}
api_types::EventTopic::LightClientOptimisticUpdate => {
event_handler.subscribe_light_client_optimistic_update()
}
api_types::EventTopic::BlockReward => { api_types::EventTopic::BlockReward => {
event_handler.subscribe_block_reward() event_handler.subscribe_block_reward()
} }
@ -4492,6 +4671,18 @@ pub fn serve<T: BeaconChainTypes>(
.uor(get_lighthouse_database_info) .uor(get_lighthouse_database_info)
.uor(get_lighthouse_block_rewards) .uor(get_lighthouse_block_rewards)
.uor(get_lighthouse_attestation_performance) .uor(get_lighthouse_attestation_performance)
.uor(
enable(ctx.config.enable_light_client_server)
.and(get_beacon_light_client_optimistic_update),
)
.uor(
enable(ctx.config.enable_light_client_server)
.and(get_beacon_light_client_finality_update),
)
.uor(
enable(ctx.config.enable_light_client_server)
.and(get_beacon_light_client_bootstrap),
)
.uor(get_lighthouse_block_packing_efficiency) .uor(get_lighthouse_block_packing_efficiency)
.uor(get_lighthouse_merge_readiness) .uor(get_lighthouse_merge_readiness)
.uor(get_events) .uor(get_events)

View File

@ -209,6 +209,7 @@ pub async fn create_api_server<T: BeaconChainTypes>(
enabled: true, enabled: true,
listen_port: port, listen_port: port,
data_dir: std::path::PathBuf::from(DEFAULT_ROOT_DIR), data_dir: std::path::PathBuf::from(DEFAULT_ROOT_DIR),
enable_light_client_server: true,
..Config::default() ..Config::default()
}, },
chain: Some(chain), chain: Some(chain),

View File

@ -1646,6 +1646,59 @@ impl ApiTester {
self self
} }
pub async fn test_get_beacon_light_client_bootstrap(self) -> Self {
let block_id = BlockId(CoreBlockId::Finalized);
let (block_root, _, _) = block_id.root(&self.chain).unwrap();
let (block, _, _) = block_id.full_block(&self.chain).await.unwrap();
let result = match self
.client
.get_light_client_bootstrap::<E>(block_root)
.await
{
Ok(result) => result.unwrap().data,
Err(e) => panic!("query failed incorrectly: {e:?}"),
};
let expected = block.slot();
assert_eq!(result.header.beacon.slot, expected);
self
}
pub async fn test_get_beacon_light_client_optimistic_update(self) -> Self {
// get_beacon_light_client_optimistic_update returns Ok(None) on 404 NOT FOUND
let result = match self
.client
.get_beacon_light_client_optimistic_update::<E>()
.await
{
Ok(result) => result.map(|res| res.data),
Err(e) => panic!("query failed incorrectly: {e:?}"),
};
let expected = self.chain.latest_seen_optimistic_update.lock().clone();
assert_eq!(result, expected);
self
}
pub async fn test_get_beacon_light_client_finality_update(self) -> Self {
let result = match self
.client
.get_beacon_light_client_finality_update::<E>()
.await
{
Ok(result) => result.map(|res| res.data),
Err(e) => panic!("query failed incorrectly: {e:?}"),
};
let expected = self.chain.latest_seen_finality_update.lock().clone();
assert_eq!(result, expected);
self
}
pub async fn test_get_beacon_pool_attestations(self) -> Self { pub async fn test_get_beacon_pool_attestations(self) -> Self {
let result = self let result = self
.client .client
@ -5701,6 +5754,42 @@ async fn node_get() {
.await; .await;
} }
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn get_light_client_bootstrap() {
let config = ApiTesterConfig {
spec: ForkName::Altair.make_genesis_spec(E::default_spec()),
..<_>::default()
};
ApiTester::new_from_config(config)
.await
.test_get_beacon_light_client_bootstrap()
.await;
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn get_light_client_optimistic_update() {
let config = ApiTesterConfig {
spec: ForkName::Altair.make_genesis_spec(E::default_spec()),
..<_>::default()
};
ApiTester::new_from_config(config)
.await
.test_get_beacon_light_client_optimistic_update()
.await;
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn get_light_client_finality_update() {
let config = ApiTesterConfig {
spec: ForkName::Altair.make_genesis_spec(E::default_spec()),
..<_>::default()
};
ApiTester::new_from_config(config)
.await
.test_get_beacon_light_client_finality_update()
.await;
}
#[tokio::test(flavor = "multi_thread", worker_threads = 2)] #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
async fn get_validator_duties_early() { async fn get_validator_duties_early() {
ApiTester::new() ApiTester::new()

View File

@ -15,11 +15,10 @@ use std::io::{Read, Write};
use std::marker::PhantomData; use std::marker::PhantomData;
use std::sync::Arc; use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder}; use tokio_util::codec::{Decoder, Encoder};
use types::{light_client_bootstrap::LightClientBootstrap, BlobSidecar};
use types::{ use types::{
EthSpec, ForkContext, ForkName, Hash256, SignedBeaconBlock, SignedBeaconBlockAltair, BlobSidecar, EthSpec, ForkContext, ForkName, Hash256, LightClientBootstrap, SignedBeaconBlock,
SignedBeaconBlockBase, SignedBeaconBlockCapella, SignedBeaconBlockDeneb, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockCapella,
SignedBeaconBlockMerge, SignedBeaconBlockDeneb, SignedBeaconBlockMerge,
}; };
use unsigned_varint::codec::Uvi; use unsigned_varint::codec::Uvi;

View File

@ -17,8 +17,8 @@ use superstruct::superstruct;
use types::blob_sidecar::BlobIdentifier; use types::blob_sidecar::BlobIdentifier;
use types::consts::deneb::MAX_BLOBS_PER_BLOCK; use types::consts::deneb::MAX_BLOBS_PER_BLOCK;
use types::{ use types::{
blob_sidecar::BlobSidecar, light_client_bootstrap::LightClientBootstrap, Epoch, EthSpec, blob_sidecar::BlobSidecar, Epoch, EthSpec, Hash256, LightClientBootstrap, SignedBeaconBlock,
Hash256, SignedBeaconBlock, Slot, Slot,
}; };
/// Maximum number of blocks in a single request. /// Maximum number of blocks in a single request.
@ -571,7 +571,11 @@ impl<T: EthSpec> std::fmt::Display for RPCResponse<T> {
RPCResponse::Pong(ping) => write!(f, "Pong: {}", ping.data), RPCResponse::Pong(ping) => write!(f, "Pong: {}", ping.data),
RPCResponse::MetaData(metadata) => write!(f, "Metadata: {}", metadata.seq_number()), RPCResponse::MetaData(metadata) => write!(f, "Metadata: {}", metadata.seq_number()),
RPCResponse::LightClientBootstrap(bootstrap) => { RPCResponse::LightClientBootstrap(bootstrap) => {
write!(f, "LightClientBootstrap Slot: {}", bootstrap.header.slot) write!(
f,
"LightClientBootstrap Slot: {}",
bootstrap.header.beacon.slot
)
} }
} }
} }

View File

@ -1,8 +1,7 @@
use std::sync::Arc; use std::sync::Arc;
use libp2p::swarm::ConnectionId; use libp2p::swarm::ConnectionId;
use types::light_client_bootstrap::LightClientBootstrap; use types::{BlobSidecar, EthSpec, LightClientBootstrap, SignedBeaconBlock};
use types::{BlobSidecar, EthSpec, SignedBeaconBlock};
use crate::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest}; use crate::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest};
use crate::rpc::{ use crate::rpc::{

View File

@ -18,9 +18,7 @@ use std::sync::Arc;
use task_executor::TaskExecutor; use task_executor::TaskExecutor;
use tokio_stream::StreamExt; use tokio_stream::StreamExt;
use types::blob_sidecar::BlobIdentifier; use types::blob_sidecar::BlobIdentifier;
use types::{ use types::{Epoch, EthSpec, ForkName, Hash256, Slot};
light_client_bootstrap::LightClientBootstrap, Epoch, EthSpec, ForkName, Hash256, Slot,
};
impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> { impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
/* Auxiliary functions */ /* Auxiliary functions */
@ -304,67 +302,33 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
request: LightClientBootstrapRequest, request: LightClientBootstrapRequest,
) { ) {
let block_root = request.root; let block_root = request.root;
let state_root = match self.chain.get_blinded_block(&block_root) { match self.chain.get_light_client_bootstrap(&block_root) {
Ok(signed_block) => match signed_block { Ok(Some((bootstrap, _))) => self.send_response(
Some(signed_block) => signed_block.state_root(),
None => {
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
);
return;
}
},
Err(_) => {
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
);
return;
}
};
let mut beacon_state = match self.chain.get_state(&state_root, None) {
Ok(beacon_state) => match beacon_state {
Some(state) => state,
None => {
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
);
return;
}
},
Err(_) => {
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
);
return;
}
};
let Ok(bootstrap) = LightClientBootstrap::from_beacon_state(&mut beacon_state) else {
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
);
return;
};
self.send_response(
peer_id, peer_id,
Response::LightClientBootstrap(bootstrap), Response::LightClientBootstrap(bootstrap),
request_id, request_id,
),
Ok(None) => self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
),
Err(e) => {
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Bootstrap not available".into(),
request_id,
);
error!(self.log, "Error getting LightClientBootstrap instance";
"block_root" => ?block_root,
"peer" => %peer_id,
"error" => ?e
) )
} }
};
}
/// Handle a `BlocksByRange` request from the peer. /// Handle a `BlocksByRange` request from the peer.
pub fn handle_blocks_by_range_request( pub fn handle_blocks_by_range_request(

View File

@ -151,6 +151,9 @@ pub fn get_config<E: EthSpec>(
client_config.http_api.duplicate_block_status_code = client_config.http_api.duplicate_block_status_code =
parse_required(cli_args, "http-duplicate-block-status")?; parse_required(cli_args, "http-duplicate-block-status")?;
client_config.http_api.enable_light_client_server =
cli_args.is_present("light-client-server");
} }
if let Some(cache_size) = clap_utils::parse_optional(cli_args, "shuffling-cache-size")? { if let Some(cache_size) = clap_utils::parse_optional(cli_args, "shuffling-cache-size")? {

View File

@ -667,6 +667,59 @@ impl BeaconNodeHttpClient {
self.get_opt(path).await self.get_opt(path).await
} }
/// `GET beacon/light_client/bootstrap`
///
/// Returns `Ok(None)` on a 404 error.
pub async fn get_light_client_bootstrap<E: EthSpec>(
&self,
block_root: Hash256,
) -> Result<Option<ForkVersionedResponse<LightClientBootstrap<E>>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("light_client")
.push("bootstrap")
.push(&format!("{:?}", block_root));
self.get_opt(path).await
}
/// `GET beacon/light_client/optimistic_update`
///
/// Returns `Ok(None)` on a 404 error.
pub async fn get_beacon_light_client_optimistic_update<E: EthSpec>(
&self,
) -> Result<Option<ForkVersionedResponse<LightClientOptimisticUpdate<E>>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("light_client")
.push("optimistic_update");
self.get_opt(path).await
}
/// `GET beacon/light_client/finality_update`
///
/// Returns `Ok(None)` on a 404 error.
pub async fn get_beacon_light_client_finality_update<E: EthSpec>(
&self,
) -> Result<Option<ForkVersionedResponse<LightClientFinalityUpdate<E>>>, Error> {
let mut path = self.eth_path(V1)?;
path.path_segments_mut()
.map_err(|()| Error::InvalidUrl(self.server.clone()))?
.push("beacon")
.push("light_client")
.push("finality_update");
self.get_opt(path).await
}
/// `GET beacon/headers?slot,parent_root` /// `GET beacon/headers?slot,parent_root`
/// ///
/// Returns `Ok(None)` on a 404 error. /// Returns `Ok(None)` on a 404 error.

View File

@ -1047,6 +1047,8 @@ pub enum EventKind<T: EthSpec> {
ChainReorg(SseChainReorg), ChainReorg(SseChainReorg),
ContributionAndProof(Box<SignedContributionAndProof<T>>), ContributionAndProof(Box<SignedContributionAndProof<T>>),
LateHead(SseLateHead), LateHead(SseLateHead),
LightClientFinalityUpdate(Box<LightClientFinalityUpdate<T>>),
LightClientOptimisticUpdate(Box<LightClientOptimisticUpdate<T>>),
#[cfg(feature = "lighthouse")] #[cfg(feature = "lighthouse")]
BlockReward(BlockReward), BlockReward(BlockReward),
PayloadAttributes(VersionedSsePayloadAttributes), PayloadAttributes(VersionedSsePayloadAttributes),
@ -1065,6 +1067,8 @@ impl<T: EthSpec> EventKind<T> {
EventKind::ContributionAndProof(_) => "contribution_and_proof", EventKind::ContributionAndProof(_) => "contribution_and_proof",
EventKind::PayloadAttributes(_) => "payload_attributes", EventKind::PayloadAttributes(_) => "payload_attributes",
EventKind::LateHead(_) => "late_head", EventKind::LateHead(_) => "late_head",
EventKind::LightClientFinalityUpdate(_) => "light_client_finality_update",
EventKind::LightClientOptimisticUpdate(_) => "light_client_optimistic_update",
#[cfg(feature = "lighthouse")] #[cfg(feature = "lighthouse")]
EventKind::BlockReward(_) => "block_reward", EventKind::BlockReward(_) => "block_reward",
} }
@ -1127,6 +1131,22 @@ impl<T: EthSpec> EventKind<T> {
ServerError::InvalidServerSentEvent(format!("Payload Attributes: {:?}", e)) ServerError::InvalidServerSentEvent(format!("Payload Attributes: {:?}", e))
})?, })?,
)), )),
"light_client_finality_update" => Ok(EventKind::LightClientFinalityUpdate(
serde_json::from_str(data).map_err(|e| {
ServerError::InvalidServerSentEvent(format!(
"Light Client Finality Update: {:?}",
e
))
})?,
)),
"light_client_optimistic_update" => Ok(EventKind::LightClientOptimisticUpdate(
serde_json::from_str(data).map_err(|e| {
ServerError::InvalidServerSentEvent(format!(
"Light Client Optimistic Update: {:?}",
e
))
})?,
)),
#[cfg(feature = "lighthouse")] #[cfg(feature = "lighthouse")]
"block_reward" => Ok(EventKind::BlockReward(serde_json::from_str(data).map_err( "block_reward" => Ok(EventKind::BlockReward(serde_json::from_str(data).map_err(
|e| ServerError::InvalidServerSentEvent(format!("Block Reward: {:?}", e)), |e| ServerError::InvalidServerSentEvent(format!("Block Reward: {:?}", e)),
@ -1158,6 +1178,8 @@ pub enum EventTopic {
ContributionAndProof, ContributionAndProof,
LateHead, LateHead,
PayloadAttributes, PayloadAttributes,
LightClientFinalityUpdate,
LightClientOptimisticUpdate,
#[cfg(feature = "lighthouse")] #[cfg(feature = "lighthouse")]
BlockReward, BlockReward,
} }
@ -1177,6 +1199,8 @@ impl FromStr for EventTopic {
"contribution_and_proof" => Ok(EventTopic::ContributionAndProof), "contribution_and_proof" => Ok(EventTopic::ContributionAndProof),
"payload_attributes" => Ok(EventTopic::PayloadAttributes), "payload_attributes" => Ok(EventTopic::PayloadAttributes),
"late_head" => Ok(EventTopic::LateHead), "late_head" => Ok(EventTopic::LateHead),
"light_client_finality_update" => Ok(EventTopic::LightClientFinalityUpdate),
"light_client_optimistic_update" => Ok(EventTopic::LightClientOptimisticUpdate),
#[cfg(feature = "lighthouse")] #[cfg(feature = "lighthouse")]
"block_reward" => Ok(EventTopic::BlockReward), "block_reward" => Ok(EventTopic::BlockReward),
_ => Err("event topic cannot be parsed.".to_string()), _ => Err("event topic cannot be parsed.".to_string()),
@ -1197,6 +1221,8 @@ impl fmt::Display for EventTopic {
EventTopic::ContributionAndProof => write!(f, "contribution_and_proof"), EventTopic::ContributionAndProof => write!(f, "contribution_and_proof"),
EventTopic::PayloadAttributes => write!(f, "payload_attributes"), EventTopic::PayloadAttributes => write!(f, "payload_attributes"),
EventTopic::LateHead => write!(f, "late_head"), EventTopic::LateHead => write!(f, "late_head"),
EventTopic::LightClientFinalityUpdate => write!(f, "light_client_finality_update"),
EventTopic::LightClientOptimisticUpdate => write!(f, "light_client_optimistic_update"),
#[cfg(feature = "lighthouse")] #[cfg(feature = "lighthouse")]
EventTopic::BlockReward => write!(f, "block_reward"), EventTopic::BlockReward => write!(f, "block_reward"),
} }

View File

@ -54,7 +54,7 @@ pub fn upgrade_to_altair<E: EthSpec>(
VariableList::new(vec![ParticipationFlags::default(); pre.validators.len()])?; VariableList::new(vec![ParticipationFlags::default(); pre.validators.len()])?;
let inactivity_scores = VariableList::new(vec![0; pre.validators.len()])?; let inactivity_scores = VariableList::new(vec![0; pre.validators.len()])?;
let temp_sync_committee = Arc::new(SyncCommittee::temporary()?); let temp_sync_committee = Arc::new(SyncCommittee::temporary());
// Where possible, use something like `mem::take` to move fields from behind the &mut // Where possible, use something like `mem::take` to move fields from behind the &mut
// reference. For other fields that don't have a good default value, use `clone`. // reference. For other fields that don't have a good default value, use `clone`.

View File

@ -99,6 +99,7 @@ pub mod slot_data;
pub mod sqlite; pub mod sqlite;
pub mod blob_sidecar; pub mod blob_sidecar;
pub mod light_client_header;
pub mod sidecar; pub mod sidecar;
pub mod signed_blob; pub mod signed_blob;
@ -154,8 +155,11 @@ pub use crate::fork_versioned_response::{ForkVersionDeserialize, ForkVersionedRe
pub use crate::graffiti::{Graffiti, GRAFFITI_BYTES_LEN}; pub use crate::graffiti::{Graffiti, GRAFFITI_BYTES_LEN};
pub use crate::historical_batch::HistoricalBatch; pub use crate::historical_batch::HistoricalBatch;
pub use crate::indexed_attestation::IndexedAttestation; pub use crate::indexed_attestation::IndexedAttestation;
pub use crate::light_client_bootstrap::LightClientBootstrap;
pub use crate::light_client_finality_update::LightClientFinalityUpdate; pub use crate::light_client_finality_update::LightClientFinalityUpdate;
pub use crate::light_client_header::LightClientHeader;
pub use crate::light_client_optimistic_update::LightClientOptimisticUpdate; pub use crate::light_client_optimistic_update::LightClientOptimisticUpdate;
pub use crate::light_client_update::{Error as LightClientError, LightClientUpdate};
pub use crate::participation_flags::ParticipationFlags; pub use crate::participation_flags::ParticipationFlags;
pub use crate::participation_list::ParticipationList; pub use crate::participation_list::ParticipationList;
pub use crate::payload::{ pub use crate::payload::{

View File

@ -1,10 +1,13 @@
use super::{BeaconBlockHeader, BeaconState, EthSpec, FixedVector, Hash256, SyncCommittee}; use super::{BeaconState, EthSpec, FixedVector, Hash256, SyncCommittee};
use crate::{light_client_update::*, test_utils::TestRandom}; use crate::{
use serde::{Deserialize, Serialize}; light_client_update::*, test_utils::TestRandom, ForkName, ForkVersionDeserialize,
LightClientHeader,
};
use serde::{Deserialize, Deserializer, Serialize};
use serde_json::Value;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use std::sync::Arc; use std::sync::Arc;
use test_random_derive::TestRandom; use test_random_derive::TestRandom;
use tree_hash::TreeHash;
/// A LightClientBootstrap is the initializer we send over to lightclient nodes /// A LightClientBootstrap is the initializer we send over to lightclient nodes
/// that are trying to generate their basic storage when booting up. /// that are trying to generate their basic storage when booting up.
@ -22,8 +25,8 @@ use tree_hash::TreeHash;
#[serde(bound = "T: EthSpec")] #[serde(bound = "T: EthSpec")]
#[arbitrary(bound = "T: EthSpec")] #[arbitrary(bound = "T: EthSpec")]
pub struct LightClientBootstrap<T: EthSpec> { pub struct LightClientBootstrap<T: EthSpec> {
/// Requested beacon block header. /// The requested beacon block header.
pub header: BeaconBlockHeader, pub header: LightClientHeader,
/// The `SyncCommittee` used in the requested period. /// The `SyncCommittee` used in the requested period.
pub current_sync_committee: Arc<SyncCommittee<T>>, pub current_sync_committee: Arc<SyncCommittee<T>>,
/// Merkle proof for sync committee /// Merkle proof for sync committee
@ -33,17 +36,37 @@ pub struct LightClientBootstrap<T: EthSpec> {
impl<T: EthSpec> LightClientBootstrap<T> { impl<T: EthSpec> LightClientBootstrap<T> {
pub fn from_beacon_state(beacon_state: &mut BeaconState<T>) -> Result<Self, Error> { pub fn from_beacon_state(beacon_state: &mut BeaconState<T>) -> Result<Self, Error> {
let mut header = beacon_state.latest_block_header().clone(); let mut header = beacon_state.latest_block_header().clone();
header.state_root = beacon_state.tree_hash_root(); header.state_root = beacon_state.update_tree_hash_cache()?;
let current_sync_committee_branch = let current_sync_committee_branch =
beacon_state.compute_merkle_proof(CURRENT_SYNC_COMMITTEE_INDEX)?; beacon_state.compute_merkle_proof(CURRENT_SYNC_COMMITTEE_INDEX)?;
Ok(LightClientBootstrap { Ok(LightClientBootstrap {
header, header: header.into(),
current_sync_committee: beacon_state.current_sync_committee()?.clone(), current_sync_committee: beacon_state.current_sync_committee()?.clone(),
current_sync_committee_branch: FixedVector::new(current_sync_committee_branch)?, current_sync_committee_branch: FixedVector::new(current_sync_committee_branch)?,
}) })
} }
} }
impl<T: EthSpec> ForkVersionDeserialize for LightClientBootstrap<T> {
fn deserialize_by_fork<'de, D: Deserializer<'de>>(
value: Value,
fork_name: ForkName,
) -> Result<Self, D::Error> {
match fork_name {
ForkName::Altair | ForkName::Merge => {
Ok(serde_json::from_value::<LightClientBootstrap<T>>(value)
.map_err(serde::de::Error::custom))?
}
ForkName::Base | ForkName::Capella | ForkName::Deneb => {
Err(serde::de::Error::custom(format!(
"LightClientBootstrap failed to deserialize: unsupported fork '{}'",
fork_name
)))
}
}
}
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;

View File

@ -1,9 +1,12 @@
use super::{ use super::{
BeaconBlockHeader, EthSpec, FixedVector, Hash256, SignedBeaconBlock, SignedBlindedBeaconBlock, EthSpec, FixedVector, Hash256, SignedBeaconBlock, SignedBlindedBeaconBlock, Slot, SyncAggregate,
Slot, SyncAggregate,
}; };
use crate::{light_client_update::*, test_utils::TestRandom, BeaconState, ChainSpec}; use crate::{
use serde::{Deserialize, Serialize}; light_client_update::*, test_utils::TestRandom, BeaconState, ChainSpec, ForkName,
ForkVersionDeserialize, LightClientHeader,
};
use serde::{Deserialize, Deserializer, Serialize};
use serde_json::Value;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom; use test_random_derive::TestRandom;
use tree_hash::TreeHash; use tree_hash::TreeHash;
@ -25,9 +28,9 @@ use tree_hash::TreeHash;
#[arbitrary(bound = "T: EthSpec")] #[arbitrary(bound = "T: EthSpec")]
pub struct LightClientFinalityUpdate<T: EthSpec> { pub struct LightClientFinalityUpdate<T: EthSpec> {
/// The last `BeaconBlockHeader` from the last attested block by the sync committee. /// The last `BeaconBlockHeader` from the last attested block by the sync committee.
pub attested_header: BeaconBlockHeader, pub attested_header: LightClientHeader,
/// The last `BeaconBlockHeader` from the last attested finalized block (end of epoch). /// The last `BeaconBlockHeader` from the last attested finalized block (end of epoch).
pub finalized_header: BeaconBlockHeader, pub finalized_header: LightClientHeader,
/// Merkle proof attesting finalized header. /// Merkle proof attesting finalized header.
pub finality_branch: FixedVector<Hash256, FinalizedRootProofLen>, pub finality_branch: FixedVector<Hash256, FinalizedRootProofLen>,
/// current sync aggreggate /// current sync aggreggate
@ -68,8 +71,8 @@ impl<T: EthSpec> LightClientFinalityUpdate<T> {
let finality_branch = attested_state.compute_merkle_proof(FINALIZED_ROOT_INDEX)?; let finality_branch = attested_state.compute_merkle_proof(FINALIZED_ROOT_INDEX)?;
Ok(Self { Ok(Self {
attested_header, attested_header: attested_header.into(),
finalized_header, finalized_header: finalized_header.into(),
finality_branch: FixedVector::new(finality_branch)?, finality_branch: FixedVector::new(finality_branch)?,
sync_aggregate: sync_aggregate.clone(), sync_aggregate: sync_aggregate.clone(),
signature_slot: block.slot(), signature_slot: block.slot(),
@ -77,6 +80,26 @@ impl<T: EthSpec> LightClientFinalityUpdate<T> {
} }
} }
impl<T: EthSpec> ForkVersionDeserialize for LightClientFinalityUpdate<T> {
fn deserialize_by_fork<'de, D: Deserializer<'de>>(
value: Value,
fork_name: ForkName,
) -> Result<Self, D::Error> {
match fork_name {
ForkName::Altair | ForkName::Merge => Ok(serde_json::from_value::<
LightClientFinalityUpdate<T>,
>(value)
.map_err(serde::de::Error::custom))?,
ForkName::Base | ForkName::Capella | ForkName::Deneb => {
Err(serde::de::Error::custom(format!(
"LightClientFinalityUpdate failed to deserialize: unsupported fork '{}'",
fork_name
)))
}
}
}
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;

View File

@ -0,0 +1,26 @@
use crate::test_utils::TestRandom;
use crate::BeaconBlockHeader;
use serde::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
#[derive(
Debug,
Clone,
PartialEq,
Serialize,
Deserialize,
Encode,
Decode,
TestRandom,
arbitrary::Arbitrary,
)]
pub struct LightClientHeader {
pub beacon: BeaconBlockHeader,
}
impl From<BeaconBlockHeader> for LightClientHeader {
fn from(beacon: BeaconBlockHeader) -> Self {
LightClientHeader { beacon }
}
}

View File

@ -1,8 +1,10 @@
use super::{BeaconBlockHeader, EthSpec, Slot, SyncAggregate}; use super::{EthSpec, ForkName, ForkVersionDeserialize, Slot, SyncAggregate};
use crate::light_client_header::LightClientHeader;
use crate::{ use crate::{
light_client_update::Error, test_utils::TestRandom, BeaconState, ChainSpec, SignedBeaconBlock, light_client_update::Error, test_utils::TestRandom, BeaconState, ChainSpec, SignedBeaconBlock,
}; };
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Deserializer, Serialize};
use serde_json::Value;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom; use test_random_derive::TestRandom;
use tree_hash::TreeHash; use tree_hash::TreeHash;
@ -24,7 +26,7 @@ use tree_hash::TreeHash;
#[arbitrary(bound = "T: EthSpec")] #[arbitrary(bound = "T: EthSpec")]
pub struct LightClientOptimisticUpdate<T: EthSpec> { pub struct LightClientOptimisticUpdate<T: EthSpec> {
/// The last `BeaconBlockHeader` from the last attested block by the sync committee. /// The last `BeaconBlockHeader` from the last attested block by the sync committee.
pub attested_header: BeaconBlockHeader, pub attested_header: LightClientHeader,
/// current sync aggreggate /// current sync aggreggate
pub sync_aggregate: SyncAggregate<T>, pub sync_aggregate: SyncAggregate<T>,
/// Slot of the sync aggregated singature /// Slot of the sync aggregated singature
@ -53,13 +55,33 @@ impl<T: EthSpec> LightClientOptimisticUpdate<T> {
let mut attested_header = attested_state.latest_block_header().clone(); let mut attested_header = attested_state.latest_block_header().clone();
attested_header.state_root = attested_state.tree_hash_root(); attested_header.state_root = attested_state.tree_hash_root();
Ok(Self { Ok(Self {
attested_header, attested_header: attested_header.into(),
sync_aggregate: sync_aggregate.clone(), sync_aggregate: sync_aggregate.clone(),
signature_slot: block.slot(), signature_slot: block.slot(),
}) })
} }
} }
impl<T: EthSpec> ForkVersionDeserialize for LightClientOptimisticUpdate<T> {
fn deserialize_by_fork<'de, D: Deserializer<'de>>(
value: Value,
fork_name: ForkName,
) -> Result<Self, D::Error> {
match fork_name {
ForkName::Altair | ForkName::Merge => Ok(serde_json::from_value::<
LightClientOptimisticUpdate<T>,
>(value)
.map_err(serde::de::Error::custom))?,
ForkName::Base | ForkName::Capella | ForkName::Deneb => {
Err(serde::de::Error::custom(format!(
"LightClientOptimisticUpdate failed to deserialize: unsupported fork '{}'",
fork_name
)))
}
}
}
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;

View File

@ -1,7 +1,11 @@
use super::{BeaconBlockHeader, EthSpec, FixedVector, Hash256, Slot, SyncAggregate, SyncCommittee}; use super::{BeaconBlockHeader, EthSpec, FixedVector, Hash256, Slot, SyncAggregate, SyncCommittee};
use crate::{beacon_state, test_utils::TestRandom, BeaconBlock, BeaconState, ChainSpec}; use crate::{
beacon_state, test_utils::TestRandom, BeaconBlock, BeaconState, ChainSpec, ForkName,
ForkVersionDeserialize, LightClientHeader,
};
use safe_arith::ArithError; use safe_arith::ArithError;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Deserializer, Serialize};
use serde_json::Value;
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use ssz_types::typenum::{U5, U6}; use ssz_types::typenum::{U5, U6};
use std::sync::Arc; use std::sync::Arc;
@ -67,13 +71,13 @@ impl From<ArithError> for Error {
#[arbitrary(bound = "T: EthSpec")] #[arbitrary(bound = "T: EthSpec")]
pub struct LightClientUpdate<T: EthSpec> { pub struct LightClientUpdate<T: EthSpec> {
/// The last `BeaconBlockHeader` from the last attested block by the sync committee. /// The last `BeaconBlockHeader` from the last attested block by the sync committee.
pub attested_header: BeaconBlockHeader, pub attested_header: LightClientHeader,
/// The `SyncCommittee` used in the next period. /// The `SyncCommittee` used in the next period.
pub next_sync_committee: Arc<SyncCommittee<T>>, pub next_sync_committee: Arc<SyncCommittee<T>>,
/// Merkle proof for next sync committee /// Merkle proof for next sync committee
pub next_sync_committee_branch: FixedVector<Hash256, NextSyncCommitteeProofLen>, pub next_sync_committee_branch: FixedVector<Hash256, NextSyncCommitteeProofLen>,
/// The last `BeaconBlockHeader` from the last attested finalized block (end of epoch). /// The last `BeaconBlockHeader` from the last attested finalized block (end of epoch).
pub finalized_header: BeaconBlockHeader, pub finalized_header: LightClientHeader,
/// Merkle proof attesting finalized header. /// Merkle proof attesting finalized header.
pub finality_branch: FixedVector<Hash256, FinalizedRootProofLen>, pub finality_branch: FixedVector<Hash256, FinalizedRootProofLen>,
/// current sync aggreggate /// current sync aggreggate
@ -128,10 +132,10 @@ impl<T: EthSpec> LightClientUpdate<T> {
attested_state.compute_merkle_proof(NEXT_SYNC_COMMITTEE_INDEX)?; attested_state.compute_merkle_proof(NEXT_SYNC_COMMITTEE_INDEX)?;
let finality_branch = attested_state.compute_merkle_proof(FINALIZED_ROOT_INDEX)?; let finality_branch = attested_state.compute_merkle_proof(FINALIZED_ROOT_INDEX)?;
Ok(Self { Ok(Self {
attested_header, attested_header: attested_header.into(),
next_sync_committee: attested_state.next_sync_committee()?.clone(), next_sync_committee: attested_state.next_sync_committee()?.clone(),
next_sync_committee_branch: FixedVector::new(next_sync_committee_branch)?, next_sync_committee_branch: FixedVector::new(next_sync_committee_branch)?,
finalized_header, finalized_header: finalized_header.into(),
finality_branch: FixedVector::new(finality_branch)?, finality_branch: FixedVector::new(finality_branch)?,
sync_aggregate: sync_aggregate.clone(), sync_aggregate: sync_aggregate.clone(),
signature_slot: block.slot(), signature_slot: block.slot(),
@ -139,6 +143,26 @@ impl<T: EthSpec> LightClientUpdate<T> {
} }
} }
impl<T: EthSpec> ForkVersionDeserialize for LightClientUpdate<T> {
fn deserialize_by_fork<'de, D: Deserializer<'de>>(
value: Value,
fork_name: ForkName,
) -> Result<Self, D::Error> {
match fork_name {
ForkName::Altair | ForkName::Merge => {
Ok(serde_json::from_value::<LightClientUpdate<T>>(value)
.map_err(serde::de::Error::custom))?
}
ForkName::Base | ForkName::Capella | ForkName::Deneb => {
Err(serde::de::Error::custom(format!(
"LightClientUpdate failed to deserialize: unsupported fork '{}'",
fork_name
)))
}
}
}
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;

View File

@ -1,5 +1,4 @@
use crate::test_utils::TestRandom; use crate::test_utils::TestRandom;
use crate::typenum::Unsigned;
use crate::{EthSpec, FixedVector, SyncSubnetId}; use crate::{EthSpec, FixedVector, SyncSubnetId};
use bls::PublicKeyBytes; use bls::PublicKeyBytes;
use safe_arith::{ArithError, SafeArith}; use safe_arith::{ArithError, SafeArith};
@ -46,14 +45,11 @@ pub struct SyncCommittee<T: EthSpec> {
impl<T: EthSpec> SyncCommittee<T> { impl<T: EthSpec> SyncCommittee<T> {
/// Create a temporary sync committee that should *never* be included in a legitimate consensus object. /// Create a temporary sync committee that should *never* be included in a legitimate consensus object.
pub fn temporary() -> Result<Self, ssz_types::Error> { pub fn temporary() -> Self {
Ok(Self { Self {
pubkeys: FixedVector::new(vec![ pubkeys: FixedVector::from_elem(PublicKeyBytes::empty()),
PublicKeyBytes::empty();
T::SyncCommitteeSize::to_usize()
])?,
aggregate_pubkey: PublicKeyBytes::empty(), aggregate_pubkey: PublicKeyBytes::empty(),
}) }
} }
/// Return the pubkeys in this `SyncCommittee` for the given `subcommittee_index`. /// Return the pubkeys in this `SyncCommittee` for the given `subcommittee_index`.

View File

@ -2346,7 +2346,10 @@ fn sync_eth1_chain_disable_deposit_contract_sync_flag() {
fn light_client_server_default() { fn light_client_server_default() {
CommandLineTest::new() CommandLineTest::new()
.run_with_zero_port() .run_with_zero_port()
.with_config(|config| assert_eq!(config.network.enable_light_client_server, false)); .with_config(|config| {
assert_eq!(config.network.enable_light_client_server, false);
assert_eq!(config.http_api.enable_light_client_server, false);
});
} }
#[test] #[test]
@ -2354,7 +2357,20 @@ fn light_client_server_enabled() {
CommandLineTest::new() CommandLineTest::new()
.flag("light-client-server", None) .flag("light-client-server", None)
.run_with_zero_port() .run_with_zero_port()
.with_config(|config| assert_eq!(config.network.enable_light_client_server, true)); .with_config(|config| {
assert_eq!(config.network.enable_light_client_server, true);
});
}
#[test]
fn light_client_http_server_enabled() {
CommandLineTest::new()
.flag("http", None)
.flag("light-client-server", None)
.run_with_zero_port()
.with_config(|config| {
assert_eq!(config.http_api.enable_light_client_server, true);
});
} }
#[test] #[test]