Compare commits

...

14 Commits

Author SHA1 Message Date
Manu NALEPA
90ef02c172 Add changelog. 2025-10-10 16:27:14 +02:00
Manu NALEPA
f389a45092 rejectInvalidSelectionProof: Do not reject on internal error. 2025-10-10 16:27:14 +02:00
Manu NALEPA
e62be1f646 validateProposerSlashing: Do not reject on internal error 2025-10-10 16:27:14 +02:00
Manu NALEPA
b0e947dd05 Data column sidecars: Do not reject if internal error. 2025-10-10 16:27:14 +02:00
Manu NALEPA
67d512fb3a Do not reject on NewRODataColumn failure. 2025-10-10 16:27:14 +02:00
Manu NALEPA
fd4c210106 validateBlob: Reject only if real validation error, else ignore. 2025-10-10 16:27:14 +02:00
Manu NALEPA
15f2f3ea9c Blob verification: Stop returning verification error when internal error. 2025-10-10 16:27:14 +02:00
Manu NALEPA
f4bb33eec9 Sidecar verification errors: Always join with ErrSidecarInvalid. 2025-10-10 16:27:14 +02:00
Manu NALEPA
2731720a97 Rename blob specific errors into sidecars errors, because they are both used in blob sidecars and data column sidecars. 2025-10-10 16:27:14 +02:00
Manu NALEPA
6c26435fca Ignore if NewROBlob fails instead of rejecting. 2025-10-10 16:27:14 +02:00
Manu NALEPA
5abd3b4f3f validateSelectionIndex: Reject or ignore depending on the error. 2025-10-10 16:27:13 +02:00
Manu NALEPA
d9ac483730 IsAggregator: Remove unused error. 2025-10-10 16:27:13 +02:00
Manu NALEPA
c76ca41bb1 validateWithBatchVerifier: Move a reject into an ignore.
Rationale:
The only way `set.Verify` returns an error is if there is a mismatch in `VerifyMultipleSignatures` arguments sizes, which is not the reponsability of the peer.
==> It is unfair to penalize the peer in such a case.
2025-10-08 10:15:15 +02:00
Manu NALEPA
f1ea5e6b61 validateWithBatchVerifier: Use Trace and WithField instead of Tracef. 2025-10-08 10:08:57 +02:00
25 changed files with 201 additions and 174 deletions

View File

@@ -58,14 +58,14 @@ func ValidateSlotTargetEpoch(data *ethpb.AttestationData) error {
// committee = get_beacon_committee(state, slot, index)
// modulo = max(1, len(committee) // TARGET_AGGREGATORS_PER_COMMITTEE)
// return bytes_to_uint64(hash(slot_signature)[0:8]) % modulo == 0
func IsAggregator(committeeCount uint64, slotSig []byte) (bool, error) {
func IsAggregator(committeeCount uint64, slotSig []byte) bool {
modulo := uint64(1)
if committeeCount/params.BeaconConfig().TargetAggregatorsPerCommittee > 1 {
modulo = committeeCount / params.BeaconConfig().TargetAggregatorsPerCommittee
}
b := hash.Hash(slotSig)
return binary.LittleEndian.Uint64(b[:8])%modulo == 0, nil
return binary.LittleEndian.Uint64(b[:8])%modulo == 0
}
// ComputeSubnetForAttestation returns the subnet for which the provided attestation will be broadcasted to.

View File

@@ -25,8 +25,7 @@ func TestAttestation_IsAggregator(t *testing.T) {
committee, err := helpers.BeaconCommitteeFromState(t.Context(), beaconState, 0, 0)
require.NoError(t, err)
sig := privKeys[0].Sign([]byte{'A'})
agg, err := helpers.IsAggregator(uint64(len(committee)), sig.Marshal())
require.NoError(t, err)
agg := helpers.IsAggregator(uint64(len(committee)), sig.Marshal())
assert.Equal(t, true, agg, "Wanted aggregator true")
})
@@ -40,8 +39,7 @@ func TestAttestation_IsAggregator(t *testing.T) {
committee, err := helpers.BeaconCommitteeFromState(t.Context(), beaconState, 0, 0)
require.NoError(t, err)
sig := privKeys[0].Sign([]byte{'A'})
agg, err := helpers.IsAggregator(uint64(len(committee)), sig.Marshal())
require.NoError(t, err)
agg := helpers.IsAggregator(uint64(len(committee)), sig.Marshal())
assert.Equal(t, false, agg, "Wanted aggregator false")
})
}

View File

@@ -132,11 +132,7 @@ func (vs *Server) processAggregateSelection(ctx context.Context, req *ethpb.Aggr
}
// Check if the validator is an aggregator
isAggregator, err := helpers.IsAggregator(uint64(len(committee)), req.SlotSignature)
if err != nil {
return 0, 0, status.Errorf(codes.Internal, "Could not get aggregator status: %v", err)
}
if !isAggregator {
if !helpers.IsAggregator(uint64(len(committee)), req.SlotSignature) {
return 0, 0, status.Errorf(codes.InvalidArgument, "Validator is not an aggregator")
}

View File

@@ -93,12 +93,12 @@ func (s *Service) validateWithBatchVerifier(ctx context.Context, message string,
// If verification fails we fallback to individual verification
// of each signature set.
if resErr != nil {
log.WithError(resErr).Tracef("Could not perform batch verification of %s", message)
log.WithError(resErr).WithField("message", message).Trace("Could not perform batch verification")
verified, err := set.Verify()
if err != nil {
verErr := errors.Wrapf(err, "Could not verify %s", message)
tracing.AnnotateError(span, verErr)
return pubsub.ValidationReject, verErr
return pubsub.ValidationIgnore, verErr
}
if !verified {
verErr := errors.Errorf("Verification of %s failed", message)

View File

@@ -346,7 +346,7 @@ func (q *blocksQueue) onDataReceivedEvent(ctx context.Context) eventHandlerFn {
q.downscorePeer(response.blocksFrom, "invalidBlocks")
}
if errors.Is(response.err, verification.ErrBlobInvalid) {
if errors.Is(response.err, verification.ErrSidecarInvalid) {
q.downscorePeer(response.blobsFrom, "invalidBlobs")
}

View File

@@ -130,7 +130,7 @@ func TestOnDataReceivedDownscore(t *testing.T) {
},
{
name: "invalid blob",
err: errors.Wrap(verification.ErrBlobInvalid, "test"),
err: errors.Wrap(verification.ErrSidecarInvalid, "test"),
downPeer: testDownscoreBlob,
},
{
@@ -149,7 +149,7 @@ func TestOnDataReceivedDownscore(t *testing.T) {
err: c.err,
}
if c.downPeer == testDownscoreBlob {
require.Equal(t, true, verification.IsBlobValidationFailure(c.err))
require.Equal(t, true, verification.IsSidecarValidationFailure(c.err))
}
ctx := t.Context()
p2p := p2pt.NewTestP2P(t)

View File

@@ -463,7 +463,7 @@ func isPunishableError(err error) bool {
// updatePeerScorerStats adjusts monitored metrics for a peer.
func (s *Service) updatePeerScorerStats(data *blocksQueueFetchedData, count uint64, err error) {
if isPunishableError(err) {
if verification.IsBlobValidationFailure(err) {
if verification.IsSidecarValidationFailure(err) {
s.downscorePeer(data.blobsFrom, "invalidBlobs")
} else {
s.downscorePeer(data.blocksFrom, "invalidBlocks")

View File

@@ -35,14 +35,14 @@ var errBlobUnmarshal = errors.New("Could not unmarshal chunk-encoded blob")
var (
// ErrInvalidFetchedData is used to signal that an error occurred which should result in peer downscoring.
ErrInvalidFetchedData = errors.New("invalid data returned from peer")
errBlobIndexOutOfBounds = errors.Wrap(verification.ErrBlobInvalid, "blob index out of range")
errMaxRequestBlobSidecarsExceeded = errors.Wrap(verification.ErrBlobInvalid, "peer exceeded req blob chunk tx limit")
errChunkResponseSlotNotAsc = errors.Wrap(verification.ErrBlobInvalid, "blob slot not higher than previous block root")
errChunkResponseIndexNotAsc = errors.Wrap(verification.ErrBlobInvalid, "blob indices for a block must start at 0 and increase by 1")
errUnrequested = errors.Wrap(verification.ErrBlobInvalid, "received BlobSidecar in response that was not requested")
errBlobResponseOutOfBounds = errors.Wrap(verification.ErrBlobInvalid, "received BlobSidecar with slot outside BlobSidecarsByRangeRequest bounds")
errChunkResponseBlockMismatch = errors.Wrap(verification.ErrBlobInvalid, "blob block details do not match")
errChunkResponseParentMismatch = errors.Wrap(verification.ErrBlobInvalid, "parent root for response element doesn't match previous element root")
errBlobIndexOutOfBounds = errors.Wrap(verification.ErrSidecarInvalid, "blob index out of range")
errMaxRequestBlobSidecarsExceeded = errors.Wrap(verification.ErrSidecarInvalid, "peer exceeded req blob chunk tx limit")
errChunkResponseSlotNotAsc = errors.Wrap(verification.ErrSidecarInvalid, "blob slot not higher than previous block root")
errChunkResponseIndexNotAsc = errors.Wrap(verification.ErrSidecarInvalid, "blob indices for a block must start at 0 and increase by 1")
errUnrequested = errors.Wrap(verification.ErrSidecarInvalid, "received BlobSidecar in response that was not requested")
errBlobResponseOutOfBounds = errors.Wrap(verification.ErrSidecarInvalid, "received BlobSidecar with slot outside BlobSidecarsByRangeRequest bounds")
errChunkResponseBlockMismatch = errors.Wrap(verification.ErrSidecarInvalid, "blob block details do not match")
errChunkResponseParentMismatch = errors.Wrap(verification.ErrSidecarInvalid, "parent root for response element doesn't match previous element root")
errDataColumnChunkedReadFailure = errors.New("failed to read stream of chunk-encoded data columns")
errMaxRequestDataColumnSidecarsExceeded = errors.New("count of requested data column sidecars exceeds MAX_REQUEST_DATA_COLUMN_SIDECARS")
errMaxResponseDataColumnSidecarsExceeded = errors.New("peer returned more data column sidecars than requested")

View File

@@ -884,7 +884,7 @@ func TestSendBlobsByRangeRequest(t *testing.T) {
}
func TestErrInvalidFetchedDataDistinction(t *testing.T) {
require.Equal(t, false, errors.Is(ErrInvalidFetchedData, verification.ErrBlobInvalid))
require.Equal(t, false, errors.Is(ErrInvalidFetchedData, verification.ErrSidecarInvalid))
}
func TestSendDataColumnSidecarsByRangeRequest(t *testing.T) {

View File

@@ -26,6 +26,8 @@ import (
"github.com/pkg/errors"
)
var errInvalidAggregator = errors.New("invalid aggregator")
// validateAggregateAndProof verifies the aggregated signature and the selection proof is valid before forwarding to the
// network and downstream services.
func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
@@ -212,11 +214,16 @@ func (s *Service) validateAggregatedAtt(ctx context.Context, signed ethpb.Signed
aggregatorIndex,
aggregateAndProof.GetSelectionProof(),
)
if errors.Is(err, errInvalidAggregator) {
attBadSelectionProofCount.Inc()
return pubsub.ValidationReject, nil
}
if err != nil {
wrappedErr := errors.Wrapf(err, "could not validate selection for validator %d", aggregateAndProof.GetAggregatorIndex())
tracing.AnnotateError(span, wrappedErr)
attBadSelectionProofCount.Inc()
return pubsub.ValidationReject, wrappedErr
return pubsub.ValidationIgnore, wrappedErr
}
// Verify selection signature, aggregator signature and attestation signature are valid.
@@ -316,12 +323,8 @@ func validateSelectionIndex(
_, span := trace.StartSpan(ctx, "sync.validateSelectionIndex")
defer span.End()
aggregator, err := helpers.IsAggregator(uint64(len(committee)), proof)
if err != nil {
return nil, err
}
if !aggregator {
return nil, fmt.Errorf("validator is not an aggregator for slot %d", slot)
if !helpers.IsAggregator(uint64(len(committee)), proof) {
return nil, errInvalidAggregator
}
domain := params.BeaconConfig().DomainSelectionProof
@@ -329,23 +332,26 @@ func validateSelectionIndex(
v, err := bs.ValidatorAtIndexReadOnly(validatorIndex)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "validator at index read only")
}
pk := v.PublicKey()
publicKey, err := bls.PublicKeyFromBytes(pk[:])
if err != nil {
return nil, err
return nil, errors.Wrap(err, "public key from bytes")
}
d, err := signing.Domain(bs.Fork(), epoch, domain, bs.GenesisValidatorsRoot())
if err != nil {
return nil, err
return nil, errors.Wrap(err, "domain")
}
sszUint := primitives.SSZUint64(slot)
root, err := signing.ComputeSigningRoot(&sszUint, d)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "compute signing root")
}
return &bls.SignatureBatch{
Signatures: [][]byte{proof},
PublicKeys: []bls.PublicKey{publicKey},

View File

@@ -188,8 +188,7 @@ func TestVerifySelection_NotAnAggregator(t *testing.T) {
committee, err := helpers.BeaconCommitteeFromState(ctx, beaconState, data.Slot, data.CommitteeIndex)
require.NoError(t, err)
_, err = validateSelectionIndex(ctx, beaconState, data.Slot, committee, 0, sig.Marshal())
wanted := "validator is not an aggregator for slot"
assert.ErrorContains(t, wanted, err)
require.ErrorIs(t, err, errInvalidAggregator)
}
func TestValidateAggregateAndProof_NoBlock(t *testing.T) {

View File

@@ -50,12 +50,12 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
}
blob, err := blocks.NewROBlob(bpb)
if err != nil {
return pubsub.ValidationReject, errors.Wrap(err, "roblob conversion failure")
return pubsub.ValidationIgnore, errors.Wrap(err, "roblob conversion failure")
}
vf := s.newBlobVerifier(blob, verification.GossipBlobSidecarRequirements)
if err := vf.BlobIndexInBounds(); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
// [REJECT] The sidecar is for the correct subnet -- i.e. compute_subnet_for_blob_sidecar(sidecar.index) == subnet_id.
@@ -94,32 +94,32 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
}
if err := vf.SidecarParentValid(s.hasBadBlock); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
if err := vf.ValidProposerSignature(ctx); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
if err := vf.SidecarParentSlotLower(); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
if err := vf.SidecarDescendsFromFinalized(); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
if err := vf.SidecarInclusionProven(); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
if err := vf.SidecarKzgProofVerified(); err != nil {
saveInvalidBlobToTemp(blob)
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
if err := vf.SidecarProposerExpected(ctx); err != nil {
return pubsub.ValidationReject, err
return rejectOrIgnore(err)
}
fields := blobFields(blob)
@@ -195,3 +195,12 @@ func saveInvalidBlobToTemp(b blocks.ROBlob) {
log.WithError(err).Error("Failed to write to disk")
}
}
// rejectOrIgnore returns ValidationReject if the error is a verification.ErrSidecarInvalid, otherwise
// it returns ValidationIgnore.
func rejectOrIgnore(err error) (pubsub.ValidationResult, error) {
if errors.Is(err, verification.ErrSidecarInvalid) {
return pubsub.ValidationReject, err
}
return pubsub.ValidationIgnore, err
}

View File

@@ -2,6 +2,7 @@ package sync
import (
"bytes"
"errors"
"reflect"
"testing"
"time"
@@ -23,7 +24,6 @@ import (
"github.com/OffchainLabs/prysm/v6/testing/util"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/pkg/errors"
)
func TestValidateBlob_FromSelf(t *testing.T) {
@@ -170,92 +170,93 @@ func TestValidateBlob_InvalidTopicIndex(t *testing.T) {
}
func TestValidateBlob_ErrorPathsWithMock(t *testing.T) {
randomErr := errors.Join(verification.ErrSidecarInvalid, errors.New("random error"))
tests := []struct {
name string
error error
verifier verification.NewBlobVerifier
result pubsub.ValidationResult
}{
{
error: errors.New("blob index out of bound"),
name: "blob index out of bound",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrBlobIndexInBounds: errors.New("blob index out of bound")}
return &verification.MockBlobVerifier{ErrBlobIndexInBounds: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("slot too early"),
name: "slot too early",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSlotTooEarly: errors.New("slot too early")}
return &verification.MockBlobVerifier{ErrSlotTooEarly: randomErr}
},
result: pubsub.ValidationIgnore,
},
{
error: errors.New("slot above finalized"),
name: "slot above finalized",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSlotAboveFinalized: errors.New("slot above finalized")}
return &verification.MockBlobVerifier{ErrSlotAboveFinalized: randomErr}
},
result: pubsub.ValidationIgnore,
},
{
error: errors.New("valid proposer signature"),
name: "valid proposer signature",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrValidProposerSignature: errors.New("valid proposer signature")}
return &verification.MockBlobVerifier{ErrValidProposerSignature: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("sidecar parent seen"),
name: "sidecar parent seen",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarParentSeen: errors.New("sidecar parent seen")}
return &verification.MockBlobVerifier{ErrSidecarParentSeen: randomErr}
},
result: pubsub.ValidationIgnore,
},
{
error: errors.New("sidecar parent valid"),
name: "sidecar parent valid",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarParentValid: errors.New("sidecar parent valid")}
return &verification.MockBlobVerifier{ErrSidecarParentValid: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("sidecar parent slot lower"),
name: "sidecar parent slot lower",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarParentSlotLower: errors.New("sidecar parent slot lower")}
return &verification.MockBlobVerifier{ErrSidecarParentSlotLower: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("descends from finalized"),
name: "descends from finalized",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarDescendsFromFinalized: errors.New("descends from finalized")}
return &verification.MockBlobVerifier{ErrSidecarDescendsFromFinalized: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("inclusion proven"),
name: "inclusion proven",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarInclusionProven: errors.New("inclusion proven")}
return &verification.MockBlobVerifier{ErrSidecarInclusionProven: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("kzg proof verified"),
name: "kzg proof verified",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarKzgProofVerified: errors.New("kzg proof verified")}
return &verification.MockBlobVerifier{ErrSidecarKzgProofVerified: randomErr}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("sidecar proposer expected"),
name: "sidecar proposer expected",
verifier: func(b blocks.ROBlob, reqs []verification.Requirement) verification.BlobVerifier {
return &verification.MockBlobVerifier{ErrSidecarProposerExpected: errors.New("sidecar proposer expected")}
return &verification.MockBlobVerifier{ErrSidecarProposerExpected: randomErr}
},
result: pubsub.ValidationReject,
},
}
for _, tt := range tests {
t.Run(tt.error.Error(), func(t *testing.T) {
t.Run(tt.name, func(t *testing.T) {
ctx := t.Context()
p := p2ptest.NewTestP2P(t)
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
@@ -280,7 +281,7 @@ func TestValidateBlob_ErrorPathsWithMock(t *testing.T) {
Data: buf.Bytes(),
Topic: &topic,
}})
require.ErrorContains(t, tt.error.Error(), err)
require.ErrorIs(t, err, randomErr)
require.Equal(t, result, tt.result)
})
}

View File

@@ -65,7 +65,7 @@ func (s *Service) validateDataColumn(ctx context.Context, pid peer.ID, msg *pubs
// Convert to a read-only data column sidecar.
roDataColumn, err := blocks.NewRODataColumn(dcsc)
if err != nil {
return pubsub.ValidationReject, errors.Wrap(err, "roDataColumn conversion failure")
return pubsub.ValidationIgnore, errors.Wrap(err, "new ro data column")
}
// Compute a batch of only one data column sidecar.

View File

@@ -56,7 +56,7 @@ func (s *Service) validateProposerSlashing(ctx context.Context, pid peer.ID, msg
}
rov, err := headState.ValidatorAtIndexReadOnly(slashing.Header_1.Header.ProposerIndex)
if err != nil {
return pubsub.ValidationReject, err
return pubsub.ValidationIgnore, err
}
if rov.Slashed() {
return pubsub.ValidationIgnore, fmt.Errorf("proposer is already slashed: %d", slashing.Header_1.Header.ProposerIndex)

View File

@@ -2,7 +2,6 @@ package sync
import (
"context"
"errors"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/altair"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed"
@@ -18,6 +17,7 @@ import (
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
// validateSyncContributionAndProof verifies the aggregated signature and the selection proof is valid before forwarding to the
@@ -202,12 +202,14 @@ func (s *Service) rejectInvalidSelectionProof(m *ethpb.SignedContributionAndProo
return func(ctx context.Context) (pubsub.ValidationResult, error) {
ctx, 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 {
valid, err := s.verifySyncSelectionData(ctx, m.Message)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
return pubsub.ValidationAccept, nil
return valid, nil
}
}
@@ -379,23 +381,23 @@ func bitListOverlaps(bitLists [][]byte, b []byte) (bool, error) {
// verifySyncSelectionData verifies that the provided sync contribution has a valid
// selection proof.
func (s *Service) verifySyncSelectionData(ctx context.Context, m *ethpb.ContributionAndProof) error {
func (s *Service) verifySyncSelectionData(ctx context.Context, m *ethpb.ContributionAndProof) (pubsub.ValidationResult, error) {
selectionData := &ethpb.SyncAggregatorSelectionData{Slot: m.Contribution.Slot, SubcommitteeIndex: m.Contribution.SubcommitteeIndex}
domain, err := s.cfg.chain.HeadSyncSelectionProofDomain(ctx, m.Contribution.Slot)
if err != nil {
return err
return pubsub.ValidationIgnore, errors.Wrap(err, "head sync selection proof domain")
}
pubkey, err := s.cfg.chain.HeadValidatorIndexToPublicKey(ctx, m.AggregatorIndex)
if err != nil {
return err
return pubsub.ValidationIgnore, errors.Wrap(err, "head validator index to public key")
}
publicKey, err := bls.PublicKeyFromBytes(pubkey[:])
if err != nil {
return err
return pubsub.ValidationIgnore, errors.Wrap(err, "public key from bytes")
}
root, err := signing.ComputeSigningRoot(selectionData, domain)
if err != nil {
return err
return pubsub.ValidationIgnore, errors.Wrap(err, "compute signing root")
}
set := &bls.SignatureBatch{
Messages: [][32]byte{root},
@@ -405,10 +407,8 @@ func (s *Service) verifySyncSelectionData(ctx context.Context, m *ethpb.Contribu
}
valid, err := s.validateWithBatchVerifier(ctx, "sync contribution selection signature", set)
if err != nil {
return err
return pubsub.ValidationIgnore, errors.Wrap(err, "validate with batch verifier")
}
if valid != pubsub.ValidationAccept {
return errors.New("invalid sync selection proof provided")
}
return nil
return valid, nil
}

View File

@@ -4,6 +4,7 @@ import (
"context"
"fmt"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/signing"
forkchoicetypes "github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
"github.com/OffchainLabs/prysm/v6/config/params"
@@ -101,7 +102,7 @@ func (bv *ROBlobVerifier) VerifiedROBlob() (blocks.VerifiedROBlob, error) {
if bv.results.allSatisfied() {
return blocks.NewVerifiedROBlob(bv.blob), nil
}
return blocks.VerifiedROBlob{}, bv.results.errors(ErrBlobInvalid)
return blocks.VerifiedROBlob{}, bv.results.errors(ErrSidecarInvalid)
}
// SatisfyRequirement allows the caller to assert that a requirement has been satisfied.
@@ -165,7 +166,7 @@ func (bv *ROBlobVerifier) SlotAboveFinalized() (err error) {
fcp := bv.fc.FinalizedCheckpoint()
fSlot, err := slots.EpochStart(fcp.Epoch)
if err != nil {
return errors.Wrapf(errSlotNotAfterFinalized, "error computing epoch start slot for finalized checkpoint (%d) %s", fcp.Epoch, err.Error())
return errors.Wrapf(err, "error computing epoch start slot for finalized checkpoint (%d)", fcp.Epoch)
}
if bv.blob.Slot() <= fSlot {
log.WithFields(logging.BlobFields(bv.blob)).Debug("Sidecar slot is not after finalized checkpoint")
@@ -197,13 +198,20 @@ func (bv *ROBlobVerifier) ValidProposerSignature(ctx context.Context) (err error
parent, err := bv.parentState(ctx)
if err != nil {
log.WithFields(logging.BlobFields(bv.blob)).WithError(err).Debug("Could not replay parent state for blob signature verification")
return ErrInvalidProposerSignature
return errors.Wrap(err, "parent state")
}
// Full verification, which will subsequently be cached for anything sharing the signature cache.
if err = bv.sc.VerifySignature(sd, parent); err != nil {
err = bv.sc.VerifySignature(sd, parent)
if err == signing.ErrSigFailedToVerify {
log.WithFields(logging.BlobFields(bv.blob)).WithError(err).Debug("Signature verification failed")
return ErrInvalidProposerSignature
}
if err != nil {
return errors.Wrap(err, "verify signature")
}
return nil
}
@@ -239,7 +247,7 @@ func (bv *ROBlobVerifier) SidecarParentSlotLower() (err error) {
defer bv.recordResult(RequireSidecarParentSlotLower, &err)
parentSlot, err := bv.fc.Slot(bv.blob.ParentRoot())
if err != nil {
return errors.Wrap(errSlotNotAfterParent, "Parent root not in forkchoice")
return errors.Wrap(err, "slot")
}
if parentSlot >= bv.blob.Slot() {
return errSlotNotAfterParent
@@ -295,7 +303,7 @@ func (bv *ROBlobVerifier) SidecarProposerExpected(ctx context.Context) (err erro
}
r, err := bv.fc.TargetRootForEpoch(bv.blob.ParentRoot(), e)
if err != nil {
return errSidecarUnexpectedProposer
return errors.Wrap(err, "target root for epoch")
}
c := &forkchoicetypes.Checkpoint{Root: r, Epoch: e}
idx, cached := bv.pc.Proposer(c, bv.blob.Slot())
@@ -303,12 +311,12 @@ func (bv *ROBlobVerifier) SidecarProposerExpected(ctx context.Context) (err erro
pst, err := bv.parentState(ctx)
if err != nil {
log.WithError(err).WithFields(logging.BlobFields(bv.blob)).Debug("State replay to parent_root failed")
return errSidecarUnexpectedProposer
return errors.Wrap(err, "parent state")
}
idx, err = bv.pc.ComputeProposer(ctx, bv.blob.ParentRoot(), bv.blob.Slot(), pst)
if err != nil {
log.WithError(err).WithFields(logging.BlobFields(bv.blob)).Debug("Error computing proposer index from parent state")
return errSidecarUnexpectedProposer
return errors.Wrap(err, "compute proposer")
}
}
if idx != bv.blob.ProposerIndex() {

View File

@@ -527,7 +527,7 @@ func TestRequirementSatisfaction(t *testing.T) {
v := ini.NewBlobVerifier(b, GossipBlobSidecarRequirements)
_, err := v.VerifiedROBlob()
require.ErrorIs(t, err, ErrBlobInvalid)
require.ErrorIs(t, err, ErrSidecarInvalid)
var me VerificationMultiError
ok := errors.As(err, &me)
require.Equal(t, true, ok)

View File

@@ -8,6 +8,7 @@ import (
"time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/signing"
forkchoicetypes "github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
@@ -60,10 +61,6 @@ var (
// SpectestDataColumnSidecarRequirements is used by the forkchoice spectests when verifying data columns used in the on_block tests.
SpectestDataColumnSidecarRequirements = requirementList(GossipDataColumnSidecarRequirements).excluding(
RequireSidecarParentSeen, RequireSidecarParentValid)
errColumnsInvalid = errors.New("data columns failed verification")
errBadTopicLength = errors.New("topic length is invalid")
errBadTopic = errors.New("topic is not of the one expected")
)
type (
@@ -85,7 +82,7 @@ var _ DataColumnsVerifier = &RODataColumnsVerifier{}
// were not run, an error will be returned.
func (dv *RODataColumnsVerifier) VerifiedRODataColumns() ([]blocks.VerifiedRODataColumn, error) {
if !dv.results.allSatisfied() {
return nil, dv.results.errors(errColumnsInvalid)
return nil, dv.results.errors(ErrSidecarInvalid)
}
verifiedRODataColumns := make([]blocks.VerifiedRODataColumn, 0, len(dv.dataColumns))
@@ -124,7 +121,7 @@ func (dv *RODataColumnsVerifier) ValidFields() (err error) {
for _, dataColumn := range dv.dataColumns {
if err := peerdas.VerifyDataColumnSidecar(dataColumn); err != nil {
return columnErrBuilder(errors.Wrap(err, "verify data column sidecar"))
return errInvalidFields
}
}
@@ -139,7 +136,7 @@ func (dv *RODataColumnsVerifier) CorrectSubnet(dataColumnSidecarSubTopic string,
defer dv.recordResult(RequireCorrectSubnet, &err)
if len(expectedTopics) != len(dv.dataColumns) {
return columnErrBuilder(errBadTopicLength)
return errBadTopicLength
}
for i := range dv.dataColumns {
@@ -152,7 +149,7 @@ func (dv *RODataColumnsVerifier) CorrectSubnet(dataColumnSidecarSubTopic string,
actualSubTopic := fmt.Sprintf(dataColumnSidecarSubTopic, actualSubnet)
if !strings.Contains(expectedTopic, actualSubTopic) {
return columnErrBuilder(errBadTopic)
return errBadTopic
}
}
@@ -188,13 +185,13 @@ func (dv *RODataColumnsVerifier) NotFromFutureSlot() (err error) {
// We lower the time by MAXIMUM_GOSSIP_CLOCK_DISPARITY in case system time is running slightly behind real time.
earliestStart, err := dv.clock.SlotStart(dataColumnSlot)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "failed to determine slot start time from clock waiter"))
return errors.Wrap(err, "slot start")
}
earliestStart = earliestStart.Add(-maximumGossipClockDisparity)
// If the system time is still before earliestStart, we consider the column from a future slot and return an error.
if now.Before(earliestStart) {
return columnErrBuilder(errFromFutureSlot)
return errFromFutureSlot
}
}
@@ -214,13 +211,13 @@ func (dv *RODataColumnsVerifier) SlotAboveFinalized() (err error) {
// Compute the first slot of the finalized checkpoint epoch.
startSlot, err := slots.EpochStart(finalizedCheckpoint.Epoch)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "epoch start"))
return errors.Wrap(err, "epoch start")
}
for _, dataColumn := range dv.dataColumns {
// Check if the data column slot is after first slot of the epoch corresponding to the finalized checkpoint.
if dataColumn.Slot() <= startSlot {
return columnErrBuilder(errSlotNotAfterFinalized)
return errSlotNotAfterFinalized
}
}
@@ -248,7 +245,7 @@ func (dv *RODataColumnsVerifier) ValidProposerSignature(ctx context.Context) (er
log.WithError(err).Debug("Reusing failed proposer signature validation from cache")
columnVerificationProposerSignatureCache.WithLabelValues("hit-invalid").Inc()
return columnErrBuilder(ErrInvalidProposerSignature)
return ErrInvalidProposerSignature
}
// If yes, we can skip the full verification.
@@ -262,12 +259,16 @@ func (dv *RODataColumnsVerifier) ValidProposerSignature(ctx context.Context) (er
// Retrieve the parent state.
parentState, err := dv.parentState(ctx, dataColumn)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "parent state"))
return errors.Wrap(err, "parent state")
}
// Full verification, which will subsequently be cached for anything sharing the signature cache.
if err = dv.sc.VerifySignature(signatureData, parentState); err != nil {
return columnErrBuilder(errors.Wrap(err, "verify signature"))
err = dv.sc.VerifySignature(signatureData, parentState)
if errors.Is(err, signing.ErrSigFailedToVerify) {
return ErrInvalidProposerSignature
}
if err != nil {
return errors.Wrap(err, "verify signature")
}
}
@@ -289,7 +290,7 @@ func (dv *RODataColumnsVerifier) SidecarParentSeen(parentSeen func([fieldparams.
}
if !dv.fc.HasNode(parentRoot) {
return columnErrBuilder(errSidecarParentNotSeen)
return errSidecarParentNotSeen
}
}
@@ -305,7 +306,7 @@ func (dv *RODataColumnsVerifier) SidecarParentValid(badParent func([fieldparams.
for _, dataColumn := range dv.dataColumns {
if badParent != nil && badParent(dataColumn.ParentRoot()) {
return columnErrBuilder(errSidecarParentInvalid)
return errSidecarParentInvalid
}
}
@@ -323,12 +324,12 @@ func (dv *RODataColumnsVerifier) SidecarParentSlotLower() (err error) {
// Compute the slot of the parent block.
parentSlot, err := dv.fc.Slot(dataColumn.ParentRoot())
if err != nil {
return columnErrBuilder(errors.Wrap(err, "slot"))
return errors.Wrap(err, "slot")
}
// Check if the data column slot is after the parent slot.
if parentSlot >= dataColumn.Slot() {
return columnErrBuilder(errSlotNotAfterParent)
return errSlotNotAfterParent
}
}
@@ -347,7 +348,7 @@ func (dv *RODataColumnsVerifier) SidecarDescendsFromFinalized() (err error) {
parentRoot := dataColumn.ParentRoot()
if !dv.fc.HasNode(parentRoot) {
return columnErrBuilder(errSidecarNotFinalizedDescendent)
return errSidecarNotFinalizedDescendent
}
}
@@ -373,8 +374,15 @@ func (dv *RODataColumnsVerifier) SidecarInclusionProven() (err error) {
log.WithError(keyErr).Error("Failed to get inclusion proof key")
}
if err = peerdas.VerifyDataColumnSidecarInclusionProof(dataColumn); err != nil {
return columnErrBuilder(ErrSidecarInclusionProofInvalid)
err = peerdas.VerifyDataColumnSidecarInclusionProof(dataColumn)
if errors.Is(err, peerdas.ErrNilBlockHeader) ||
errors.Is(err, peerdas.ErrBadRootLength) ||
errors.Is(err, peerdas.ErrInvalidInclusionProof) {
return ErrSidecarInclusionProofInvalid
}
if err != nil {
return errors.Wrap(err, "verify data column sidecar inclusion proof")
}
if keyErr == nil {
@@ -396,9 +404,8 @@ func (dv *RODataColumnsVerifier) SidecarKzgProofVerified() (err error) {
startTime := time.Now()
err = dv.verifyDataColumnsCommitment(dv.dataColumns)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "verify data column commitment"))
if err = dv.verifyDataColumnsCommitment(dv.dataColumns); err != nil {
return ErrSidecarKzgProofInvalid
}
dataColumnBatchKZGVerificationHistogram.Observe(float64(time.Since(startTime).Milliseconds()))
@@ -435,7 +442,7 @@ func (dv *RODataColumnsVerifier) SidecarProposerExpected(ctx context.Context) (e
// Compute the target root for the epoch.
targetRoot, err := dv.fc.TargetRootForEpoch(parentRoot, dataColumnEpoch)
if err != nil {
return [fieldparams.RootLength]byte{}, columnErrBuilder(errors.Wrap(err, "target root from epoch"))
return [fieldparams.RootLength]byte{}, errors.Wrap(err, "target root for epoch")
}
// Store the target root in the cache.
@@ -454,7 +461,7 @@ func (dv *RODataColumnsVerifier) SidecarProposerExpected(ctx context.Context) (e
// Compute the target root for the data column.
targetRoot, err := targetRootFromCache(dataColumnSlot, parentRoot)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "target root"))
return errors.Wrap(err, "target root from cache")
}
// Compute the epoch of the data column slot.
@@ -473,17 +480,17 @@ func (dv *RODataColumnsVerifier) SidecarProposerExpected(ctx context.Context) (e
// Retrieve the parent state.
parentState, err := dv.parentState(ctx, dataColumn)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "parent state"))
return errors.Wrap(err, "parent state")
}
idx, err = dv.pc.ComputeProposer(ctx, parentRoot, dataColumnSlot, parentState)
if err != nil {
return columnErrBuilder(errors.Wrap(err, "compute proposer"))
return errors.Wrap(err, "compute proposer")
}
}
if idx != dataColumn.ProposerIndex() {
return columnErrBuilder(errSidecarUnexpectedProposer)
return errSidecarUnexpectedProposer
}
}
@@ -521,11 +528,7 @@ func columnToSignatureData(d blocks.RODataColumn) signatureData {
}
}
func columnErrBuilder(baseErr error) error {
return errors.Wrap(baseErr, errColumnsInvalid.Error())
}
// incluseionProofKey computes a unique key based on the KZG commitments,
// inclusionProofKey computes a unique key based on the KZG commitments,
// the KZG commitments inclusion proof, and the signed block header root.
func inclusionProofKey(c blocks.RODataColumn) ([32]byte, error) {
const (
@@ -535,7 +538,7 @@ func inclusionProofKey(c blocks.RODataColumn) ([32]byte, error) {
if len(c.KzgCommitmentsInclusionProof) != commsIncProofLen {
// This should be already enforced by ssz unmarshaling; still check so we don't panic on array bounds.
return [32]byte{}, columnErrBuilder(ErrSidecarInclusionProofInvalid)
return [32]byte{}, ErrSidecarInclusionProofInvalid
}
commsByteCount := len(c.KzgCommitments) * fieldparams.KzgCommitmentSize
@@ -549,7 +552,7 @@ func inclusionProofKey(c blocks.RODataColumn) ([32]byte, error) {
// Include the block root in the key.
root, err := c.SignedBlockHeader.HashTreeRoot()
if err != nil {
return [32]byte{}, columnErrBuilder(errors.Wrap(err, "hash tree root"))
return [32]byte{}, errors.Wrap(err, "hash tree root")
}
unhashedKey = append(unhashedKey, root[:]...)

View File

@@ -940,7 +940,7 @@ func TestColumnRequirementSatisfaction(t *testing.T) {
// We haven't performed any verification, VerifiedRODataColumns should error.
_, err := verifier.VerifiedRODataColumns()
require.ErrorIs(t, err, errColumnsInvalid)
require.ErrorIs(t, err, ErrSidecarInvalid)
var me VerificationMultiError
ok := errors.As(err, &me)
@@ -959,7 +959,7 @@ func TestColumnRequirementSatisfaction(t *testing.T) {
// One requirement is missing, VerifiedRODataColumns should still error.
_, err = verifier.VerifiedRODataColumns()
require.ErrorIs(t, err, errColumnsInvalid)
require.ErrorIs(t, err, ErrSidecarInvalid)
// Now, satisfy the first requirement.
verifier.results.record(GossipDataColumnSidecarRequirements[0], nil)

View File

@@ -16,64 +16,73 @@ func AsVerificationFailure(err error) error {
return errors.Join(ErrInvalid, err)
}
// IsBlobValidationFailure checks if the given error is a blob validation failure.
func IsBlobValidationFailure(err error) bool {
return errors.Is(err, ErrBlobInvalid)
// IsSidecarValidationFailure checks if the given error is a blob validation failure.
func IsSidecarValidationFailure(err error) bool {
return errors.Is(err, ErrSidecarInvalid)
}
var (
// ErrBlobInvalid is joined with all other blob verification errors. This enables other packages to check for any sort of
// ErrSidecarInvalid is joined with all other sidecar verification errors. This enables other packages to check for any sort of
// verification error at one point, like sync code checking for peer scoring purposes.
ErrBlobInvalid = AsVerificationFailure(errors.New("invalid blob"))
ErrSidecarInvalid = AsVerificationFailure(errors.New("invalid sidecar"))
// ErrBlobIndexInvalid means RequireBlobIndexInBounds failed.
ErrBlobIndexInvalid = errors.Join(ErrBlobInvalid, errors.New("incorrect blob sidecar index"))
ErrBlobIndexInvalid = errors.Join(ErrSidecarInvalid, errors.New("incorrect blob sidecar index"))
// errFromFutureSlot means RequireSlotNotTooEarly failed.
errFromFutureSlot = errors.New("slot is too far in the future")
errFromFutureSlot = errors.Join(ErrSidecarInvalid, errors.New("slot is too far in the future"))
// errSlotNotAfterFinalized means RequireSlotAboveFinalized failed.
errSlotNotAfterFinalized = errors.New("slot <= finalized checkpoint")
errSlotNotAfterFinalized = errors.Join(ErrSidecarInvalid, errors.New("slot <= finalized checkpoint"))
// ErrInvalidProposerSignature means RequireValidProposerSignature failed.
ErrInvalidProposerSignature = errors.Join(ErrBlobInvalid, errors.New("proposer signature could not be verified"))
ErrInvalidProposerSignature = errors.Join(ErrSidecarInvalid, errors.New("proposer signature could not be verified"))
// errSidecarParentNotSeen means RequireSidecarParentSeen failed.
errSidecarParentNotSeen = errors.New("parent root has not been seen")
errSidecarParentNotSeen = errors.Join(ErrSidecarInvalid, errors.New("parent root has not been seen"))
// errSidecarParentInvalid means RequireSidecarParentValid failed.
errSidecarParentInvalid = errors.Join(ErrBlobInvalid, errors.New("parent block is not valid"))
errSidecarParentInvalid = errors.Join(ErrSidecarInvalid, errors.New("parent block is not valid"))
// errSlotNotAfterParent means RequireSidecarParentSlotLower failed.
errSlotNotAfterParent = errors.Join(ErrBlobInvalid, errors.New("slot <= slot"))
errSlotNotAfterParent = errors.Join(ErrSidecarInvalid, errors.New("slot <= slot"))
// errSidecarNotFinalizedDescendent means RequireSidecarDescendsFromFinalized failed.
errSidecarNotFinalizedDescendent = errors.Join(ErrBlobInvalid, errors.New("parent is not descended from the finalized block"))
errSidecarNotFinalizedDescendent = errors.Join(ErrSidecarInvalid, errors.New("parent is not descended from the finalized block"))
// ErrSidecarInclusionProofInvalid means RequireSidecarInclusionProven failed.
ErrSidecarInclusionProofInvalid = errors.Join(ErrBlobInvalid, errors.New("sidecar inclusion proof verification failed"))
ErrSidecarInclusionProofInvalid = errors.Join(ErrSidecarInvalid, errors.New("sidecar inclusion proof verification failed"))
// ErrSidecarKzgProofInvalid means RequireSidecarKzgProofVerified failed.
ErrSidecarKzgProofInvalid = errors.Join(ErrBlobInvalid, errors.New("sidecar kzg commitment proof verification failed"))
ErrSidecarKzgProofInvalid = errors.Join(ErrSidecarInvalid, errors.New("sidecar kzg commitment proof verification failed"))
// errSidecarUnexpectedProposer means RequireSidecarProposerExpected failed.
errSidecarUnexpectedProposer = errors.Join(ErrBlobInvalid, errors.New("sidecar was not proposed by the expected proposer_index"))
errSidecarUnexpectedProposer = errors.Join(ErrSidecarInvalid, errors.New("sidecar was not proposed by the expected proposer_index"))
// ErrMissingVerification indicates that the given verification function was never performed on the value.
ErrMissingVerification = errors.Join(ErrBlobInvalid, errors.New("verification was not performed for requirement"))
ErrMissingVerification = errors.Join(ErrSidecarInvalid, errors.New("verification was not performed for requirement"))
// errBatchSignatureMismatch is returned by VerifiedROBlobs when any of the blobs in the batch have a signature
// errBatchSignatureMismatch is returned by verified sidecars when any of the sidecars in the batch have a signature
// which does not match the signature for the block with a corresponding root.
errBatchSignatureMismatch = errors.Join(ErrBlobInvalid, errors.New("sidecar block header signature does not match signed block"))
errBatchSignatureMismatch = errors.Join(ErrSidecarInvalid, errors.New("sidecar block header signature does not match signed block"))
// errBatchBlockRootMismatch is returned by VerifiedROBlobs in the scenario where the root of the given signed block
// errBatchBlockRootMismatch is returned by verified sidecars in the scenario where the root of the given signed block
// does not match the block header in one of the corresponding sidecars.
errBatchBlockRootMismatch = errors.Join(ErrBlobInvalid, errors.New("sidecar block header root does not match signed block"))
errBatchBlockRootMismatch = errors.Join(ErrSidecarInvalid, errors.New("sidecar block header root does not match signed block"))
// errInvalidFields indicates that the sidecar has invalid fields, such as a nil field or a field with an invalid value.
errInvalidFields = errors.Join(ErrSidecarInvalid, errors.New("sidecar has invalid fields"))
// errBadTopicLength indicates that the length of the topic is invalid.
errBadTopicLength = errors.Join(ErrSidecarInvalid, errors.New("topic length is invalid"))
// errBadTopic indicates that the topic is not of the one expected.
errBadTopic = errors.Join(ErrSidecarInvalid, errors.New("topic is not of the one expected"))
)
// errVerificationImplementationFault indicates that a code path yielding VerifiedROBlobs has an implementation
// error, leading it to call VerifiedROBlobError with a nil error.
var errVerificationImplementationFault = errors.New("could not verify blob data or create a valid VerifiedROBlob")
// errVerificationImplementationFault indicates that a code path yielding verified sidecars has an implementation
// error, leading it to call verified RO sidecar with a nil error.
var errVerificationImplementationFault = errors.New("could not verify sidecar data or create a valid verified sidecar")
// VerificationMultiError is a custom error that can be used to access individual verification failures.
type VerificationMultiError struct {

View File

@@ -45,11 +45,11 @@ func (m *MockBlobVerifier) ValidProposerSignature(_ context.Context) (err error)
return m.ErrValidProposerSignature
}
func (m *MockBlobVerifier) SidecarParentSeen(_ func([32]byte) bool) (err error) {
func (m *MockBlobVerifier) SidecarParentSeen(func([fieldparams.RootLength]byte) bool) (err error) {
return m.ErrSidecarParentSeen
}
func (m *MockBlobVerifier) SidecarParentValid(_ func([32]byte) bool) (err error) {
func (m *MockBlobVerifier) SidecarParentValid(func([fieldparams.RootLength]byte) bool) (err error) {
return m.ErrSidecarParentValid
}
@@ -73,7 +73,7 @@ func (m *MockBlobVerifier) SidecarProposerExpected(_ context.Context) (err error
return m.ErrSidecarProposerExpected
}
func (*MockBlobVerifier) SatisfyRequirement(_ Requirement) {}
func (*MockBlobVerifier) SatisfyRequirement(Requirement) {}
// Data column sidecars
// --------------------

View File

@@ -0,0 +1,2 @@
### Fixed
- Stop unfairly penalize peers if hitting and internal error in a gossip message validator

View File

@@ -230,7 +230,7 @@ func runBlobStep(t *testing.T,
require.NoError(t, err)
sh, err := beaconBlock.Header()
require.NoError(t, err)
requireVerifyExpected := errAssertionForStep(step, verification.ErrBlobInvalid)
requireVerifyExpected := errAssertionForStep(step, verification.ErrSidecarInvalid)
for index := 0; index*fieldparams.BlobLength < len(blobsSSZ); index++ {
var proof []byte
if index < len(proofs) {
@@ -452,7 +452,7 @@ func errAssertionForStep(step Step, expect error) func(t *testing.T, err error)
}
return func(t *testing.T, err error) {
if err != nil {
require.ErrorIs(t, err, verification.ErrBlobInvalid)
require.ErrorIs(t, err, verification.ErrSidecarInvalid)
var me verification.VerificationMultiError
ok := errors.As(err, &me)
require.Equal(t, true, ok)

View File

@@ -98,11 +98,7 @@ func (c *beaconApiValidatorClient) getAttestationDataRootFromRequest(ctx context
return nil, errors.New("the node is currently optimistic and cannot serve validators")
}
isAggregator, err := helpers.IsAggregator(committeeLength, in.SlotSignature)
if err != nil {
return nil, errors.Wrap(err, "failed to get aggregator status")
}
if !isAggregator {
if !helpers.IsAggregator(committeeLength, in.SlotSignature) {
return nil, errors.New("validator is not an aggregator")
}