Clean up committee helpers (#9504)

* Clean up committee related helper funcs

* Go fmt

Co-authored-by: Nishant Das <nishdas93@gmail.com>
This commit is contained in:
terence tsao
2021-09-01 08:17:10 -07:00
committed by GitHub
parent 44375a3d67
commit 46b22bb649
7 changed files with 686 additions and 630 deletions

View File

@@ -44,6 +44,7 @@ go_library(
"//shared/copyutil:go_default_library",
"//shared/featureconfig:go_default_library",
"//shared/hashutil:go_default_library",
"//shared/mathutil:go_default_library",
"//shared/params:go_default_library",
"//shared/rand:go_default_library",
"//shared/sliceutil:go_default_library",

View File

@@ -10,6 +10,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/shared/mathutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/sliceutil"
)
@@ -83,7 +84,10 @@ func (c *CommitteeCache) Committee(slot types.Slot, seed [32]byte, index types.C
committeeCountPerSlot = item.CommitteeCount / uint64(params.BeaconConfig().SlotsPerEpoch)
}
indexOffSet := uint64(index) + uint64(slot.ModSlot(params.BeaconConfig().SlotsPerEpoch).Mul(committeeCountPerSlot))
indexOffSet, err := mathutil.Add64(uint64(index), uint64(slot.ModSlot(params.BeaconConfig().SlotsPerEpoch).Mul(committeeCountPerSlot)))
if err != nil {
return nil, err
}
start, end := startEndIndices(item, indexOffSet)
if end > uint64(len(item.ShuffledIndices)) || end < start {

View File

@@ -4,14 +4,15 @@ go_library(
name = "go_default_library",
srcs = [
"attestation.go",
"beacon_committee.go",
"block.go",
"committee.go",
"genesis.go",
"randao.go",
"rewards_penalties.go",
"shuffle.go",
"signing_root.go",
"slot_epoch.go",
"sync_committee.go",
"validators.go",
"weak_subjectivity.go",
],
@@ -60,13 +61,14 @@ go_test(
size = "medium",
srcs = [
"attestation_test.go",
"beacon_committee_test.go",
"block_test.go",
"committee_test.go",
"randao_test.go",
"rewards_penalties_test.go",
"shuffle_test.go",
"signing_root_test.go",
"slot_epoch_test.go",
"sync_committee_test.go",
"validators_test.go",
"weak_subjectivity_test.go",
],

View File

@@ -15,22 +15,22 @@ import (
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/mathutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/sliceutil"
log "github.com/sirupsen/logrus"
)
var committeeCache = cache.NewCommitteesCache()
var proposerIndicesCache = cache.NewProposerIndicesCache()
var syncCommitteeCache = cache.NewSyncCommittee()
var (
committeeCache = cache.NewCommitteesCache()
proposerIndicesCache = cache.NewProposerIndicesCache()
)
// SlotCommitteeCount returns the number of crosslink committees of a slot. The
// active validator count is provided as an argument rather than a imported implementation
// SlotCommitteeCount returns the number of beacon committees of a slot. The
// active validator count is provided as an argument rather than an imported implementation
// from the spec definition. Having the active validator count as an argument allows for
// cheaper computation, instead of retrieving head state, one can retrieve the validator
// count.
//
//
// Spec pseudocode definition:
// def get_committee_count_per_slot(state: BeaconState, epoch: Epoch) -> uint64:
// """
@@ -41,16 +41,16 @@ var syncCommitteeCache = cache.NewSyncCommittee()
// uint64(len(get_active_validator_indices(state, epoch))) // SLOTS_PER_EPOCH // TARGET_COMMITTEE_SIZE,
// ))
func SlotCommitteeCount(activeValidatorCount uint64) uint64 {
var committeePerSlot = activeValidatorCount / uint64(params.BeaconConfig().SlotsPerEpoch) / params.BeaconConfig().TargetCommitteeSize
var committeesPerSlot = activeValidatorCount / uint64(params.BeaconConfig().SlotsPerEpoch) / params.BeaconConfig().TargetCommitteeSize
if committeePerSlot > params.BeaconConfig().MaxCommitteesPerSlot {
if committeesPerSlot > params.BeaconConfig().MaxCommitteesPerSlot {
return params.BeaconConfig().MaxCommitteesPerSlot
}
if committeePerSlot == 0 {
if committeesPerSlot == 0 {
return 1
}
return committeePerSlot
return committeesPerSlot
}
// BeaconCommitteeFromState returns the crosslink committee of a given slot and committee index. This
@@ -77,12 +77,12 @@ func BeaconCommitteeFromState(state state.ReadOnlyBeaconState, slot types.Slot,
return nil, errors.Wrap(err, "could not get seed")
}
indices, err := committeeCache.Committee(slot, seed, committeeIndex)
committee, err := committeeCache.Committee(slot, seed, committeeIndex)
if err != nil {
return nil, errors.Wrap(err, "could not interface with committee cache")
}
if indices != nil {
return indices, nil
if committee != nil {
return committee, nil
}
activeIndices, err := ActiveValidatorIndices(state, epoch)
@@ -93,73 +93,49 @@ func BeaconCommitteeFromState(state state.ReadOnlyBeaconState, slot types.Slot,
return BeaconCommittee(activeIndices, seed, slot, committeeIndex)
}
// BeaconCommittee returns the crosslink committee of a given slot and committee index. The
// validator indices and seed are provided as an argument rather than a imported implementation
// BeaconCommittee returns the beacon committee of a given slot and committee index. The
// validator indices and seed are provided as an argument rather than an imported implementation
// from the spec definition. Having them as an argument allows for cheaper computation run time.
//
// Spec pseudocode definition:
// def get_beacon_committee(state: BeaconState, slot: Slot, index: CommitteeIndex) -> Sequence[ValidatorIndex]:
// """
// Return the beacon committee at ``slot`` for ``index``.
// """
// epoch = compute_epoch_at_slot(slot)
// committees_per_slot = get_committee_count_per_slot(state, epoch)
// return compute_committee(
// indices=get_active_validator_indices(state, epoch),
// seed=get_seed(state, epoch, DOMAIN_BEACON_ATTESTER),
// index=(slot % SLOTS_PER_EPOCH) * committees_per_slot + index,
// count=committees_per_slot * SLOTS_PER_EPOCH,
// )
func BeaconCommittee(
validatorIndices []types.ValidatorIndex,
seed [32]byte,
slot types.Slot,
committeeIndex types.CommitteeIndex,
) ([]types.ValidatorIndex, error) {
indices, err := committeeCache.Committee(slot, seed, committeeIndex)
committee, err := committeeCache.Committee(slot, seed, committeeIndex)
if err != nil {
return nil, errors.Wrap(err, "could not interface with committee cache")
}
if indices != nil {
return indices, nil
if committee != nil {
return committee, nil
}
committeesPerSlot := SlotCommitteeCount(uint64(len(validatorIndices)))
epochOffset := uint64(committeeIndex) + uint64(slot.ModSlot(params.BeaconConfig().SlotsPerEpoch).Mul(committeesPerSlot))
indexOffset, err := mathutil.Add64(uint64(committeeIndex), uint64(slot.ModSlot(params.BeaconConfig().SlotsPerEpoch).Mul(committeesPerSlot)))
if err != nil {
return nil, errors.Wrap(err, "could not add calculate index offset")
}
count := committeesPerSlot * uint64(params.BeaconConfig().SlotsPerEpoch)
return ComputeCommittee(validatorIndices, seed, epochOffset, count)
return computeCommittee(validatorIndices, seed, indexOffset, count)
}
// ComputeCommittee returns the requested shuffled committee out of the total committees using
// validator indices and seed.
//
// Spec pseudocode definition:
// def compute_committee(indices: Sequence[ValidatorIndex],
// seed: Bytes32,
// index: uint64,
// count: uint64) -> Sequence[ValidatorIndex]:
// """
// Return the committee corresponding to ``indices``, ``seed``, ``index``, and committee ``count``.
// """
// start = (len(indices) * index) // count
// end = (len(indices) * uint64(index + 1)) // count
// return [indices[compute_shuffled_index(uint64(i), uint64(len(indices)), seed)] for i in range(start, end)]
func ComputeCommittee(
indices []types.ValidatorIndex,
seed [32]byte,
index, count uint64,
) ([]types.ValidatorIndex, error) {
validatorCount := uint64(len(indices))
start := sliceutil.SplitOffset(validatorCount, count, index)
end := sliceutil.SplitOffset(validatorCount, count, index+1)
if start > validatorCount || end > validatorCount {
return nil, errors.New("index out of range")
}
// Save the shuffled indices in cache, this is only needed once per epoch or once per new committee index.
shuffledIndices := make([]types.ValidatorIndex, len(indices))
copy(shuffledIndices, indices)
// UnshuffleList is used here as it is an optimized implementation created
// for fast computation of committees.
// Reference implementation: https://github.com/protolambda/eth2-shuffle
shuffledList, err := UnshuffleList(shuffledIndices, seed)
if err != nil {
return nil, err
}
return shuffledList[start:end], nil
}
// CommitteeAssignmentContainer represents a committee, index, and attester slot for a given epoch.
// CommitteeAssignmentContainer represents a committee list, committee index, and to be attested slot for a given epoch.
type CommitteeAssignmentContainer struct {
Committee []types.ValidatorIndex
AttesterSlot types.Slot
@@ -392,193 +368,45 @@ func ClearCache() {
syncCommitteeCache = cache.NewSyncCommittee()
}
// IsCurrentPeriodSyncCommittee returns true if the input validator index belongs in the current period sync committee
// along with the sync committee root.
// 1.) Checks if the public key exists in the sync committee cache
// 2.) If 1 fails, checks if the public key exists in the input current sync committee object
func IsCurrentPeriodSyncCommittee(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) (bool, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return false, err
}
indices, err := syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return false, nil
}
committee, err := st.CurrentSyncCommittee()
if err != nil {
return false, err
}
// computeCommittee returns the requested shuffled committee out of the total committees using
// validator indices and seed.
//
// Spec pseudocode definition:
// def compute_committee(indices: Sequence[ValidatorIndex],
// seed: Bytes32,
// index: uint64,
// count: uint64) -> Sequence[ValidatorIndex]:
// """
// Return the committee corresponding to ``indices``, ``seed``, ``index``, and committee ``count``.
// """
// start = (len(indices) * index) // count
// end = (len(indices) * uint64(index + 1)) // count
// return [indices[compute_shuffled_index(uint64(i), uint64(len(indices)), seed)] for i in range(start, end)]
func computeCommittee(
indices []types.ValidatorIndex,
seed [32]byte,
index, count uint64,
) ([]types.ValidatorIndex, error) {
validatorCount := uint64(len(indices))
start := sliceutil.SplitOffset(validatorCount, count, index)
end := sliceutil.SplitOffset(validatorCount, count, index+1)
// Fill in the cache on miss.
go func() {
if err := syncCommitteeCache.UpdatePositionsInCommittee(bytesutil.ToBytes32(root), st); err != nil {
log.Errorf("Could not fill sync committee cache on miss: %v", err)
}
}()
return len(findSubCommitteeIndices(val.PublicKey, committee.Pubkeys)) > 0, nil
}
if err != nil {
return false, err
}
return len(indices) > 0, nil
}
// IsNextPeriodSyncCommittee returns true if the input validator index belongs in the next period sync committee
// along with the sync period boundary root.
// 1.) Checks if the public key exists in the sync committee cache
// 2.) If 1 fails, checks if the public key exists in the input next sync committee object
func IsNextPeriodSyncCommittee(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) (bool, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return false, err
}
indices, err := syncCommitteeCache.NextPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return false, nil
}
committee, err := st.NextSyncCommittee()
if err != nil {
return false, err
}
return len(findSubCommitteeIndices(val.PublicKey, committee.Pubkeys)) > 0, nil
}
if err != nil {
return false, err
}
return len(indices) > 0, nil
}
// CurrentPeriodSyncSubcommitteeIndices returns the subcommittee indices of the
// current period sync committee for input validator.
func CurrentPeriodSyncSubcommitteeIndices(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) ([]types.CommitteeIndex, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return nil, err
}
indices, err := syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return nil, nil
}
committee, err := st.CurrentSyncCommittee()
if err != nil {
return nil, err
}
// Fill in the cache on miss.
go func() {
if err := syncCommitteeCache.UpdatePositionsInCommittee(bytesutil.ToBytes32(root), st); err != nil {
log.Errorf("Could not fill sync committee cache on miss: %v", err)
}
}()
return findSubCommitteeIndices(val.PublicKey, committee.Pubkeys), nil
}
if err != nil {
return nil, err
}
return indices, nil
}
// NextPeriodSyncSubcommitteeIndices returns the subcommittee indices of the next period sync committee for input validator.
func NextPeriodSyncSubcommitteeIndices(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) ([]types.CommitteeIndex, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return nil, err
}
indices, err := syncCommitteeCache.NextPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return nil, nil
}
committee, err := st.NextSyncCommittee()
if err != nil {
return nil, err
}
return findSubCommitteeIndices(val.PublicKey, committee.Pubkeys), nil
}
if err != nil {
return nil, err
}
return indices, nil
}
// UpdateSyncCommitteeCache updates sync committee cache.
// It uses `state`'s latest block header root as key. To avoid misuse, it disallows
// block header with state root zeroed out.
func UpdateSyncCommitteeCache(st state.BeaconStateAltair) error {
nextSlot := st.Slot() + 1
if nextSlot%params.BeaconConfig().SlotsPerEpoch != 0 {
return errors.New("not at the end of the epoch to update cache")
}
if SlotToEpoch(nextSlot)%params.BeaconConfig().EpochsPerSyncCommitteePeriod != 0 {
return errors.New("not at sync committee period boundary to update cache")
if start > validatorCount || end > validatorCount {
return nil, errors.New("index out of range")
}
header := st.LatestBlockHeader()
if bytes.Equal(header.StateRoot, params.BeaconConfig().ZeroHash[:]) {
return errors.New("zero hash state root can't be used to update cache")
}
prevBlockRoot, err := header.HashTreeRoot()
if err != nil {
return err
}
return syncCommitteeCache.UpdatePositionsInCommittee(prevBlockRoot, st)
}
// Loop through `pubKeys` for matching `pubKey` and get the indices where it matches.
func findSubCommitteeIndices(pubKey []byte, pubKeys [][]byte) []types.CommitteeIndex {
var indices []types.CommitteeIndex
for i, k := range pubKeys {
if bytes.Equal(k, pubKey) {
indices = append(indices, types.CommitteeIndex(i))
}
}
return indices
}
// Retrieve the current sync period boundary root by calculating sync period start epoch
// and calling `BlockRoot`.
// It uses the boundary slot - 1 for block root. (Ex: SlotsPerEpoch * EpochsPerSyncCommitteePeriod - 1)
func syncPeriodBoundaryRoot(st state.ReadOnlyBeaconState) ([]byte, error) {
// Can't call `BlockRoot` until the first slot.
if st.Slot() == params.BeaconConfig().GenesisSlot {
return params.BeaconConfig().ZeroHash[:], nil
}
startEpoch, err := SyncCommitteePeriodStartEpoch(CurrentEpoch(st))
if err != nil {
return nil, err
}
startEpochSlot, err := StartSlot(startEpoch)
// Save the shuffled indices in cache, this is only needed once per epoch or once per new committee index.
shuffledIndices := make([]types.ValidatorIndex, len(indices))
copy(shuffledIndices, indices)
// UnshuffleList is used here as it is an optimized implementation created
// for fast computation of committees.
// Reference implementation: https://github.com/protolambda/eth2-shuffle
shuffledList, err := UnshuffleList(shuffledIndices, seed)
if err != nil {
return nil, err
}
// Prevent underflow
if startEpochSlot >= 1 {
startEpochSlot--
}
return BlockRootAtSlot(st, startEpochSlot)
return shuffledList[start:end], nil
}
// This computes proposer indices of the current epoch and returns a list of proposer indices,

View File

@@ -4,13 +4,10 @@ import (
"fmt"
"strconv"
"testing"
"time"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/beacon-chain/cache"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
v2 "github.com/prysmaticlabs/prysm/beacon-chain/state/v2"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/hashutil"
@@ -50,12 +47,12 @@ func TestComputeCommittee_WithoutCache(t *testing.T) {
require.NoError(t, err)
seed, err := Seed(state, epoch, params.BeaconConfig().DomainBeaconAttester)
require.NoError(t, err)
committees, err := ComputeCommittee(indices, seed, 0, 1 /* Total committee*/)
committees, err := computeCommittee(indices, seed, 0, 1 /* Total committee*/)
assert.NoError(t, err, "Could not compute committee")
// Test shuffled indices are correct for index 5 committee
index := uint64(5)
committee5, err := ComputeCommittee(indices, seed, index, committeeCount)
committee5, err := computeCommittee(indices, seed, index, committeeCount)
assert.NoError(t, err, "Could not compute committee")
start := sliceutil.SplitOffset(validatorCount, committeeCount, index)
end := sliceutil.SplitOffset(validatorCount, committeeCount, index+1)
@@ -63,7 +60,7 @@ func TestComputeCommittee_WithoutCache(t *testing.T) {
// Test shuffled indices are correct for index 9 committee
index = uint64(9)
committee9, err := ComputeCommittee(indices, seed, index, committeeCount)
committee9, err := computeCommittee(indices, seed, index, committeeCount)
assert.NoError(t, err, "Could not compute committee")
start = sliceutil.SplitOffset(validatorCount, committeeCount, index)
end = sliceutil.SplitOffset(validatorCount, committeeCount, index+1)
@@ -75,7 +72,7 @@ func TestComputeCommittee_RegressionTest(t *testing.T) {
seed := [32]byte{68, 110, 161, 250, 98, 230, 161, 172, 227, 226, 99, 11, 138, 124, 201, 134, 38, 197, 0, 120, 6, 165, 122, 34, 19, 216, 43, 226, 210, 114, 165, 183}
index := uint64(215)
count := uint64(32)
_, err := ComputeCommittee(indices, seed, index, count)
_, err := computeCommittee(indices, seed, index, count)
require.ErrorContains(t, "index out of range", err)
}
@@ -418,14 +415,14 @@ func BenchmarkComputeCommittee300000_WithPreCache(b *testing.B) {
require.NoError(b, err)
index := uint64(3)
_, err = ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err = computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
b.ResetTimer()
for n := 0; n < b.N; n++ {
_, err := ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err := computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
@@ -452,14 +449,14 @@ func BenchmarkComputeCommittee3000000_WithPreCache(b *testing.B) {
require.NoError(b, err)
index := uint64(3)
_, err = ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err = computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
b.ResetTimer()
for n := 0; n < b.N; n++ {
_, err := ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err := computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
@@ -490,7 +487,7 @@ func BenchmarkComputeCommittee128000_WithOutPreCache(b *testing.B) {
b.ResetTimer()
for n := 0; n < b.N; n++ {
i++
_, err := ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err := computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
@@ -525,7 +522,7 @@ func BenchmarkComputeCommittee1000000_WithOutCache(b *testing.B) {
b.ResetTimer()
for n := 0; n < b.N; n++ {
i++
_, err := ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err := computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
@@ -560,7 +557,7 @@ func BenchmarkComputeCommittee4000000_WithOutCache(b *testing.B) {
b.ResetTimer()
for n := 0; n < b.N; n++ {
i++
_, err := ComputeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
_, err := computeCommittee(indices, seed, index, params.BeaconConfig().MaxCommitteesPerSlot)
if err != nil {
panic(err)
}
@@ -629,373 +626,3 @@ func TestPrecomputeProposerIndices_Ok(t *testing.T) {
}
assert.DeepEqual(t, wantedProposerIndices, proposerIndices, "Did not precompute proposer indices correctly")
}
func TestIsCurrentEpochSyncCommittee_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
ok, err := IsCurrentPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsCurrentEpochSyncCommittee_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsCurrentPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsCurrentEpochSyncCommittee_DoesNotExist(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsCurrentPeriodSyncCommittee(state, 12390192)
require.NoError(t, err)
require.Equal(t, false, ok)
}
func TestIsNextEpochSyncCommittee_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
ok, err := IsNextPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsNextEpochSyncCommittee_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsNextPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsNextEpochSyncCommittee_DoesNotExist(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsNextPeriodSyncCommittee(state, 120391029)
require.NoError(t, err)
require.Equal(t, false, ok)
}
func TestCurrentEpochSyncSubcommitteeIndices_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
index, err := CurrentPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestCurrentEpochSyncSubcommitteeIndices_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
root, err := syncPeriodBoundaryRoot(state)
require.NoError(t, err)
// Test that cache was empty.
_, err = syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), 0)
require.Equal(t, cache.ErrNonExistingSyncCommitteeKey, err)
// Test that helper can retrieve the index given empty cache.
index, err := CurrentPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
// Test that cache was able to fill on miss.
time.Sleep(100 * time.Millisecond)
index, err = syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestCurrentEpochSyncSubcommitteeIndices_DoesNotExist(t *testing.T) {
ClearCache()
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
index, err := CurrentPeriodSyncSubcommitteeIndices(state, 129301923)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex(nil), index)
}
func TestNextEpochSyncSubcommitteeIndices_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
index, err := NextPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestNextEpochSyncSubcommitteeIndices_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
index, err := NextPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestNextEpochSyncSubcommitteeIndices_DoesNotExist(t *testing.T) {
ClearCache()
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
index, err := NextPeriodSyncSubcommitteeIndices(state, 21093019)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex(nil), index)
}
func TestUpdateSyncCommitteeCache_BadSlot(t *testing.T) {
state, err := v1.InitializeFromProto(&ethpb.BeaconState{
Slot: 1,
})
require.NoError(t, err)
err = UpdateSyncCommitteeCache(state)
require.ErrorContains(t, "not at the end of the epoch to update cache", err)
state, err = v1.InitializeFromProto(&ethpb.BeaconState{
Slot: params.BeaconConfig().SlotsPerEpoch - 1,
})
require.NoError(t, err)
err = UpdateSyncCommitteeCache(state)
require.ErrorContains(t, "not at sync committee period boundary to update cache", err)
}
func TestUpdateSyncCommitteeCache_BadRoot(t *testing.T) {
state, err := v1.InitializeFromProto(&ethpb.BeaconState{
Slot: types.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*params.BeaconConfig().SlotsPerEpoch - 1,
LatestBlockHeader: &ethpb.BeaconBlockHeader{StateRoot: params.BeaconConfig().ZeroHash[:]},
})
require.NoError(t, err)
err = UpdateSyncCommitteeCache(state)
require.ErrorContains(t, "zero hash state root can't be used to update cache", err)
}

View File

@@ -0,0 +1,208 @@
// Package helpers contains helper functions outlined in the Ethereum Beacon Chain spec, such as
// computing committees, randao, rewards/penalties, and more.
package helpers
import (
"bytes"
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/params"
log "github.com/sirupsen/logrus"
)
var (
syncCommitteeCache = cache.NewSyncCommittee()
)
// IsCurrentPeriodSyncCommittee returns true if the input validator index belongs in the current period sync committee
// along with the sync committee root.
// 1. Checks if the public key exists in the sync committee cache
// 2. If 1 fails, checks if the public key exists in the input current sync committee object
func IsCurrentPeriodSyncCommittee(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) (bool, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return false, err
}
indices, err := syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return false, nil
}
committee, err := st.CurrentSyncCommittee()
if err != nil {
return false, err
}
// Fill in the cache on miss.
go func() {
if err := syncCommitteeCache.UpdatePositionsInCommittee(bytesutil.ToBytes32(root), st); err != nil {
log.Errorf("Could not fill sync committee cache on miss: %v", err)
}
}()
return len(findSubCommitteeIndices(val.PublicKey, committee.Pubkeys)) > 0, nil
}
if err != nil {
return false, err
}
return len(indices) > 0, nil
}
// IsNextPeriodSyncCommittee returns true if the input validator index belongs in the next period sync committee
// along with the sync period boundary root.
// 1. Checks if the public key exists in the sync committee cache
// 2. If 1 fails, checks if the public key exists in the input next sync committee object
func IsNextPeriodSyncCommittee(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) (bool, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return false, err
}
indices, err := syncCommitteeCache.NextPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return false, nil
}
committee, err := st.NextSyncCommittee()
if err != nil {
return false, err
}
return len(findSubCommitteeIndices(val.PublicKey, committee.Pubkeys)) > 0, nil
}
if err != nil {
return false, err
}
return len(indices) > 0, nil
}
// CurrentPeriodSyncSubcommitteeIndices returns the subcommittee indices of the
// current period sync committee for input validator.
func CurrentPeriodSyncSubcommitteeIndices(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) ([]types.CommitteeIndex, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return nil, err
}
indices, err := syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return nil, nil
}
committee, err := st.CurrentSyncCommittee()
if err != nil {
return nil, err
}
// Fill in the cache on miss.
go func() {
if err := syncCommitteeCache.UpdatePositionsInCommittee(bytesutil.ToBytes32(root), st); err != nil {
log.Errorf("Could not fill sync committee cache on miss: %v", err)
}
}()
return findSubCommitteeIndices(val.PublicKey, committee.Pubkeys), nil
}
if err != nil {
return nil, err
}
return indices, nil
}
// NextPeriodSyncSubcommitteeIndices returns the subcommittee indices of the next period sync committee for input validator.
func NextPeriodSyncSubcommitteeIndices(
st state.BeaconStateAltair, valIdx types.ValidatorIndex,
) ([]types.CommitteeIndex, error) {
root, err := syncPeriodBoundaryRoot(st)
if err != nil {
return nil, err
}
indices, err := syncCommitteeCache.NextPeriodIndexPosition(bytesutil.ToBytes32(root), valIdx)
if err == cache.ErrNonExistingSyncCommitteeKey {
val, err := st.ValidatorAtIndex(valIdx)
if err != nil {
return nil, nil
}
committee, err := st.NextSyncCommittee()
if err != nil {
return nil, err
}
return findSubCommitteeIndices(val.PublicKey, committee.Pubkeys), nil
}
if err != nil {
return nil, err
}
return indices, nil
}
// UpdateSyncCommitteeCache updates sync committee cache.
// It uses `state`'s latest block header root as key. To avoid misuse, it disallows
// block header with state root zeroed out.
func UpdateSyncCommitteeCache(st state.BeaconStateAltair) error {
nextSlot := st.Slot() + 1
if nextSlot%params.BeaconConfig().SlotsPerEpoch != 0 {
return errors.New("not at the end of the epoch to update cache")
}
if SlotToEpoch(nextSlot)%params.BeaconConfig().EpochsPerSyncCommitteePeriod != 0 {
return errors.New("not at sync committee period boundary to update cache")
}
header := st.LatestBlockHeader()
if bytes.Equal(header.StateRoot, params.BeaconConfig().ZeroHash[:]) {
return errors.New("zero hash state root can't be used to update cache")
}
prevBlockRoot, err := header.HashTreeRoot()
if err != nil {
return err
}
return syncCommitteeCache.UpdatePositionsInCommittee(prevBlockRoot, st)
}
// Loop through `pubKeys` for matching `pubKey` and get the indices where it matches.
func findSubCommitteeIndices(pubKey []byte, pubKeys [][]byte) []types.CommitteeIndex {
var indices []types.CommitteeIndex
for i, k := range pubKeys {
if bytes.Equal(k, pubKey) {
indices = append(indices, types.CommitteeIndex(i))
}
}
return indices
}
// Retrieve the current sync period boundary root by calculating sync period start epoch
// and calling `BlockRoot`.
// It uses the boundary slot - 1 for block root. (Ex: SlotsPerEpoch * EpochsPerSyncCommitteePeriod - 1)
func syncPeriodBoundaryRoot(st state.ReadOnlyBeaconState) ([]byte, error) {
// Can't call `BlockRoot` until the first slot.
if st.Slot() == params.BeaconConfig().GenesisSlot {
return params.BeaconConfig().ZeroHash[:], nil
}
startEpoch, err := SyncCommitteePeriodStartEpoch(CurrentEpoch(st))
if err != nil {
return nil, err
}
startEpochSlot, err := StartSlot(startEpoch)
if err != nil {
return nil, err
}
// Prevent underflow
if startEpochSlot >= 1 {
startEpochSlot--
}
return BlockRootAtSlot(st, startEpochSlot)
}

View File

@@ -0,0 +1,386 @@
package helpers
import (
"strconv"
"testing"
"time"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/cache"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
v2 "github.com/prysmaticlabs/prysm/beacon-chain/state/v2"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/testutil/require"
)
func TestIsCurrentEpochSyncCommittee_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
ok, err := IsCurrentPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsCurrentEpochSyncCommittee_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsCurrentPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsCurrentEpochSyncCommittee_DoesNotExist(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsCurrentPeriodSyncCommittee(state, 12390192)
require.NoError(t, err)
require.Equal(t, false, ok)
}
func TestIsNextEpochSyncCommittee_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
ok, err := IsNextPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsNextEpochSyncCommittee_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsNextPeriodSyncCommittee(state, 0)
require.NoError(t, err)
require.Equal(t, true, ok)
}
func TestIsNextEpochSyncCommittee_DoesNotExist(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ok, err := IsNextPeriodSyncCommittee(state, 120391029)
require.NoError(t, err)
require.Equal(t, false, ok)
}
func TestCurrentEpochSyncSubcommitteeIndices_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
index, err := CurrentPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestCurrentEpochSyncSubcommitteeIndices_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
root, err := syncPeriodBoundaryRoot(state)
require.NoError(t, err)
// Test that cache was empty.
_, err = syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), 0)
require.Equal(t, cache.ErrNonExistingSyncCommitteeKey, err)
// Test that helper can retrieve the index given empty cache.
index, err := CurrentPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
// Test that cache was able to fill on miss.
time.Sleep(100 * time.Millisecond)
index, err = syncCommitteeCache.CurrentPeriodIndexPosition(bytesutil.ToBytes32(root), 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestCurrentEpochSyncSubcommitteeIndices_DoesNotExist(t *testing.T) {
ClearCache()
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
index, err := CurrentPeriodSyncSubcommitteeIndices(state, 129301923)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex(nil), index)
}
func TestNextEpochSyncSubcommitteeIndices_UsingCache(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
ClearCache()
r := [32]byte{'a'}
require.NoError(t, err, syncCommitteeCache.UpdatePositionsInCommittee(r, state))
index, err := NextPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestNextEpochSyncSubcommitteeIndices_UsingCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
index, err := NextPeriodSyncSubcommitteeIndices(state, 0)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex{0}, index)
}
func TestNextEpochSyncSubcommitteeIndices_DoesNotExist(t *testing.T) {
ClearCache()
validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize)
syncCommittee := &ethpb.SyncCommittee{
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
}
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
}
syncCommittee.Pubkeys = append(syncCommittee.Pubkeys, bytesutil.PadTo(k, 48))
}
state, err := v2.InitializeFromProto(&ethpb.BeaconStateAltair{
Validators: validators,
})
require.NoError(t, err)
require.NoError(t, state.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, state.SetNextSyncCommittee(syncCommittee))
index, err := NextPeriodSyncSubcommitteeIndices(state, 21093019)
require.NoError(t, err)
require.DeepEqual(t, []types.CommitteeIndex(nil), index)
}
func TestUpdateSyncCommitteeCache_BadSlot(t *testing.T) {
state, err := v1.InitializeFromProto(&ethpb.BeaconState{
Slot: 1,
})
require.NoError(t, err)
err = UpdateSyncCommitteeCache(state)
require.ErrorContains(t, "not at the end of the epoch to update cache", err)
state, err = v1.InitializeFromProto(&ethpb.BeaconState{
Slot: params.BeaconConfig().SlotsPerEpoch - 1,
})
require.NoError(t, err)
err = UpdateSyncCommitteeCache(state)
require.ErrorContains(t, "not at sync committee period boundary to update cache", err)
}
func TestUpdateSyncCommitteeCache_BadRoot(t *testing.T) {
state, err := v1.InitializeFromProto(&ethpb.BeaconState{
Slot: types.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)*params.BeaconConfig().SlotsPerEpoch - 1,
LatestBlockHeader: &ethpb.BeaconBlockHeader{StateRoot: params.BeaconConfig().ZeroHash[:]},
})
require.NoError(t, err)
err = UpdateSyncCommitteeCache(state)
require.ErrorContains(t, "zero hash state root can't be used to update cache", err)
}