Correctly filter sync committee message at period boundary (#9626)

* Add head sync committee indices with slot consideration

* Apply rpc

* Apply sync

* Update tests

* Update mock.go

* Update sync_committee_test.go

* Update validator_test.go

Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>
This commit is contained in:
terence tsao 2021-09-20 11:06:43 -07:00 committed by GitHub
parent cb631360e9
commit 161a13ac09
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 103 additions and 103 deletions

View File

@ -23,8 +23,7 @@ var syncCommitteeHeadStateCache = cache.NewSyncCommitteeHeadState()
// HeadSyncCommitteeFetcher is the interface that wraps the head sync committee related functions.
// The head sync committee functions return callers sync committee indices and public keys with respect to current head state.
type HeadSyncCommitteeFetcher interface {
HeadCurrentSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error)
HeadNextSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error)
HeadSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error)
HeadSyncCommitteePubKeys(ctx context.Context, slot types.Slot, committeeIndex types.CommitteeIndex) ([][]byte, error)
}
@ -51,9 +50,35 @@ func (s *Service) HeadSyncContributionProofDomain(ctx context.Context, slot type
return s.domainWithHeadState(ctx, slot, params.BeaconConfig().DomainContributionAndProof)
}
// HeadCurrentSyncCommitteeIndices returns the input validator `index`'s position indices in the current sync committee with respect to `slot`.
// HeadSyncCommitteeIndices returns the sync committee index position using the head state. Input `slot` is taken in consideration
// where validator's duty for `slot - 1` is used for block inclusion in `slot`. That means when a validator is at epoch boundary
// across EPOCHS_PER_SYNC_COMMITTEE_PERIOD then the valiator will be considered using next period sync committee.
//
// Spec definition:
// Being assigned to a sync committee for a given slot means that the validator produces and broadcasts signatures for slot - 1 for inclusion in slot.
// This means that when assigned to an epoch sync committee signatures must be produced and broadcast for slots on range
// [compute_start_slot_at_epoch(epoch) - 1, compute_start_slot_at_epoch(epoch) + SLOTS_PER_EPOCH - 1)
// rather than for the range
// [compute_start_slot_at_epoch(epoch), compute_start_slot_at_epoch(epoch) + SLOTS_PER_EPOCH)
func (s *Service) HeadSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
nextSlotEpoch := core.SlotToEpoch(slot + 1)
currentEpoch := core.SlotToEpoch(slot)
switch {
case core.SyncCommitteePeriod(nextSlotEpoch) == core.SyncCommitteePeriod(currentEpoch):
return s.headCurrentSyncCommitteeIndices(ctx, index, slot)
// At sync committee period boundary, validator should sample the next epoch sync committee.
case core.SyncCommitteePeriod(nextSlotEpoch) == core.SyncCommitteePeriod(currentEpoch)+1:
return s.headNextSyncCommitteeIndices(ctx, index, slot)
default:
// Impossible condition.
return nil, errors.New("could get calculate sync subcommittee based on the period")
}
}
// headCurrentSyncCommitteeIndices returns the input validator `index`'s position indices in the current sync committee with respect to `slot`.
// Head state advanced up to `slot` is used for calculation.
func (s *Service) HeadCurrentSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
func (s *Service) headCurrentSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
headState, err := s.getSyncCommitteeHeadState(ctx, slot)
if err != nil {
return nil, err
@ -61,9 +86,9 @@ func (s *Service) HeadCurrentSyncCommitteeIndices(ctx context.Context, index typ
return helpers.CurrentPeriodSyncSubcommitteeIndices(headState, index)
}
// HeadNextSyncCommitteeIndices returns the input validator `index`'s position indices in the next sync committee with respect to `slot`.
// headNextSyncCommitteeIndices returns the input validator `index`'s position indices in the next sync committee with respect to `slot`.
// Head state advanced up to `slot` is used for calculation.
func (s *Service) HeadNextSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
func (s *Service) headNextSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
headState, err := s.getSyncCommitteeHeadState(ctx, slot)
if err != nil {
return nil, err

View File

@ -15,7 +15,7 @@ import (
"github.com/prysmaticlabs/prysm/shared/testutil/require"
)
func TestService_HeadSyncCommitteeFetcher_Errors(t *testing.T) {
func TestService_headSyncCommitteeFetcher_Errors(t *testing.T) {
beaconDB := dbtest.SetupDB(t)
c := &Service{
cfg: &Config{
@ -23,10 +23,10 @@ func TestService_HeadSyncCommitteeFetcher_Errors(t *testing.T) {
},
}
c.head = &head{}
_, err := c.HeadCurrentSyncCommitteeIndices(context.Background(), types.ValidatorIndex(0), types.Slot(0))
_, err := c.headCurrentSyncCommitteeIndices(context.Background(), types.ValidatorIndex(0), types.Slot(0))
require.ErrorContains(t, "nil state", err)
_, err = c.HeadNextSyncCommitteeIndices(context.Background(), types.ValidatorIndex(0), types.Slot(0))
_, err = c.headNextSyncCommitteeIndices(context.Background(), types.ValidatorIndex(0), types.Slot(0))
require.ErrorContains(t, "nil state", err)
_, err = c.HeadSyncCommitteePubKeys(context.Background(), types.Slot(0), types.CommitteeIndex(0))
@ -51,28 +51,51 @@ func TestService_HeadDomainFetcher_Errors(t *testing.T) {
require.ErrorContains(t, "nil state", err)
}
func TestService_HeadCurrentSyncCommitteeIndices(t *testing.T) {
func TestService_HeadSyncCommitteeIndices(t *testing.T) {
s, _ := testutil.DeterministicGenesisStateAltair(t, params.BeaconConfig().TargetCommitteeSize)
c := &Service{}
c.head = &head{state: s}
// Current period
slot := 2*uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*uint64(params.BeaconConfig().SlotsPerEpoch) + 1
a, err := c.HeadSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
require.NoError(t, err)
// Current period where slot-2 across EPOCHS_PER_SYNC_COMMITTEE_PERIOD
slot = 3*uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*uint64(params.BeaconConfig().SlotsPerEpoch) - 2
b, err := c.HeadSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
require.NoError(t, err)
require.DeepEqual(t, a, b)
// Next period where slot-1 across EPOCHS_PER_SYNC_COMMITTEE_PERIOD
slot = 3*uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*uint64(params.BeaconConfig().SlotsPerEpoch) - 1
b, err = c.HeadSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
require.NoError(t, err)
require.DeepNotEqual(t, a, b)
}
func TestService_headCurrentSyncCommitteeIndices(t *testing.T) {
s, _ := testutil.DeterministicGenesisStateAltair(t, params.BeaconConfig().TargetCommitteeSize)
c := &Service{}
c.head = &head{state: s}
// Process slot up to `EpochsPerSyncCommitteePeriod` so it can `ProcessSyncCommitteeUpdates`.
slot := uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*uint64(params.BeaconConfig().SlotsPerEpoch) + 1
indices, err := c.HeadCurrentSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
indices, err := c.headCurrentSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
require.NoError(t, err)
// NextSyncCommittee becomes CurrentSyncCommittee so it should be empty by default.
require.Equal(t, 0, len(indices))
}
func TestService_HeadNextSyncCommitteeIndices(t *testing.T) {
func TestService_headNextSyncCommitteeIndices(t *testing.T) {
s, _ := testutil.DeterministicGenesisStateAltair(t, params.BeaconConfig().TargetCommitteeSize)
c := &Service{}
c.head = &head{state: s}
// Process slot up to `EpochsPerSyncCommitteePeriod` so it can `ProcessSyncCommitteeUpdates`.
slot := uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*uint64(params.BeaconConfig().SlotsPerEpoch) + 1
indices, err := c.HeadNextSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
indices, err := c.headNextSyncCommitteeIndices(context.Background(), 0, types.Slot(slot))
require.NoError(t, err)
// NextSyncCommittee should be be empty after `ProcessSyncCommitteeUpdates`. Validator should get indices.

View File

@ -51,8 +51,7 @@ type ChainService struct {
ForkChoiceStore *protoarray.Store
VerifyBlkDescendantErr error
Slot *types.Slot // Pointer because 0 is a useful value, so checking against it can be incorrect.
CurrentSyncCommitteeIndices []types.CommitteeIndex
NextSyncCommitteeIndices []types.CommitteeIndex
SyncCommitteeIndices []types.CommitteeIndex
SyncCommitteeDomain []byte
SyncSelectionProofDomain []byte
SyncContributionProofDomain []byte
@ -410,14 +409,9 @@ func (s *ChainService) HeadValidatorIndexToPublicKey(ctx context.Context, index
return s.PublicKey, nil
}
// HeadCurrentSyncCommitteeIndices mocks HeadCurrentSyncCommitteeIndices and always return `CurrentSyncCommitteeIndices`.
func (s *ChainService) HeadCurrentSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
return s.CurrentSyncCommitteeIndices, nil
}
// HeadNextSyncCommitteeIndices mocks HeadNextSyncCommitteeIndices and always return `HeadNextSyncCommitteeIndices`.
func (s *ChainService) HeadNextSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
return s.NextSyncCommitteeIndices, nil
// HeadSyncCommitteeIndices mocks HeadSyncCommitteeIndices and always return `HeadNextSyncCommitteeIndices`.
func (s *ChainService) HeadSyncCommitteeIndices(ctx context.Context, index types.ValidatorIndex, slot types.Slot) ([]types.CommitteeIndex, error) {
return s.SyncCommitteeIndices, nil
}
// HeadSyncCommitteePubKeys mocks HeadSyncCommitteePubKeys and always return empty nil.

View File

@ -1526,7 +1526,7 @@ func TestProduceSyncCommitteeContribution(t *testing.T) {
v1Server := &v1alpha1validator.Server{
SyncCommitteePool: syncCommitteePool,
HeadFetcher: &mockChain.ChainService{
CurrentSyncCommitteeIndices: []types.CommitteeIndex{0},
SyncCommitteeIndices: []types.CommitteeIndex{0},
},
}
server := Server{

View File

@ -6,7 +6,6 @@ import (
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core"
"github.com/prysmaticlabs/prysm/crypto/bls"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
@ -36,15 +35,15 @@ func (vs *Server) GetSyncMessageBlockRoot(
func (vs *Server) SubmitSyncMessage(ctx context.Context, msg *ethpb.SyncCommitteeMessage) (*emptypb.Empty, error) {
errs, ctx := errgroup.WithContext(ctx)
idxResp, err := vs.syncSubcommitteeIndex(ctx, msg.ValidatorIndex, msg.Slot)
headSyncCommitteeIndices, err := vs.HeadFetcher.HeadSyncCommitteeIndices(ctx, msg.ValidatorIndex, msg.Slot)
if err != nil {
return &emptypb.Empty{}, err
}
// Broadcasting and saving message into the pool in parallel. As one fail should not affect another.
// This broadcasts for all subnets.
for _, id := range idxResp.Indices {
for _, index := range headSyncCommitteeIndices {
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
subnet := uint64(id) / subCommitteeSize
subnet := uint64(index) / subCommitteeSize
errs.Go(func() error {
return vs.P2P.BroadcastSyncCommitteeMessage(ctx, subnet, msg)
})
@ -68,43 +67,11 @@ func (vs *Server) GetSyncSubcommitteeIndex(
if !exists {
return nil, errors.New("public key does not exist in state")
}
indices, err := vs.syncSubcommitteeIndex(ctx, index, req.Slot)
indices, err := vs.HeadFetcher.HeadSyncCommitteeIndices(ctx, index, req.Slot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get sync subcommittee index: %v", err)
}
return indices, nil
}
// syncSubcommitteeIndex returns a list of subcommittee index of a validator and slot for sync message aggregation duty.
func (vs *Server) syncSubcommitteeIndex(
ctx context.Context, index types.ValidatorIndex, slot types.Slot,
) (*ethpb.SyncSubcommitteeIndexResponse, error) {
nextSlotEpoch := core.SlotToEpoch(slot + 1)
currentEpoch := core.SlotToEpoch(slot)
switch {
case core.SyncCommitteePeriod(nextSlotEpoch) == core.SyncCommitteePeriod(currentEpoch):
indices, err := vs.HeadFetcher.HeadCurrentSyncCommitteeIndices(ctx, index, slot)
if err != nil {
return nil, err
}
return &ethpb.SyncSubcommitteeIndexResponse{
Indices: indices,
}, nil
// At sync committee period boundary, validator should sample the next epoch sync committee.
case core.SyncCommitteePeriod(nextSlotEpoch) == core.SyncCommitteePeriod(currentEpoch)+1:
indices, err := vs.HeadFetcher.HeadNextSyncCommitteeIndices(ctx, index, slot)
if err != nil {
return nil, err
}
return &ethpb.SyncSubcommitteeIndexResponse{
Indices: indices,
}, nil
default:
// Impossible condition.
return nil, errors.New("could get calculate sync subcommittee based on the period")
}
return &ethpb.SyncSubcommitteeIndexResponse{Indices: indices}, nil
}
// GetSyncCommitteeContribution is called by a sync committee aggregator
@ -170,11 +137,11 @@ func (vs *Server) AggregatedSigAndAggregationBits(
bits := ethpb.NewSyncCommitteeAggregationBits()
for _, msg := range msgs {
if bytes.Equal(blockRoot, msg.BlockRoot) {
idxResp, err := vs.syncSubcommitteeIndex(ctx, msg.ValidatorIndex, slot)
headSyncCommitteeIndices, err := vs.HeadFetcher.HeadSyncCommitteeIndices(ctx, msg.ValidatorIndex, slot)
if err != nil {
return []byte{}, nil, errors.Wrapf(err, "could not get sync subcommittee index")
}
for _, index := range idxResp.Indices {
for _, index := range headSyncCommitteeIndices {
i := uint64(index)
subnetIndex := i / subCommitteeSize
if subnetIndex == subnetId {

View File

@ -54,8 +54,7 @@ func TestGetSyncSubcommitteeIndex_Ok(t *testing.T) {
server := &Server{
HeadFetcher: &mock.ChainService{
CurrentSyncCommitteeIndices: []types.CommitteeIndex{0},
NextSyncCommitteeIndices: []types.CommitteeIndex{1},
SyncCommitteeIndices: []types.CommitteeIndex{0},
},
}
pubKey := [48]byte{}
@ -65,14 +64,6 @@ func TestGetSyncSubcommitteeIndex_Ok(t *testing.T) {
})
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, res.Indices)
// Request at period boundary, should get index 1 for validator 0.
periodBoundary := types.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*params.BeaconConfig().SlotsPerEpoch - 1
res, err = server.GetSyncSubcommitteeIndex(context.Background(), &ethpb.SyncSubcommitteeIndexRequest{
PublicKey: pubKey[:], Slot: periodBoundary,
})
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{1}, res.Indices)
}
func TestSubmitSignedContributionAndProof_OK(t *testing.T) {

View File

@ -79,7 +79,7 @@ func (s *Service) validateSyncCommitteeMessage(
return pubsub.ValidationIgnore
}
committeeIndices, err := s.cfg.Chain.HeadCurrentSyncCommitteeIndices(ctx, m.ValidatorIndex, m.Slot)
committeeIndices, err := s.cfg.Chain.HeadSyncCommitteeIndices(ctx, m.ValidatorIndex, m.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore

View File

@ -216,9 +216,9 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
hState, err := beaconDB.State(context.Background(), headRoot)
assert.NoError(t, err)
s.cfg.Chain = &mockChain.ChainService{
CurrentSyncCommitteeIndices: []types.CommitteeIndex{0},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
SyncCommitteeIndices: []types.CommitteeIndex{0},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
}
numOfVals := hState.NumValidators()
@ -323,11 +323,11 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
assert.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
SyncCommitteeDomain: d,
PublicKey: bytesutil.ToBytes48(keys[chosenVal].PublicKey().Marshal()),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
SyncCommitteeDomain: d,
PublicKey: bytesutil.ToBytes48(keys[chosenVal].PublicKey().Marshal()),
}
// Set Topic and Subnet
@ -377,11 +377,11 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
assert.NoError(t, err)
s.cfg.Chain = &mockChain.ChainService{
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
SyncCommitteeDomain: d,
PublicKey: bytesutil.ToBytes48(keys[chosenVal].PublicKey().Marshal()),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(hState.Slot()-1)),
SyncCommitteeDomain: d,
PublicKey: bytesutil.ToBytes48(keys[chosenVal].PublicKey().Marshal()),
}
msg.Signature = keys[chosenVal].Sign(sigRoot[:]).Marshal()

View File

@ -152,7 +152,7 @@ func (s *Service) rejectInvalidIndexInSubCommittee(m *ethpb.SignedContributionAn
_, span := trace.StartSpan(ctx, "sync.rejectInvalidIndexInSubCommittee")
defer span.End()
// The aggregator's validator index is in the declared subcommittee of the current sync committee.
committeeIndices, err := s.cfg.Chain.HeadCurrentSyncCommitteeIndices(ctx, m.Message.AggregatorIndex, m.Message.Contribution.Slot)
committeeIndices, err := s.cfg.Chain.HeadSyncCommitteeIndices(ctx, m.Message.AggregatorIndex, m.Message.Contribution.Slot)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationIgnore

View File

@ -395,9 +395,9 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
}
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
}
msg.Message.Contribution.AggregationBits.SetBitAt(1, true)
@ -474,11 +474,11 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
require.NoError(t, err)
subCommitteeSize := params.BeaconConfig().SyncCommitteeSize / params.BeaconConfig().SyncCommitteeSubnetCount
s.cfg.Chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
PublicKey: bytesutil.ToBytes48(pubkey),
SyncSelectionProofDomain: d,
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
PublicKey: bytesutil.ToBytes48(pubkey),
SyncSelectionProofDomain: d,
}
s.initCaches()
@ -558,9 +558,9 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
}
}
s.cfg.Chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
CurrentSyncCommitteeIndices: []types.CommitteeIndex{1},
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
SyncCommitteeIndices: []types.CommitteeIndex{1},
}
s.initCaches()
@ -641,7 +641,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
s.cfg.Chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
PublicKey: bytesutil.ToBytes48(keys[msg.Message.AggregatorIndex].PublicKey().Marshal()),
SyncSelectionProofDomain: d,
SyncContributionProofDomain: cd,
@ -736,7 +736,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
s.cfg.Chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
PublicKey: bytesutil.ToBytes48(keys[msg.Message.AggregatorIndex].PublicKey().Marshal()),
SyncSelectionProofDomain: pd,
SyncContributionProofDomain: cd,
@ -834,7 +834,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
s.cfg.Chain = &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Now().Add(-time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(msg.Message.Contribution.Slot)),
CurrentSyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
SyncCommitteeIndices: []types.CommitteeIndex{types.CommitteeIndex(msg.Message.Contribution.SubcommitteeIndex * subCommitteeSize)},
PublicKey: bytesutil.ToBytes48(keys[msg.Message.AggregatorIndex].PublicKey().Marshal()),
SyncSelectionProofDomain: pd,
SyncContributionProofDomain: cd,