mirror of
https://gitlab.com/pulsechaincom/prysm-pulse.git
synced 2024-12-25 12:57:18 +00:00
30b2adc5d6
* add contribution handlers * fix it * fmt * fix more * add clean up * Apply suggestions from code review Co-authored-by: Radosław Kapka <rkapka@wp.pl> * Update beacon-chain/sync/validate_sync_contribution_proof.go Co-authored-by: Radosław Kapka <rkapka@wp.pl> * use validation pipeline * deepsource * fix all tests * clean up * comment Co-authored-by: Radosław Kapka <rkapka@wp.pl> Co-authored-by: Raul Jordan <raul@prysmaticlabs.com>
297 lines
12 KiB
Go
297 lines
12 KiB
Go
package sync
|
|
|
|
import (
|
|
"context"
|
|
|
|
"github.com/libp2p/go-libp2p-core/peer"
|
|
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
|
types "github.com/prysmaticlabs/eth2-types"
|
|
"github.com/prysmaticlabs/prysm/beacon-chain/core/altair"
|
|
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
|
|
p2ptypes "github.com/prysmaticlabs/prysm/beacon-chain/p2p/types"
|
|
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
|
|
"github.com/prysmaticlabs/prysm/shared/bls"
|
|
"github.com/prysmaticlabs/prysm/shared/bytesutil"
|
|
"github.com/prysmaticlabs/prysm/shared/params"
|
|
"github.com/prysmaticlabs/prysm/shared/traceutil"
|
|
"go.opencensus.io/trace"
|
|
)
|
|
|
|
// validateSyncContributionAndProof verifies the aggregated signature and the selection proof is valid before forwarding to the
|
|
// network and downstream services.
|
|
// Gossip Validation Conditions:
|
|
// [IGNORE] The contribution's slot is for the current slot (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance), i.e. contribution.slot == current_slot.
|
|
// [REJECT] The subcommittee index is in the allowed range, i.e. contribution.subcommittee_index < SYNC_COMMITTEE_SUBNET_COUNT.
|
|
// [REJECT] The contribution has participants -- that is, any(contribution.aggregation_bits).
|
|
// [REJECT] contribution_and_proof.selection_proof selects the validator as an aggregator for the slot -- i.e.
|
|
// is_sync_committee_aggregator(contribution_and_proof.selection_proof) returns True.
|
|
// [REJECT] The aggregator's validator index is in the declared subcommittee of the current sync committee -- i.e.
|
|
// state.validators[contribution_and_proof.aggregator_index].pubkey in get_sync_subcommittee_pubkeys(state, contribution.subcommittee_index).
|
|
// [IGNORE] The sync committee contribution is the first valid contribution received for the aggregator with
|
|
// index contribution_and_proof.aggregator_index for the slot contribution.slot and subcommittee index contribution.subcommittee_index
|
|
// (this requires maintaining a cache of size SYNC_COMMITTEE_SIZE for this topic that can be flushed after each slot).
|
|
// [REJECT] The contribution_and_proof.selection_proof is a valid signature of the SyncAggregatorSelectionData derived from
|
|
// the contribution by the validator with index contribution_and_proof.aggregator_index.
|
|
// [REJECT] The aggregator signature, signed_contribution_and_proof.signature, is valid.
|
|
// [REJECT] The aggregate signature is valid for the message beacon_block_root and aggregate pubkey derived from the participation
|
|
// info in aggregation_bits for the subcommittee specified by the contribution.subcommittee_index.
|
|
func (s *Service) validateSyncContributionAndProof(ctx context.Context, pid peer.ID, msg *pubsub.Message) pubsub.ValidationResult {
|
|
ctx, span := trace.StartSpan(ctx, "sync.validateSyncContributionAndProof")
|
|
defer span.End()
|
|
|
|
// Accept the sync committee contribution if the contribution came from itself.
|
|
if pid == s.cfg.P2P.PeerID() {
|
|
return pubsub.ValidationAccept
|
|
}
|
|
|
|
// Ignore the sync committee contribution if the beacon node is syncing.
|
|
if s.cfg.InitialSync.Syncing() {
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
m, err := s.readSyncContributionMessage(msg)
|
|
if err != nil {
|
|
log.WithError(err).Debug("Could not decode message")
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationReject
|
|
}
|
|
|
|
// The contribution's slot is for the current slot (with a `MAXIMUM_GOSSIP_CLOCK_DISPARITY` allowance).
|
|
if err := altair.ValidateSyncMessageTime(m.Message.Contribution.Slot, s.cfg.Chain.GenesisTime(), params.BeaconNetworkConfig().MaximumGossipClockDisparity); err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
// Validate the message's data according to the p2p specification.
|
|
if result := validationPipeline(
|
|
ctx,
|
|
rejectIncorrectSubcommitteeIndex(m),
|
|
rejectEmptyContribution(m),
|
|
s.ignoreSeenSyncContribution(m),
|
|
rejectInvalidAggregator(m),
|
|
s.rejectInvalidIndexInSubCommittee(m),
|
|
s.rejectInvalidSelectionProof(m),
|
|
s.rejectInvalidContributionSignature(m),
|
|
s.rejectInvalidSyncAggregateSignature(m),
|
|
); result != pubsub.ValidationAccept {
|
|
return result
|
|
}
|
|
|
|
s.setSyncContributionIndexSlotSeen(m.Message.Contribution.Slot, m.Message.AggregatorIndex, types.CommitteeIndex(m.Message.Contribution.SubcommitteeIndex))
|
|
|
|
msg.ValidatorData = m
|
|
|
|
return pubsub.ValidationAccept
|
|
}
|
|
|
|
// Parse a sync contribution message from a pubsub message.
|
|
func (s *Service) readSyncContributionMessage(msg *pubsub.Message) (*ethpb.SignedContributionAndProof, error) {
|
|
raw, err := s.decodePubsubMessage(msg)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
m, ok := raw.(*ethpb.SignedContributionAndProof)
|
|
if !ok {
|
|
return nil, errWrongMessage
|
|
}
|
|
if err := altair.ValidateNilSyncContribution(m); err != nil {
|
|
return nil, errNilMessage
|
|
}
|
|
return m, nil
|
|
}
|
|
|
|
func rejectIncorrectSubcommitteeIndex(
|
|
m *ethpb.SignedContributionAndProof,
|
|
) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
_, span := trace.StartSpan(ctx, "sync.rejectIncorrectSubcommitteeIndex")
|
|
defer span.End()
|
|
// The subcommittee index is in the allowed range, i.e. `contribution.subcommittee_index < SYNC_COMMITTEE_SUBNET_COUNT`.
|
|
if m.Message.Contribution.SubcommitteeIndex >= params.BeaconConfig().SyncCommitteeSubnetCount {
|
|
return pubsub.ValidationReject
|
|
}
|
|
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func rejectEmptyContribution(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
bVector := m.Message.Contribution.AggregationBits
|
|
// In the event no bit is set for the
|
|
// sync contribution, we reject the message.
|
|
if bVector.Count() == 0 {
|
|
return pubsub.ValidationReject
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func (s *Service) ignoreSeenSyncContribution(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
seen := s.hasSeenSyncContributionIndexSlot(m.Message.Contribution.Slot, m.Message.AggregatorIndex, types.CommitteeIndex(m.Message.Contribution.SubcommitteeIndex))
|
|
if seen {
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func rejectInvalidAggregator(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
// The `contribution_and_proof.selection_proof` selects the validator as an aggregator for the slot.
|
|
if isAggregator, err := altair.IsSyncCommitteeAggregator(m.Message.SelectionProof); err != nil || !isAggregator {
|
|
return pubsub.ValidationReject
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func (s *Service) rejectInvalidIndexInSubCommittee(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
_, span := trace.StartSpan(ctx, "sync.rejectInvalidIndexInSubCommittee")
|
|
defer span.End()
|
|
// The aggregator's validator index is in the declared subcommittee of the current sync committee.
|
|
committeeIndices, err := s.cfg.Chain.HeadCurrentSyncCommitteeIndices(ctx, m.Message.AggregatorIndex, m.Message.Contribution.Slot)
|
|
if err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
if len(committeeIndices) == 0 {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
isValid := false
|
|
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
|
|
for _, i := range committeeIndices {
|
|
if uint64(i)/subCommitteeSize == m.Message.Contribution.SubcommitteeIndex {
|
|
isValid = true
|
|
break
|
|
}
|
|
}
|
|
if !isValid {
|
|
return pubsub.ValidationReject
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func (s *Service) rejectInvalidSelectionProof(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
_, span := trace.StartSpan(ctx, "sync.rejectInvalidSelectionProof")
|
|
defer span.End()
|
|
// The `contribution_and_proof.selection_proof` is a valid signature of the `SyncAggregatorSelectionData`.
|
|
if err := s.verifySyncSelectionData(ctx, m.Message); err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationReject
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func (s *Service) rejectInvalidContributionSignature(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
_, span := trace.StartSpan(ctx, "sync.rejectInvalidContributionSignature")
|
|
defer span.End()
|
|
// The aggregator signature, `signed_contribution_and_proof.signature`, is valid.
|
|
d, err := s.cfg.Chain.HeadSyncContributionProofDomain(ctx, m.Message.Contribution.Slot)
|
|
if err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
pubkey, err := s.cfg.Chain.HeadValidatorIndexToPublicKey(ctx, m.Message.AggregatorIndex)
|
|
if err != nil {
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
if err := helpers.VerifySigningRoot(m.Message, pubkey[:], m.Signature, d); err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationReject
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
func (s *Service) rejectInvalidSyncAggregateSignature(m *ethpb.SignedContributionAndProof) validationFn {
|
|
return func(ctx context.Context) pubsub.ValidationResult {
|
|
_, span := trace.StartSpan(ctx, "sync.rejectInvalidSyncAggregateSignature")
|
|
defer span.End()
|
|
// The aggregate signature is valid for the message `beacon_block_root` and aggregate pubkey
|
|
// derived from the participation info in `aggregation_bits` for the subcommittee specified by the `contribution.subcommittee_index`.
|
|
activePubkeys := []bls.PublicKey{}
|
|
syncPubkeys, err := s.cfg.Chain.HeadSyncCommitteePubKeys(ctx, m.Message.Contribution.Slot, types.CommitteeIndex(m.Message.Contribution.SubcommitteeIndex))
|
|
if err != nil {
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
bVector := m.Message.Contribution.AggregationBits
|
|
// In the event no bit is set for the
|
|
// sync contribution, we reject the message.
|
|
if bVector.Count() == 0 {
|
|
return pubsub.ValidationReject
|
|
}
|
|
for i, pk := range syncPubkeys {
|
|
if bVector.BitAt(uint64(i)) {
|
|
pubK, err := bls.PublicKeyFromBytes(pk)
|
|
if err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
activePubkeys = append(activePubkeys, pubK)
|
|
}
|
|
}
|
|
sig, err := bls.SignatureFromBytes(m.Message.Contribution.Signature)
|
|
if err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationReject
|
|
}
|
|
d, err := s.cfg.Chain.HeadSyncCommitteeDomain(ctx, m.Message.Contribution.Slot)
|
|
if err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
rawBytes := p2ptypes.SSZBytes(m.Message.Contribution.BlockRoot)
|
|
sigRoot, err := helpers.ComputeSigningRoot(&rawBytes, d)
|
|
if err != nil {
|
|
traceutil.AnnotateError(span, err)
|
|
return pubsub.ValidationIgnore
|
|
}
|
|
verified := sig.Eth2FastAggregateVerify(activePubkeys, sigRoot)
|
|
if !verified {
|
|
return pubsub.ValidationReject
|
|
}
|
|
return pubsub.ValidationAccept
|
|
}
|
|
}
|
|
|
|
// Returns true if the node has received sync contribution for the aggregator with index, slot and subcommittee index.
|
|
func (s *Service) hasSeenSyncContributionIndexSlot(slot types.Slot, aggregatorIndex types.ValidatorIndex, subComIdx types.CommitteeIndex) bool {
|
|
s.seenSyncContributionLock.RLock()
|
|
defer s.seenSyncContributionLock.RUnlock()
|
|
|
|
b := append(bytesutil.Bytes32(uint64(aggregatorIndex)), bytesutil.Bytes32(uint64(slot))...)
|
|
b = append(b, bytesutil.Bytes32(uint64(subComIdx))...)
|
|
_, seen := s.seenSyncContributionCache.Get(string(b))
|
|
return seen
|
|
}
|
|
|
|
// Set sync contributor's aggregate index, slot and subcommittee index as seen.
|
|
func (s *Service) setSyncContributionIndexSlotSeen(slot types.Slot, aggregatorIndex types.ValidatorIndex, subComIdx types.CommitteeIndex) {
|
|
s.seenSyncContributionLock.Lock()
|
|
defer s.seenSyncContributionLock.Unlock()
|
|
b := append(bytesutil.Bytes32(uint64(aggregatorIndex)), bytesutil.Bytes32(uint64(slot))...)
|
|
b = append(b, bytesutil.Bytes32(uint64(subComIdx))...)
|
|
s.seenSyncContributionCache.Add(string(b), true)
|
|
}
|
|
|
|
// verifySyncSelectionData verifies that the provided sync contribution has a valid
|
|
// selection proof.
|
|
func (s *Service) verifySyncSelectionData(ctx context.Context, m *ethpb.ContributionAndProof) error {
|
|
selectionData := ðpb.SyncAggregatorSelectionData{Slot: m.Contribution.Slot, SubcommitteeIndex: uint64(m.Contribution.SubcommitteeIndex)}
|
|
domain, err := s.cfg.Chain.HeadSyncSelectionProofDomain(ctx, m.Contribution.Slot)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
pubkey, err := s.cfg.Chain.HeadValidatorIndexToPublicKey(ctx, m.AggregatorIndex)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return helpers.VerifySigningRoot(selectionData, pubkey[:], m.SelectionProof, domain)
|
|
}
|