Skip to content

Commit

Permalink
Correctly filter sync committee message at period boundary (#9626)
Browse files Browse the repository at this point in the history
* 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>
  • Loading branch information
terencechain and prylabs-bulldozer[bot] authored Sep 20, 2021
1 parent cb63136 commit 161a13a
Show file tree
Hide file tree
Showing 10 changed files with 103 additions and 103 deletions.
37 changes: 31 additions & 6 deletions beacon-chain/blockchain/head_sync_committee_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand All @@ -51,19 +50,45 @@ 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
}
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
Expand Down
37 changes: 30 additions & 7 deletions beacon-chain/blockchain/head_sync_committee_info_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,18 +15,18 @@ 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{
StateGen: stategen.New(beaconDB),
},
}
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))
Expand All @@ -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.
Expand Down
14 changes: 4 additions & 10 deletions beacon-chain/blockchain/testing/mock.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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.
Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/rpc/eth/validator/validator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down
47 changes: 7 additions & 40 deletions beacon-chain/rpc/prysm/v1alpha1/validator/sync_committee.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
})
Expand All @@ -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
Expand Down Expand Up @@ -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 {
Expand Down
11 changes: 1 addition & 10 deletions beacon-chain/rpc/prysm/v1alpha1/validator/sync_committee_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{}
Expand All @@ -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) {
Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/sync/validate_sync_committee_message.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
26 changes: 13 additions & 13 deletions beacon-chain/sync/validate_sync_committee_message_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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()
Expand Down
2 changes: 1 addition & 1 deletion beacon-chain/sync/validate_sync_contribution_proof.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Loading

0 comments on commit 161a13a

Please sign in to comment.