Skip to content

Commit

Permalink
Data column gossip validation and error handling (#6181)
Browse files Browse the repository at this point in the history
* Add gossip verification and error handling.

* Merge branch 'unstable' into das-gossip-validation

* Add inclusion proof verification and some renames for consistency
  • Loading branch information
jimmygchen authored Jul 29, 2024
1 parent a3b1ef3 commit 19b3ab3
Show file tree
Hide file tree
Showing 7 changed files with 494 additions and 59 deletions.
6 changes: 4 additions & 2 deletions beacon_node/beacon_chain/src/beacon_chain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,8 @@ use crate::observed_aggregates::{
use crate::observed_attesters::{
ObservedAggregators, ObservedAttesters, ObservedSyncAggregators, ObservedSyncContributors,
};
use crate::observed_blob_sidecars::ObservedBlobSidecars;
use crate::observed_block_producers::ObservedBlockProducers;
use crate::observed_data_sidecars::ObservedDataSidecars;
use crate::observed_operations::{ObservationOutcome, ObservedOperations};
use crate::observed_slashable::ObservedSlashable;
use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT};
Expand Down Expand Up @@ -415,7 +415,9 @@ pub struct BeaconChain<T: BeaconChainTypes> {
/// Maintains a record of which validators have proposed blocks for each slot.
pub observed_block_producers: RwLock<ObservedBlockProducers<T::EthSpec>>,
/// Maintains a record of blob sidecars seen over the gossip network.
pub observed_blob_sidecars: RwLock<ObservedBlobSidecars<T::EthSpec>>,
pub observed_blob_sidecars: RwLock<ObservedDataSidecars<BlobSidecar<T::EthSpec>>>,
/// Maintains a record of column sidecars seen over the gossip network.
pub observed_column_sidecars: RwLock<ObservedDataSidecars<DataColumnSidecar<T::EthSpec>>>,
/// Maintains a record of slashable message seen over the gossip network or RPC.
pub observed_slashable: RwLock<ObservedSlashable<T::EthSpec>>,
/// Maintains a record of which validators have submitted voluntary exits.
Expand Down
4 changes: 3 additions & 1 deletion beacon_node/beacon_chain/src/builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ use crate::graffiti_calculator::{GraffitiCalculator, GraffitiOrigin};
use crate::head_tracker::HeadTracker;
use crate::light_client_server_cache::LightClientServerCache;
use crate::migrate::{BackgroundMigrator, MigratorConfig};
use crate::observed_data_sidecars::ObservedDataSidecars;
use crate::persisted_beacon_chain::PersistedBeaconChain;
use crate::shuffling_cache::{BlockShufflingIds, ShufflingCache};
use crate::validator_monitor::{ValidatorMonitor, ValidatorMonitorConfig};
Expand Down Expand Up @@ -917,7 +918,8 @@ where
observed_sync_aggregators: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_block_producers: <_>::default(),
observed_blob_sidecars: <_>::default(),
observed_column_sidecars: RwLock::new(ObservedDataSidecars::new(self.spec.clone())),
observed_blob_sidecars: RwLock::new(ObservedDataSidecars::new(self.spec.clone())),
observed_slashable: <_>::default(),
observed_voluntary_exits: <_>::default(),
observed_proposer_slashings: <_>::default(),
Expand Down
323 changes: 318 additions & 5 deletions beacon_node/beacon_chain/src/data_column_verification.rs
Original file line number Diff line number Diff line change
@@ -1,13 +1,21 @@
use crate::block_verification::{process_block_slash_info, BlockSlashInfo};
use crate::block_verification::{
cheap_state_advance_to_obtain_committees, get_validator_pubkey_cache, process_block_slash_info,
BlockSlashInfo,
};
use crate::{BeaconChain, BeaconChainError, BeaconChainTypes};
use derivative::Derivative;
use fork_choice::ProtoBlock;
use kzg::{Error as KzgError, Kzg};
use proto_array::Block;
use slasher::test_utils::E;
use slog::debug;
use slot_clock::SlotClock;
use ssz_derive::{Decode, Encode};
use std::sync::Arc;
use types::data_column_sidecar::{ColumnIndex, DataColumnIdentifier};
use types::{
BeaconStateError, DataColumnSidecar, EthSpec, Hash256, RuntimeVariableList,
SignedBeaconBlockHeader, Slot,
BeaconStateError, ChainSpec, DataColumnSidecar, DataColumnSubnetId, EthSpec, Hash256,
RuntimeVariableList, SignedBeaconBlockHeader, Slot,
};

/// An error occurred while validating a gossip data column.
Expand Down Expand Up @@ -54,6 +62,75 @@ pub enum GossipDataColumnError {
///
/// The data column sidecar is invalid and the peer is faulty.
InvalidKzgProof(kzg::Error),
/// The column was gossiped over an incorrect subnet.
///
/// ## Peer scoring
///
/// The column is invalid or the peer is faulty.
InvalidSubnetId { received: u64, expected: u64 },
/// The column sidecar is from a slot that is later than the current slot (with respect to the
/// gossip clock disparity).
///
/// ## Peer scoring
///
/// Assuming the local clock is correct, the peer has sent an invalid message.
FutureSlot {
message_slot: Slot,
latest_permissible_slot: Slot,
},
/// The sidecar corresponds to a slot older than the finalized head slot.
///
/// ## Peer scoring
///
/// It's unclear if this column is valid, but this column is for a finalized slot and is
/// therefore useless to us.
PastFinalizedSlot {
column_slot: Slot,
finalized_slot: Slot,
},
/// The pubkey cache timed out.
///
/// ## Peer scoring
///
/// The column sidecar may be valid, this is an internal error.
PubkeyCacheTimeout,
/// The proposer index specified in the sidecar does not match the locally computed
/// proposer index.
///
/// ## Peer scoring
///
/// The column is invalid and the peer is faulty.
ProposerIndexMismatch { sidecar: usize, local: usize },
/// The provided columns's parent block is unknown.
///
/// ## Peer scoring
///
/// We cannot process the columns without validating its parent, the peer isn't necessarily faulty.
ParentUnknown { parent_root: Hash256 },
/// The column conflicts with finalization, no need to propagate.
///
/// ## Peer scoring
///
/// It's unclear if this column is valid, but it conflicts with finality and shouldn't be
/// imported.
NotFinalizedDescendant { block_parent_root: Hash256 },
/// Invalid kzg commitment inclusion proof
///
/// ## Peer scoring
///
/// The column sidecar is invalid and the peer is faulty
InvalidInclusionProof,
/// A column has already been seen for the given `(sidecar.block_root, sidecar.index)` tuple
/// over gossip or no gossip sources.
///
/// ## Peer scoring
///
/// The peer isn't faulty, but we do not forward it over gossip.
PriorKnown {
proposer: u64,
slot: Slot,
index: ColumnIndex,
},
}

impl From<BeaconChainError> for GossipDataColumnError {
Expand Down Expand Up @@ -183,18 +260,254 @@ where

pub fn validate_data_column_sidecar_for_gossip<T: BeaconChainTypes>(
data_column: Arc<DataColumnSidecar<T::EthSpec>>,
_subnet: u64,
subnet: u64,
chain: &BeaconChain<T>,
) -> Result<GossipVerifiedDataColumn<T>, GossipDataColumnError> {
// TODO(das): implement gossip verification
let column_slot = data_column.slot();

verify_index_matches_subnet(&data_column, subnet, &chain.spec)?;
verify_sidecar_not_from_future_slot(chain, column_slot)?;
verify_slot_greater_than_latest_finalized_slot(chain, column_slot)?;
verify_is_first_sidecar(chain, &data_column)?;
verify_column_inclusion_proof(&data_column)?;
let parent_block = verify_parent_block_and_finalized_descendant(data_column.clone(), chain)?;
verify_slot_higher_than_parent(&parent_block, column_slot)?;
verify_proposer_and_signature(&data_column, &parent_block, chain)?;
let kzg = chain
.kzg
.clone()
.ok_or(GossipDataColumnError::KzgNotInitialized)?;
let kzg_verified_data_column = verify_kzg_for_data_column(data_column.clone(), &kzg)
.map_err(GossipDataColumnError::InvalidKzgProof)?;

chain
.observed_slashable
.write()
.observe_slashable(
column_slot,
data_column.block_proposer_index(),
data_column.block_root(),
)
.map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))?;

Ok(GossipVerifiedDataColumn {
block_root: data_column.block_root(),
data_column: kzg_verified_data_column,
})
}

// Verify that this is the first column sidecar received for the tuple:
// (block_header.slot, block_header.proposer_index, column_sidecar.index)
fn verify_is_first_sidecar<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
data_column: &DataColumnSidecar<T::EthSpec>,
) -> Result<(), GossipDataColumnError> {
if chain
.observed_column_sidecars
.read()
.proposer_is_known(data_column)
.map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))?
{
return Err(GossipDataColumnError::PriorKnown {
proposer: data_column.block_proposer_index(),
slot: data_column.slot(),
index: data_column.index,
});
}
Ok(())
}

fn verify_column_inclusion_proof<E: EthSpec>(
data_column: &DataColumnSidecar<E>,
) -> Result<(), GossipDataColumnError> {
if !data_column.verify_inclusion_proof() {
return Err(GossipDataColumnError::InvalidInclusionProof);
}
Ok(())
}

fn verify_slot_higher_than_parent(
parent_block: &Block,
data_column_slot: Slot,
) -> Result<(), GossipDataColumnError> {
if parent_block.slot >= data_column_slot {
return Err(GossipDataColumnError::IsNotLaterThanParent {
data_column_slot,
parent_slot: parent_block.slot,
});
}
Ok(())
}

fn verify_parent_block_and_finalized_descendant<T: BeaconChainTypes>(
data_column: Arc<DataColumnSidecar<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<ProtoBlock, GossipDataColumnError> {
let fork_choice = chain.canonical_head.fork_choice_read_lock();

// We have already verified that the column is past finalization, so we can
// just check fork choice for the block's parent.
let block_parent_root = data_column.block_parent_root();
let Some(parent_block) = fork_choice.get_block(&block_parent_root) else {
return Err(GossipDataColumnError::ParentUnknown {
parent_root: block_parent_root,
});
};

// Do not process a column that does not descend from the finalized root.
// We just loaded the parent_block, so we can be sure that it exists in fork choice.
if !fork_choice.is_finalized_checkpoint_or_descendant(block_parent_root) {
return Err(GossipDataColumnError::NotFinalizedDescendant { block_parent_root });
}

Ok(parent_block)
}

fn verify_proposer_and_signature<T: BeaconChainTypes>(
data_column: &DataColumnSidecar<T::EthSpec>,
parent_block: &ProtoBlock,
chain: &BeaconChain<T>,
) -> Result<(), GossipDataColumnError> {
let column_slot = data_column.slot();
let column_epoch = column_slot.epoch(E::slots_per_epoch());
let column_index = data_column.index;
let block_root = data_column.block_root();
let block_parent_root = data_column.block_parent_root();

let proposer_shuffling_root =
if parent_block.slot.epoch(T::EthSpec::slots_per_epoch()) == column_epoch {
parent_block
.next_epoch_shuffling_id
.shuffling_decision_block
} else {
parent_block.root
};

let proposer_opt = chain
.beacon_proposer_cache
.lock()
.get_slot::<T::EthSpec>(proposer_shuffling_root, column_slot);

let (proposer_index, fork) = if let Some(proposer) = proposer_opt {
(proposer.index, proposer.fork)
} else {
debug!(
chain.log,
"Proposer shuffling cache miss for column verification";
"block_root" => %block_root,
"index" => %column_index,
);
let (parent_state_root, mut parent_state) = chain
.store
.get_advanced_hot_state(block_parent_root, column_slot, parent_block.state_root)
.map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))?
.ok_or_else(|| {
BeaconChainError::DBInconsistent(format!(
"Missing state for parent block {block_parent_root:?}",
))
})?;

let state = cheap_state_advance_to_obtain_committees::<_, GossipDataColumnError>(
&mut parent_state,
Some(parent_state_root),
column_slot,
&chain.spec,
)?;

let proposers = state.get_beacon_proposer_indices(&chain.spec)?;
let proposer_index = *proposers
.get(column_slot.as_usize() % T::EthSpec::slots_per_epoch() as usize)
.ok_or_else(|| BeaconChainError::NoProposerForSlot(column_slot))?;

// Prime the proposer shuffling cache with the newly-learned value.
chain.beacon_proposer_cache.lock().insert(
column_epoch,
proposer_shuffling_root,
proposers,
state.fork(),
)?;
(proposer_index, state.fork())
};

// Signature verify the signed block header.
let signature_is_valid = {
let pubkey_cache = get_validator_pubkey_cache(chain)
.map_err(|_| GossipDataColumnError::PubkeyCacheTimeout)?;

let pubkey = pubkey_cache
.get(proposer_index)
.ok_or_else(|| GossipDataColumnError::UnknownValidator(proposer_index as u64))?;
let signed_block_header = &data_column.signed_block_header;
signed_block_header.verify_signature::<T::EthSpec>(
pubkey,
&fork,
chain.genesis_validators_root,
&chain.spec,
)
};

if !signature_is_valid {
return Err(GossipDataColumnError::ProposalSignatureInvalid);
}

let column_proposer_index = data_column.block_proposer_index();
if proposer_index != column_proposer_index as usize {
return Err(GossipDataColumnError::ProposerIndexMismatch {
sidecar: column_proposer_index as usize,
local: proposer_index,
});
}

Ok(())
}

fn verify_index_matches_subnet<E: EthSpec>(
data_column: &DataColumnSidecar<E>,
subnet: u64,
spec: &ChainSpec,
) -> Result<(), GossipDataColumnError> {
let expected_subnet: u64 =
DataColumnSubnetId::from_column_index::<E>(data_column.index as usize, spec).into();
if expected_subnet != subnet {
return Err(GossipDataColumnError::InvalidSubnetId {
received: subnet,
expected: expected_subnet,
});
}
Ok(())
}

fn verify_slot_greater_than_latest_finalized_slot<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
column_slot: Slot,
) -> Result<(), GossipDataColumnError> {
let latest_finalized_slot = chain
.head()
.finalized_checkpoint()
.epoch
.start_slot(T::EthSpec::slots_per_epoch());
if column_slot <= latest_finalized_slot {
return Err(GossipDataColumnError::PastFinalizedSlot {
column_slot,
finalized_slot: latest_finalized_slot,
});
}
Ok(())
}

fn verify_sidecar_not_from_future_slot<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
column_slot: Slot,
) -> Result<(), GossipDataColumnError> {
let latest_permissible_slot = chain
.slot_clock
.now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity())
.ok_or(BeaconChainError::UnableToReadSlot)?;
if column_slot > latest_permissible_slot {
return Err(GossipDataColumnError::FutureSlot {
message_slot: column_slot,
latest_permissible_slot,
});
}
Ok(())
}
Loading

0 comments on commit 19b3ab3

Please sign in to comment.