mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-08 23:18:15 -05:00
Redesign the pending attestation queue (#15024)
* Redesign pending attestation queue * fix bad signature test * equality tests * changelog <3 * rename functions * change logs * fix fuzzing * fixes after rebasing * build fix * review * James' review * fix imports
This commit is contained in:
@@ -6,7 +6,6 @@ import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
|
||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
gcache "github.com/patrickmn/go-cache"
|
||||
@@ -22,7 +21,7 @@ func NewRegularSyncFuzz(opts ...Option) *Service {
|
||||
cancel: cancel,
|
||||
slotToPendingBlocks: gcache.New(time.Second, 2*time.Second),
|
||||
seenPendingBlocks: make(map[[32]byte]bool),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
}
|
||||
r.rateLimiter = newRateLimiter(r.cfg.p2p)
|
||||
|
||||
|
||||
@@ -28,14 +28,14 @@ import (
|
||||
var processPendingAttsPeriod = slots.DivideSlotBy(2 /* twice per slot */)
|
||||
var pendingAttsLimit = 10000
|
||||
|
||||
// This processes pending attestation queues on every `processPendingAttsPeriod`.
|
||||
func (s *Service) processPendingAttsQueue() {
|
||||
// This processes pending attestation queues on every processPendingAttsPeriod.
|
||||
func (s *Service) runPendingAttsQueue() {
|
||||
// Prevents multiple queue processing goroutines (invoked by RunEvery) from contending for data.
|
||||
mutex := new(sync.Mutex)
|
||||
async.RunEvery(s.ctx, processPendingAttsPeriod, func() {
|
||||
mutex.Lock()
|
||||
if err := s.processPendingAtts(s.ctx); err != nil {
|
||||
log.WithError(err).Debugf("Could not process pending attestation: %v", err)
|
||||
log.WithError(err).Debug("Could not process pending attestation")
|
||||
}
|
||||
mutex.Unlock()
|
||||
})
|
||||
@@ -51,7 +51,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
|
||||
|
||||
// Before a node processes pending attestations queue, it verifies
|
||||
// the attestations in the queue are still valid. Attestations will
|
||||
// be deleted from the queue if invalid (ie. getting staled from falling too many slots behind).
|
||||
// be deleted from the queue if invalid (i.e. getting stalled from falling too many slots behind).
|
||||
s.validatePendingAtts(ctx, s.cfg.clock.CurrentSlot())
|
||||
|
||||
s.pendingAttsLock.RLock()
|
||||
@@ -91,52 +91,59 @@ func (s *Service) processPendingAtts(ctx context.Context) error {
|
||||
return s.sendBatchRootRequest(ctx, pendingRoots, randGen)
|
||||
}
|
||||
|
||||
func (s *Service) processAttestations(ctx context.Context, attestations []ethpb.SignedAggregateAttAndProof) {
|
||||
func (s *Service) processAttestations(ctx context.Context, attestations []any) {
|
||||
for _, signedAtt := range attestations {
|
||||
att := signedAtt.AggregateAttestationAndProof().AggregateVal()
|
||||
// The pending attestations can arrive in both aggregated and unaggregated forms,
|
||||
// each from has distinct validation steps.
|
||||
if att.IsAggregated() {
|
||||
s.processAggregated(ctx, signedAtt)
|
||||
} else {
|
||||
s.processUnaggregated(ctx, att)
|
||||
// The pending attestations can arrive as both aggregates and attestations,
|
||||
// and each form has to be processed differently.
|
||||
switch t := signedAtt.(type) {
|
||||
case ethpb.Att:
|
||||
s.processAtt(ctx, t)
|
||||
case ethpb.SignedAggregateAttAndProof:
|
||||
s.processAggregate(ctx, t)
|
||||
default:
|
||||
log.Warnf("Unexpected item of type %T in pending attestation queue. Item will not be processed", t)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Service) processAggregated(ctx context.Context, att ethpb.SignedAggregateAttAndProof) {
|
||||
aggregate := att.AggregateAttestationAndProof().AggregateVal()
|
||||
func (s *Service) processAggregate(ctx context.Context, aggregate ethpb.SignedAggregateAttAndProof) {
|
||||
att := aggregate.AggregateAttestationAndProof().AggregateVal()
|
||||
|
||||
// Save the pending aggregated attestation to the pool if it passes the aggregated
|
||||
// validation steps.
|
||||
valRes, err := s.validateAggregatedAtt(ctx, att)
|
||||
valRes, err := s.validateAggregatedAtt(ctx, aggregate)
|
||||
if err != nil {
|
||||
log.WithError(err).Debug("Pending aggregated attestation failed validation")
|
||||
}
|
||||
aggValid := pubsub.ValidationAccept == valRes
|
||||
if s.validateBlockInAttestation(ctx, att) && aggValid {
|
||||
if s.validateBlockInAttestation(ctx, aggregate) && aggValid {
|
||||
if features.Get().EnableExperimentalAttestationPool {
|
||||
if err = s.cfg.attestationCache.Add(aggregate); err != nil {
|
||||
log.WithError(err).Debug("Could not save aggregate attestation")
|
||||
if err = s.cfg.attestationCache.Add(att); err != nil {
|
||||
log.WithError(err).Debug("Could not save aggregated attestation")
|
||||
return
|
||||
}
|
||||
} else {
|
||||
if err := s.cfg.attPool.SaveAggregatedAttestation(aggregate); err != nil {
|
||||
log.WithError(err).Debug("Could not save aggregate attestation")
|
||||
if att.IsAggregated() {
|
||||
if err = s.cfg.attPool.SaveAggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Debug("Could not save aggregated attestation")
|
||||
return
|
||||
}
|
||||
} else if err = s.cfg.attPool.SaveUnaggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Debug("Could not save unaggregated attestation")
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
s.setAggregatorIndexEpochSeen(aggregate.GetData().Target.Epoch, att.AggregateAttestationAndProof().GetAggregatorIndex())
|
||||
s.setAggregatorIndexEpochSeen(att.GetData().Target.Epoch, aggregate.AggregateAttestationAndProof().GetAggregatorIndex())
|
||||
|
||||
// Broadcasting the signed attestation again once a node is able to process it.
|
||||
if err := s.cfg.p2p.Broadcast(ctx, att); err != nil {
|
||||
if err := s.cfg.p2p.Broadcast(ctx, aggregate); err != nil {
|
||||
log.WithError(err).Debug("Could not broadcast")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Service) processUnaggregated(ctx context.Context, att ethpb.Att) {
|
||||
func (s *Service) processAtt(ctx context.Context, att ethpb.Att) {
|
||||
data := att.GetData()
|
||||
|
||||
// This is an important validation before retrieving attestation pre state to defend against
|
||||
@@ -239,13 +246,41 @@ func (s *Service) processUnaggregated(ctx context.Context, att ethpb.Att) {
|
||||
}
|
||||
}
|
||||
|
||||
// This defines how pending attestations is saved in the map. The key is the
|
||||
// root of the missing block. The value is the list of pending attestations
|
||||
// This defines how pending aggregates are saved in the map. The key is the
|
||||
// root of the missing block. The value is the list of pending attestations/aggregates
|
||||
// that voted for that block root. The caller of this function is responsible
|
||||
// for not sending repeated aggregates to the pending queue.
|
||||
func (s *Service) savePendingAggregate(agg ethpb.SignedAggregateAttAndProof) {
|
||||
root := bytesutil.ToBytes32(agg.AggregateAttestationAndProof().AggregateVal().GetData().BeaconBlockRoot)
|
||||
|
||||
s.savePending(root, agg, func(other any) bool {
|
||||
a, ok := other.(ethpb.SignedAggregateAttAndProof)
|
||||
return ok && pendingAggregatesAreEqual(agg, a)
|
||||
})
|
||||
}
|
||||
|
||||
// This defines how pending attestations are saved in the map. The key is the
|
||||
// root of the missing block. The value is the list of pending attestations/aggregates
|
||||
// that voted for that block root. The caller of this function is responsible
|
||||
// for not sending repeated attestations to the pending queue.
|
||||
func (s *Service) savePendingAtt(att ethpb.SignedAggregateAttAndProof) {
|
||||
root := bytesutil.ToBytes32(att.AggregateAttestationAndProof().AggregateVal().GetData().BeaconBlockRoot)
|
||||
func (s *Service) savePendingAtt(att ethpb.Att) {
|
||||
if att.Version() >= version.Electra && !att.IsSingle() {
|
||||
log.Debug("Non-single attestation sent to pending attestation pool. Attestation will be ignored")
|
||||
return
|
||||
}
|
||||
|
||||
root := bytesutil.ToBytes32(att.GetData().BeaconBlockRoot)
|
||||
|
||||
s.savePending(root, att, func(other any) bool {
|
||||
a, ok := other.(ethpb.Att)
|
||||
return ok && pendingAttsAreEqual(att, a)
|
||||
})
|
||||
}
|
||||
|
||||
// We want to avoid saving duplicate items, which is the purpose of the passed-in closure.
|
||||
// It is the responsibility of the caller to provide a function that correctly determines quality
|
||||
// in the context of the pending queue.
|
||||
func (s *Service) savePending(root [32]byte, pending any, isEqual func(other any) bool) {
|
||||
s.pendingAttsLock.Lock()
|
||||
defer s.pendingAttsLock.Unlock()
|
||||
|
||||
@@ -261,62 +296,60 @@ func (s *Service) savePendingAtt(att ethpb.SignedAggregateAttAndProof) {
|
||||
_, ok := s.blkRootToPendingAtts[root]
|
||||
if !ok {
|
||||
pendingAttCount.Inc()
|
||||
s.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{att}
|
||||
s.blkRootToPendingAtts[root] = []any{pending}
|
||||
return
|
||||
}
|
||||
// Skip if the attestation from the same aggregator already exists in
|
||||
|
||||
// Skip if the attestation/aggregate from the same validator already exists in
|
||||
// the pending queue.
|
||||
for _, a := range s.blkRootToPendingAtts[root] {
|
||||
if attsAreEqual(att, a) {
|
||||
if isEqual(a) {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
pendingAttCount.Inc()
|
||||
s.blkRootToPendingAtts[root] = append(s.blkRootToPendingAtts[root], att)
|
||||
s.blkRootToPendingAtts[root] = append(s.blkRootToPendingAtts[root], pending)
|
||||
}
|
||||
|
||||
func attsAreEqual(a, b ethpb.SignedAggregateAttAndProof) bool {
|
||||
func pendingAggregatesAreEqual(a, b ethpb.SignedAggregateAttAndProof) bool {
|
||||
if a.Version() != b.Version() {
|
||||
return false
|
||||
}
|
||||
|
||||
if a.GetSignature() != nil {
|
||||
return b.GetSignature() != nil && a.AggregateAttestationAndProof().GetAggregatorIndex() == b.AggregateAttestationAndProof().GetAggregatorIndex()
|
||||
}
|
||||
if b.GetSignature() != nil {
|
||||
if a.AggregateAttestationAndProof().GetAggregatorIndex() != b.AggregateAttestationAndProof().GetAggregatorIndex() {
|
||||
return false
|
||||
}
|
||||
|
||||
aAggregate := a.AggregateAttestationAndProof().AggregateVal()
|
||||
bAggregate := b.AggregateAttestationAndProof().AggregateVal()
|
||||
aData := aAggregate.GetData()
|
||||
bData := bAggregate.GetData()
|
||||
|
||||
if aData.Slot != bData.Slot {
|
||||
aAtt := a.AggregateAttestationAndProof().AggregateVal()
|
||||
bAtt := b.AggregateAttestationAndProof().AggregateVal()
|
||||
if aAtt.GetData().Slot != bAtt.GetData().Slot {
|
||||
return false
|
||||
}
|
||||
if aAtt.GetCommitteeIndex() != bAtt.GetCommitteeIndex() {
|
||||
return false
|
||||
}
|
||||
return bytes.Equal(aAtt.GetAggregationBits(), bAtt.GetAggregationBits())
|
||||
}
|
||||
|
||||
func pendingAttsAreEqual(a, b ethpb.Att) bool {
|
||||
if a.Version() != b.Version() {
|
||||
return false
|
||||
}
|
||||
if a.GetData().Slot != b.GetData().Slot {
|
||||
return false
|
||||
}
|
||||
if a.Version() >= version.Electra {
|
||||
if aAggregate.IsSingle() != bAggregate.IsSingle() {
|
||||
return false
|
||||
}
|
||||
if aAggregate.IsSingle() && aAggregate.GetAttestingIndex() != bAggregate.GetAttestingIndex() {
|
||||
return false
|
||||
}
|
||||
if !bytes.Equal(aAggregate.CommitteeBitsVal().Bytes(), bAggregate.CommitteeBitsVal().Bytes()) {
|
||||
return false
|
||||
}
|
||||
} else if aData.CommitteeIndex != bData.CommitteeIndex {
|
||||
return a.GetAttestingIndex() == b.GetAttestingIndex()
|
||||
}
|
||||
if a.GetCommitteeIndex() != b.GetCommitteeIndex() {
|
||||
return false
|
||||
}
|
||||
|
||||
return bytes.Equal(aAggregate.GetAggregationBits(), bAggregate.GetAggregationBits())
|
||||
return bytes.Equal(a.GetAggregationBits(), b.GetAggregationBits())
|
||||
}
|
||||
|
||||
// This validates the pending attestations in the queue are still valid.
|
||||
// If not valid, a node will remove it in the queue in place. The validity
|
||||
// check specifies the pending attestation could not fall one epoch behind
|
||||
// of the current slot.
|
||||
// If not valid, a node will remove it from the queue in place. The validity
|
||||
// check specifies the pending attestation cannot fall one epoch behind
|
||||
// the current slot.
|
||||
func (s *Service) validatePendingAtts(ctx context.Context, slot primitives.Slot) {
|
||||
_, span := trace.StartSpan(ctx, "validatePendingAtts")
|
||||
defer span.End()
|
||||
@@ -326,9 +359,23 @@ func (s *Service) validatePendingAtts(ctx context.Context, slot primitives.Slot)
|
||||
|
||||
for bRoot, atts := range s.blkRootToPendingAtts {
|
||||
for i := len(atts) - 1; i >= 0; i-- {
|
||||
if slot >= atts[i].AggregateAttestationAndProof().AggregateVal().GetData().Slot+params.BeaconConfig().SlotsPerEpoch {
|
||||
// Remove the pending attestation from the list in place.
|
||||
atts = append(atts[:i], atts[i+1:]...)
|
||||
var attSlot primitives.Slot
|
||||
switch t := atts[i].(type) {
|
||||
case ethpb.Att:
|
||||
attSlot = t.GetData().Slot
|
||||
case ethpb.SignedAggregateAttAndProof:
|
||||
attSlot = t.AggregateAttestationAndProof().AggregateVal().GetData().Slot
|
||||
default:
|
||||
log.Debugf("Unexpected item of type %T in pending attestation queue. Item will be removed", t)
|
||||
// Remove the pending attestation from the map in place.
|
||||
atts[i] = atts[len(atts)-1]
|
||||
atts = atts[:len(atts)-1]
|
||||
continue
|
||||
}
|
||||
if slot >= attSlot+params.BeaconConfig().SlotsPerEpoch {
|
||||
// Remove the pending attestation from the map in place.
|
||||
atts[i] = atts[len(atts)-1]
|
||||
atts = atts[:len(atts)-1]
|
||||
}
|
||||
}
|
||||
s.blkRootToPendingAtts[bRoot] = atts
|
||||
|
||||
@@ -56,17 +56,17 @@ func TestProcessPendingAtts_NoBlockRequestBlock(t *testing.T) {
|
||||
chain := &mock.ChainService{Genesis: prysmTime.Now(), FinalizedCheckPoint: ðpb.Checkpoint{}}
|
||||
r := &Service{
|
||||
cfg: &config{p2p: p1, beaconDB: db, chain: chain, clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot)},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
chainStarted: abool.New(),
|
||||
}
|
||||
|
||||
a := ðpb.AggregateAttestationAndProof{Aggregate: ðpb.Attestation{Data: ðpb.AttestationData{Target: ðpb.Checkpoint{Root: make([]byte, 32)}}}}
|
||||
r.blkRootToPendingAtts[[32]byte{'A'}] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: a}}
|
||||
a := ðpb.Attestation{Data: ðpb.AttestationData{Target: ðpb.Checkpoint{Root: make([]byte, 32)}}}
|
||||
r.blkRootToPendingAtts[[32]byte{'A'}] = []any{a}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
require.LogsContain(t, hook, "Requesting block by root")
|
||||
}
|
||||
|
||||
func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
|
||||
func TestProcessPendingAtts_HasBlockSaveUnaggregatedAtt(t *testing.T) {
|
||||
hook := logTest.NewGlobal()
|
||||
db := dbtest.SetupDB(t)
|
||||
p1 := p2ptest.NewTestP2P(t)
|
||||
@@ -102,16 +102,12 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
|
||||
att.Signature = privKeys[i].Sign(hashTreeRoot[:]).Marshal()
|
||||
}
|
||||
|
||||
aggregateAndProof := ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: att,
|
||||
}
|
||||
|
||||
require.NoError(t, beaconState.SetGenesisTime(time.Now()))
|
||||
|
||||
chain := &mock.ChainService{Genesis: time.Now(),
|
||||
State: beaconState,
|
||||
FinalizedCheckPoint: ðpb.Checkpoint{
|
||||
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
|
||||
Root: att.Data.BeaconBlockRoot,
|
||||
Epoch: 0,
|
||||
},
|
||||
}
|
||||
@@ -132,7 +128,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
|
||||
attPool: attestations.NewPool(),
|
||||
attestationNotifier: &mock.SimpleNotifier{Feed: opn},
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenUnAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
@@ -142,7 +138,9 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof}}
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
r.blkRootToPendingAtts[root] = []any{att}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
|
||||
var wg sync.WaitGroup
|
||||
@@ -169,7 +167,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) {
|
||||
cancel()
|
||||
}
|
||||
|
||||
func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra(t *testing.T) {
|
||||
func TestProcessPendingAtts_HasBlockSaveUnaggregatedAttElectra(t *testing.T) {
|
||||
hook := logTest.NewGlobal()
|
||||
db := dbtest.SetupDB(t)
|
||||
p1 := p2ptest.NewTestP2P(t)
|
||||
@@ -189,9 +187,6 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra(t *testing.T) {
|
||||
Target: ðpb.Checkpoint{Epoch: 0, Root: root[:]},
|
||||
},
|
||||
}
|
||||
aggregateAndProof := ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: att,
|
||||
}
|
||||
|
||||
committee, err := helpers.BeaconCommitteeFromState(t.Context(), beaconState, att.Data.Slot, att.Data.CommitteeIndex)
|
||||
assert.NoError(t, err)
|
||||
@@ -207,7 +202,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra(t *testing.T) {
|
||||
chain := &mock.ChainService{Genesis: time.Now(),
|
||||
State: beaconState,
|
||||
FinalizedCheckPoint: ðpb.Checkpoint{
|
||||
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
|
||||
Root: att.Data.BeaconBlockRoot,
|
||||
Epoch: 0,
|
||||
},
|
||||
}
|
||||
@@ -227,7 +222,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra(t *testing.T) {
|
||||
attPool: attestations.NewPool(),
|
||||
attestationNotifier: &mock.SimpleNotifier{Feed: opn},
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenUnAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
@@ -237,7 +232,9 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProofSingle{Message: aggregateAndProof}}
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
r.blkRootToPendingAtts[root] = []any{att}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(1)
|
||||
@@ -308,9 +305,6 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra_VerifyAlreadySeen
|
||||
CommitteeIndex: 0,
|
||||
},
|
||||
}
|
||||
aggregateAndProof := ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: att,
|
||||
}
|
||||
|
||||
// Retrieve the beacon committee and set the attester index.
|
||||
committee, err := helpers.BeaconCommitteeFromState(t.Context(), beaconState, att.Data.Slot, att.CommitteeId)
|
||||
@@ -332,7 +326,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra_VerifyAlreadySeen
|
||||
Genesis: time.Now(),
|
||||
State: beaconState,
|
||||
FinalizedCheckPoint: ðpb.Checkpoint{
|
||||
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
|
||||
Root: att.Data.BeaconBlockRoot,
|
||||
Epoch: 0,
|
||||
},
|
||||
}
|
||||
@@ -358,7 +352,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra_VerifyAlreadySeen
|
||||
attPool: attestations.NewPool(),
|
||||
attestationNotifier: &mock.SimpleNotifier{Feed: opn},
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenUnAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
@@ -370,8 +364,8 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra_VerifyAlreadySeen
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
// Add the pending attestation.
|
||||
r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{
|
||||
ðpb.SignedAggregateAttestationAndProofSingle{Message: aggregateAndProof},
|
||||
r.blkRootToPendingAtts[root] = []any{
|
||||
att,
|
||||
}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
|
||||
@@ -426,54 +420,72 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAttElectra_VerifyAlreadySeen
|
||||
}
|
||||
|
||||
func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) {
|
||||
db := dbtest.SetupDB(t)
|
||||
p1 := p2ptest.NewTestP2P(t)
|
||||
ctx, cancel := context.WithCancel(t.Context())
|
||||
|
||||
db := dbtest.SetupDB(t)
|
||||
p2p := p2ptest.NewTestP2P(t)
|
||||
st, privKeys := util.DeterministicGenesisState(t, 256)
|
||||
require.NoError(t, st.SetGenesisTime(time.Now()))
|
||||
b := util.NewBeaconBlock()
|
||||
r32, err := b.Block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
util.SaveBlock(t, t.Context(), db, b)
|
||||
require.NoError(t, db.SaveState(t.Context(), st, r32))
|
||||
|
||||
s, _ := util.DeterministicGenesisState(t, 256)
|
||||
chain := &mock.ChainService{
|
||||
State: s,
|
||||
Genesis: prysmTime.Now(), FinalizedCheckPoint: ðpb.Checkpoint{Root: make([]byte, 32)}}
|
||||
r := &Service{
|
||||
State: st,
|
||||
Genesis: prysmTime.Now(),
|
||||
DB: db,
|
||||
FinalizedCheckPoint: ðpb.Checkpoint{
|
||||
Root: r32[:],
|
||||
Epoch: 0,
|
||||
},
|
||||
}
|
||||
|
||||
s := &Service{
|
||||
ctx: ctx,
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
p2p: p2p,
|
||||
beaconDB: db,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
attPool: attestations.NewPool(),
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
seenAggregatedAttestationCache: lruwrpr.New(10),
|
||||
}
|
||||
go s.verifierRoutine()
|
||||
|
||||
committee, err := helpers.BeaconCommitteeFromState(t.Context(), st, 0, 0)
|
||||
assert.NoError(t, err)
|
||||
// Arbitrary aggregator index for testing purposes.
|
||||
aggregatorIndex := committee[0]
|
||||
|
||||
priv, err := bls.RandKey()
|
||||
require.NoError(t, err)
|
||||
aggBits := bitfield.NewBitlist(8)
|
||||
aggBits.SetBitAt(1, true)
|
||||
|
||||
a := ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Signature: priv.Sign([]byte("foo")).Marshal(),
|
||||
AggregationBits: bitfield.Bitlist{0x02},
|
||||
AggregationBits: aggBits,
|
||||
Data: util.HydrateAttestationData(ðpb.AttestationData{}),
|
||||
},
|
||||
SelectionProof: make([]byte, fieldparams.BLSSignatureLength),
|
||||
AggregatorIndex: aggregatorIndex,
|
||||
SelectionProof: make([]byte, fieldparams.BLSSignatureLength),
|
||||
}
|
||||
|
||||
b := util.NewBeaconBlock()
|
||||
r32, err := b.Block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
util.SaveBlock(t, t.Context(), r.cfg.beaconDB, b)
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, r32))
|
||||
s.blkRootToPendingAtts[r32] = []any{ðpb.SignedAggregateAttestationAndProof{Message: a, Signature: make([]byte, fieldparams.BLSSignatureLength)}}
|
||||
require.NoError(t, s.processPendingAtts(t.Context()))
|
||||
|
||||
r.blkRootToPendingAtts[r32] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: a, Signature: make([]byte, fieldparams.BLSSignatureLength)}}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
assert.Equal(t, false, p2p.BroadcastCalled.Load(), "Broadcasted bad aggregate")
|
||||
|
||||
assert.Equal(t, false, p1.BroadcastCalled.Load(), "Broadcasted bad aggregate")
|
||||
// Clear pool.
|
||||
err = r.cfg.attPool.DeleteUnaggregatedAttestation(a.Aggregate)
|
||||
err = s.cfg.attPool.DeleteUnaggregatedAttestation(a.Aggregate)
|
||||
require.NoError(t, err)
|
||||
|
||||
validators := uint64(256)
|
||||
|
||||
_, privKeys := util.DeterministicGenesisState(t, validators)
|
||||
aggBits := bitfield.NewBitlist(8)
|
||||
aggBits.SetBitAt(1, true)
|
||||
att := ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
BeaconBlockRoot: r32[:],
|
||||
@@ -482,11 +494,10 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) {
|
||||
},
|
||||
AggregationBits: aggBits,
|
||||
}
|
||||
committee, err := helpers.BeaconCommitteeFromState(t.Context(), s, att.Data.Slot, att.Data.CommitteeIndex)
|
||||
assert.NoError(t, err)
|
||||
|
||||
attestingIndices, err := attestation.AttestingIndices(att, committee)
|
||||
require.NoError(t, err)
|
||||
attesterDomain, err := signing.Domain(s.Fork(), 0, params.BeaconConfig().DomainBeaconAttester, s.GenesisValidatorsRoot())
|
||||
attesterDomain, err := signing.Domain(st.Fork(), 0, params.BeaconConfig().DomainBeaconAttester, st.GenesisValidatorsRoot())
|
||||
require.NoError(t, err)
|
||||
hashTreeRoot, err := signing.ComputeSigningRoot(att.Data, attesterDomain)
|
||||
assert.NoError(t, err)
|
||||
@@ -494,47 +505,22 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) {
|
||||
att.Signature = privKeys[i].Sign(hashTreeRoot[:]).Marshal()
|
||||
}
|
||||
|
||||
// Arbitrary aggregator index for testing purposes.
|
||||
aggregatorIndex := committee[0]
|
||||
sszSlot := primitives.SSZUint64(att.Data.Slot)
|
||||
sig, err := signing.ComputeDomainAndSign(s, 0, &sszSlot, params.BeaconConfig().DomainSelectionProof, privKeys[aggregatorIndex])
|
||||
sig, err := signing.ComputeDomainAndSign(st, 0, &sszSlot, params.BeaconConfig().DomainSelectionProof, privKeys[aggregatorIndex])
|
||||
require.NoError(t, err)
|
||||
aggregateAndProof := ðpb.AggregateAttestationAndProof{
|
||||
SelectionProof: sig,
|
||||
Aggregate: att,
|
||||
AggregatorIndex: aggregatorIndex,
|
||||
}
|
||||
aggreSig, err := signing.ComputeDomainAndSign(s, 0, aggregateAndProof, params.BeaconConfig().DomainAggregateAndProof, privKeys[aggregatorIndex])
|
||||
aggreSig, err := signing.ComputeDomainAndSign(st, 0, aggregateAndProof, params.BeaconConfig().DomainAggregateAndProof, privKeys[aggregatorIndex])
|
||||
require.NoError(t, err)
|
||||
|
||||
require.NoError(t, s.SetGenesisTime(time.Now()))
|
||||
ctx, cancel := context.WithCancel(t.Context())
|
||||
chain2 := &mock.ChainService{Genesis: time.Now(),
|
||||
State: s,
|
||||
FinalizedCheckPoint: ðpb.Checkpoint{
|
||||
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
|
||||
Epoch: 0,
|
||||
}}
|
||||
r = &Service{
|
||||
ctx: ctx,
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
beaconDB: db,
|
||||
chain: chain2,
|
||||
clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot),
|
||||
attPool: attestations.NewPool(),
|
||||
attestationNotifier: &mock.MockOperationNotifier{},
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
seenUnAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
go r.verifierRoutine()
|
||||
s.blkRootToPendingAtts[r32] = []any{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}}
|
||||
require.NoError(t, s.processPendingAtts(t.Context()))
|
||||
|
||||
r.blkRootToPendingAtts[r32] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
assert.Equal(t, true, p2p.BroadcastCalled.Load(), "The good aggregate was not broadcasted")
|
||||
|
||||
assert.Equal(t, true, p1.BroadcastCalled.Load(), "Could not broadcast the good aggregate")
|
||||
cancel()
|
||||
}
|
||||
|
||||
@@ -610,7 +596,7 @@ func TestProcessPendingAtts_HasBlockSaveAggregatedAtt(t *testing.T) {
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
attPool: attestations.NewPool(),
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
@@ -619,7 +605,9 @@ func TestProcessPendingAtts_HasBlockSaveAggregatedAtt(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}}
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
r.blkRootToPendingAtts[root] = []any{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
|
||||
assert.Equal(t, 1, len(r.cfg.attPool.AggregatedAttestations()), "Did not save aggregated att")
|
||||
@@ -630,112 +618,99 @@ func TestProcessPendingAtts_HasBlockSaveAggregatedAtt(t *testing.T) {
|
||||
cancel()
|
||||
}
|
||||
|
||||
func TestValidatePendingAtts_CanPruneOldAtts(t *testing.T) {
|
||||
s := &Service{
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
func TestProcessPendingAtts_HasBlockSaveAggregatedAttElectra(t *testing.T) {
|
||||
hook := logTest.NewGlobal()
|
||||
db := dbtest.SetupDB(t)
|
||||
p1 := p2ptest.NewTestP2P(t)
|
||||
validators := uint64(256)
|
||||
|
||||
beaconState, privKeys := util.DeterministicGenesisState(t, validators)
|
||||
|
||||
sb := util.NewBeaconBlock()
|
||||
util.SaveBlock(t, t.Context(), db, sb)
|
||||
root, err := sb.Block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
committeeBits := primitives.NewAttestationCommitteeBits()
|
||||
committeeBits.SetBitAt(0, true)
|
||||
aggBits := bitfield.NewBitlist(validators / uint64(params.BeaconConfig().SlotsPerEpoch))
|
||||
aggBits.SetBitAt(0, true)
|
||||
aggBits.SetBitAt(1, true)
|
||||
att := ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{
|
||||
BeaconBlockRoot: root[:],
|
||||
Source: ðpb.Checkpoint{Epoch: 0, Root: bytesutil.PadTo([]byte("hello-world"), 32)},
|
||||
Target: ðpb.Checkpoint{Epoch: 0, Root: root[:]},
|
||||
},
|
||||
CommitteeBits: committeeBits,
|
||||
AggregationBits: aggBits,
|
||||
}
|
||||
|
||||
// 100 Attestations per block root.
|
||||
r1 := [32]byte{'A'}
|
||||
r2 := [32]byte{'B'}
|
||||
r3 := [32]byte{'C'}
|
||||
|
||||
for i := primitives.Slot(0); i < 100; i++ {
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: primitives.ValidatorIndex(i),
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: i, BeaconBlockRoot: r1[:]}}}})
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: primitives.ValidatorIndex(i*2 + i),
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: i, BeaconBlockRoot: r2[:]}}}})
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: primitives.ValidatorIndex(i*3 + i),
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: i, BeaconBlockRoot: r3[:]}}}})
|
||||
committee, err := helpers.BeaconCommitteeFromState(t.Context(), beaconState, att.Data.Slot, att.GetCommitteeIndex())
|
||||
assert.NoError(t, err)
|
||||
attestingIndices, err := attestation.AttestingIndices(att, committee)
|
||||
require.NoError(t, err)
|
||||
attesterDomain, err := signing.Domain(beaconState.Fork(), 0, params.BeaconConfig().DomainBeaconAttester, beaconState.GenesisValidatorsRoot())
|
||||
require.NoError(t, err)
|
||||
hashTreeRoot, err := signing.ComputeSigningRoot(att.Data, attesterDomain)
|
||||
assert.NoError(t, err)
|
||||
sigs := make([]bls.Signature, len(attestingIndices))
|
||||
for i, indice := range attestingIndices {
|
||||
sig := privKeys[indice].Sign(hashTreeRoot[:])
|
||||
sigs[i] = sig
|
||||
}
|
||||
att.Signature = bls.AggregateSignatures(sigs).Marshal()
|
||||
|
||||
assert.Equal(t, 100, len(s.blkRootToPendingAtts[r1]), "Did not save pending atts")
|
||||
assert.Equal(t, 100, len(s.blkRootToPendingAtts[r2]), "Did not save pending atts")
|
||||
assert.Equal(t, 100, len(s.blkRootToPendingAtts[r3]), "Did not save pending atts")
|
||||
|
||||
// Set current slot to 50, it should prune 19 attestations. (50 - 31)
|
||||
s.validatePendingAtts(t.Context(), 50)
|
||||
assert.Equal(t, 81, len(s.blkRootToPendingAtts[r1]), "Did not delete pending atts")
|
||||
assert.Equal(t, 81, len(s.blkRootToPendingAtts[r2]), "Did not delete pending atts")
|
||||
assert.Equal(t, 81, len(s.blkRootToPendingAtts[r3]), "Did not delete pending atts")
|
||||
|
||||
// Set current slot to 100 + slot_duration, it should prune all the attestations.
|
||||
s.validatePendingAtts(t.Context(), 100+params.BeaconConfig().SlotsPerEpoch)
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r1]), "Did not delete pending atts")
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r2]), "Did not delete pending atts")
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r3]), "Did not delete pending atts")
|
||||
|
||||
// Verify the keys are deleted.
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts), "Did not delete block keys")
|
||||
}
|
||||
|
||||
func TestValidatePendingAtts_NoDuplicatingAtts(t *testing.T) {
|
||||
s := &Service{
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
// Arbitrary aggregator index for testing purposes.
|
||||
aggregatorIndex := committee[0]
|
||||
sszUint := primitives.SSZUint64(att.Data.Slot)
|
||||
sig, err := signing.ComputeDomainAndSign(beaconState, 0, &sszUint, params.BeaconConfig().DomainSelectionProof, privKeys[aggregatorIndex])
|
||||
require.NoError(t, err)
|
||||
aggregateAndProof := ðpb.AggregateAttestationAndProofElectra{
|
||||
SelectionProof: sig,
|
||||
Aggregate: att,
|
||||
AggregatorIndex: aggregatorIndex,
|
||||
}
|
||||
aggreSig, err := signing.ComputeDomainAndSign(beaconState, 0, aggregateAndProof, params.BeaconConfig().DomainAggregateAndProof, privKeys[aggregatorIndex])
|
||||
require.NoError(t, err)
|
||||
|
||||
r1 := [32]byte{'A'}
|
||||
r2 := [32]byte{'B'}
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: 1,
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: 1, BeaconBlockRoot: r1[:]}}}})
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: 2,
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: 2, BeaconBlockRoot: r2[:]}}}})
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: 2,
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: 2, BeaconBlockRoot: r2[:]}}}})
|
||||
require.NoError(t, beaconState.SetGenesisTime(time.Now()))
|
||||
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r1]), "Did not save pending atts")
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r2]), "Did not save pending atts")
|
||||
}
|
||||
|
||||
func TestSavePendingAtts_BeyondLimit(t *testing.T) {
|
||||
s := &Service{
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
chain := &mock.ChainService{Genesis: time.Now(),
|
||||
DB: db,
|
||||
State: beaconState,
|
||||
FinalizedCheckPoint: ðpb.Checkpoint{
|
||||
Root: aggregateAndProof.Aggregate.Data.BeaconBlockRoot,
|
||||
Epoch: 0,
|
||||
}}
|
||||
ctx, cancel := context.WithCancel(t.Context())
|
||||
r := &Service{
|
||||
ctx: ctx,
|
||||
cfg: &config{
|
||||
p2p: p1,
|
||||
beaconDB: db,
|
||||
chain: chain,
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
attPool: attestations.NewPool(),
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
go r.verifierRoutine()
|
||||
s, err := util.NewBeaconState()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
for i := 0; i < pendingAttsLimit; i++ {
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: primitives.ValidatorIndex(i),
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: 1, BeaconBlockRoot: bytesutil.Bytes32(uint64(i))}}}})
|
||||
}
|
||||
r1 := [32]byte(bytesutil.Bytes32(0))
|
||||
r2 := [32]byte(bytesutil.Bytes32(uint64(pendingAttsLimit) - 1))
|
||||
r.blkRootToPendingAtts[root] = []any{ðpb.SignedAggregateAttestationAndProofElectra{Message: aggregateAndProof, Signature: aggreSig}}
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r1]), "Did not save pending atts")
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r2]), "Did not save pending atts")
|
||||
|
||||
for i := pendingAttsLimit; i < pendingAttsLimit+20; i++ {
|
||||
s.savePendingAtt(ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
AggregatorIndex: primitives.ValidatorIndex(i),
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{Slot: 1, BeaconBlockRoot: bytesutil.Bytes32(uint64(i))}}}})
|
||||
}
|
||||
|
||||
r1 = [32]byte(bytesutil.Bytes32(uint64(pendingAttsLimit)))
|
||||
r2 = [32]byte(bytesutil.Bytes32(uint64(pendingAttsLimit) + 10))
|
||||
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r1]), "Saved pending atts")
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r2]), "Saved pending atts")
|
||||
assert.Equal(t, 1, len(r.cfg.attPool.AggregatedAttestations()), "Did not save aggregated att")
|
||||
assert.DeepEqual(t, att, r.cfg.attPool.AggregatedAttestations()[0], "Incorrect saved att")
|
||||
atts := r.cfg.attPool.UnaggregatedAttestations()
|
||||
assert.Equal(t, 0, len(atts), "Did save aggregated att")
|
||||
require.LogsContain(t, hook, "Verified and saved pending attestations to pool")
|
||||
cancel()
|
||||
}
|
||||
|
||||
func TestProcessPendingAtts_BlockNotInForkChoice(t *testing.T) {
|
||||
@@ -802,7 +777,7 @@ func TestProcessPendingAtts_BlockNotInForkChoice(t *testing.T) {
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
attPool: attestations.NewPool(),
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
}
|
||||
|
||||
s, err := util.NewBeaconState()
|
||||
@@ -810,8 +785,8 @@ func TestProcessPendingAtts_BlockNotInForkChoice(t *testing.T) {
|
||||
require.NoError(t, r.cfg.beaconDB.SaveState(t.Context(), s, root))
|
||||
|
||||
// Add pending attestation
|
||||
r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof}}
|
||||
|
||||
r.blkRootToPendingAtts[root] = []any{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof}}
|
||||
|
||||
// Process pending attestations - should not process because block is not in fork choice
|
||||
require.NoError(t, r.processPendingAtts(t.Context()))
|
||||
|
||||
@@ -822,137 +797,213 @@ func TestProcessPendingAtts_BlockNotInForkChoice(t *testing.T) {
|
||||
require.LogsDoNotContain(t, hook, "Verified and saved pending attestations to pool")
|
||||
}
|
||||
|
||||
func Test_attsAreEqual_Committee(t *testing.T) {
|
||||
t.Run("Phase 0 equal", func(t *testing.T) {
|
||||
att1 := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
CommitteeIndex: 0}}}}
|
||||
att2 := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
CommitteeIndex: 0}}}}
|
||||
assert.Equal(t, true, attsAreEqual(att1, att2))
|
||||
})
|
||||
t.Run("Phase 0 not equal", func(t *testing.T) {
|
||||
att1 := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
CommitteeIndex: 0}}}}
|
||||
att2 := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
CommitteeIndex: 1}}}}
|
||||
assert.Equal(t, false, attsAreEqual(att1, att2))
|
||||
})
|
||||
t.Run("Electra equal", func(t *testing.T) {
|
||||
cb1 := primitives.NewAttestationCommitteeBits()
|
||||
cb1.SetBitAt(0, true)
|
||||
att1 := ðpb.SignedAggregateAttestationAndProofElectra{
|
||||
Message: ðpb.AggregateAttestationAndProofElectra{
|
||||
Aggregate: ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{},
|
||||
CommitteeBits: cb1,
|
||||
}}}
|
||||
cb2 := primitives.NewAttestationCommitteeBits()
|
||||
cb2.SetBitAt(0, true)
|
||||
att2 := ðpb.SignedAggregateAttestationAndProofElectra{
|
||||
Message: ðpb.AggregateAttestationAndProofElectra{
|
||||
Aggregate: ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{},
|
||||
CommitteeBits: cb2,
|
||||
}}}
|
||||
assert.Equal(t, true, attsAreEqual(att1, att2))
|
||||
})
|
||||
t.Run("Electra not equal", func(t *testing.T) {
|
||||
cb1 := primitives.NewAttestationCommitteeBits()
|
||||
cb1.SetBitAt(0, true)
|
||||
att1 := ðpb.SignedAggregateAttestationAndProofElectra{
|
||||
Message: ðpb.AggregateAttestationAndProofElectra{
|
||||
Aggregate: ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{},
|
||||
CommitteeBits: cb1,
|
||||
}}}
|
||||
cb2 := primitives.NewAttestationCommitteeBits()
|
||||
cb2.SetBitAt(1, true)
|
||||
att2 := ðpb.SignedAggregateAttestationAndProofElectra{
|
||||
Message: ðpb.AggregateAttestationAndProofElectra{
|
||||
Aggregate: ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{},
|
||||
CommitteeBits: cb2,
|
||||
}}}
|
||||
assert.Equal(t, false, attsAreEqual(att1, att2))
|
||||
})
|
||||
t.Run("Single and Electra not equal", func(t *testing.T) {
|
||||
cb := primitives.NewAttestationCommitteeBits()
|
||||
cb.SetBitAt(0, true)
|
||||
att1 := ðpb.SignedAggregateAttestationAndProofElectra{
|
||||
Message: ðpb.AggregateAttestationAndProofElectra{
|
||||
Aggregate: ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{},
|
||||
CommitteeBits: cb,
|
||||
}}}
|
||||
att2 := ðpb.SignedAggregateAttestationAndProofSingle{
|
||||
Message: ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: ðpb.SingleAttestation{
|
||||
CommitteeId: 0,
|
||||
AttesterIndex: 0,
|
||||
Data: ðpb.AttestationData{},
|
||||
},
|
||||
},
|
||||
}
|
||||
assert.Equal(t, false, attsAreEqual(att1, att2))
|
||||
})
|
||||
t.Run("Single equal", func(t *testing.T) {
|
||||
att1 := ðpb.SignedAggregateAttestationAndProofSingle{
|
||||
Message: ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: ðpb.SingleAttestation{
|
||||
CommitteeId: 0,
|
||||
AttesterIndex: 0,
|
||||
Data: ðpb.AttestationData{},
|
||||
},
|
||||
},
|
||||
}
|
||||
att2 := ðpb.SignedAggregateAttestationAndProofSingle{
|
||||
Message: ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: ðpb.SingleAttestation{
|
||||
CommitteeId: 0,
|
||||
AttesterIndex: 0,
|
||||
Data: ðpb.AttestationData{},
|
||||
},
|
||||
},
|
||||
}
|
||||
assert.Equal(t, true, attsAreEqual(att1, att2))
|
||||
})
|
||||
t.Run("Single not equal", func(t *testing.T) {
|
||||
// Same AttesterIndex but different CommitteeId
|
||||
att1 := ðpb.SignedAggregateAttestationAndProofSingle{
|
||||
Message: ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: ðpb.SingleAttestation{
|
||||
CommitteeId: 0,
|
||||
AttesterIndex: 0,
|
||||
Data: ðpb.AttestationData{},
|
||||
},
|
||||
},
|
||||
}
|
||||
att2 := ðpb.SignedAggregateAttestationAndProofSingle{
|
||||
Message: ðpb.AggregateAttestationAndProofSingle{
|
||||
Aggregate: ðpb.SingleAttestation{
|
||||
CommitteeId: 1,
|
||||
AttesterIndex: 0,
|
||||
Data: ðpb.AttestationData{},
|
||||
},
|
||||
},
|
||||
}
|
||||
assert.Equal(t, false, attsAreEqual(att1, att2))
|
||||
func TestValidatePendingAtts_CanPruneOldAtts(t *testing.T) {
|
||||
s := &Service{
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
}
|
||||
|
||||
// Same CommitteeId but different AttesterIndex
|
||||
att2.Message.Aggregate.CommitteeId = 0
|
||||
att2.Message.Aggregate.AttesterIndex = 1
|
||||
assert.Equal(t, false, attsAreEqual(att1, att2))
|
||||
// 100 Attestations per block root.
|
||||
r1 := [32]byte{'A'}
|
||||
r2 := [32]byte{'B'}
|
||||
r3 := [32]byte{'C'}
|
||||
|
||||
for i := primitives.Slot(0); i < 100; i++ {
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: i, BeaconBlockRoot: r1[:]}})
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: i, BeaconBlockRoot: r2[:]}})
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: i, BeaconBlockRoot: r3[:]}})
|
||||
}
|
||||
|
||||
assert.Equal(t, 100, len(s.blkRootToPendingAtts[r1]), "Did not save pending atts")
|
||||
assert.Equal(t, 100, len(s.blkRootToPendingAtts[r2]), "Did not save pending atts")
|
||||
assert.Equal(t, 100, len(s.blkRootToPendingAtts[r3]), "Did not save pending atts")
|
||||
|
||||
// Set current slot to 50, it should prune 19 attestations. (50 - 31)
|
||||
s.validatePendingAtts(t.Context(), 50)
|
||||
assert.Equal(t, 81, len(s.blkRootToPendingAtts[r1]), "Did not delete pending atts")
|
||||
assert.Equal(t, 81, len(s.blkRootToPendingAtts[r2]), "Did not delete pending atts")
|
||||
assert.Equal(t, 81, len(s.blkRootToPendingAtts[r3]), "Did not delete pending atts")
|
||||
|
||||
// Set current slot to 100 + slot_duration, it should prune all the attestations.
|
||||
s.validatePendingAtts(t.Context(), 100+params.BeaconConfig().SlotsPerEpoch)
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r1]), "Did not delete pending atts")
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r2]), "Did not delete pending atts")
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r3]), "Did not delete pending atts")
|
||||
|
||||
// Verify the keys are deleted.
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts), "Did not delete block keys")
|
||||
}
|
||||
|
||||
func TestValidatePendingAtts_NoDuplicatingAtts(t *testing.T) {
|
||||
s := &Service{
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
}
|
||||
|
||||
r1 := [32]byte{'A'}
|
||||
r2 := [32]byte{'B'}
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, BeaconBlockRoot: r1[:]}})
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: 2, BeaconBlockRoot: r2[:]}})
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: 2, BeaconBlockRoot: r2[:]}})
|
||||
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r1]), "Did not save pending atts")
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r2]), "Did not save pending atts")
|
||||
}
|
||||
|
||||
func TestSavePendingAtts_BeyondLimit(t *testing.T) {
|
||||
s := &Service{
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
}
|
||||
|
||||
for i := 0; i < pendingAttsLimit; i++ {
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, BeaconBlockRoot: bytesutil.Bytes32(uint64(i))}})
|
||||
}
|
||||
r1 := [32]byte(bytesutil.Bytes32(0))
|
||||
r2 := [32]byte(bytesutil.Bytes32(uint64(pendingAttsLimit) - 1))
|
||||
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r1]), "Did not save pending atts")
|
||||
assert.Equal(t, 1, len(s.blkRootToPendingAtts[r2]), "Did not save pending atts")
|
||||
|
||||
for i := pendingAttsLimit; i < pendingAttsLimit+20; i++ {
|
||||
s.savePendingAtt(ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, BeaconBlockRoot: bytesutil.Bytes32(uint64(i))}})
|
||||
}
|
||||
|
||||
r1 = [32]byte(bytesutil.Bytes32(uint64(pendingAttsLimit)))
|
||||
r2 = [32]byte(bytesutil.Bytes32(uint64(pendingAttsLimit) + 10))
|
||||
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r1]), "Saved pending atts")
|
||||
assert.Equal(t, 0, len(s.blkRootToPendingAtts[r2]), "Saved pending atts")
|
||||
}
|
||||
|
||||
func Test_pendingAggregatesAreEqual(t *testing.T) {
|
||||
t.Run("equal", func(t *testing.T) {
|
||||
a := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
b := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
assert.Equal(t, true, pendingAggregatesAreEqual(a, b))
|
||||
})
|
||||
t.Run("different version", func(t *testing.T) {
|
||||
a := ðpb.SignedAggregateAttestationAndProof{Message: ðpb.AggregateAttestationAndProof{AggregatorIndex: 1}}
|
||||
b := ðpb.SignedAggregateAttestationAndProofElectra{Message: ðpb.AggregateAttestationAndProofElectra{AggregatorIndex: 1}}
|
||||
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
|
||||
})
|
||||
t.Run("different aggregator index", func(t *testing.T) {
|
||||
a := ðpb.SignedAggregateAttestationAndProof{Message: ðpb.AggregateAttestationAndProof{AggregatorIndex: 1}}
|
||||
b := ðpb.SignedAggregateAttestationAndProof{Message: ðpb.AggregateAttestationAndProof{AggregatorIndex: 2}}
|
||||
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
|
||||
})
|
||||
t.Run("different slot", func(t *testing.T) {
|
||||
a := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
b := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 2,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
|
||||
})
|
||||
t.Run("different committee index", func(t *testing.T) {
|
||||
a := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
b := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 2,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
|
||||
})
|
||||
t.Run("different aggregation bits", func(t *testing.T) {
|
||||
a := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
}}}
|
||||
b := ðpb.SignedAggregateAttestationAndProof{
|
||||
Message: ðpb.AggregateAttestationAndProof{
|
||||
Aggregate: ðpb.Attestation{
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
},
|
||||
AggregationBits: bitfield.Bitlist{0b1000},
|
||||
}}}
|
||||
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
|
||||
})
|
||||
}
|
||||
|
||||
func Test_pendingAttsAreEqual(t *testing.T) {
|
||||
t.Run("equal Phase0", func(t *testing.T) {
|
||||
a := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
b := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
assert.Equal(t, true, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
t.Run("equal Electra", func(t *testing.T) {
|
||||
a := ðpb.SingleAttestation{Data: ðpb.AttestationData{Slot: 1}, AttesterIndex: 1}
|
||||
b := ðpb.SingleAttestation{Data: ðpb.AttestationData{Slot: 1}, AttesterIndex: 1}
|
||||
assert.Equal(t, true, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
t.Run("different version", func(t *testing.T) {
|
||||
a := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
b := ðpb.SingleAttestation{Data: ðpb.AttestationData{Slot: 1}, AttesterIndex: 1}
|
||||
assert.Equal(t, false, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
t.Run("different slot", func(t *testing.T) {
|
||||
a := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
b := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 2, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
assert.Equal(t, false, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
t.Run("different committee index", func(t *testing.T) {
|
||||
a := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
b := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 2}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
assert.Equal(t, false, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
t.Run("different aggregation bits", func(t *testing.T) {
|
||||
a := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1111}}
|
||||
b := ðpb.Attestation{Data: ðpb.AttestationData{Slot: 1, CommitteeIndex: 1}, AggregationBits: bitfield.Bitlist{0b1000}}
|
||||
assert.Equal(t, false, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
t.Run("different attester index", func(t *testing.T) {
|
||||
a := ðpb.SingleAttestation{Data: ðpb.AttestationData{Slot: 1}, AttesterIndex: 1}
|
||||
b := ðpb.SingleAttestation{Data: ðpb.AttestationData{Slot: 1}, AttesterIndex: 2}
|
||||
assert.Equal(t, false, pendingAttsAreEqual(a, b))
|
||||
})
|
||||
}
|
||||
|
||||
@@ -38,7 +38,6 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
|
||||
leakybucket "github.com/OffchainLabs/prysm/v6/container/leaky-bucket"
|
||||
"github.com/OffchainLabs/prysm/v6/crypto/rand"
|
||||
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
|
||||
"github.com/OffchainLabs/prysm/v6/runtime"
|
||||
prysmTime "github.com/OffchainLabs/prysm/v6/time"
|
||||
"github.com/OffchainLabs/prysm/v6/time/slots"
|
||||
@@ -134,7 +133,7 @@ type Service struct {
|
||||
cancel context.CancelFunc
|
||||
slotToPendingBlocks *gcache.Cache
|
||||
seenPendingBlocks map[[32]byte]bool
|
||||
blkRootToPendingAtts map[[32]byte][]ethpb.SignedAggregateAttAndProof
|
||||
blkRootToPendingAtts map[[32]byte][]any
|
||||
subHandler *subTopicHandler
|
||||
pendingAttsLock sync.RWMutex
|
||||
pendingQueueLock sync.RWMutex
|
||||
@@ -190,7 +189,7 @@ func NewService(ctx context.Context, opts ...Option) *Service {
|
||||
cfg: &config{clock: startup.NewClock(time.Unix(0, 0), [32]byte{})},
|
||||
slotToPendingBlocks: gcache.New(pendingBlockExpTime /* exp time */, 0 /* disable janitor */),
|
||||
seenPendingBlocks: make(map[[32]byte]bool),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
dataColumnLogCh: make(chan dataColumnLogEntry, 1000),
|
||||
reconstructionRandGen: rand.NewGenerator(),
|
||||
}
|
||||
@@ -265,7 +264,7 @@ func (s *Service) Start() {
|
||||
s.cfg.p2p.AddPingMethod(s.sendPingRequest)
|
||||
|
||||
s.processPendingBlocksQueue()
|
||||
s.processPendingAttsQueue()
|
||||
s.runPendingAttsQueue()
|
||||
s.maintainPeerStatuses()
|
||||
|
||||
if params.FuluEnabled() {
|
||||
|
||||
@@ -246,7 +246,7 @@ func (s *Service) validateBlockInAttestation(ctx context.Context, satt ethpb.Sig
|
||||
blockRoot := bytesutil.ToBytes32(satt.AggregateAttestationAndProof().AggregateVal().GetData().BeaconBlockRoot)
|
||||
if !s.hasBlockAndState(ctx, blockRoot) {
|
||||
// A node doesn't have the block, it'll request from peer while saving the pending attestation to a queue.
|
||||
s.savePendingAtt(satt)
|
||||
s.savePendingAggregate(satt)
|
||||
return false
|
||||
}
|
||||
return true
|
||||
|
||||
@@ -219,7 +219,7 @@ func TestValidateAggregateAndProof_NoBlock(t *testing.T) {
|
||||
attPool: attestations.NewPool(),
|
||||
chain: &mock.ChainService{},
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenAggregatedAttestationCache: c,
|
||||
}
|
||||
r.initCaches()
|
||||
@@ -372,7 +372,7 @@ func TestValidateAggregateAndProof_ExistedInPool(t *testing.T) {
|
||||
attestationNotifier: (&mock.ChainService{}).OperationNotifier(),
|
||||
},
|
||||
seenAggregatedAttestationCache: lruwrpr.New(10),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
}
|
||||
r.initCaches()
|
||||
|
||||
|
||||
@@ -105,7 +105,7 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(
|
||||
// Verify the block being voted and the processed state is in beaconDB and the block has passed validation if it's in the beaconDB.
|
||||
blockRoot := bytesutil.ToBytes32(data.BeaconBlockRoot)
|
||||
if !s.hasBlockAndState(ctx, blockRoot) {
|
||||
return s.saveToPendingAttPool(att)
|
||||
s.savePendingAtt(att)
|
||||
}
|
||||
if !s.cfg.chain.InForkchoice(blockRoot) {
|
||||
tracing.AnnotateError(span, blockchain.ErrNotDescendantOfFinalized)
|
||||
@@ -403,31 +403,3 @@ func (s *Service) hasBlockAndState(ctx context.Context, blockRoot [32]byte) bool
|
||||
hasState := hasStateSummary || s.cfg.beaconDB.HasState(ctx, blockRoot)
|
||||
return hasState && s.cfg.chain.HasBlock(ctx, blockRoot)
|
||||
}
|
||||
|
||||
func (s *Service) saveToPendingAttPool(att eth.Att) (pubsub.ValidationResult, error) {
|
||||
// A node doesn't have the block, it'll request from peer while saving the pending attestation to a queue.
|
||||
if att.Version() >= version.Electra {
|
||||
a, ok := att.(*eth.SingleAttestation)
|
||||
// This will never fail in practice because we asserted the version
|
||||
if !ok {
|
||||
return pubsub.ValidationIgnore, fmt.Errorf("attestation has wrong type (expected %T, got %T)", ð.SingleAttestation{}, att)
|
||||
}
|
||||
// Even though there is no AggregateAndProof type to hold a single attestation, our design of pending atts pool
|
||||
// requires to have an AggregateAndProof object, even for unaggregated attestations.
|
||||
// Because of this we need to have a single attestation version of it to be able to save single attestations into the pool.
|
||||
// It's not possible to convert the single attestation into an electra attestation before saving to the pool
|
||||
// because crucial verification steps can't be performed without the block, and converting prior to these checks
|
||||
// opens up DoS attacks.
|
||||
// The AggregateAndProof object is discarded once we process the pending attestation and code paths dealing
|
||||
// with "real" AggregateAndProof objects (ones that hold actual aggregates) don't use the single attestation version anywhere.
|
||||
s.savePendingAtt(ð.SignedAggregateAttestationAndProofSingle{Message: ð.AggregateAttestationAndProofSingle{Aggregate: a}})
|
||||
} else {
|
||||
a, ok := att.(*eth.Attestation)
|
||||
// This will never fail in practice because we asserted the version
|
||||
if !ok {
|
||||
return pubsub.ValidationIgnore, fmt.Errorf("attestation has wrong type (expected %T, got %T)", ð.Attestation{}, att)
|
||||
}
|
||||
s.savePendingAtt(ð.SignedAggregateAttestationAndProof{Message: ð.AggregateAttestationAndProof{Aggregate: a}})
|
||||
}
|
||||
return pubsub.ValidationIgnore, nil
|
||||
}
|
||||
|
||||
@@ -51,7 +51,7 @@ func TestService_validateCommitteeIndexBeaconAttestation(t *testing.T) {
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
attestationNotifier: (&mockChain.ChainService{}).OperationNotifier(),
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenUnAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
@@ -343,7 +343,7 @@ func TestService_validateCommitteeIndexBeaconAttestationElectra(t *testing.T) {
|
||||
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
|
||||
attestationNotifier: (&mockChain.ChainService{}).OperationNotifier(),
|
||||
},
|
||||
blkRootToPendingAtts: make(map[[32]byte][]ethpb.SignedAggregateAttAndProof),
|
||||
blkRootToPendingAtts: make(map[[32]byte][]any),
|
||||
seenUnAggregatedAttestationCache: lruwrpr.New(10),
|
||||
signatureChan: make(chan *signatureVerifier, verifierLimit),
|
||||
}
|
||||
|
||||
3
changelog/radek_redesign-pending-att-queue.md
Normal file
3
changelog/radek_redesign-pending-att-queue.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Changed
|
||||
|
||||
- Redesign the pending attestation queue.
|
||||
@@ -530,21 +530,6 @@ func (a *AggregateAttestationAndProofElectra) AggregateVal() Att {
|
||||
return a.Aggregate
|
||||
}
|
||||
|
||||
// Version --
|
||||
func (a *AggregateAttestationAndProofSingle) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *AggregateAttestationAndProofSingle) IsNil() bool {
|
||||
return a == nil || a.Aggregate == nil || a.Aggregate.IsNil()
|
||||
}
|
||||
|
||||
// AggregateVal --
|
||||
func (a *AggregateAttestationAndProofSingle) AggregateVal() Att {
|
||||
return a.Aggregate
|
||||
}
|
||||
|
||||
// Version --
|
||||
func (a *SignedAggregateAttestationAndProof) Version() int {
|
||||
return version.Phase0
|
||||
@@ -574,18 +559,3 @@ func (a *SignedAggregateAttestationAndProofElectra) IsNil() bool {
|
||||
func (a *SignedAggregateAttestationAndProofElectra) AggregateAttestationAndProof() AggregateAttAndProof {
|
||||
return a.Message
|
||||
}
|
||||
|
||||
// Version --
|
||||
func (a *SignedAggregateAttestationAndProofSingle) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *SignedAggregateAttestationAndProofSingle) IsNil() bool {
|
||||
return a == nil || a.Message == nil || a.Message.IsNil()
|
||||
}
|
||||
|
||||
// AggregateAttestationAndProof --
|
||||
func (a *SignedAggregateAttestationAndProofSingle) AggregateAttestationAndProof() AggregateAttAndProof {
|
||||
return a.Message
|
||||
}
|
||||
|
||||
278
proto/prysm/v1alpha1/attestation.pb.go
generated
278
proto/prysm/v1alpha1/attestation.pb.go
generated
@@ -528,124 +528,6 @@ func (x *AttestationElectra) GetCommitteeBits() github_com_prysmaticlabs_go_bitf
|
||||
return github_com_prysmaticlabs_go_bitfield.Bitvector64(nil)
|
||||
}
|
||||
|
||||
type SignedAggregateAttestationAndProofSingle struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Message *AggregateAttestationAndProofSingle `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"`
|
||||
Signature []byte `protobuf:"bytes,2,opt,name=signature,proto3" json:"signature,omitempty" ssz-size:"96"`
|
||||
}
|
||||
|
||||
func (x *SignedAggregateAttestationAndProofSingle) Reset() {
|
||||
*x = SignedAggregateAttestationAndProofSingle{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *SignedAggregateAttestationAndProofSingle) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*SignedAggregateAttestationAndProofSingle) ProtoMessage() {}
|
||||
|
||||
func (x *SignedAggregateAttestationAndProofSingle) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use SignedAggregateAttestationAndProofSingle.ProtoReflect.Descriptor instead.
|
||||
func (*SignedAggregateAttestationAndProofSingle) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_attestation_proto_rawDescGZIP(), []int{8}
|
||||
}
|
||||
|
||||
func (x *SignedAggregateAttestationAndProofSingle) GetMessage() *AggregateAttestationAndProofSingle {
|
||||
if x != nil {
|
||||
return x.Message
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *SignedAggregateAttestationAndProofSingle) GetSignature() []byte {
|
||||
if x != nil {
|
||||
return x.Signature
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type AggregateAttestationAndProofSingle struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
AggregatorIndex github_com_OffchainLabs_prysm_v6_consensus_types_primitives.ValidatorIndex `protobuf:"varint,1,opt,name=aggregator_index,json=aggregatorIndex,proto3" json:"aggregator_index,omitempty" cast-type:"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.ValidatorIndex"`
|
||||
Aggregate *SingleAttestation `protobuf:"bytes,3,opt,name=aggregate,proto3" json:"aggregate,omitempty"`
|
||||
SelectionProof []byte `protobuf:"bytes,2,opt,name=selection_proof,json=selectionProof,proto3" json:"selection_proof,omitempty" ssz-size:"96"`
|
||||
}
|
||||
|
||||
func (x *AggregateAttestationAndProofSingle) Reset() {
|
||||
*x = AggregateAttestationAndProofSingle{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *AggregateAttestationAndProofSingle) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*AggregateAttestationAndProofSingle) ProtoMessage() {}
|
||||
|
||||
func (x *AggregateAttestationAndProofSingle) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[9]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use AggregateAttestationAndProofSingle.ProtoReflect.Descriptor instead.
|
||||
func (*AggregateAttestationAndProofSingle) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_attestation_proto_rawDescGZIP(), []int{9}
|
||||
}
|
||||
|
||||
func (x *AggregateAttestationAndProofSingle) GetAggregatorIndex() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.ValidatorIndex {
|
||||
if x != nil {
|
||||
return x.AggregatorIndex
|
||||
}
|
||||
return github_com_OffchainLabs_prysm_v6_consensus_types_primitives.ValidatorIndex(0)
|
||||
}
|
||||
|
||||
func (x *AggregateAttestationAndProofSingle) GetAggregate() *SingleAttestation {
|
||||
if x != nil {
|
||||
return x.Aggregate
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *AggregateAttestationAndProofSingle) GetSelectionProof() []byte {
|
||||
if x != nil {
|
||||
return x.SelectionProof
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type SingleAttestation struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@@ -660,7 +542,7 @@ type SingleAttestation struct {
|
||||
func (x *SingleAttestation) Reset() {
|
||||
*x = SingleAttestation{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[10]
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -673,7 +555,7 @@ func (x *SingleAttestation) String() string {
|
||||
func (*SingleAttestation) ProtoMessage() {}
|
||||
|
||||
func (x *SingleAttestation) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[10]
|
||||
mi := &file_proto_prysm_v1alpha1_attestation_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -686,7 +568,7 @@ func (x *SingleAttestation) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use SingleAttestation.ProtoReflect.Descriptor instead.
|
||||
func (*SingleAttestation) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_attestation_proto_rawDescGZIP(), []int{10}
|
||||
return file_proto_prysm_v1alpha1_attestation_proto_rawDescGZIP(), []int{8}
|
||||
}
|
||||
|
||||
func (x *SingleAttestation) GetCommitteeId() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.CommitteeIndex {
|
||||
@@ -848,67 +730,39 @@ var file_proto_prysm_v1alpha1_attestation_proto_rawDesc = []byte{
|
||||
0x6c, 0x61, 0x62, 0x73, 0x2f, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64,
|
||||
0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x36, 0x34, 0x8a, 0xb5, 0x18, 0x01,
|
||||
0x38, 0x52, 0x0d, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x42, 0x69, 0x74, 0x73,
|
||||
0x22, 0xa5, 0x01, 0x0a, 0x28, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x67, 0x67, 0x72, 0x65,
|
||||
0x67, 0x61, 0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41,
|
||||
0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x12, 0x53, 0x0a,
|
||||
0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39,
|
||||
0x22, 0xdf, 0x02, 0x0a, 0x11, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73,
|
||||
0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x71, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74,
|
||||
0x74, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4e, 0x82, 0xb5,
|
||||
0x18, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66,
|
||||
0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f,
|
||||
0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70,
|
||||
0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x43, 0x6f,
|
||||
0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0b, 0x63, 0x6f,
|
||||
0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x49, 0x64, 0x12, 0x75, 0x0a, 0x0e, 0x61, 0x74, 0x74,
|
||||
0x65, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28,
|
||||
0x04, 0x42, 0x4e, 0x82, 0xb5, 0x18, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f,
|
||||
0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70,
|
||||
0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75,
|
||||
0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76,
|
||||
0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65,
|
||||
0x78, 0x52, 0x0d, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78,
|
||||
0x12, 0x3a, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26,
|
||||
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
|
||||
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65,
|
||||
0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72,
|
||||
0x6f, 0x6f, 0x66, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61,
|
||||
0x67, 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73,
|
||||
0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x98, 0x02, 0x0a, 0x22, 0x41, 0x67, 0x67,
|
||||
0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x12,
|
||||
0x79, 0x0a, 0x10, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4e, 0x82, 0xb5, 0x18, 0x4a, 0x67,
|
||||
0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61,
|
||||
0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f,
|
||||
0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f,
|
||||
0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64,
|
||||
0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0f, 0x61, 0x67, 0x67, 0x72, 0x65,
|
||||
0x67, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x46, 0x0a, 0x09, 0x61, 0x67,
|
||||
0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e,
|
||||
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
|
||||
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x41, 0x74, 0x74, 0x65,
|
||||
0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,
|
||||
0x74, 0x65, 0x12, 0x2f, 0x0a, 0x0f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
|
||||
0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
|
||||
0x02, 0x39, 0x36, 0x52, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72,
|
||||
0x6f, 0x6f, 0x66, 0x22, 0xdf, 0x02, 0x0a, 0x11, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x41, 0x74,
|
||||
0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x71, 0x0a, 0x0c, 0x63, 0x6f, 0x6d,
|
||||
0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42,
|
||||
0x4e, 0x82, 0xb5, 0x18, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
|
||||
0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
|
||||
0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d,
|
||||
0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73,
|
||||
0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52,
|
||||
0x0b, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x49, 0x64, 0x12, 0x75, 0x0a, 0x0e,
|
||||
0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02,
|
||||
0x20, 0x01, 0x28, 0x04, 0x42, 0x4e, 0x82, 0xb5, 0x18, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
|
||||
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62,
|
||||
0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65,
|
||||
0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69,
|
||||
0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49,
|
||||
0x6e, 0x64, 0x65, 0x78, 0x52, 0x0d, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x12, 0x3a, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x26, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
|
||||
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74,
|
||||
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12,
|
||||
0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e,
|
||||
0x61, 0x74, 0x75, 0x72, 0x65, 0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74,
|
||||
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70,
|
||||
0x68, 0x61, 0x31, 0x42, 0x10, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
|
||||
0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73,
|
||||
0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f,
|
||||
0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65,
|
||||
0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74,
|
||||
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68,
|
||||
0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
|
||||
0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x24, 0x0a, 0x09,
|
||||
0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42,
|
||||
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75,
|
||||
0x72, 0x65, 0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
|
||||
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
|
||||
0x42, 0x10, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f,
|
||||
0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
|
||||
0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72,
|
||||
0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79,
|
||||
0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa,
|
||||
0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76,
|
||||
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65,
|
||||
0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62,
|
||||
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
@@ -923,7 +777,7 @@ func file_proto_prysm_v1alpha1_attestation_proto_rawDescGZIP() []byte {
|
||||
return file_proto_prysm_v1alpha1_attestation_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_proto_prysm_v1alpha1_attestation_proto_msgTypes = make([]protoimpl.MessageInfo, 11)
|
||||
var file_proto_prysm_v1alpha1_attestation_proto_msgTypes = make([]protoimpl.MessageInfo, 9)
|
||||
var file_proto_prysm_v1alpha1_attestation_proto_goTypes = []interface{}{
|
||||
(*SignedAggregateAttestationAndProof)(nil), // 0: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProof
|
||||
(*AggregateAttestationAndProof)(nil), // 1: ethereum.eth.v1alpha1.AggregateAttestationAndProof
|
||||
@@ -933,27 +787,23 @@ var file_proto_prysm_v1alpha1_attestation_proto_goTypes = []interface{}{
|
||||
(*SignedAggregateAttestationAndProofElectra)(nil), // 5: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProofElectra
|
||||
(*AggregateAttestationAndProofElectra)(nil), // 6: ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra
|
||||
(*AttestationElectra)(nil), // 7: ethereum.eth.v1alpha1.AttestationElectra
|
||||
(*SignedAggregateAttestationAndProofSingle)(nil), // 8: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProofSingle
|
||||
(*AggregateAttestationAndProofSingle)(nil), // 9: ethereum.eth.v1alpha1.AggregateAttestationAndProofSingle
|
||||
(*SingleAttestation)(nil), // 10: ethereum.eth.v1alpha1.SingleAttestation
|
||||
(*SingleAttestation)(nil), // 8: ethereum.eth.v1alpha1.SingleAttestation
|
||||
}
|
||||
var file_proto_prysm_v1alpha1_attestation_proto_depIdxs = []int32{
|
||||
1, // 0: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProof.message:type_name -> ethereum.eth.v1alpha1.AggregateAttestationAndProof
|
||||
2, // 1: ethereum.eth.v1alpha1.AggregateAttestationAndProof.aggregate:type_name -> ethereum.eth.v1alpha1.Attestation
|
||||
3, // 2: ethereum.eth.v1alpha1.Attestation.data:type_name -> ethereum.eth.v1alpha1.AttestationData
|
||||
4, // 3: ethereum.eth.v1alpha1.AttestationData.source:type_name -> ethereum.eth.v1alpha1.Checkpoint
|
||||
4, // 4: ethereum.eth.v1alpha1.AttestationData.target:type_name -> ethereum.eth.v1alpha1.Checkpoint
|
||||
6, // 5: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProofElectra.message:type_name -> ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra
|
||||
7, // 6: ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra.aggregate:type_name -> ethereum.eth.v1alpha1.AttestationElectra
|
||||
3, // 7: ethereum.eth.v1alpha1.AttestationElectra.data:type_name -> ethereum.eth.v1alpha1.AttestationData
|
||||
9, // 8: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProofSingle.message:type_name -> ethereum.eth.v1alpha1.AggregateAttestationAndProofSingle
|
||||
10, // 9: ethereum.eth.v1alpha1.AggregateAttestationAndProofSingle.aggregate:type_name -> ethereum.eth.v1alpha1.SingleAttestation
|
||||
3, // 10: ethereum.eth.v1alpha1.SingleAttestation.data:type_name -> ethereum.eth.v1alpha1.AttestationData
|
||||
11, // [11:11] is the sub-list for method output_type
|
||||
11, // [11:11] is the sub-list for method input_type
|
||||
11, // [11:11] is the sub-list for extension type_name
|
||||
11, // [11:11] is the sub-list for extension extendee
|
||||
0, // [0:11] is the sub-list for field type_name
|
||||
1, // 0: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProof.message:type_name -> ethereum.eth.v1alpha1.AggregateAttestationAndProof
|
||||
2, // 1: ethereum.eth.v1alpha1.AggregateAttestationAndProof.aggregate:type_name -> ethereum.eth.v1alpha1.Attestation
|
||||
3, // 2: ethereum.eth.v1alpha1.Attestation.data:type_name -> ethereum.eth.v1alpha1.AttestationData
|
||||
4, // 3: ethereum.eth.v1alpha1.AttestationData.source:type_name -> ethereum.eth.v1alpha1.Checkpoint
|
||||
4, // 4: ethereum.eth.v1alpha1.AttestationData.target:type_name -> ethereum.eth.v1alpha1.Checkpoint
|
||||
6, // 5: ethereum.eth.v1alpha1.SignedAggregateAttestationAndProofElectra.message:type_name -> ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra
|
||||
7, // 6: ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra.aggregate:type_name -> ethereum.eth.v1alpha1.AttestationElectra
|
||||
3, // 7: ethereum.eth.v1alpha1.AttestationElectra.data:type_name -> ethereum.eth.v1alpha1.AttestationData
|
||||
3, // 8: ethereum.eth.v1alpha1.SingleAttestation.data:type_name -> ethereum.eth.v1alpha1.AttestationData
|
||||
9, // [9:9] is the sub-list for method output_type
|
||||
9, // [9:9] is the sub-list for method input_type
|
||||
9, // [9:9] is the sub-list for extension type_name
|
||||
9, // [9:9] is the sub-list for extension extendee
|
||||
0, // [0:9] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_proto_prysm_v1alpha1_attestation_proto_init() }
|
||||
@@ -1059,30 +909,6 @@ func file_proto_prysm_v1alpha1_attestation_proto_init() {
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_attestation_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*SignedAggregateAttestationAndProofSingle); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_attestation_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*AggregateAttestationAndProofSingle); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_attestation_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*SingleAttestation); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
@@ -1101,7 +927,7 @@ func file_proto_prysm_v1alpha1_attestation_proto_init() {
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_proto_prysm_v1alpha1_attestation_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 11,
|
||||
NumMessages: 9,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
|
||||
@@ -154,29 +154,6 @@ message AttestationElectra {
|
||||
];
|
||||
}
|
||||
|
||||
message SignedAggregateAttestationAndProofSingle {
|
||||
// The aggregated attestation and selection proof itself.
|
||||
AggregateAttestationAndProofSingle message = 1;
|
||||
|
||||
// 96 byte BLS aggregate signature signed by the aggregator over the message.
|
||||
bytes signature = 2 [ (ethereum.eth.ext.ssz_size) = "96" ];
|
||||
}
|
||||
|
||||
message AggregateAttestationAndProofSingle {
|
||||
// The aggregator index that submitted this aggregated attestation and proof.
|
||||
uint64 aggregator_index = 1
|
||||
[ (ethereum.eth.ext.cast_type) =
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/"
|
||||
"primitives.ValidatorIndex" ];
|
||||
|
||||
// The aggregated attestation that was submitted.
|
||||
SingleAttestation aggregate = 3;
|
||||
|
||||
// 96 byte selection proof signed by the aggregator, which is the signature of
|
||||
// the slot to aggregate.
|
||||
bytes selection_proof = 2 [ (ethereum.eth.ext.ssz_size) = "96" ];
|
||||
}
|
||||
|
||||
message SingleAttestation {
|
||||
uint64 committee_id = 1 [ (ethereum.eth.ext.cast_type) =
|
||||
"github.com/OffchainLabs/prysm/v6/"
|
||||
|
||||
@@ -383,181 +383,6 @@ func (a *AttestationElectra) HashTreeRootWith(hh *ssz.Hasher) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
// MarshalSSZ ssz marshals the SignedAggregateAttestationAndProofSingle object
|
||||
func (s *SignedAggregateAttestationAndProofSingle) MarshalSSZ() ([]byte, error) {
|
||||
return ssz.MarshalSSZ(s)
|
||||
}
|
||||
|
||||
// MarshalSSZTo ssz marshals the SignedAggregateAttestationAndProofSingle object to a target array
|
||||
func (s *SignedAggregateAttestationAndProofSingle) MarshalSSZTo(buf []byte) (dst []byte, err error) {
|
||||
dst = buf
|
||||
|
||||
// Field (0) 'Message'
|
||||
if s.Message == nil {
|
||||
s.Message = new(AggregateAttestationAndProofSingle)
|
||||
}
|
||||
if dst, err = s.Message.MarshalSSZTo(dst); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (1) 'Signature'
|
||||
if size := len(s.Signature); size != 96 {
|
||||
err = ssz.ErrBytesLengthFn("--.Signature", size, 96)
|
||||
return
|
||||
}
|
||||
dst = append(dst, s.Signature...)
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// UnmarshalSSZ ssz unmarshals the SignedAggregateAttestationAndProofSingle object
|
||||
func (s *SignedAggregateAttestationAndProofSingle) UnmarshalSSZ(buf []byte) error {
|
||||
var err error
|
||||
size := uint64(len(buf))
|
||||
if size != 440 {
|
||||
return ssz.ErrSize
|
||||
}
|
||||
|
||||
// Field (0) 'Message'
|
||||
if s.Message == nil {
|
||||
s.Message = new(AggregateAttestationAndProofSingle)
|
||||
}
|
||||
if err = s.Message.UnmarshalSSZ(buf[0:344]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Field (1) 'Signature'
|
||||
if cap(s.Signature) == 0 {
|
||||
s.Signature = make([]byte, 0, len(buf[344:440]))
|
||||
}
|
||||
s.Signature = append(s.Signature, buf[344:440]...)
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// SizeSSZ returns the ssz encoded size in bytes for the SignedAggregateAttestationAndProofSingle object
|
||||
func (s *SignedAggregateAttestationAndProofSingle) SizeSSZ() (size int) {
|
||||
size = 440
|
||||
return
|
||||
}
|
||||
|
||||
// HashTreeRoot ssz hashes the SignedAggregateAttestationAndProofSingle object
|
||||
func (s *SignedAggregateAttestationAndProofSingle) HashTreeRoot() ([32]byte, error) {
|
||||
return ssz.HashWithDefaultHasher(s)
|
||||
}
|
||||
|
||||
// HashTreeRootWith ssz hashes the SignedAggregateAttestationAndProofSingle object with a hasher
|
||||
func (s *SignedAggregateAttestationAndProofSingle) HashTreeRootWith(hh *ssz.Hasher) (err error) {
|
||||
indx := hh.Index()
|
||||
|
||||
// Field (0) 'Message'
|
||||
if err = s.Message.HashTreeRootWith(hh); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (1) 'Signature'
|
||||
if size := len(s.Signature); size != 96 {
|
||||
err = ssz.ErrBytesLengthFn("--.Signature", size, 96)
|
||||
return
|
||||
}
|
||||
hh.PutBytes(s.Signature)
|
||||
|
||||
hh.Merkleize(indx)
|
||||
return
|
||||
}
|
||||
|
||||
// MarshalSSZ ssz marshals the AggregateAttestationAndProofSingle object
|
||||
func (a *AggregateAttestationAndProofSingle) MarshalSSZ() ([]byte, error) {
|
||||
return ssz.MarshalSSZ(a)
|
||||
}
|
||||
|
||||
// MarshalSSZTo ssz marshals the AggregateAttestationAndProofSingle object to a target array
|
||||
func (a *AggregateAttestationAndProofSingle) MarshalSSZTo(buf []byte) (dst []byte, err error) {
|
||||
dst = buf
|
||||
|
||||
// Field (0) 'AggregatorIndex'
|
||||
dst = ssz.MarshalUint64(dst, uint64(a.AggregatorIndex))
|
||||
|
||||
// Field (1) 'Aggregate'
|
||||
if a.Aggregate == nil {
|
||||
a.Aggregate = new(SingleAttestation)
|
||||
}
|
||||
if dst, err = a.Aggregate.MarshalSSZTo(dst); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (2) 'SelectionProof'
|
||||
if size := len(a.SelectionProof); size != 96 {
|
||||
err = ssz.ErrBytesLengthFn("--.SelectionProof", size, 96)
|
||||
return
|
||||
}
|
||||
dst = append(dst, a.SelectionProof...)
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// UnmarshalSSZ ssz unmarshals the AggregateAttestationAndProofSingle object
|
||||
func (a *AggregateAttestationAndProofSingle) UnmarshalSSZ(buf []byte) error {
|
||||
var err error
|
||||
size := uint64(len(buf))
|
||||
if size != 344 {
|
||||
return ssz.ErrSize
|
||||
}
|
||||
|
||||
// Field (0) 'AggregatorIndex'
|
||||
a.AggregatorIndex = github_com_OffchainLabs_prysm_v6_consensus_types_primitives.ValidatorIndex(ssz.UnmarshallUint64(buf[0:8]))
|
||||
|
||||
// Field (1) 'Aggregate'
|
||||
if a.Aggregate == nil {
|
||||
a.Aggregate = new(SingleAttestation)
|
||||
}
|
||||
if err = a.Aggregate.UnmarshalSSZ(buf[8:248]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Field (2) 'SelectionProof'
|
||||
if cap(a.SelectionProof) == 0 {
|
||||
a.SelectionProof = make([]byte, 0, len(buf[248:344]))
|
||||
}
|
||||
a.SelectionProof = append(a.SelectionProof, buf[248:344]...)
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// SizeSSZ returns the ssz encoded size in bytes for the AggregateAttestationAndProofSingle object
|
||||
func (a *AggregateAttestationAndProofSingle) SizeSSZ() (size int) {
|
||||
size = 344
|
||||
return
|
||||
}
|
||||
|
||||
// HashTreeRoot ssz hashes the AggregateAttestationAndProofSingle object
|
||||
func (a *AggregateAttestationAndProofSingle) HashTreeRoot() ([32]byte, error) {
|
||||
return ssz.HashWithDefaultHasher(a)
|
||||
}
|
||||
|
||||
// HashTreeRootWith ssz hashes the AggregateAttestationAndProofSingle object with a hasher
|
||||
func (a *AggregateAttestationAndProofSingle) HashTreeRootWith(hh *ssz.Hasher) (err error) {
|
||||
indx := hh.Index()
|
||||
|
||||
// Field (0) 'AggregatorIndex'
|
||||
hh.PutUint64(uint64(a.AggregatorIndex))
|
||||
|
||||
// Field (1) 'Aggregate'
|
||||
if err = a.Aggregate.HashTreeRootWith(hh); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (2) 'SelectionProof'
|
||||
if size := len(a.SelectionProof); size != 96 {
|
||||
err = ssz.ErrBytesLengthFn("--.SelectionProof", size, 96)
|
||||
return
|
||||
}
|
||||
hh.PutBytes(a.SelectionProof)
|
||||
|
||||
hh.Merkleize(indx)
|
||||
return
|
||||
}
|
||||
|
||||
// MarshalSSZ ssz marshals the SingleAttestation object
|
||||
func (s *SingleAttestation) MarshalSSZ() ([]byte, error) {
|
||||
return ssz.MarshalSSZ(s)
|
||||
|
||||
Reference in New Issue
Block a user