Some gossip work

This commit is contained in:
realbigsean 2022-02-20 14:11:10 -07:00
parent ebc0ccd02a
commit 3f1e5cee78
No known key found for this signature in database
GPG Key ID: B372B64D866BF8CC
14 changed files with 138 additions and 34 deletions

View File

@ -602,7 +602,7 @@ where
///
/// If type inference errors are being raised, see the comment on the definition of `Self`.
#[allow(clippy::type_complexity)]
pub fn build(
pub fn build(
mut self,
) -> Result<Client<Witness<TSlotClock, TEth1Backend, TEthSpec, THotStore, TColdStore>>, String>
{

View File

@ -15,7 +15,10 @@ use std::io::{Read, Write};
use std::marker::PhantomData;
use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder};
use types::{BlobWrapper, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockMerge, SignedBeaconBlockShanghai};
use types::{
BlobWrapper, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockMerge, SignedBeaconBlockShanghai,
};
use unsigned_varint::codec::Uvi;
const CONTEXT_BYTES_LEN: usize = 4;
@ -603,13 +606,9 @@ fn handle_v2_response<T: EthSpec>(
)?),
)))),
},
Protocol::TxBlobsByRange => {
Ok(Some(RPCResponse::TxBlobsByRange(Box::new(
BlobWrapper::from_ssz_bytes(
decoded_buffer,
)?
))))
},
Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Box::new(
BlobWrapper::from_ssz_bytes(decoded_buffer)?,
)))),
Protocol::BlocksByRoot => match fork_name {
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Altair(SignedBeaconBlockAltair::from_ssz_bytes(

View File

@ -73,9 +73,11 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZSnappy),
],
//FIXME(sean) what should the protocol version be?
OutboundRequest::TxBlobsByRange(_) => vec![
ProtocolId::new(Protocol::TxBlobsByRange, Version::V2, Encoding::SSZSnappy),
],
OutboundRequest::TxBlobsByRange(_) => vec![ProtocolId::new(
Protocol::TxBlobsByRange,
Version::V2,
Encoding::SSZSnappy,
)],
OutboundRequest::BlocksByRoot(_) => vec![
ProtocolId::new(Protocol::BlocksByRoot, Version::V2, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),

View File

@ -20,7 +20,10 @@ use tokio_util::{
codec::Framed,
compat::{Compat, FuturesAsyncReadCompatExt},
};
use types::{BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BlobWrapper, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, SignedBeaconBlock};
use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BlobWrapper, EthSpec,
ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, SignedBeaconBlock,
};
lazy_static! {
// Note: Hardcoding the `EthSpec` type for `SignedBeaconBlock` as min/max values is
@ -288,12 +291,10 @@ impl ProtocolId {
<OldBlocksByRangeRequest as Encode>::ssz_fixed_len(),
<OldBlocksByRangeRequest as Encode>::ssz_fixed_len(),
),
Protocol::TxBlobsByRange => {
RpcLimits::new(
<TxBlobsByRangeRequest as Encode>::ssz_fixed_len(),
<TxBlobsByRangeRequest as Encode>::ssz_fixed_len(),
)
}
Protocol::TxBlobsByRange => RpcLimits::new(
<TxBlobsByRangeRequest as Encode>::ssz_fixed_len(),
<TxBlobsByRangeRequest as Encode>::ssz_fixed_len(),
),
Protocol::BlocksByRoot => {
RpcLimits::new(*BLOCKS_BY_ROOT_REQUEST_MIN, *BLOCKS_BY_ROOT_REQUEST_MAX)
}

View File

@ -151,6 +151,8 @@ impl GossipCache {
pub fn insert(&mut self, topic: GossipTopic, data: Vec<u8>) {
let expire_timeout = match topic.kind() {
GossipKind::BeaconBlock => self.beacon_block,
//FIXME(sean) use its own timeout
GossipKind::Blob => self.beacon_block,
GossipKind::BeaconAggregateAndProof => self.aggregates,
GossipKind::Attestation(_) => self.attestation,
GossipKind::VoluntaryExit => self.voluntary_exit,

View File

@ -9,6 +9,8 @@ use crate::peer_manager::{
use crate::peer_manager::{MIN_OUTBOUND_ONLY_FACTOR, PEER_EXCESS_FACTOR, PRIORITY_PEER_EXCESS};
use crate::service::behaviour::BehaviourEvent;
pub use crate::service::behaviour::Gossipsub;
use crate::rpc::*;
use crate::service::{Context as ServiceContext, METADATA_FILENAME};
use crate::types::{
subnet_from_topic_hash, GossipEncoding, GossipKind, GossipTopic, SnappyTransform, Subnet,
SubnetDiscovery,
@ -42,7 +44,7 @@ use types::{
BlobWrapper, SignedBeaconBlock, SyncSubnetId
};
use crate::rpc::methods::TxBlobsByRangeRequest;
use utils::{build_transport, strip_peer_id, Context as ServiceContext, MAX_CONNECTIONS_PER_PEER};
use utils::{build_transport, strip_peer_id, MAX_CONNECTIONS_PER_PEER};
use self::behaviour::Behaviour;
use self::gossip_cache::GossipCache;

View File

@ -9,7 +9,7 @@ use std::boxed::Box;
use std::io::{Error, ErrorKind};
use std::sync::Arc;
use types::{
Attestation, AttesterSlashing, EthSpec, ForkContext, ForkName, ProposerSlashing,
Attestation, AttesterSlashing, BlobWrapper, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedBeaconBlockMerge, SignedBeaconBlockShanghai, SignedContributionAndProof,
SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
@ -106,6 +106,7 @@ impl<T: EthSpec> PubsubMessage<T> {
pub fn kind(&self) -> GossipKind {
match self {
PubsubMessage::BeaconBlock(_) => GossipKind::BeaconBlock,
PubsubMessage::Blob(_) => GossipKind::Blob,
PubsubMessage::AggregateAndProofAttestation(_) => GossipKind::BeaconAggregateAndProof,
PubsubMessage::Attestation(attestation_data) => {
GossipKind::Attestation(attestation_data.0)
@ -180,6 +181,12 @@ impl<T: EthSpec> PubsubMessage<T> {
};
Ok(PubsubMessage::BeaconBlock(Arc::new(beacon_block)))
}
GossipKind::Blob => {
//FIXME(sean) verify against fork context
let blob =
BlobWrapper::from_ssz_bytes(data).map_err(|e| format!("{:?}", e))?;
Ok(PubsubMessage::Blob(Box::new(blob)))
}
GossipKind::VoluntaryExit => {
let voluntary_exit = SignedVoluntaryExit::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
@ -224,6 +231,7 @@ impl<T: EthSpec> PubsubMessage<T> {
// messages for us.
match &self {
PubsubMessage::BeaconBlock(data) => data.as_ssz_bytes(),
PubsubMessage::Blob(data) => data.as_ssz_bytes(),
PubsubMessage::AggregateAndProofAttestation(data) => data.as_ssz_bytes(),
PubsubMessage::VoluntaryExit(data) => data.as_ssz_bytes(),
PubsubMessage::ProposerSlashing(data) => data.as_ssz_bytes(),
@ -244,6 +252,11 @@ impl<T: EthSpec> std::fmt::Display for PubsubMessage<T> {
block.slot(),
block.message().proposer_index()
),
PubsubMessage::Blob(blob) => write!(
f,
"Tx Blob: slot: {}, beacon_block_root: {}",
blob.beacon_block_slot, blob.beacon_block_root
),
PubsubMessage::AggregateAndProofAttestation(att) => write!(
f,
"Aggregate and Proof: slot: {}, index: {}, aggregator_index: {}",

View File

@ -11,6 +11,8 @@ use crate::Subnet;
pub const TOPIC_PREFIX: &str = "eth2";
pub const SSZ_SNAPPY_ENCODING_POSTFIX: &str = "ssz_snappy";
pub const BEACON_BLOCK_TOPIC: &str = "beacon_block";
//FIXME(sean) check this name
pub const BLOB_TOPIC: &str = "tx_blob";
pub const BEACON_AGGREGATE_AND_PROOF_TOPIC: &str = "beacon_aggregate_and_proof";
pub const BEACON_ATTESTATION_PREFIX: &str = "beacon_attestation_";
pub const VOLUNTARY_EXIT_TOPIC: &str = "voluntary_exit";
@ -47,6 +49,7 @@ pub struct GossipTopic {
pub enum GossipKind {
/// Topic for publishing beacon blocks.
BeaconBlock,
Blob,
/// Topic for publishing aggregate attestations and proofs.
BeaconAggregateAndProof,
/// Topic for publishing raw attestations on a particular subnet.
@ -178,6 +181,7 @@ impl From<GossipTopic> for String {
let kind = match topic.kind {
GossipKind::BeaconBlock => BEACON_BLOCK_TOPIC.into(),
GossipKind::Blob => BLOB_TOPIC.into(),
GossipKind::BeaconAggregateAndProof => BEACON_AGGREGATE_AND_PROOF_TOPIC.into(),
GossipKind::VoluntaryExit => VOLUNTARY_EXIT_TOPIC.into(),
GossipKind::ProposerSlashing => PROPOSER_SLASHING_TOPIC.into(),
@ -206,6 +210,7 @@ impl std::fmt::Display for GossipTopic {
let kind = match self.kind {
GossipKind::BeaconBlock => BEACON_BLOCK_TOPIC.into(),
GossipKind::Blob => BLOB_TOPIC.into(),
GossipKind::BeaconAggregateAndProof => BEACON_AGGREGATE_AND_PROOF_TOPIC.into(),
GossipKind::VoluntaryExit => VOLUNTARY_EXIT_TOPIC.into(),
GossipKind::ProposerSlashing => PROPOSER_SLASHING_TOPIC.into(),

View File

@ -45,6 +45,7 @@ use beacon_chain::{BeaconChain, BeaconChainTypes, GossipVerifiedBlock};
use derivative::Derivative;
use futures::stream::{Stream, StreamExt};
use futures::task::Poll;
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use lighthouse_network::{
rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage},
Client, MessageId, NetworkGlobals, PeerId, PeerRequestId,
@ -60,9 +61,8 @@ use std::time::Duration;
use std::{cmp, collections::HashSet};
use task_executor::TaskExecutor;
use tokio::sync::{mpsc, oneshot};
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use types::{
Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof,
Attestation, AttesterSlashing, BlobWrapper, Hash256, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit, SubnetId,
SyncCommitteeMessage, SyncSubnetId,
};
@ -113,6 +113,9 @@ const MAX_AGGREGATED_ATTESTATION_REPROCESS_QUEUE_LEN: usize = 1_024;
/// before we start dropping them.
const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024;
//FIXME(sean) verify
const MAX_GOSSIP_BLOB_QUEUE_LEN: usize = 1_024;
/// The maximum number of queued `SignedBeaconBlock` objects received prior to their slot (but
/// within acceptable clock disparity) that will be queued before we start dropping them.
const MAX_DELAYED_BLOCK_QUEUE_LEN: usize = 1_024;
@ -187,6 +190,7 @@ pub const GOSSIP_ATTESTATION_BATCH: &str = "gossip_attestation_batch";
pub const GOSSIP_AGGREGATE: &str = "gossip_aggregate";
pub const GOSSIP_AGGREGATE_BATCH: &str = "gossip_aggregate_batch";
pub const GOSSIP_BLOCK: &str = "gossip_block";
pub const GOSSIP_BLOB: &str = "gossip_blob";
pub const DELAYED_IMPORT_BLOCK: &str = "delayed_import_block";
pub const GOSSIP_VOLUNTARY_EXIT: &str = "gossip_voluntary_exit";
pub const GOSSIP_PROPOSER_SLASHING: &str = "gossip_proposer_slashing";
@ -404,6 +408,25 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
}
}
pub fn gossip_tx_blob_block(
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: Box<BlobWrapper<T::EthSpec>>,
seen_timestamp: Duration,
) -> Self {
Self {
drop_during_sync: false,
work: Work::GossipBlob {
message_id,
peer_id,
peer_client,
blob,
seen_timestamp,
},
}
}
/// Create a new `Work` event for some sync committee signature.
pub fn gossip_sync_signature(
message_id: MessageId,
@ -694,6 +717,13 @@ pub enum Work<T: BeaconChainTypes> {
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
},
GossipBlob {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: Box<BlobWrapper<T::EthSpec>>,
seen_timestamp: Duration,
},
DelayedImportBlock {
peer_id: PeerId,
block: Box<GossipVerifiedBlock<T>>,
@ -768,6 +798,7 @@ impl<T: BeaconChainTypes> Work<T> {
Work::GossipAggregate { .. } => GOSSIP_AGGREGATE,
Work::GossipAggregateBatch { .. } => GOSSIP_AGGREGATE_BATCH,
Work::GossipBlock { .. } => GOSSIP_BLOCK,
Work::GossipBlob { .. } => GOSSIP_BLOB,
Work::DelayedImportBlock { .. } => DELAYED_IMPORT_BLOCK,
Work::GossipVoluntaryExit { .. } => GOSSIP_VOLUNTARY_EXIT,
Work::GossipProposerSlashing { .. } => GOSSIP_PROPOSER_SLASHING,
@ -918,6 +949,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
let mut chain_segment_queue = FifoQueue::new(MAX_CHAIN_SEGMENT_QUEUE_LEN);
let mut backfill_chain_segment = FifoQueue::new(MAX_CHAIN_SEGMENT_QUEUE_LEN);
let mut gossip_block_queue = FifoQueue::new(MAX_GOSSIP_BLOCK_QUEUE_LEN);
let mut gossip_blob_queue = FifoQueue::new(MAX_GOSSIP_BLOB_QUEUE_LEN);
let mut delayed_block_queue = FifoQueue::new(MAX_DELAYED_BLOCK_QUEUE_LEN);
let mut status_queue = FifoQueue::new(MAX_STATUS_QUEUE_LEN);
@ -1026,6 +1058,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
// required to verify some attestations.
} else if let Some(item) = gossip_block_queue.pop() {
self.spawn_worker(item, toolbox);
//FIXME(sean)
} else if let Some(item) = gossip_blob_queue.pop() {
self.spawn_worker(item, toolbox);
// Check the aggregates, *then* the unaggregates since we assume that
// aggregates are more valuable to local validators and effectively give us
// more information with less signature verification time.
@ -1232,6 +1267,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
Work::GossipBlock { .. } => {
gossip_block_queue.push(work, work_id, &self.log)
}
Work::GossipBlob { .. } => {
gossip_blob_queue.push(work, work_id, &self.log)
}
Work::DelayedImportBlock { .. } => {
delayed_block_queue.push(work, work_id, &self.log)
}
@ -1302,6 +1340,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL,
gossip_block_queue.len() as i64,
);
//FIXME(sean) blob metrics
metrics::set_gauge(
&metrics::BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL,
rpc_block_queue.len() as i64,

View File

@ -18,9 +18,9 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, EthSpec, Hash256, IndexedAttestation, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId,
Attestation, AttesterSlashing, BlobWrapper, EthSpec, Hash256, IndexedAttestation,
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof,
SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId,
};
use super::{
@ -692,6 +692,20 @@ impl<T: BeaconChainTypes> Worker<T> {
}
}
#[allow(clippy::too_many_arguments)]
pub fn process_gossip_blob(
self,
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: BlobWrapper<T::EthSpec>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
duplicate_cache: DuplicateCache,
seen_duration: Duration,
) {
//FIXME(sean)
}
/// Process the beacon block received from the gossip network and
/// if it passes gossip propagation criteria, tell the network thread to forward it.
///

View File

@ -236,6 +236,14 @@ impl<T: BeaconChainTypes> Router<T> {
block,
);
}
PubsubMessage::Blob(blob) => {
self.processor.on_tx_blob_gossip(
id,
peer_id,
self.network_globals.client(&peer_id),
blob,
);
}
PubsubMessage::VoluntaryExit(exit) => {
debug!(self.log, "Received a voluntary exit"; "peer_id" => %peer_id);
self.processor.on_voluntary_exit_gossip(id, peer_id, exit);

View File

@ -5,7 +5,8 @@ use crate::service::{NetworkMessage, RequestId};
use crate::status::status_message;
use crate::sync::manager::RequestId as SyncId;
use crate::sync::SyncMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use lighthouse_network::rpc::*;
use lighthouse_network::{
Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, Request, Response,
@ -16,8 +17,10 @@ use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::SyncCommitteeMessage;
use tokio::sync::mpsc;
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use types::{Attestation, AttesterSlashing, BlobWrapper, EthSpec, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncSubnetId};
use types::{
Attestation, AttesterSlashing, BlobWrapper, EthSpec, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncSubnetId,
};
/// Processes validated messages from the network. It relays necessary data to the syncing thread
/// and processes blocks from the pubsub network.
@ -291,6 +294,22 @@ impl<T: BeaconChainTypes> Processor<T> {
))
}
pub fn on_tx_blob_gossip(
&mut self,
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: Box<BlobWrapper<T::EthSpec>>,
) {
self.send_beacon_processor_work(BeaconWorkEvent::gossip_tx_blob_block(
message_id,
peer_id,
peer_client,
blob,
timestamp_now(),
))
}
pub fn on_unaggregated_attestation_gossip(
&mut self,
message_id: MessageId,

View File

@ -1,10 +1,10 @@
use crate::{Blob, EthSpec, Hash256, SignedBeaconBlock, Slot};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Encode, Decode};
use ssz::{Decode, Encode};
use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use tree_hash::TreeHash;
use tree_hash_derive::TreeHash;
use ssz::{Decode, Encode};
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq, Default)]
@ -14,7 +14,7 @@ pub struct BlobWrapper<E: EthSpec> {
pub blobs: VariableList<Blob<E::ChunksPerBlob>, E::MaxObjectListSize>,
}
impl <E: EthSpec> BlobWrapper<E> {
impl<E: EthSpec> BlobWrapper<E> {
pub fn empty() -> Self {
Self::default()
}
@ -24,4 +24,4 @@ impl <E: EthSpec> BlobWrapper<E> {
// Max size of variable length `blobs` field
+ (E::max_object_list_size() * <Blob<E::ChunksPerBlob> as Encode>::ssz_fixed_len())
}
}
}

View File

@ -111,6 +111,7 @@ pub use crate::beacon_block_body::{
pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
pub use crate::beacon_state::{BeaconTreeHashCache, Error as BeaconStateError, *};
pub use crate::blob_wrapper::BlobWrapper;
pub use crate::chain_spec::{ChainSpec, Config, Domain};
pub use crate::checkpoint::Checkpoint;
pub use crate::config_and_preset::{
@ -169,7 +170,6 @@ pub use crate::validator_registration_data::*;
pub use crate::validator_subscription::ValidatorSubscription;
pub use crate::voluntary_exit::VoluntaryExit;
use serde_big_array::BigArray;
pub use crate::blob_wrapper::BlobWrapper;
pub type CommitteeIndex = u64;
pub type Hash256 = H256;