Use fieldparams for BLS public key (#10042)

* Use fieldparams for pubkey length

* Fix validator tests

* fix more tests

* fix mock validator

* Fix typo

* bunch of typos

* Update bytes.go

* Update BUILD.bazel

Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>
This commit is contained in:
terence tsao
2022-01-06 09:33:08 -08:00
committed by GitHub
parent ad06230291
commit c69bce5d84
133 changed files with 672 additions and 539 deletions

View File

@@ -54,6 +54,7 @@ go_library(
"//beacon-chain/state/stategen:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -8,6 +8,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/forkchoice/protoarray"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -49,8 +50,8 @@ type HeadFetcher interface {
HeadSeed(ctx context.Context, epoch types.Epoch) ([32]byte, error)
HeadGenesisValidatorRoot() [32]byte
HeadETH1Data() *ethpb.Eth1Data
HeadPublicKeyToValidatorIndex(ctx context.Context, pubKey [48]byte) (types.ValidatorIndex, bool)
HeadValidatorIndexToPublicKey(ctx context.Context, index types.ValidatorIndex) ([48]byte, error)
HeadPublicKeyToValidatorIndex(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool)
HeadValidatorIndexToPublicKey(ctx context.Context, index types.ValidatorIndex) ([fieldparams.BLSPubkeyLength]byte, error)
ProtoArrayStore() *protoarray.Store
ChainHeads() ([][32]byte, []types.Slot)
HeadSyncCommitteeFetcher
@@ -290,7 +291,7 @@ func (s *Service) ChainHeads() ([][32]byte, []types.Slot) {
}
// HeadPublicKeyToValidatorIndex returns the validator index of the `pubkey` in current head state.
func (s *Service) HeadPublicKeyToValidatorIndex(ctx context.Context, pubKey [48]byte) (types.ValidatorIndex, bool) {
func (s *Service) HeadPublicKeyToValidatorIndex(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool) {
s.headLock.RLock()
defer s.headLock.RUnlock()
if !s.hasHeadState() {
@@ -300,15 +301,15 @@ func (s *Service) HeadPublicKeyToValidatorIndex(ctx context.Context, pubKey [48]
}
// HeadValidatorIndexToPublicKey returns the pubkey of the validator `index` in current head state.
func (s *Service) HeadValidatorIndexToPublicKey(_ context.Context, index types.ValidatorIndex) ([48]byte, error) {
func (s *Service) HeadValidatorIndexToPublicKey(_ context.Context, index types.ValidatorIndex) ([fieldparams.BLSPubkeyLength]byte, error) {
s.headLock.RLock()
defer s.headLock.RUnlock()
if !s.hasHeadState() {
return [48]byte{}, nil
return [fieldparams.BLSPubkeyLength]byte{}, nil
}
v, err := s.headValidatorAtIndex(index)
if err != nil {
return [48]byte{}, err
return [fieldparams.BLSPubkeyLength]byte{}, err
}
return v.PublicKey(), nil
}

View File

@@ -10,6 +10,7 @@ import (
testDB "github.com/prysmaticlabs/prysm/beacon-chain/db/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/forkchoice/protoarray"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -302,7 +303,7 @@ func TestService_HeadPublicKeyToValidatorIndex(t *testing.T) {
c := &Service{}
c.head = &head{state: s}
_, e := c.HeadPublicKeyToValidatorIndex(context.Background(), [48]byte{})
_, e := c.HeadPublicKeyToValidatorIndex(context.Background(), [fieldparams.BLSPubkeyLength]byte{})
require.Equal(t, false, e)
v, err := s.ValidatorAtIndex(0)
@@ -317,12 +318,12 @@ func TestService_HeadPublicKeyToValidatorIndexNil(t *testing.T) {
c := &Service{}
c.head = nil
idx, e := c.HeadPublicKeyToValidatorIndex(context.Background(), [48]byte{})
idx, e := c.HeadPublicKeyToValidatorIndex(context.Background(), [fieldparams.BLSPubkeyLength]byte{})
require.Equal(t, false, e)
require.Equal(t, types.ValidatorIndex(0), idx)
c.head = &head{state: nil}
i, e := c.HeadPublicKeyToValidatorIndex(context.Background(), [48]byte{})
i, e := c.HeadPublicKeyToValidatorIndex(context.Background(), [fieldparams.BLSPubkeyLength]byte{})
require.Equal(t, false, e)
require.Equal(t, types.ValidatorIndex(0), i)
}
@@ -347,10 +348,10 @@ func TestService_HeadValidatorIndexToPublicKeyNil(t *testing.T) {
p, err := c.HeadValidatorIndexToPublicKey(context.Background(), 0)
require.NoError(t, err)
require.Equal(t, [48]byte{}, p)
require.Equal(t, [fieldparams.BLSPubkeyLength]byte{}, p)
c.head = &head{state: nil}
p, err = c.HeadValidatorIndexToPublicKey(context.Background(), 0)
require.NoError(t, err)
require.Equal(t, [48]byte{}, p)
require.Equal(t, [fieldparams.BLSPubkeyLength]byte{}, p)
}

View File

@@ -21,6 +21,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -1027,7 +1028,7 @@ func TestInsertFinalizedDeposits(t *testing.T) {
for i := uint64(0); i < uint64(4*params.BeaconConfig().SlotsPerEpoch); i++ {
root := []byte(strconv.Itoa(int(i)))
assert.NoError(t, depositCache.InsertDeposit(ctx, &ethpb.Deposit{Data: &ethpb.Deposit_Data{
PublicKey: bytesutil.FromBytes48([48]byte{}),
PublicKey: bytesutil.FromBytes48([fieldparams.BLSPubkeyLength]byte{}),
WithdrawalCredentials: params.BeaconConfig().ZeroHash[:],
Amount: 0,
Signature: zeroSig[:],

View File

@@ -22,6 +22,7 @@ go_library(
"//beacon-chain/forkchoice/protoarray:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/v1:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",

View File

@@ -21,6 +21,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/forkchoice/protoarray"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -55,7 +56,7 @@ type ChainService struct {
SyncCommitteeDomain []byte
SyncSelectionProofDomain []byte
SyncContributionProofDomain []byte
PublicKey [48]byte
PublicKey [fieldparams.BLSPubkeyLength]byte
SyncCommitteePubkeys [][]byte
InitSyncBlockRoots map[[32]byte]bool
}
@@ -404,12 +405,12 @@ func (_ *ChainService) ChainHeads() ([][32]byte, []types.Slot) {
}
// HeadPublicKeyToValidatorIndex mocks HeadPublicKeyToValidatorIndex and always return 0 and true.
func (_ *ChainService) HeadPublicKeyToValidatorIndex(_ context.Context, _ [48]byte) (types.ValidatorIndex, bool) {
func (_ *ChainService) HeadPublicKeyToValidatorIndex(_ context.Context, _ [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool) {
return 0, true
}
// HeadValidatorIndexToPublicKey mocks HeadValidatorIndexToPublicKey and always return empty and nil.
func (s *ChainService) HeadValidatorIndexToPublicKey(_ context.Context, _ types.ValidatorIndex) ([48]byte, error) {
func (s *ChainService) HeadValidatorIndexToPublicKey(_ context.Context, _ types.ValidatorIndex) ([fieldparams.BLSPubkeyLength]byte, error) {
return s.PublicKey, nil
}

View File

@@ -85,6 +85,7 @@ go_test(
"//beacon-chain/state/v2:go_default_library",
"//beacon-chain/state/v3:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",

View File

@@ -10,6 +10,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/cache/depositcache",
visibility = ["//beacon-chain:__subpackages__"],
deps = [
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//container/trie:go_default_library",
"//crypto/hash:go_default_library",

View File

@@ -14,6 +14,7 @@ import (
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/container/trie"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
@@ -52,7 +53,7 @@ type DepositCache struct {
pendingDeposits []*ethpb.DepositContainer
deposits []*ethpb.DepositContainer
finalizedDeposits *FinalizedDeposits
depositsByKey map[[48]byte][]*ethpb.DepositContainer
depositsByKey map[[fieldparams.BLSPubkeyLength]byte][]*ethpb.DepositContainer
depositsLock sync.RWMutex
}
@@ -68,7 +69,7 @@ func New() (*DepositCache, error) {
return &DepositCache{
pendingDeposits: []*ethpb.DepositContainer{},
deposits: []*ethpb.DepositContainer{},
depositsByKey: map[[48]byte][]*ethpb.DepositContainer{},
depositsByKey: map[[fieldparams.BLSPubkeyLength]byte][]*ethpb.DepositContainer{},
finalizedDeposits: &FinalizedDeposits{Deposits: finalizedDepositsTrie, MerkleTrieIndex: -1},
}, nil
}

View File

@@ -4,6 +4,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
)
@@ -46,12 +47,12 @@ func TestSubnetIDsCache_PersistentCommitteeRoundtrip(t *testing.T) {
c := newSubnetIDs()
for i := 0; i < 20; i++ {
pubkey := [48]byte{byte(i)}
pubkey := [fieldparams.BLSPubkeyLength]byte{byte(i)}
c.AddPersistentCommittee(pubkey[:], []uint64{uint64(i)}, 0)
}
for i := uint64(0); i < 20; i++ {
pubkey := [48]byte{byte(i)}
pubkey := [fieldparams.BLSPubkeyLength]byte{byte(i)}
idxs, ok, _ := c.GetPersistentSubnets(pubkey[:])
if !ok {

View File

@@ -3,6 +3,7 @@ package cache
import (
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
@@ -12,12 +13,12 @@ func TestSyncSubnetIDsCache_Roundtrip(t *testing.T) {
c := newSyncSubnetIDs()
for i := 0; i < 20; i++ {
pubkey := [48]byte{byte(i)}
pubkey := [fieldparams.BLSPubkeyLength]byte{byte(i)}
c.AddSyncCommitteeSubnets(pubkey[:], 100, []uint64{uint64(i)}, 0)
}
for i := uint64(0); i < 20; i++ {
pubkey := [48]byte{byte(i)}
pubkey := [fieldparams.BLSPubkeyLength]byte{byte(i)}
idxs, _, ok, _ := c.GetSyncCommitteeSubnets(pubkey[:], 100)
if !ok {
@@ -34,7 +35,7 @@ func TestSyncSubnetIDsCache_ValidateCurrentEpoch(t *testing.T) {
c := newSyncSubnetIDs()
for i := 0; i < 20; i++ {
pubkey := [48]byte{byte(i)}
pubkey := [fieldparams.BLSPubkeyLength]byte{byte(i)}
c.AddSyncCommitteeSubnets(pubkey[:], 100, []uint64{uint64(i)}, 0)
}
@@ -42,7 +43,7 @@ func TestSyncSubnetIDsCache_ValidateCurrentEpoch(t *testing.T) {
assert.Equal(t, 0, len(coms))
for i := uint64(0); i < 20; i++ {
pubkey := [48]byte{byte(i)}
pubkey := [fieldparams.BLSPubkeyLength]byte{byte(i)}
_, jEpoch, ok, _ := c.GetSyncCommitteeSubnets(pubkey[:], 100)
if !ok {

View File

@@ -12,6 +12,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/time"
p2pType "github.com/prysmaticlabs/prysm/beacon-chain/p2p/types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
@@ -184,7 +185,7 @@ func TestProcessSyncCommittee_FilterSyncCommitteeVotes(t *testing.T) {
votedKeys, votedIndices, didntVoteIndices, err := altair.FilterSyncCommitteeVotes(beaconState, syncAggregate)
require.NoError(t, err)
votedMap := make(map[[48]byte]bool)
votedMap := make(map[[fieldparams.BLSPubkeyLength]byte]bool)
for _, key := range votedKeys {
votedMap[bytesutil.ToBytes48(key.Marshal())] = true
}

View File

@@ -8,6 +8,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/wrapper"
@@ -49,7 +50,7 @@ func TestFuzzProcessBlockHeader_10000(t *testing.T) {
func TestFuzzverifyDepositDataSigningRoot_10000(_ *testing.T) {
fuzzer := fuzz.NewWithSeed(0)
var ba []byte
pubkey := [48]byte{}
pubkey := [fieldparams.BLSPubkeyLength]byte{}
sig := [96]byte{}
domain := [4]byte{}
var p []byte

View File

@@ -59,7 +59,7 @@ func TestProcessDeposits_SameValidatorMultipleDepositsSameBlock(t *testing.T) {
func TestProcessDeposits_MerkleBranchFailsVerification(t *testing.T) {
deposit := &ethpb.Deposit{
Data: &ethpb.Deposit_Data{
PublicKey: bytesutil.PadTo([]byte{1, 2, 3}, 48),
PublicKey: bytesutil.PadTo([]byte{1, 2, 3}, fieldparams.BLSPubkeyLength),
WithdrawalCredentials: make([]byte, 32),
Signature: make([]byte, fieldparams.BLSSignatureLength),
},

View File

@@ -31,6 +31,7 @@ go_test(
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -10,6 +10,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
@@ -112,7 +113,7 @@ func TestSigningRoot_ComputeForkDigest(t *testing.T) {
func TestFuzzverifySigningRoot_10000(_ *testing.T) {
fuzzer := fuzz.NewWithSeed(0)
state := &ethpb.BeaconState{}
pubkey := [48]byte{}
pubkey := [fieldparams.BLSPubkeyLength]byte{}
sig := [96]byte{}
domain := [4]byte{}
var p []byte

View File

@@ -9,6 +9,7 @@ go_library(
"//testing/util:__pkg__",
],
deps = [
"//config/fieldparams:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_prysmaticlabs_eth2_types//:go_default_library",
@@ -22,6 +23,7 @@ go_test(
deps = [
":go_default_library",
"//beacon-chain/state/v1:go_default_library",
"//config/fieldparams:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/assert:go_default_library",

View File

@@ -5,14 +5,15 @@ package stateutils
import (
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
// ValidatorIndexMap builds a lookup map for quickly determining the index of
// a validator by their public key.
func ValidatorIndexMap(validators []*ethpb.Validator) map[[48]byte]types.ValidatorIndex {
m := make(map[[48]byte]types.ValidatorIndex, len(validators))
func ValidatorIndexMap(validators []*ethpb.Validator) map[[fieldparams.BLSPubkeyLength]byte]types.ValidatorIndex {
m := make(map[[fieldparams.BLSPubkeyLength]byte]types.ValidatorIndex, len(validators))
if validators == nil {
return m
}

View File

@@ -6,6 +6,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/transition/stateutils"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
@@ -27,7 +28,7 @@ func TestValidatorIndexMap_OK(t *testing.T) {
require.NoError(t, err)
tests := []struct {
key [48]byte
key [fieldparams.BLSPubkeyLength]byte
val types.ValidatorIndex
ok bool
}{

View File

@@ -73,7 +73,7 @@ func (vs *Server) GetAttesterDuties(ctx context.Context, req *ethpbv1.AttesterDu
duties := make([]*ethpbv1.AttesterDuty, 0, len(req.Index))
for _, index := range req.Index {
pubkey := s.PubkeyAtIndex(index)
zeroPubkey := [48]byte{}
zeroPubkey := [fieldparams.BLSPubkeyLength]byte{}
if bytes.Equal(pubkey[:], zeroPubkey[:]) {
return nil, status.Errorf(codes.InvalidArgument, "Invalid validator index")
}
@@ -230,7 +230,7 @@ func (vs *Server) GetSyncCommitteeDuties(ctx context.Context, req *ethpbv2.SyncC
return nil, status.Errorf(codes.Internal, "Could not get sync committee: %v", err)
}
}
committeePubkeys := make(map[[48]byte][]uint64)
committeePubkeys := make(map[[fieldparams.BLSPubkeyLength]byte][]uint64)
for j, pubkey := range committee.Pubkeys {
pubkey48 := bytesutil.ToBytes48(pubkey)
committeePubkeys[pubkey48] = append(committeePubkeys[pubkey48], uint64(j))
@@ -713,7 +713,7 @@ func syncCommitteeDutiesLastValidEpoch(currentEpoch types.Epoch) types.Epoch {
func syncCommitteeDuties(
valIndices []types.ValidatorIndex,
st state.BeaconState,
committeePubkeys map[[48]byte][]uint64,
committeePubkeys map[[fieldparams.BLSPubkeyLength]byte][]uint64,
) ([]*ethpbv2.SyncCommitteeDuty, error) {
duties := make([]*ethpbv2.SyncCommitteeDuty, 0)
for _, index := range valIndices {
@@ -721,7 +721,7 @@ func syncCommitteeDuties(
ValidatorIndex: index,
}
valPubkey48 := st.PubkeyAtIndex(index)
zeroPubkey := [48]byte{}
zeroPubkey := [fieldparams.BLSPubkeyLength]byte{}
if bytes.Equal(valPubkey48[:], zeroPubkey[:]) {
return nil, errInvalidValIndex
}

View File

@@ -43,6 +43,7 @@ go_library(
"//beacon-chain/sync:go_default_library",
"//cmd:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//container/slice:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -24,6 +24,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/powchain"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -192,7 +193,7 @@ func (is *infostream) handleMessage(msg *ethpb.ValidatorChangeSet) {
func (is *infostream) handleAddValidatorKeys(reqPubKeys [][]byte) error {
is.pubKeysMutex.Lock()
// Create existence map to ensure we don't duplicate keys.
pubKeysMap := make(map[[48]byte]bool, len(is.pubKeys))
pubKeysMap := make(map[[fieldparams.BLSPubkeyLength]byte]bool, len(is.pubKeys))
for _, pubKey := range is.pubKeys {
pubKeysMap[bytesutil.ToBytes48(pubKey)] = true
}
@@ -222,7 +223,7 @@ func (is *infostream) handleSetValidatorKeys(reqPubKeys [][]byte) error {
func (is *infostream) handleRemoveValidatorKeys(reqPubKeys [][]byte) {
is.pubKeysMutex.Lock()
// Create existence map to track what we have to delete.
pubKeysMap := make(map[[48]byte]bool, len(reqPubKeys))
pubKeysMap := make(map[[fieldparams.BLSPubkeyLength]byte]bool, len(reqPubKeys))
for _, pubKey := range reqPubKeys {
pubKeysMap[bytesutil.ToBytes48(pubKey)] = true
}
@@ -376,7 +377,7 @@ func (is *infostream) generatePendingValidatorInfo(info *ethpb.ValidatorInfo) (*
func (is *infostream) calculateActivationTimeForPendingValidators(res []*ethpb.ValidatorInfo, headState state.ReadOnlyBeaconState, epoch types.Epoch) error {
// pendingValidatorsMap is map from the validator pubkey to the index in our return array
pendingValidatorsMap := make(map[[48]byte]int)
pendingValidatorsMap := make(map[[fieldparams.BLSPubkeyLength]byte]int)
for i, info := range res {
if info.Status == ethpb.ValidatorStatus_PENDING {
pendingValidatorsMap[bytesutil.ToBytes48(info.PublicKey)] = i

View File

@@ -133,7 +133,7 @@ func TestGetAltairDuties_SyncCommitteeOK(t *testing.T) {
require.NoError(t, bs.SetSlot(params.BeaconConfig().SlotsPerEpoch*types.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)-1))
require.NoError(t, helpers.UpdateSyncCommitteeCache(bs))
pubkeysAs48ByteType := make([][48]byte, len(pubKeys))
pubkeysAs48ByteType := make([][fieldparams.BLSPubkeyLength]byte, len(pubKeys))
for i, pk := range pubKeys {
pubkeysAs48ByteType[i] = bytesutil.ToBytes48(pk)
}
@@ -282,7 +282,7 @@ func TestGetDuties_CurrentEpoch_ShouldNotFail(t *testing.T) {
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
pubKeys := make([][48]byte, len(deposits))
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = bytesutil.ToBytes48(deposits[i].Data.PublicKey)
@@ -320,7 +320,7 @@ func TestGetDuties_MultipleKeys_OK(t *testing.T) {
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
pubKeys := make([][48]byte, len(deposits))
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = bytesutil.ToBytes48(deposits[i].Data.PublicKey)
@@ -387,7 +387,7 @@ func TestStreamDuties_OK(t *testing.T) {
indices[i] = uint64(i)
}
pubkeysAs48ByteType := make([][48]byte, len(pubKeys))
pubkeysAs48ByteType := make([][fieldparams.BLSPubkeyLength]byte, len(pubKeys))
for i, pk := range pubKeys {
pubkeysAs48ByteType[i] = bytesutil.ToBytes48(pk)
}
@@ -444,7 +444,7 @@ func TestStreamDuties_OK_ChainReorg(t *testing.T) {
indices[i] = uint64(i)
}
pubkeysAs48ByteType := make([][48]byte, len(pubKeys))
pubkeysAs48ByteType := make([][fieldparams.BLSPubkeyLength]byte, len(pubKeys))
for i, pk := range pubKeys {
pubkeysAs48ByteType[i] = bytesutil.ToBytes48(pk)
}
@@ -542,7 +542,7 @@ func BenchmarkCommitteeAssignment(b *testing.B) {
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(b, err, "Could not get signing root")
pubKeys := make([][48]byte, len(deposits))
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = bytesutil.ToBytes48(deposits[i].Data.PublicKey)

View File

@@ -14,6 +14,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/operations/voluntaryexits"
mockp2p "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -136,7 +137,7 @@ func TestProposeExit_NoPanic(t *testing.T) {
_, err = server.ProposeExit(context.Background(), req)
require.ErrorContains(t, "invalid signature provided", err, "Expected error for no signature exists")
req.Signature = bytesutil.FromBytes48([48]byte{})
req.Signature = bytesutil.FromBytes48([fieldparams.BLSPubkeyLength]byte{})
_, err = server.ProposeExit(context.Background(), req)
require.ErrorContains(t, "invalid signature provided", err, "Expected error for invalid signature length")

View File

@@ -10,6 +10,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/contracts/deposit"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
@@ -62,8 +63,8 @@ func (vs *Server) MultipleValidatorStatus(
}
responseCap := len(req.PublicKeys) + len(req.Indices)
pubKeys := make([][]byte, 0, responseCap)
filtered := make(map[[48]byte]bool)
filtered[[48]byte{}] = true // Filter out keys with all zeros.
filtered := make(map[[fieldparams.BLSPubkeyLength]byte]bool)
filtered[[fieldparams.BLSPubkeyLength]byte{}] = true // Filter out keys with all zeros.
// Filter out duplicate public keys.
for _, pubKey := range req.PublicKeys {
pubkeyBytes := bytesutil.ToBytes48(pubKey)

View File

@@ -11,6 +11,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/synccommittee"
mockp2p "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
@@ -60,7 +61,7 @@ func TestGetSyncSubcommitteeIndex_Ok(t *testing.T) {
SyncCommitteeIndices: []types.CommitteeIndex{0},
},
}
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
// Request slot 0, should get the index 0 for validator 0.
res, err := server.GetSyncSubcommitteeIndex(context.Background(), &ethpb.SyncSubcommitteeIndexRequest{
PublicKey: pubKey[:], Slot: types.Slot(0),

View File

@@ -23,6 +23,7 @@ go_library(
"//tools/pcli:__pkg__",
],
deps = [
"//config/fieldparams:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_prysmaticlabs_eth2_types//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",

View File

@@ -8,6 +8,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/go-bitfield"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
@@ -83,7 +84,7 @@ type ReadOnlyValidator interface {
ActivationEpoch() types.Epoch
WithdrawableEpoch() types.Epoch
ExitEpoch() types.Epoch
PublicKey() [48]byte
PublicKey() [fieldparams.BLSPubkeyLength]byte
WithdrawalCredentials() []byte
Slashed() bool
IsNil() bool
@@ -94,8 +95,8 @@ type ReadOnlyValidators interface {
Validators() []*ethpb.Validator
ValidatorAtIndex(idx types.ValidatorIndex) (*ethpb.Validator, error)
ValidatorAtIndexReadOnly(idx types.ValidatorIndex) (ReadOnlyValidator, error)
ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex, bool)
PubkeyAtIndex(idx types.ValidatorIndex) [48]byte
ValidatorIndexByPubkey(key [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool)
PubkeyAtIndex(idx types.ValidatorIndex) [fieldparams.BLSPubkeyLength]byte
NumValidators() int
ReadFromEveryValidator(f func(idx int, val ReadOnlyValidator) error) error
}

View File

@@ -63,6 +63,7 @@ go_test(
embed = [":go_default_library"],
deps = [
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//crypto/hash:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -6,6 +6,7 @@ import (
"strconv"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/runtime/interop"
@@ -63,7 +64,7 @@ func setupGenesisState(tb testing.TB, count uint64) *ethpb.BeaconState {
require.NoError(tb, err, "Could not generate genesis beacon state")
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
genesisState.Validators = append(genesisState.Validators, &ethpb.Validator{

View File

@@ -5,13 +5,14 @@ import (
types "github.com/prysmaticlabs/eth2-types"
coreutils "github.com/prysmaticlabs/prysm/beacon-chain/core/transition/stateutils"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
// ValidatorMapHandler is a container to hold the map and a reference tracker for how many
// states shared this.
type ValidatorMapHandler struct {
valIdxMap map[[48]byte]types.ValidatorIndex
valIdxMap map[[fieldparams.BLSPubkeyLength]byte]types.ValidatorIndex
mapRef *Reference
*sync.RWMutex
}
@@ -38,11 +39,11 @@ func (v *ValidatorMapHandler) IsNil() bool {
// Copy the whole map and returns a map handler with the copied map.
func (v *ValidatorMapHandler) Copy() *ValidatorMapHandler {
if v == nil || v.valIdxMap == nil {
return &ValidatorMapHandler{valIdxMap: map[[48]byte]types.ValidatorIndex{}, mapRef: new(Reference), RWMutex: new(sync.RWMutex)}
return &ValidatorMapHandler{valIdxMap: map[[fieldparams.BLSPubkeyLength]byte]types.ValidatorIndex{}, mapRef: new(Reference), RWMutex: new(sync.RWMutex)}
}
v.RLock()
defer v.RUnlock()
m := make(map[[48]byte]types.ValidatorIndex, len(v.valIdxMap))
m := make(map[[fieldparams.BLSPubkeyLength]byte]types.ValidatorIndex, len(v.valIdxMap))
for k, v := range v.valIdxMap {
m[k] = v
}
@@ -54,7 +55,7 @@ func (v *ValidatorMapHandler) Copy() *ValidatorMapHandler {
}
// Get the validator index using the corresponding public key.
func (v *ValidatorMapHandler) Get(key [48]byte) (types.ValidatorIndex, bool) {
func (v *ValidatorMapHandler) Get(key [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool) {
v.RLock()
defer v.RUnlock()
idx, ok := v.valIdxMap[key]
@@ -65,7 +66,7 @@ func (v *ValidatorMapHandler) Get(key [48]byte) (types.ValidatorIndex, bool) {
}
// Set the validator index using the corresponding public key.
func (v *ValidatorMapHandler) Set(key [48]byte, index types.ValidatorIndex) {
func (v *ValidatorMapHandler) Set(key [fieldparams.BLSPubkeyLength]byte, index types.ValidatorIndex) {
v.Lock()
defer v.Unlock()
v.valIdxMap[key] = index

View File

@@ -6,6 +6,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
@@ -55,7 +56,7 @@ func TestNilState_NoPanic(t *testing.T) {
_ = err
_, err = st.ValidatorAtIndexReadOnly(0)
_ = err
_, _ = st.ValidatorIndexByPubkey([48]byte{})
_, _ = st.ValidatorIndexByPubkey([fieldparams.BLSPubkeyLength]byte{})
_ = st.PubkeyAtIndex(0)
_ = st.NumValidators()
_ = st.Balances()
@@ -113,22 +114,22 @@ func TestBeaconState_MarshalSSZ_NilState(t *testing.T) {
}
func TestBeaconState_ValidatorByPubkey(t *testing.T) {
keyCreator := func(input []byte) [48]byte {
nKey := [48]byte{}
keyCreator := func(input []byte) [fieldparams.BLSPubkeyLength]byte {
nKey := [fieldparams.BLSPubkeyLength]byte{}
copy(nKey[:1], input)
return nKey
}
tests := []struct {
name string
modifyFunc func(b *BeaconState, k [48]byte)
modifyFunc func(b *BeaconState, k [fieldparams.BLSPubkeyLength]byte)
exists bool
expectedIdx types.ValidatorIndex
largestIdxInSet types.ValidatorIndex
}{
{
name: "retrieve validator",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
},
exists: true,
@@ -136,7 +137,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators from the start",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
@@ -148,7 +149,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
@@ -160,7 +161,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators from the start with shared state",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
@@ -173,7 +174,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators with shared state",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
@@ -188,7 +189,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators with shared state at boundary",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
n := b.Copy()

View File

@@ -6,6 +6,7 @@ import (
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
@@ -134,7 +135,7 @@ func (b *BeaconState) ValidatorAtIndexReadOnly(idx types.ValidatorIndex) (state.
}
// ValidatorIndexByPubkey returns a given validator by its 48-byte public key.
func (b *BeaconState) ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex, bool) {
func (b *BeaconState) ValidatorIndexByPubkey(key [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool) {
if b == nil || b.valMapHandler == nil || b.valMapHandler.IsNil() {
return 0, false
}
@@ -151,18 +152,18 @@ func (b *BeaconState) ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex
// PubkeyAtIndex returns the pubkey at the given
// validator index.
func (b *BeaconState) PubkeyAtIndex(idx types.ValidatorIndex) [48]byte {
func (b *BeaconState) PubkeyAtIndex(idx types.ValidatorIndex) [fieldparams.BLSPubkeyLength]byte {
if !b.hasInnerState() {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
if uint64(idx) >= uint64(len(b.state.Validators)) {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
b.lock.RLock()
defer b.lock.RUnlock()
if b.state.Validators[idx] == nil {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
return bytesutil.ToBytes48(b.state.Validators[idx].PublicKey)
}

View File

@@ -4,6 +4,7 @@ import (
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
@@ -63,8 +64,8 @@ func (v readOnlyValidator) ExitEpoch() types.Epoch {
// PublicKey returns the public key of the
// read only validator.
func (v readOnlyValidator) PublicKey() [48]byte {
var pubkey [48]byte
func (v readOnlyValidator) PublicKey() [fieldparams.BLSPubkeyLength]byte {
var pubkey [fieldparams.BLSPubkeyLength]byte
copy(pubkey[:], v.validator.PublicKey)
return pubkey
}

View File

@@ -5,6 +5,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
@@ -52,7 +53,7 @@ func TestReadOnlyValidator_ExitEpoch(t *testing.T) {
}
func TestReadOnlyValidator_PublicKey(t *testing.T) {
key := [48]byte{0xFA, 0xCC}
key := [fieldparams.BLSPubkeyLength]byte{0xFA, 0xCC}
v, err := v1.NewValidator(&ethpb.Validator{PublicKey: key[:]})
require.NoError(t, err)
assert.Equal(t, key, v.PublicKey())

View File

@@ -21,7 +21,7 @@ func TestValidatorMap_DistinctCopy(t *testing.T) {
vals := make([]*ethpb.Validator, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{
@@ -49,7 +49,7 @@ func TestBeaconState_NoDeadlock(t *testing.T) {
vals := make([]*ethpb.Validator, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{
@@ -115,7 +115,7 @@ func TestBeaconState_AppendBalanceWithTrie(t *testing.T) {
bals := make([]uint64, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{

View File

@@ -7,6 +7,7 @@ import (
"testing"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -56,7 +57,7 @@ func setupGenesisState(tb testing.TB, count uint64) *ethpb.BeaconState {
require.NoError(tb, err, "Could not generate genesis beacon state")
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
genesisState.Validators = append(genesisState.Validators, &ethpb.Validator{
@@ -77,7 +78,7 @@ func setupGenesisState(tb testing.TB, count uint64) *ethpb.BeaconState {
func BenchmarkCloneValidators_Proto(b *testing.B) {
b.StopTimer()
validators := make([]*ethpb.Validator, 16384)
somePubKey := [48]byte{1, 2, 3}
somePubKey := [fieldparams.BLSPubkeyLength]byte{1, 2, 3}
someRoot := [32]byte{3, 4, 5}
for i := 0; i < len(validators); i++ {
validators[i] = &ethpb.Validator{
@@ -100,7 +101,7 @@ func BenchmarkCloneValidators_Proto(b *testing.B) {
func BenchmarkCloneValidators_Manual(b *testing.B) {
b.StopTimer()
validators := make([]*ethpb.Validator, 16384)
somePubKey := [48]byte{1, 2, 3}
somePubKey := [fieldparams.BLSPubkeyLength]byte{1, 2, 3}
someRoot := [32]byte{3, 4, 5}
for i := 0; i < len(validators); i++ {
validators[i] = &ethpb.Validator{

View File

@@ -6,6 +6,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
@@ -57,7 +58,7 @@ func TestNilState_NoPanic(t *testing.T) {
_ = err
_, err = st.ValidatorAtIndexReadOnly(0)
_ = err
_, _ = st.ValidatorIndexByPubkey([48]byte{})
_, _ = st.ValidatorIndexByPubkey([fieldparams.BLSPubkeyLength]byte{})
_ = st.PubkeyAtIndex(0)
_ = st.NumValidators()
_ = st.Balances()
@@ -90,22 +91,22 @@ func TestNilState_NoPanic(t *testing.T) {
}
func TestBeaconState_ValidatorByPubkey(t *testing.T) {
keyCreator := func(input []byte) [48]byte {
nKey := [48]byte{}
keyCreator := func(input []byte) [fieldparams.BLSPubkeyLength]byte {
nKey := [fieldparams.BLSPubkeyLength]byte{}
copy(nKey[:1], input)
return nKey
}
tests := []struct {
name string
modifyFunc func(b *BeaconState, k [48]byte)
modifyFunc func(b *BeaconState, k [fieldparams.BLSPubkeyLength]byte)
exists bool
expectedIdx types.ValidatorIndex
largestIdxInSet types.ValidatorIndex
}{
{
name: "retrieve validator",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
},
exists: true,
@@ -113,7 +114,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators from the start",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
@@ -125,7 +126,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
@@ -137,7 +138,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators from the start with shared state",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
@@ -150,7 +151,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators with shared state",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
@@ -165,7 +166,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators with shared state at boundary",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
n := b.Copy()

View File

@@ -7,6 +7,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
@@ -135,7 +136,7 @@ func (b *BeaconState) ValidatorAtIndexReadOnly(idx types.ValidatorIndex) (state.
}
// ValidatorIndexByPubkey returns a given validator by its 48-byte public key.
func (b *BeaconState) ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex, bool) {
func (b *BeaconState) ValidatorIndexByPubkey(key [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool) {
if b == nil || b.valMapHandler == nil || b.valMapHandler.IsNil() {
return 0, false
}
@@ -152,18 +153,18 @@ func (b *BeaconState) ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex
// PubkeyAtIndex returns the pubkey at the given
// validator index.
func (b *BeaconState) PubkeyAtIndex(idx types.ValidatorIndex) [48]byte {
func (b *BeaconState) PubkeyAtIndex(idx types.ValidatorIndex) [fieldparams.BLSPubkeyLength]byte {
if !b.hasInnerState() {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
if uint64(idx) >= uint64(len(b.state.Validators)) {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
b.lock.RLock()
defer b.lock.RUnlock()
if b.state.Validators[idx] == nil {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
return bytesutil.ToBytes48(b.state.Validators[idx].PublicKey)
}

View File

@@ -69,7 +69,7 @@ func TestBeaconState_AppendBalanceWithTrie(t *testing.T) {
bals := make([]uint64, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{

View File

@@ -7,6 +7,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/state/stateutil"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -25,7 +26,7 @@ func TestValidatorMap_DistinctCopy(t *testing.T) {
vals := make([]*ethpb.Validator, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{
@@ -90,7 +91,7 @@ func TestBeaconState_NoDeadlock(t *testing.T) {
vals := make([]*ethpb.Validator, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{

View File

@@ -78,6 +78,7 @@ go_test(
"//beacon-chain/state/types:go_default_library",
"//beacon-chain/state/v1:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//container/trie:go_default_library",
"//crypto/bls:go_default_library",

View File

@@ -6,6 +6,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
@@ -57,7 +58,7 @@ func TestNilState_NoPanic(t *testing.T) {
_ = err
_, err = st.ValidatorAtIndexReadOnly(0)
_ = err
_, _ = st.ValidatorIndexByPubkey([48]byte{})
_, _ = st.ValidatorIndexByPubkey([fieldparams.BLSPubkeyLength]byte{})
_ = st.PubkeyAtIndex(0)
_ = st.NumValidators()
_ = st.Balances()
@@ -90,22 +91,22 @@ func TestNilState_NoPanic(t *testing.T) {
}
func TestBeaconState_ValidatorByPubkey(t *testing.T) {
keyCreator := func(input []byte) [48]byte {
nKey := [48]byte{}
keyCreator := func(input []byte) [fieldparams.BLSPubkeyLength]byte {
nKey := [fieldparams.BLSPubkeyLength]byte{}
copy(nKey[:1], input)
return nKey
}
tests := []struct {
name string
modifyFunc func(b *BeaconState, k [48]byte)
modifyFunc func(b *BeaconState, k [fieldparams.BLSPubkeyLength]byte)
exists bool
expectedIdx types.ValidatorIndex
largestIdxInSet types.ValidatorIndex
}{
{
name: "retrieve validator",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
},
exists: true,
@@ -113,7 +114,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators from the start",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
@@ -125,7 +126,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
@@ -137,7 +138,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators from the start with shared state",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key[:]}))
@@ -150,7 +151,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators with shared state",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
key2 := keyCreator([]byte{'D'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
@@ -165,7 +166,7 @@ func TestBeaconState_ValidatorByPubkey(t *testing.T) {
},
{
name: "retrieve validator with multiple validators with shared state at boundary",
modifyFunc: func(b *BeaconState, key [48]byte) {
modifyFunc: func(b *BeaconState, key [fieldparams.BLSPubkeyLength]byte) {
key1 := keyCreator([]byte{'C'})
assert.NoError(t, b.AppendValidator(&ethpb.Validator{PublicKey: key1[:]}))
n := b.Copy()

View File

@@ -7,6 +7,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
v1 "github.com/prysmaticlabs/prysm/beacon-chain/state/v1"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
@@ -135,7 +136,7 @@ func (b *BeaconState) ValidatorAtIndexReadOnly(idx types.ValidatorIndex) (state.
}
// ValidatorIndexByPubkey returns a given validator by its 48-byte public key.
func (b *BeaconState) ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex, bool) {
func (b *BeaconState) ValidatorIndexByPubkey(key [fieldparams.BLSPubkeyLength]byte) (types.ValidatorIndex, bool) {
if b == nil || b.valMapHandler == nil || b.valMapHandler.IsNil() {
return 0, false
}
@@ -152,18 +153,18 @@ func (b *BeaconState) ValidatorIndexByPubkey(key [48]byte) (types.ValidatorIndex
// PubkeyAtIndex returns the pubkey at the given
// validator index.
func (b *BeaconState) PubkeyAtIndex(idx types.ValidatorIndex) [48]byte {
func (b *BeaconState) PubkeyAtIndex(idx types.ValidatorIndex) [fieldparams.BLSPubkeyLength]byte {
if !b.hasInnerState() {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
if uint64(idx) >= uint64(len(b.state.Validators)) {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
b.lock.RLock()
defer b.lock.RUnlock()
if b.state.Validators[idx] == nil {
return [48]byte{}
return [fieldparams.BLSPubkeyLength]byte{}
}
return bytesutil.ToBytes48(b.state.Validators[idx].PublicKey)
}

View File

@@ -9,6 +9,7 @@ import (
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stateutil"
stateTypes "github.com/prysmaticlabs/prysm/beacon-chain/state/types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -80,7 +81,7 @@ func TestBeaconState_AppendBalanceWithTrie(t *testing.T) {
bals := make([]uint64, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{

View File

@@ -7,6 +7,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/state/stateutil"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -25,7 +26,7 @@ func TestValidatorMap_DistinctCopy(t *testing.T) {
vals := make([]*ethpb.Validator, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{
@@ -89,7 +90,7 @@ func TestBeaconState_NoDeadlock(t *testing.T) {
vals := make([]*ethpb.Validator, 0, count)
for i := uint64(1); i < count; i++ {
someRoot := [32]byte{}
someKey := [48]byte{}
someKey := [fieldparams.BLSPubkeyLength]byte{}
copy(someRoot[:], strconv.Itoa(int(i)))
copy(someKey[:], strconv.Itoa(int(i)))
vals = append(vals, &ethpb.Validator{

View File

@@ -15,6 +15,7 @@ const (
SyncCommitteeLength = 512 // SYNC_COMMITTEE_SIZE
RootLength = 32 // RootLength defines the byte length of a Merkle root.
BLSSignatureLength = 96 // BLSSignatureLength defines the byte length of a BLSSignature.
BLSPubkeyLength = 48 // BLSPubkeyLength defines the byte length of a BLSSignature.
MaxTxsPerPayloadLength = 1048576 // MaxTxsPerPayloadLength defines the maximum number of transactions that can be included in a payload.
MaxBytesPerTxLength = 1073741824 // MaxBytesPerTxLength defines the maximum number of bytes that can be included in a transaction.
FeeRecipientLength = 20 // FeeRecipientLength defines the byte length of a fee recipient.

View File

@@ -15,6 +15,7 @@ const (
SyncCommitteeLength = 32 // SYNC_COMMITTEE_SIZE
RootLength = 32 // RootLength defines the byte length of a Merkle root.
BLSSignatureLength = 96 // BLSSignatureLength defines the byte length of a BLSSignature.
BLSPubkeyLength = 48 // BLSPubkeyLength defines the byte length of a BLSSignature.
MaxTxsPerPayloadLength = 1048576 // MaxTxsPerPayloadLength defines the maximum number of transactions that can be included in a payload.
MaxBytesPerTxLength = 1073741824 // MaxBytesPerTxLength defines the maximum number of bytes that can be included in a transaction.
FeeRecipientLength = 20 // FeeRecipientLength defines the byte length of a fee recipient.

View File

@@ -22,6 +22,7 @@ go_test(
srcs = ["sparse_merkle_test.go"],
deps = [
":go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//contracts/deposit:go_default_library",
"//crypto/hash:go_default_library",

View File

@@ -5,6 +5,7 @@ import (
"testing"
"github.com/ethereum/go-ethereum/accounts/abi/bind"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/container/trie"
contracts "github.com/prysmaticlabs/prysm/contracts/deposit"
@@ -32,7 +33,7 @@ func TestMarshalDepositWithProof(t *testing.T) {
require.Equal(t, len(proof), int(params.BeaconConfig().DepositContractTreeDepth)+1)
someRoot := [32]byte{1, 2, 3, 4}
someSig := [96]byte{1, 2, 3, 4}
someKey := [48]byte{1, 2, 3, 4}
someKey := [fieldparams.BLSPubkeyLength]byte{1, 2, 3, 4}
dep := &ethpb.Deposit{
Proof: proof,
Data: &ethpb.Deposit_Data{

View File

@@ -12,4 +12,5 @@ go_library(
"//crypto/bls:__subpackages__",
"//testing:__subpackages__",
],
deps = ["//config/fieldparams:go_default_library"],
)

View File

@@ -1,10 +1,12 @@
package common
import fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
// ZeroSecretKey represents a zero secret key.
var ZeroSecretKey = [32]byte{}
// InfinitePublicKey represents an infinite public key (G1 Point at Infinity).
var InfinitePublicKey = [48]byte{0xC0}
var InfinitePublicKey = [fieldparams.BLSPubkeyLength]byte{0xC0}
// InfiniteSignature represents an infinite signature (G2 Point at Infinity).
var InfiniteSignature = [96]byte{0xC0}

View File

@@ -27,6 +27,7 @@ go_library(
"//beacon-chain/core/blocks:go_default_library",
"//cmd:go_default_library",
"//cmd/validator/flags:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",
@@ -74,6 +75,7 @@ go_test(
deps = [
"//async/event:go_default_library",
"//cmd/validator/flags:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -13,6 +13,7 @@ import (
"github.com/manifoldco/promptui"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/cmd/validator/flags"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/io/file"
@@ -121,7 +122,7 @@ func BackupAccountsCli(cliCtx *cli.Context) error {
}
// Ask user to select accounts via an interactive userprompt.
func selectAccounts(selectionPrompt string, pubKeys [][48]byte) (filteredPubKeys []bls.PublicKey, err error) {
func selectAccounts(selectionPrompt string, pubKeys [][fieldparams.BLSPubkeyLength]byte) (filteredPubKeys []bls.PublicKey, err error) {
pubKeyStrings := make([]string, len(pubKeys))
for i, pk := range pubKeys {
name := petnames.DeterministicName(pk[:], "-")

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/cmd"
"github.com/prysmaticlabs/prysm/cmd/validator/flags"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/io/prompt"
@@ -127,7 +128,7 @@ func PerformVoluntaryExit(
return rawExitedKeys, formattedExitedKeys, nil
}
func prepareWallet(cliCtx *cli.Context) (validatingPublicKeys [][48]byte, km keymanager.IKeymanager, err error) {
func prepareWallet(cliCtx *cli.Context) (validatingPublicKeys [][fieldparams.BLSPubkeyLength]byte, km keymanager.IKeymanager, err error) {
w, err := wallet.OpenWalletOrElseCli(cliCtx, func(cliCtx *cli.Context) (*wallet.Wallet, error) {
return nil, wallet.ErrNoWalletFound
})
@@ -153,7 +154,7 @@ func prepareWallet(cliCtx *cli.Context) (validatingPublicKeys [][48]byte, km key
func interact(
cliCtx *cli.Context,
r io.Reader,
validatingPublicKeys [][48]byte,
validatingPublicKeys [][fieldparams.BLSPubkeyLength]byte,
) (rawPubKeys [][]byte, formattedPubKeys []string, err error) {
if !cliCtx.IsSet(flags.ExitAllFlag.Name) {
// Allow the user to interactively select the accounts to exit or optionally
@@ -231,7 +232,7 @@ func interact(
return rawPubKeys, formattedPubKeys, nil
}
func prepareAllKeys(validatingKeys [][48]byte) (raw [][]byte, formatted []string) {
func prepareAllKeys(validatingKeys [][fieldparams.BLSPubkeyLength]byte) (raw [][]byte, formatted []string) {
raw = make([][]byte, len(validatingKeys))
formatted = make([]string, len(validatingKeys))
for i, pk := range validatingKeys {

View File

@@ -9,6 +9,7 @@ import (
"time"
"github.com/golang/mock/gomock"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
@@ -271,7 +272,7 @@ func TestDisplayExitInfo_NoKeys(t *testing.T) {
func TestPrepareAllKeys(t *testing.T) {
key1 := bytesutil.ToBytes48([]byte("key1"))
key2 := bytesutil.ToBytes48([]byte("key2"))
raw, formatted := prepareAllKeys([][48]byte{key1, key2})
raw, formatted := prepareAllKeys([][fieldparams.BLSPubkeyLength]byte{key1, key2})
require.Equal(t, 2, len(raw))
require.Equal(t, 2, len(formatted))
assert.DeepEqual(t, bytesutil.ToBytes48([]byte{107, 101, 121, 49}), bytesutil.ToBytes48(raw[0]))

View File

@@ -6,6 +6,7 @@ import (
"strings"
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/urfave/cli/v2"
)
@@ -13,7 +14,7 @@ import (
func filterPublicKeysFromUserInput(
cliCtx *cli.Context,
publicKeysFlag *cli.StringFlag,
validatingPublicKeys [][48]byte,
validatingPublicKeys [][fieldparams.BLSPubkeyLength]byte,
selectionPrompt string,
) ([]bls.PublicKey, error) {
var filteredPubKeys []bls.PublicKey

View File

@@ -14,6 +14,7 @@ import (
"github.com/google/uuid"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/async/event"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -32,11 +33,11 @@ import (
)
type mockRemoteKeymanager struct {
publicKeys [][48]byte
publicKeys [][fieldparams.BLSPubkeyLength]byte
opts *remote.KeymanagerOpts
}
func (m *mockRemoteKeymanager) FetchValidatingPublicKeys(_ context.Context) ([][48]byte, error) {
func (m *mockRemoteKeymanager) FetchValidatingPublicKeys(_ context.Context) ([][fieldparams.BLSPubkeyLength]byte, error) {
return m.publicKeys, nil
}
@@ -44,7 +45,7 @@ func (_ *mockRemoteKeymanager) Sign(context.Context, *validatorpb.SignRequest) (
return nil, nil
}
func (_ *mockRemoteKeymanager) SubscribeAccountChanges(_ chan [][48]byte) event.Subscription {
func (_ *mockRemoteKeymanager) SubscribeAccountChanges(_ chan [][fieldparams.BLSPubkeyLength]byte) event.Subscription {
return nil
}
@@ -383,7 +384,7 @@ func TestListAccounts_RemoteKeymanager(t *testing.T) {
os.Stdout = writer
numAccounts := 3
pubKeys := make([][48]byte, numAccounts)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numAccounts)
for i := 0; i < numAccounts; i++ {
key := make([]byte, 48)
copy(key, strconv.Itoa(i))
@@ -483,7 +484,7 @@ func TestListAccounts_ListValidatorIndices(t *testing.T) {
defer ctrl.Finish()
numAccounts := 3
pubKeys := make([][48]byte, numAccounts)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numAccounts)
pks := make([][]byte, numAccounts)
for i := 0; i < numAccounts; i++ {

View File

@@ -7,6 +7,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/monitoring/tracing"
@@ -23,7 +24,7 @@ import (
// via gRPC. Beacon node will verify the slot signature and determine if the validator is also
// an aggregator. If yes, then beacon node will broadcast aggregated signature and
// proof on the validator's behalf.
func (v *validator) SubmitAggregateAndProof(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) SubmitAggregateAndProof(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
ctx, span := trace.StartSpan(ctx, "validator.SubmitAggregateAndProof")
defer span.End()
@@ -116,7 +117,7 @@ func (v *validator) SubmitAggregateAndProof(ctx context.Context, slot types.Slot
}
// Signs input slot with domain selection proof. This is used to create the signature for aggregator selection.
func (v *validator) signSlotWithSelectionProof(ctx context.Context, pubKey [48]byte, slot types.Slot) (signature []byte, err error) {
func (v *validator) signSlotWithSelectionProof(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, slot types.Slot) (signature []byte, err error) {
domain, err := v.domainData(ctx, slots.ToEpoch(slot), params.BeaconConfig().DomainSelectionProof[:])
if err != nil {
return nil, err
@@ -171,7 +172,7 @@ func (v *validator) waitToSlotTwoThirds(ctx context.Context, slot types.Slot) {
// This returns the signature of validator signing over aggregate and
// proof object.
func (v *validator) aggregateAndProofSig(ctx context.Context, pubKey [48]byte, agg *ethpb.AggregateAttestationAndProof) ([]byte, error) {
func (v *validator) aggregateAndProofSig(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, agg *ethpb.AggregateAttestationAndProof) ([]byte, error) {
d, err := v.domainData(ctx, slots.ToEpoch(agg.Aggregate.Data.Slot), params.BeaconConfig().DomainAggregateAndProof[:])
if err != nil {
return nil, err

View File

@@ -8,6 +8,7 @@ import (
"github.com/golang/mock/gomock"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/go-bitfield"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -25,7 +26,7 @@ func TestSubmitAggregateAndProof_GetDutiesRequestFailure(t *testing.T) {
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitAggregateAndProof(context.Background(), 0, pubKey)
@@ -35,7 +36,7 @@ func TestSubmitAggregateAndProof_GetDutiesRequestFailure(t *testing.T) {
func TestSubmitAggregateAndProof_SignFails(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{
Duties: []*ethpb.DutiesResponse_Duty{
@@ -74,7 +75,7 @@ func TestSubmitAggregateAndProof_SignFails(t *testing.T) {
func TestSubmitAggregateAndProof_Ok(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{
Duties: []*ethpb.DutiesResponse_Duty{
@@ -149,7 +150,7 @@ func TestAggregateAndProofSignature_CanSignValidSignature(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
gomock.Any(), // ctx

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/prysm/async"
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/hash"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
@@ -31,7 +32,7 @@ import (
// It fetches the latest beacon block head along with the latest canonical beacon state
// information in order to sign the block and include information about the validator's
// participation in voting on the block.
func (v *validator) SubmitAttestation(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) SubmitAttestation(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
ctx, span := trace.StartSpan(ctx, "validator.SubmitAttestation")
defer span.End()
span.AddAttributes(trace.StringAttribute("validator", fmt.Sprintf("%#x", pubKey)))
@@ -180,7 +181,7 @@ func (v *validator) SubmitAttestation(ctx context.Context, slot types.Slot, pubK
}
// Given the validator public key, this gets the validator assignment.
func (v *validator) duty(pubKey [48]byte) (*ethpb.DutiesResponse_Duty, error) {
func (v *validator) duty(pubKey [fieldparams.BLSPubkeyLength]byte) (*ethpb.DutiesResponse_Duty, error) {
if v.duties == nil {
return nil, errors.New("no duties for validators")
}
@@ -195,7 +196,7 @@ func (v *validator) duty(pubKey [48]byte) (*ethpb.DutiesResponse_Duty, error) {
}
// Given validator's public key, this function returns the signature of an attestation data and its signing root.
func (v *validator) signAtt(ctx context.Context, pubKey [48]byte, data *ethpb.AttestationData) ([]byte, [32]byte, error) {
func (v *validator) signAtt(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, data *ethpb.AttestationData) ([]byte, [32]byte, error) {
domain, root, err := v.getDomainAndSigningRoot(ctx, data)
if err != nil {
return nil, [32]byte{}, err
@@ -291,7 +292,7 @@ func (v *validator) waitOneThirdOrValidBlock(ctx context.Context, slot types.Slo
}
}
func attestationLogFields(pubKey [48]byte, indexedAtt *ethpb.IndexedAttestation) logrus.Fields {
func attestationLogFields(pubKey [fieldparams.BLSPubkeyLength]byte, indexedAtt *ethpb.IndexedAttestation) logrus.Fields {
return logrus.Fields{
"attesterPublicKey": fmt.Sprintf("%#x", pubKey),
"attestationSlot": indexedAtt.Data.Slot,

View File

@@ -7,6 +7,7 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/slashings"
"github.com/prysmaticlabs/prysm/validator/db/kv"
@@ -22,7 +23,7 @@ var failedPostAttSignExternalErr = "attempted to make slashable attestation, rej
func (v *validator) slashableAttestationCheck(
ctx context.Context,
indexedAtt *ethpb.IndexedAttestation,
pubKey [48]byte,
pubKey [fieldparams.BLSPubkeyLength]byte,
signingRoot [32]byte,
) error {
ctx, span := trace.StartSpan(ctx, "validator.postAttSignUpdate")

View File

@@ -7,6 +7,7 @@ import (
"github.com/golang/mock/gomock"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/require"
@@ -20,7 +21,7 @@ func Test_slashableAttestationCheck(t *testing.T) {
defer reset()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
att := &ethpb.IndexedAttestation{
AttestingIndices: []uint64{1, 2},
@@ -68,7 +69,7 @@ func Test_slashableAttestationCheck_UpdatesLowestSignedEpochs(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
ctx := context.Background()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
att := &ethpb.IndexedAttestation{
AttestingIndices: []uint64{1, 2},

View File

@@ -37,7 +37,7 @@ func TestRequestAttestation_ValidatorDutiesRequestFailure(t *testing.T) {
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitAttestation(context.Background(), 30, pubKey)
require.LogsContain(t, hook, "Could not fetch validator assignment")
@@ -48,7 +48,7 @@ func TestAttestToBlockHead_SubmitAttestation_EmptyCommittee(t *testing.T) {
validator, _, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
{
@@ -90,7 +90,7 @@ func TestAttestToBlockHead_SubmitAttestation_RequestFailure(t *testing.T) {
gomock.AssignableToTypeOf(&ethpb.Attestation{}),
).Return(nil, errors.New("something went wrong"))
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitAttestation(context.Background(), 30, pubKey)
require.LogsContain(t, hook, "Could not submit attestation to beacon node")
@@ -102,7 +102,7 @@ func TestAttestToBlockHead_AttestsCorrectly(t *testing.T) {
hook := logTest.NewGlobal()
validatorIndex := types.ValidatorIndex(7)
committee := []types.ValidatorIndex{0, 3, 4, 2, validatorIndex, 6, 8, 9, 10}
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
{
@@ -175,7 +175,7 @@ func TestAttestToBlockHead_BlocksDoubleAtt(t *testing.T) {
defer finish()
validatorIndex := types.ValidatorIndex(7)
committee := []types.ValidatorIndex{0, 3, 4, 2, validatorIndex, 6, 8, 9, 10}
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
{
@@ -227,7 +227,7 @@ func TestAttestToBlockHead_BlocksSurroundAtt(t *testing.T) {
defer finish()
validatorIndex := types.ValidatorIndex(7)
committee := []types.ValidatorIndex{0, 3, 4, 2, validatorIndex, 6, 8, 9, 10}
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
{
@@ -278,7 +278,7 @@ func TestAttestToBlockHead_BlocksSurroundedAtt(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
validatorIndex := types.ValidatorIndex(7)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
committee := []types.ValidatorIndex{0, 3, 4, 2, validatorIndex, 6, 8, 9, 10}
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
@@ -332,7 +332,7 @@ func TestAttestToBlockHead_DoesNotAttestBeforeDelay(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.genesisTime = uint64(prysmTime.Now().Unix())
m.validatorClient.EXPECT().GetDuties(
@@ -367,7 +367,7 @@ func TestAttestToBlockHead_DoesAttestAfterDelay(t *testing.T) {
validator.genesisTime = uint64(prysmTime.Now().Unix())
validatorIndex := types.ValidatorIndex(5)
committee := []types.ValidatorIndex{0, 3, 4, 2, validatorIndex, 6, 8, 9, 10}
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
{
@@ -406,7 +406,7 @@ func TestAttestToBlockHead_CorrectBitfieldLength(t *testing.T) {
defer finish()
validatorIndex := types.ValidatorIndex(2)
committee := []types.ValidatorIndex{0, 3, 4, 2, validatorIndex, 6, 8, 9, 10}
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{
{
@@ -466,10 +466,10 @@ func TestSignAttestation(t *testing.T) {
att.Data.Target.Epoch = 200
att.Data.Slot = 999
att.Data.BeaconBlockRoot = bytesutil.PadTo([]byte("blockRoot"), 32)
var pubKey [48]byte
var pubKey [fieldparams.BLSPubkeyLength]byte
copy(pubKey[:], publicKey.Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: secretKey,
},
}

View File

@@ -6,6 +6,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/validator/client/iface",
visibility = ["//validator:__subpackages__"],
deps = [
"//config/fieldparams:go_default_library",
"//validator/keymanager:go_default_library",
"@com_github_prysmaticlabs_eth2_types//:go_default_library",
],

View File

@@ -6,6 +6,7 @@ import (
"time"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/validator/keymanager"
)
@@ -35,18 +36,18 @@ type Validator interface {
Done()
WaitForChainStart(ctx context.Context) error
WaitForSync(ctx context.Context) error
WaitForActivation(ctx context.Context, accountsChangedChan chan [][48]byte) error
WaitForActivation(ctx context.Context, accountsChangedChan chan [][fieldparams.BLSPubkeyLength]byte) error
CanonicalHeadSlot(ctx context.Context) (types.Slot, error)
NextSlot() <-chan types.Slot
SlotDeadline(slot types.Slot) time.Time
LogValidatorGainsAndLosses(ctx context.Context, slot types.Slot) error
UpdateDuties(ctx context.Context, slot types.Slot) error
RolesAt(ctx context.Context, slot types.Slot) (map[[48]byte][]ValidatorRole, error) // validator pubKey -> roles
SubmitAttestation(ctx context.Context, slot types.Slot, pubKey [48]byte)
ProposeBlock(ctx context.Context, slot types.Slot, pubKey [48]byte)
SubmitAggregateAndProof(ctx context.Context, slot types.Slot, pubKey [48]byte)
SubmitSyncCommitteeMessage(ctx context.Context, slot types.Slot, pubKey [48]byte)
SubmitSignedContributionAndProof(ctx context.Context, slot types.Slot, pubKey [48]byte)
RolesAt(ctx context.Context, slot types.Slot) (map[[fieldparams.BLSPubkeyLength]byte][]ValidatorRole, error) // validator pubKey -> roles
SubmitAttestation(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)
ProposeBlock(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)
SubmitAggregateAndProof(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)
SubmitSyncCommitteeMessage(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)
SubmitSignedContributionAndProof(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)
LogAttestationsSubmitted()
LogNextDutyTimeLeft(slot types.Slot) error
UpdateDomainDataCaches(ctx context.Context, slot types.Slot)
@@ -54,6 +55,6 @@ type Validator interface {
AllValidatorsAreExited(ctx context.Context) (bool, error)
GetKeymanager() keymanager.IKeymanager
ReceiveBlocks(ctx context.Context, connectionErrorChannel chan<- error)
HandleKeyReload(ctx context.Context, newKeys [][48]byte) (bool, error)
HandleKeyReload(ctx context.Context, newKeys [][fieldparams.BLSPubkeyLength]byte) (bool, error)
CheckDoppelGanger(ctx context.Context) error
}

View File

@@ -3,13 +3,14 @@ package client
import (
"context"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
eth "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"go.opencensus.io/trace"
)
// HandleKeyReload makes sure the validator keeps operating correctly after a change to the underlying keys.
// It is also responsible for logging out information about the new state of keys.
func (v *validator) HandleKeyReload(ctx context.Context, newKeys [][48]byte) (anyActive bool, err error) {
func (v *validator) HandleKeyReload(ctx context.Context, newKeys [][fieldparams.BLSPubkeyLength]byte) (anyActive bool, err error) {
ctx, span := trace.StartSpan(ctx, "validator.HandleKeyReload")
defer span.End()

View File

@@ -6,6 +6,7 @@ import (
"github.com/golang/mock/gomock"
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/crypto/bls"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
@@ -24,14 +25,14 @@ func TestValidator_HandleKeyReload(t *testing.T) {
inactivePrivKey, err := bls.RandKey()
require.NoError(t, err)
inactivePubKey := [48]byte{}
inactivePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(inactivePubKey[:], inactivePrivKey.PublicKey().Marshal())
activePrivKey, err := bls.RandKey()
require.NoError(t, err)
activePubKey := [48]byte{}
activePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(activePubKey[:], activePrivKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
inactivePubKey: inactivePrivKey,
},
}
@@ -52,7 +53,7 @@ func TestValidator_HandleKeyReload(t *testing.T) {
},
).Return(resp, nil)
anyActive, err := v.HandleKeyReload(context.Background(), [][48]byte{inactivePubKey, activePubKey})
anyActive, err := v.HandleKeyReload(context.Background(), [][fieldparams.BLSPubkeyLength]byte{inactivePubKey, activePubKey})
require.NoError(t, err)
assert.Equal(t, true, anyActive)
assert.LogsContain(t, hook, "Waiting for deposit to be observed by beacon node")
@@ -64,10 +65,10 @@ func TestValidator_HandleKeyReload(t *testing.T) {
inactivePrivKey, err := bls.RandKey()
require.NoError(t, err)
inactivePubKey := [48]byte{}
inactivePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(inactivePubKey[:], inactivePrivKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
inactivePubKey: inactivePrivKey,
},
}
@@ -87,7 +88,7 @@ func TestValidator_HandleKeyReload(t *testing.T) {
},
).Return(resp, nil)
anyActive, err := v.HandleKeyReload(context.Background(), [][48]byte{inactivePubKey})
anyActive, err := v.HandleKeyReload(context.Background(), [][fieldparams.BLSPubkeyLength]byte{inactivePubKey})
require.NoError(t, err)
assert.Equal(t, false, anyActive)
assert.LogsContain(t, hook, "Waiting for deposit to be observed by beacon node")
@@ -97,10 +98,10 @@ func TestValidator_HandleKeyReload(t *testing.T) {
t.Run("error when getting status", func(t *testing.T) {
inactivePrivKey, err := bls.RandKey()
require.NoError(t, err)
inactivePubKey := [48]byte{}
inactivePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(inactivePubKey[:], inactivePrivKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
inactivePubKey: inactivePrivKey,
},
}
@@ -118,7 +119,7 @@ func TestValidator_HandleKeyReload(t *testing.T) {
},
).Return(nil, errors.New("error"))
_, err = v.HandleKeyReload(context.Background(), [][48]byte{inactivePubKey})
_, err = v.HandleKeyReload(context.Background(), [][fieldparams.BLSPubkeyLength]byte{inactivePubKey})
assert.ErrorContains(t, "error", err)
})
}

View File

@@ -7,6 +7,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -226,7 +227,7 @@ func (v *validator) LogValidatorGainsAndLosses(ctx context.Context, slot types.S
return nil
}
var pks [][48]byte
var pks [][fieldparams.BLSPubkeyLength]byte
var err error
pks, err = v.keyManager.FetchValidatingPublicKeys(ctx)
if err != nil {

View File

@@ -4,6 +4,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -15,14 +16,14 @@ import (
func TestUpdateLogAggregateStats(t *testing.T) {
v := &validator{
logValidatorBalances: true,
startBalances: make(map[[48]byte]uint64),
prevBalance: make(map[[48]byte]uint64),
startBalances: make(map[[fieldparams.BLSPubkeyLength]byte]uint64),
prevBalance: make(map[[fieldparams.BLSPubkeyLength]byte]uint64),
voteStats: voteStats{
startEpoch: 0, // this would otherwise have been previously set in LogValidatorGainsAndLosses()
},
}
pubKeyBytes := [][48]byte{
pubKeyBytes := [][fieldparams.BLSPubkeyLength]byte{
bytesutil.ToBytes48([]byte("000000000000000000000000000000000000000012345678")),
bytesutil.ToBytes48([]byte("000000000000000000000000000000000000000099999999")),
bytesutil.ToBytes48([]byte("000000000000000000000000000000000000000055555555")),
@@ -95,14 +96,14 @@ func TestUpdateLogAggregateStats(t *testing.T) {
func TestUpdateLogAltairAggregateStats(t *testing.T) {
v := &validator{
logValidatorBalances: true,
startBalances: make(map[[48]byte]uint64),
prevBalance: make(map[[48]byte]uint64),
startBalances: make(map[[fieldparams.BLSPubkeyLength]byte]uint64),
prevBalance: make(map[[fieldparams.BLSPubkeyLength]byte]uint64),
voteStats: voteStats{
startEpoch: params.BeaconConfig().AltairForkEpoch, // this would otherwise have been previously set in LogValidatorGainsAndLosses()
},
}
pubKeyBytes := [][48]byte{
pubKeyBytes := [][fieldparams.BLSPubkeyLength]byte{
bytesutil.ToBytes48([]byte("000000000000000000000000000000000000000012345678")),
bytesutil.ToBytes48([]byte("000000000000000000000000000000000000000099999999")),
bytesutil.ToBytes48([]byte("000000000000000000000000000000000000000055555555")),

View File

@@ -10,6 +10,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/async"
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/crypto/rand"
@@ -38,7 +39,7 @@ const signExitErr = "could not sign voluntary exit proposal"
// chain node to construct the new block. The new block is then processed with
// the state root computation, and finally signed by the validator before being
// sent back to the beacon node for broadcasting.
func (v *validator) ProposeBlock(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) ProposeBlock(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
currEpoch := slots.ToEpoch(slot)
switch {
case currEpoch >= params.BeaconConfig().AltairForkEpoch:
@@ -48,7 +49,7 @@ func (v *validator) ProposeBlock(ctx context.Context, slot types.Slot, pubKey [4
}
}
func (v *validator) proposeBlockPhase0(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) proposeBlockPhase0(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
if slot == 0 {
log.Debug("Assigned to genesis slot, skipping proposal")
return
@@ -157,7 +158,7 @@ func (v *validator) proposeBlockPhase0(ctx context.Context, slot types.Slot, pub
}
// This is a routine to propose altair compatible beacon blocks.
func (v *validator) proposeBlockAltair(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) proposeBlockAltair(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
if slot == 0 {
log.Debug("Assigned to genesis slot, skipping proposal")
return
@@ -340,7 +341,7 @@ func ProposeExit(
}
// Sign randao reveal with randao domain and private key.
func (v *validator) signRandaoReveal(ctx context.Context, pubKey [48]byte, epoch types.Epoch) ([]byte, error) {
func (v *validator) signRandaoReveal(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, epoch types.Epoch) ([]byte, error) {
domain, err := v.domainData(ctx, epoch, params.BeaconConfig().DomainRandao[:])
if err != nil {
return nil, errors.Wrap(err, domainDataErr)
@@ -368,7 +369,7 @@ func (v *validator) signRandaoReveal(ctx context.Context, pubKey [48]byte, epoch
}
// Sign block with proposer domain and private key.
func (v *validator) signBlock(ctx context.Context, pubKey [48]byte, epoch types.Epoch, b block.BeaconBlock) ([]byte, *ethpb.DomainResponse, error) {
func (v *validator) signBlock(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, epoch types.Epoch, b block.BeaconBlock) ([]byte, *ethpb.DomainResponse, error) {
domain, err := v.domainData(ctx, epoch, params.BeaconConfig().DomainBeaconProposer[:])
if err != nil {
return nil, nil, errors.Wrap(err, domainDataErr)
@@ -461,7 +462,7 @@ func signVoluntaryExit(
}
// Gets the graffiti from cli or file for the validator public key.
func (v *validator) getGraffiti(ctx context.Context, pubKey [48]byte) ([]byte, error) {
func (v *validator) getGraffiti(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte) ([]byte, error) {
// When specified, default graffiti from the command line takes the first priority.
if len(v.graffiti) != 0 {
return v.graffiti, nil

View File

@@ -6,6 +6,7 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
"github.com/sirupsen/logrus"
@@ -15,7 +16,7 @@ var failedBlockSignLocalErr = "attempted to sign a double proposal, block reject
var failedBlockSignExternalErr = "attempted a double proposal, block rejected by remote slashing protection"
func (v *validator) slashableProposalCheck(
ctx context.Context, pubKey [48]byte, signedBlock block.SignedBeaconBlock, signingRoot [32]byte,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signedBlock block.SignedBeaconBlock, signingRoot [32]byte,
) error {
fmtKey := fmt.Sprintf("%#x", pubKey[:])
@@ -82,7 +83,7 @@ func (v *validator) slashableProposalCheck(
return nil
}
func blockLogFields(pubKey [48]byte, blk block.BeaconBlock, sig []byte) logrus.Fields {
func blockLogFields(pubKey [fieldparams.BLSPubkeyLength]byte, blk block.BeaconBlock, sig []byte) logrus.Fields {
fields := logrus.Fields{
"proposerPublicKey": fmt.Sprintf("%#x", pubKey),
"proposerIndex": blk.ProposerIndex(),

View File

@@ -7,6 +7,7 @@ import (
"github.com/golang/mock/gomock"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/block"
@@ -20,7 +21,7 @@ func Test_slashableProposalCheck_PreventsLowerThanMinProposal(t *testing.T) {
validator, _, validatorKey, finish := setup(t)
defer finish()
lowestSignedSlot := types.Slot(10)
pubKeyBytes := [48]byte{}
pubKeyBytes := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKeyBytes[:], validatorKey.PublicKey().Marshal())
// We save a proposal at the lowest signed slot in the DB.
@@ -88,7 +89,7 @@ func Test_slashableProposalCheck(t *testing.T) {
Signature: params.BeaconConfig().EmptySignature[:],
})
pubKeyBytes := [48]byte{}
pubKeyBytes := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKeyBytes[:], validatorKey.PublicKey().Marshal())
// We save a proposal at slot 1 as our lowest proposal.
@@ -99,7 +100,7 @@ func Test_slashableProposalCheck(t *testing.T) {
dummySigningRoot := [32]byte{1}
err = validator.db.SaveProposalHistoryForSlot(ctx, pubKeyBytes, 10, dummySigningRoot[:])
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
sBlock := wrapper.WrappedPhase0SignedBeaconBlock(blk)
blockHdr, err := block.SignedBeaconBlockHeaderFromBlockInterface(sBlock)
@@ -151,7 +152,7 @@ func Test_slashableProposalCheck_RemoteProtection(t *testing.T) {
defer reset()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
blk := util.NewBeaconBlock()

View File

@@ -11,6 +11,7 @@ import (
"github.com/golang/mock/gomock"
types "github.com/prysmaticlabs/eth2-types"
lruwrpr "github.com/prysmaticlabs/prysm/cache/lru"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
@@ -63,9 +64,9 @@ func setup(t *testing.T) (*validator, *mocks, bls.SecretKey, func()) {
}
func setupWithKey(t *testing.T, validatorKey bls.SecretKey) (*validator, *mocks, bls.SecretKey, func()) {
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
valDB := testing2.SetupDB(t, [][48]byte{pubKey})
valDB := testing2.SetupDB(t, [][fieldparams.BLSPubkeyLength]byte{pubKey})
ctrl := gomock.NewController(t)
m := &mocks{
validatorClient: mock.NewMockBeaconNodeValidatorClient(ctrl),
@@ -79,7 +80,7 @@ func setupWithKey(t *testing.T, validatorKey bls.SecretKey) (*validator, *mocks,
aggregatedSlotCommitteeIDCache := lruwrpr.New(int(params.BeaconConfig().MaxCommitteesPerSlot))
copy(pubKey[:], validatorKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: validatorKey,
},
}
@@ -100,7 +101,7 @@ func TestProposeBlock_DoesNotProposeGenesisBlock(t *testing.T) {
hook := logTest.NewGlobal()
validator, _, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.ProposeBlock(context.Background(), 0, pubKey)
@@ -111,7 +112,7 @@ func TestProposeBlock_DomainDataFailed(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -127,7 +128,7 @@ func TestProposeBlock_DomainDataIsNil(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -143,7 +144,7 @@ func TestProposeBlock_RequestBlockFailed(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -168,7 +169,7 @@ func TestProposeBlockAltair_RequestBlockFailed(t *testing.T) {
params.OverrideBeaconConfig(cfg)
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -189,7 +190,7 @@ func TestProposeBlock_ProposeBlockFailed(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -224,7 +225,7 @@ func TestProposeBlockAltair_ProposeBlockFailed(t *testing.T) {
params.OverrideBeaconConfig(cfg)
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -257,7 +258,7 @@ func TestProposeBlock_BlocksDoubleProposal(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
dummyRoot := [32]byte{}
@@ -313,7 +314,7 @@ func TestProposeBlockAltair_BlocksDoubleProposal(t *testing.T) {
params.OverrideBeaconConfig(cfg)
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
dummyRoot := [32]byte{}
@@ -369,7 +370,7 @@ func TestProposeBlock_BlocksDoubleProposal_After54KEpochs(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
dummyRoot := [32]byte{}
@@ -421,7 +422,7 @@ func TestProposeBlock_AllowsPastProposals(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
// Save a dummy proposal history at slot 0.
@@ -469,7 +470,7 @@ func TestProposeBlock_AllowsSameEpoch(t *testing.T) {
hook := logTest.NewGlobal()
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
// Save a dummy proposal history at slot 0.
@@ -516,7 +517,7 @@ func TestProposeBlock_AllowsSameEpoch(t *testing.T) {
func TestProposeBlock_BroadcastsBlock(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().DomainData(
@@ -545,7 +546,7 @@ func TestProposeBlock_BroadcastsBlock(t *testing.T) {
func TestProposeBlock_BroadcastsBlock_WithGraffiti(t *testing.T) {
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.graffiti = []byte("12345678901234567890123456789012")
@@ -588,7 +589,7 @@ func TestProposeBlockAltair_BroadcastsBlock_WithGraffiti(t *testing.T) {
params.OverrideBeaconConfig(cfg)
validator, m, validatorKey, finish := setup(t)
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.graffiti = []byte("12345678901234567890123456789012")
@@ -827,10 +828,10 @@ func TestSignBlock(t *testing.T) {
blk := util.NewBeaconBlock()
blk.Block.Slot = 1
blk.Block.ProposerIndex = 100
var pubKey [48]byte
var pubKey [fieldparams.BLSPubkeyLength]byte
copy(pubKey[:], publicKey.Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: secretKey,
},
}
@@ -860,10 +861,10 @@ func TestSignAltairBlock(t *testing.T) {
blk := util.NewBeaconBlockAltair()
blk.Block.Slot = 1
blk.Block.ProposerIndex = 100
var pubKey [48]byte
var pubKey [fieldparams.BLSPubkeyLength]byte
copy(pubKey[:], publicKey.Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: secretKey,
},
}
@@ -880,7 +881,7 @@ func TestGetGraffiti_Ok(t *testing.T) {
m := &mocks{
validatorClient: mock.NewMockBeaconNodeValidatorClient(ctrl),
}
pubKey := [48]byte{'a'}
pubKey := [fieldparams.BLSPubkeyLength]byte{'a'}
tests := []struct {
name string
v *validator
@@ -957,8 +958,8 @@ func TestGetGraffiti_Ok(t *testing.T) {
}
func TestGetGraffitiOrdered_Ok(t *testing.T) {
pubKey := [48]byte{'a'}
valDB := testing2.SetupDB(t, [][48]byte{pubKey})
pubKey := [fieldparams.BLSPubkeyLength]byte{'a'}
valDB := testing2.SetupDB(t, [][fieldparams.BLSPubkeyLength]byte{pubKey})
ctrl := gomock.NewController(t)
m := &mocks{
validatorClient: mock.NewMockBeaconNodeValidatorClient(ctrl),

View File

@@ -8,6 +8,7 @@ import (
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/time/slots"
@@ -103,7 +104,7 @@ func run(ctx context.Context, v iface.Validator) {
handleAssignmentError(err, headSlot)
}
accountsChangedChan := make(chan [][48]byte, 1)
accountsChangedChan := make(chan [][fieldparams.BLSPubkeyLength]byte, 1)
sub := v.GetKeymanager().SubscribeAccountChanges(accountsChangedChan)
for {
slotCtx, cancel := context.WithCancel(ctx)
@@ -185,7 +186,7 @@ func run(ctx context.Context, v iface.Validator) {
for pubKey, roles := range allRoles {
wg.Add(len(roles))
for _, role := range roles {
go func(role iface.ValidatorRole, pubKey [48]byte) {
go func(role iface.ValidatorRole, pubKey [fieldparams.BLSPubkeyLength]byte) {
defer wg.Done()
switch role {
case iface.RoleAttester:

View File

@@ -8,6 +8,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/async/event"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
"github.com/prysmaticlabs/prysm/validator/client/iface"
@@ -203,7 +204,7 @@ func TestKeyReload_ActiveKey(t *testing.T) {
km := &mockKeymanager{}
v := &testutil.FakeValidator{Keymanager: km}
go func() {
km.SimulateAccountChanges([][48]byte{testutil.ActiveKey})
km.SimulateAccountChanges([][fieldparams.BLSPubkeyLength]byte{testutil.ActiveKey})
cancel()
}()
@@ -219,7 +220,7 @@ func TestKeyReload_NoActiveKey(t *testing.T) {
km := &mockKeymanager{}
v := &testutil.FakeValidator{Keymanager: km}
go func() {
km.SimulateAccountChanges(make([][48]byte, 0))
km.SimulateAccountChanges(make([][fieldparams.BLSPubkeyLength]byte, 0))
cancel()
}()

View File

@@ -16,6 +16,7 @@ import (
grpcutil "github.com/prysmaticlabs/prysm/api/grpc"
"github.com/prysmaticlabs/prysm/async/event"
lruwrpr "github.com/prysmaticlabs/prysm/cache/lru"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -159,7 +160,7 @@ func (v *ValidatorService) Start() {
log.Errorf("Could not read slashable public keys from disk: %v", err)
return
}
slashablePublicKeys := make(map[[48]byte]bool)
slashablePublicKeys := make(map[[fieldparams.BLSPubkeyLength]byte]bool)
for _, pubKey := range sPubKeys {
slashablePublicKeys[pubKey] = true
}
@@ -180,8 +181,8 @@ func (v *ValidatorService) Start() {
graffiti: v.graffiti,
logValidatorBalances: v.logValidatorBalances,
emitAccountMetrics: v.emitAccountMetrics,
startBalances: make(map[[48]byte]uint64),
prevBalance: make(map[[48]byte]uint64),
startBalances: make(map[[fieldparams.BLSPubkeyLength]byte]uint64),
prevBalance: make(map[[fieldparams.BLSPubkeyLength]byte]uint64),
attLogs: make(map[[32]byte]*attSubmitted),
domainDataCache: cache,
aggregatedSlotCommitteeIDCache: aggregatedSlotCommitteeIDCache,
@@ -228,7 +229,7 @@ func (v *ValidatorService) Status() error {
}
func (v *ValidatorService) recheckKeys(ctx context.Context) {
var validatingKeys [][48]byte
var validatingKeys [][fieldparams.BLSPubkeyLength]byte
var err error
if v.useWeb {
initializedChan := make(chan *wallet.Wallet)
@@ -337,7 +338,7 @@ func recheckValidatingKeysBucket(ctx context.Context, valDB db.Database, km keym
if !ok {
return
}
validatingPubKeysChan := make(chan [][48]byte, 1)
validatingPubKeysChan := make(chan [][fieldparams.BLSPubkeyLength]byte, 1)
sub := importedKeymanager.SubscribeAccountChanges(validatingPubKeysChan)
defer func() {
sub.Unsubscribe()

View File

@@ -8,6 +8,7 @@ import (
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/altair"
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/monitoring/tracing"
@@ -19,7 +20,7 @@ import (
)
// SubmitSyncCommitteeMessage submits the sync committee message to the beacon chain.
func (v *validator) SubmitSyncCommitteeMessage(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) SubmitSyncCommitteeMessage(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
ctx, span := trace.StartSpan(ctx, "validator.SubmitSyncCommitteeMessage")
defer span.End()
span.AddAttributes(trace.StringAttribute("validator", fmt.Sprintf("%#x", pubKey)))
@@ -80,7 +81,7 @@ func (v *validator) SubmitSyncCommitteeMessage(ctx context.Context, slot types.S
}
// SubmitSignedContributionAndProof submits the signed sync committee contribution and proof to the beacon chain.
func (v *validator) SubmitSignedContributionAndProof(ctx context.Context, slot types.Slot, pubKey [48]byte) {
func (v *validator) SubmitSignedContributionAndProof(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) {
ctx, span := trace.StartSpan(ctx, "validator.SubmitSignedContributionAndProof")
defer span.End()
span.AddAttributes(trace.StringAttribute("validator", fmt.Sprintf("%#x", pubKey)))
@@ -171,7 +172,7 @@ func (v *validator) SubmitSignedContributionAndProof(ctx context.Context, slot t
}
// Signs and returns selection proofs per validator for slot and pub key.
func (v *validator) selectionProofs(ctx context.Context, slot types.Slot, pubKey [48]byte, indexRes *ethpb.SyncSubcommitteeIndexResponse) ([][]byte, error) {
func (v *validator) selectionProofs(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte, indexRes *ethpb.SyncSubcommitteeIndexResponse) ([][]byte, error) {
selectionProofs := make([][]byte, len(indexRes.Indices))
cfg := params.BeaconConfig()
size := cfg.SyncCommitteeSize
@@ -189,7 +190,7 @@ func (v *validator) selectionProofs(ctx context.Context, slot types.Slot, pubKey
}
// Signs input slot with domain sync committee selection proof. This is used to create the signature for sync committee selection.
func (v *validator) signSyncSelectionData(ctx context.Context, pubKey [48]byte, index uint64, slot types.Slot) (signature []byte, err error) {
func (v *validator) signSyncSelectionData(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, index uint64, slot types.Slot) (signature []byte, err error) {
domain, err := v.domainData(ctx, slots.ToEpoch(slot), params.BeaconConfig().DomainSyncCommitteeSelectionProof[:])
if err != nil {
return nil, err
@@ -215,7 +216,7 @@ func (v *validator) signSyncSelectionData(ctx context.Context, pubKey [48]byte,
}
// This returns the signature of validator signing over sync committee contribution and proof object.
func (v *validator) signContributionAndProof(ctx context.Context, pubKey [48]byte, c *ethpb.ContributionAndProof) ([]byte, error) {
func (v *validator) signContributionAndProof(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, c *ethpb.ContributionAndProof) ([]byte, error) {
d, err := v.domainData(ctx, slots.ToEpoch(c.Contribution.Slot), params.BeaconConfig().DomainContributionAndProof[:])
if err != nil {
return nil, err

View File

@@ -33,7 +33,7 @@ func TestSubmitSyncCommitteeMessage_ValidatorDutiesRequestFailure(t *testing.T)
Root: bytesutil.PadTo([]byte{}, 32),
}, nil)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitSyncCommitteeMessage(context.Background(), 1, pubKey)
require.LogsContain(t, hook, "Could not fetch validator assignment")
@@ -65,7 +65,7 @@ func TestSubmitSyncCommitteeMessage_BadDomainData(t *testing.T) {
DomainData(gomock.Any(), gomock.Any()).
Return(nil, errors.New("uh oh"))
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitSyncCommitteeMessage(context.Background(), 1, pubKey)
require.LogsContain(t, hook, "Could not get sync committee domain data")
@@ -105,7 +105,7 @@ func TestSubmitSyncCommitteeMessage_CouldNotSubmit(t *testing.T) {
gomock.AssignableToTypeOf(&ethpb.SyncCommitteeMessage{}),
).Return(&emptypb.Empty{}, errors.New("uh oh") /* error */)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitSyncCommitteeMessage(context.Background(), 1, pubKey)
@@ -149,7 +149,7 @@ func TestSubmitSyncCommitteeMessage_OK(t *testing.T) {
generatedMsg = msg
}).Return(&emptypb.Empty{}, nil /* error */)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitSyncCommitteeMessage(context.Background(), 1, pubKey)
@@ -165,7 +165,7 @@ func TestSubmitSignedContributionAndProof_ValidatorDutiesRequestFailure(t *testi
validator.duties = &ethpb.DutiesResponse{Duties: []*ethpb.DutiesResponse_Duty{}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
validator.SubmitSignedContributionAndProof(context.Background(), 1, pubKey)
require.LogsContain(t, hook, "Could not fetch validator assignment")
@@ -185,7 +185,7 @@ func TestSubmitSignedContributionAndProof_GetSyncSubcommitteeIndexFailure(t *tes
}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().GetSyncSubcommitteeIndex(
gomock.Any(), // ctx
@@ -213,7 +213,7 @@ func TestSubmitSignedContributionAndProof_NothingToDo(t *testing.T) {
}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().GetSyncSubcommitteeIndex(
gomock.Any(), // ctx
@@ -241,7 +241,7 @@ func TestSubmitSignedContributionAndProof_BadDomain(t *testing.T) {
}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().GetSyncSubcommitteeIndex(
gomock.Any(), // ctx
@@ -282,7 +282,7 @@ func TestSubmitSignedContributionAndProof_CouldNotGetContribution(t *testing.T)
}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().GetSyncSubcommitteeIndex(
gomock.Any(), // ctx
@@ -332,7 +332,7 @@ func TestSubmitSignedContributionAndProof_CouldNotSubmitContribution(t *testing.
}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().GetSyncSubcommitteeIndex(
gomock.Any(), // ctx
@@ -410,7 +410,7 @@ func TestSubmitSignedContributionAndProof_Ok(t *testing.T) {
}}
defer finish()
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], validatorKey.PublicKey().Marshal())
m.validatorClient.EXPECT().GetSyncSubcommitteeIndex(
gomock.Any(), // ctx

View File

@@ -10,6 +10,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/validator/client/testutil",
visibility = ["//validator:__subpackages__"],
deps = [
"//config/fieldparams:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time:go_default_library",

View File

@@ -6,6 +6,7 @@ import (
"time"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
prysmTime "github.com/prysmaticlabs/prysm/time"
"github.com/prysmaticlabs/prysm/validator/client/iface"
@@ -44,10 +45,10 @@ type FakeValidator struct {
PublicKey string
UpdateDutiesRet error
RolesAtRet []iface.ValidatorRole
Balances map[[48]byte]uint64
IndexToPubkeyMap map[uint64][48]byte
PubkeyToIndexMap map[[48]byte]uint64
PubkeysToStatusesMap map[[48]byte]ethpb.ValidatorStatus
Balances map[[fieldparams.BLSPubkeyLength]byte]uint64
IndexToPubkeyMap map[uint64][fieldparams.BLSPubkeyLength]byte
PubkeyToIndexMap map[[fieldparams.BLSPubkeyLength]byte]uint64
PubkeysToStatusesMap map[[fieldparams.BLSPubkeyLength]byte]ethpb.ValidatorStatus
Keymanager keymanager.IKeymanager
}
@@ -77,7 +78,7 @@ func (fv *FakeValidator) WaitForChainStart(_ context.Context) error {
}
// WaitForActivation for mocking.
func (fv *FakeValidator) WaitForActivation(_ context.Context, _ chan [][48]byte) error {
func (fv *FakeValidator) WaitForActivation(_ context.Context, _ chan [][fieldparams.BLSPubkeyLength]byte) error {
fv.WaitForActivationCalled++
if fv.RetryTillSuccess >= fv.WaitForActivationCalled {
return iface.ErrConnectionIssue
@@ -146,31 +147,33 @@ func (fv *FakeValidator) ResetAttesterProtectionData() {
}
// RolesAt for mocking.
func (fv *FakeValidator) RolesAt(_ context.Context, slot types.Slot) (map[[48]byte][]iface.ValidatorRole, error) {
func (fv *FakeValidator) RolesAt(_ context.Context, slot types.Slot) (map[[fieldparams.BLSPubkeyLength]byte][]iface.ValidatorRole, error) {
fv.RoleAtCalled = true
fv.RoleAtArg1 = uint64(slot)
vr := make(map[[48]byte][]iface.ValidatorRole)
vr[[48]byte{1}] = fv.RolesAtRet
vr := make(map[[fieldparams.BLSPubkeyLength]byte][]iface.ValidatorRole)
vr[[fieldparams.BLSPubkeyLength]byte{1}] = fv.RolesAtRet
return vr, nil
}
// SubmitAttestation for mocking.
func (fv *FakeValidator) SubmitAttestation(_ context.Context, slot types.Slot, _ [48]byte) {
func (fv *FakeValidator) SubmitAttestation(_ context.Context, slot types.Slot, _ [fieldparams.BLSPubkeyLength]byte) {
fv.AttestToBlockHeadCalled = true
fv.AttestToBlockHeadArg1 = uint64(slot)
}
// ProposeBlock for mocking.
func (fv *FakeValidator) ProposeBlock(_ context.Context, slot types.Slot, _ [48]byte) {
func (fv *FakeValidator) ProposeBlock(_ context.Context, slot types.Slot, _ [fieldparams.BLSPubkeyLength]byte) {
fv.ProposeBlockCalled = true
fv.ProposeBlockArg1 = uint64(slot)
}
// SubmitAggregateAndProof for mocking.
func (_ *FakeValidator) SubmitAggregateAndProof(_ context.Context, _ types.Slot, _ [48]byte) {}
func (_ *FakeValidator) SubmitAggregateAndProof(_ context.Context, _ types.Slot, _ [fieldparams.BLSPubkeyLength]byte) {
}
// SubmitSyncCommitteeMessage for mocking.
func (_ *FakeValidator) SubmitSyncCommitteeMessage(_ context.Context, _ types.Slot, _ [48]byte) {}
func (_ *FakeValidator) SubmitSyncCommitteeMessage(_ context.Context, _ types.Slot, _ [fieldparams.BLSPubkeyLength]byte) {
}
// LogAttestationsSubmitted for mocking.
func (_ *FakeValidator) LogAttestationsSubmitted() {}
@@ -184,22 +187,22 @@ func (_ *FakeValidator) LogNextDutyTimeLeft(_ types.Slot) error {
func (_ *FakeValidator) UpdateDomainDataCaches(context.Context, types.Slot) {}
// BalancesByPubkeys for mocking.
func (fv *FakeValidator) BalancesByPubkeys(_ context.Context) map[[48]byte]uint64 {
func (fv *FakeValidator) BalancesByPubkeys(_ context.Context) map[[fieldparams.BLSPubkeyLength]byte]uint64 {
return fv.Balances
}
// IndicesToPubkeys for mocking.
func (fv *FakeValidator) IndicesToPubkeys(_ context.Context) map[uint64][48]byte {
func (fv *FakeValidator) IndicesToPubkeys(_ context.Context) map[uint64][fieldparams.BLSPubkeyLength]byte {
return fv.IndexToPubkeyMap
}
// PubkeysToIndices for mocking.
func (fv *FakeValidator) PubkeysToIndices(_ context.Context) map[[48]byte]uint64 {
func (fv *FakeValidator) PubkeysToIndices(_ context.Context) map[[fieldparams.BLSPubkeyLength]byte]uint64 {
return fv.PubkeyToIndexMap
}
// PubkeysToStatuses for mocking.
func (fv *FakeValidator) PubkeysToStatuses(_ context.Context) map[[48]byte]ethpb.ValidatorStatus {
func (fv *FakeValidator) PubkeysToStatuses(_ context.Context) map[[fieldparams.BLSPubkeyLength]byte]ethpb.ValidatorStatus {
return fv.PubkeysToStatusesMap
}
@@ -230,7 +233,7 @@ func (fv *FakeValidator) ReceiveBlocks(_ context.Context, connectionErrorChannel
}
// HandleKeyReload for mocking
func (fv *FakeValidator) HandleKeyReload(_ context.Context, newKeys [][48]byte) (anyActive bool, err error) {
func (fv *FakeValidator) HandleKeyReload(_ context.Context, newKeys [][fieldparams.BLSPubkeyLength]byte) (anyActive bool, err error) {
fv.HandleKeyReloadCalled = true
for _, key := range newKeys {
if bytes.Equal(key[:], ActiveKey[:]) {
@@ -241,5 +244,5 @@ func (fv *FakeValidator) HandleKeyReload(_ context.Context, newKeys [][48]byte)
}
// SubmitSignedContributionAndProof for mocking
func (_ *FakeValidator) SubmitSignedContributionAndProof(_ context.Context, _ types.Slot, _ [48]byte) {
func (_ *FakeValidator) SubmitSignedContributionAndProof(_ context.Context, _ types.Slot, _ [fieldparams.BLSPubkeyLength]byte) {
}

View File

@@ -70,9 +70,9 @@ type validator struct {
domainDataCache *ristretto.Cache
aggregatedSlotCommitteeIDCache *lru.Cache
ticker slots.Ticker
prevBalance map[[48]byte]uint64
prevBalance map[[fieldparams.BLSPubkeyLength]byte]uint64
duties *ethpb.DutiesResponse
startBalances map[[48]byte]uint64
startBalances map[[fieldparams.BLSPubkeyLength]byte]uint64
attLogs map[[32]byte]*attSubmitted
node ethpb.NodeClient
keyManager keymanager.IKeymanager
@@ -84,7 +84,7 @@ type validator struct {
voteStats voteStats
graffitiStruct *graffiti.Graffiti
graffitiOrderedIndex uint64
eipImportBlacklistedPublicKeys map[[48]byte]bool
eipImportBlacklistedPublicKeys map[[fieldparams.BLSPubkeyLength]byte]bool
}
type validatorStatus struct {
@@ -415,7 +415,7 @@ func buildDuplicateError(response []*ethpb.DoppelGangerResponse_ValidatorRespons
duplicates := make([][]byte, 0)
for _, valRes := range response {
if valRes.DuplicateExists {
copiedKey := [48]byte{}
copiedKey := [fieldparams.BLSPubkeyLength]byte{}
copy(copiedKey[:], valRes.PublicKey)
duplicates = append(duplicates, copiedKey[:])
}
@@ -473,7 +473,7 @@ func (v *validator) UpdateDuties(ctx context.Context, slot types.Slot) error {
}
// Filter out the slashable public keys from the duties request.
filteredKeys := make([][48]byte, 0, len(validatingKeys))
filteredKeys := make([][fieldparams.BLSPubkeyLength]byte, 0, len(validatingKeys))
v.slashableKeysLock.RLock()
for _, pubKey := range validatingKeys {
if ok := v.eipImportBlacklistedPublicKeys[pubKey]; !ok {
@@ -582,8 +582,8 @@ func (v *validator) subscribeToSubnets(ctx context.Context, res *ethpb.DutiesRes
// RolesAt slot returns the validator roles at the given slot. Returns nil if the
// validator is known to not have a roles at the slot. Returns UNKNOWN if the
// validator assignments are unknown. Otherwise returns a valid ValidatorRole map.
func (v *validator) RolesAt(ctx context.Context, slot types.Slot) (map[[48]byte][]iface.ValidatorRole, error) {
rolesAt := make(map[[48]byte][]iface.ValidatorRole)
func (v *validator) RolesAt(ctx context.Context, slot types.Slot) (map[[fieldparams.BLSPubkeyLength]byte][]iface.ValidatorRole, error) {
rolesAt := make(map[[fieldparams.BLSPubkeyLength]byte][]iface.ValidatorRole)
for validator, duty := range v.duties.Duties {
var roles []iface.ValidatorRole
@@ -640,7 +640,7 @@ func (v *validator) RolesAt(ctx context.Context, slot types.Slot) (map[[48]byte]
roles = append(roles, iface.RoleUnknown)
}
var pubKey [48]byte
var pubKey [fieldparams.BLSPubkeyLength]byte
copy(pubKey[:], duty.PublicKey)
rolesAt[pubKey] = roles
}
@@ -654,7 +654,7 @@ func (v *validator) GetKeymanager() keymanager.IKeymanager {
// isAggregator checks if a validator is an aggregator of a given slot and committee,
// it uses a modulo calculated by validator count in committee and samples randomness around it.
func (v *validator) isAggregator(ctx context.Context, committee []types.ValidatorIndex, slot types.Slot, pubKey [48]byte) (bool, error) {
func (v *validator) isAggregator(ctx context.Context, committee []types.ValidatorIndex, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) (bool, error) {
modulo := uint64(1)
if len(committee)/int(params.BeaconConfig().TargetAggregatorsPerCommittee) > 1 {
modulo = uint64(len(committee)) / params.BeaconConfig().TargetAggregatorsPerCommittee
@@ -677,7 +677,7 @@ func (v *validator) isAggregator(ctx context.Context, committee []types.Validato
// def is_sync_committee_aggregator(signature: BLSSignature) -> bool:
// modulo = max(1, SYNC_COMMITTEE_SIZE // SYNC_COMMITTEE_SUBNET_COUNT // TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE)
// return bytes_to_uint64(hash(signature)[0:8]) % modulo == 0
func (v *validator) isSyncCommitteeAggregator(ctx context.Context, slot types.Slot, pubKey [48]byte) (bool, error) {
func (v *validator) isSyncCommitteeAggregator(ctx context.Context, slot types.Slot, pubKey [fieldparams.BLSPubkeyLength]byte) (bool, error) {
res, err := v.validatorClient.GetSyncSubcommitteeIndex(ctx, &ethpb.SyncSubcommitteeIndexRequest{
PublicKey: pubKey[:],
Slot: slot,

View File

@@ -42,7 +42,7 @@ var _ iface.Validator = (*validator)(nil)
const cancelledCtx = "context has been canceled"
func genMockKeymanager(numKeys int) *mockKeymanager {
km := make(map[[48]byte]bls.SecretKey, numKeys)
km := make(map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey, numKeys)
for i := 0; i < numKeys; i++ {
k, err := bls.RandKey()
if err != nil {
@@ -56,15 +56,15 @@ func genMockKeymanager(numKeys int) *mockKeymanager {
type mockKeymanager struct {
lock sync.RWMutex
keysMap map[[48]byte]bls.SecretKey
keysMap map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey
fetchNoKeys bool
accountsChangedFeed *event.Feed
}
func (m *mockKeymanager) FetchValidatingPublicKeys(_ context.Context) ([][48]byte, error) {
func (m *mockKeymanager) FetchValidatingPublicKeys(_ context.Context) ([][fieldparams.BLSPubkeyLength]byte, error) {
m.lock.RLock()
defer m.lock.RUnlock()
keys := make([][48]byte, 0)
keys := make([][fieldparams.BLSPubkeyLength]byte, 0)
if m.fetchNoKeys {
m.fetchNoKeys = false
return keys, nil
@@ -76,7 +76,7 @@ func (m *mockKeymanager) FetchValidatingPublicKeys(_ context.Context) ([][48]byt
}
func (m *mockKeymanager) Sign(_ context.Context, req *validatorpb.SignRequest) (bls.Signature, error) {
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], req.PublicKey)
privKey, ok := m.keysMap[pubKey]
if !ok {
@@ -86,14 +86,14 @@ func (m *mockKeymanager) Sign(_ context.Context, req *validatorpb.SignRequest) (
return sig, nil
}
func (m *mockKeymanager) SubscribeAccountChanges(pubKeysChan chan [][48]byte) event.Subscription {
func (m *mockKeymanager) SubscribeAccountChanges(pubKeysChan chan [][fieldparams.BLSPubkeyLength]byte) event.Subscription {
if m.accountsChangedFeed == nil {
m.accountsChangedFeed = &event.Feed{}
}
return m.accountsChangedFeed.Subscribe(pubKeysChan)
}
func (m *mockKeymanager) SimulateAccountChanges(newKeys [][48]byte) {
func (m *mockKeymanager) SimulateAccountChanges(newKeys [][fieldparams.BLSPubkeyLength]byte) {
m.accountsChangedFeed.Send(newKeys)
}
@@ -115,7 +115,7 @@ func TestWaitForChainStart_SetsGenesisInfo(t *testing.T) {
defer ctrl.Finish()
client := mock2.NewMockBeaconNodeValidatorClient(ctrl)
db := dbTest.SetupDB(t, [][48]byte{})
db := dbTest.SetupDB(t, [][fieldparams.BLSPubkeyLength]byte{})
v := validator{
validatorClient: client,
db: db,
@@ -170,7 +170,7 @@ func TestWaitForChainStart_SetsGenesisInfo_IncorrectSecondTry(t *testing.T) {
defer ctrl.Finish()
client := mock2.NewMockBeaconNodeValidatorClient(ctrl)
db := dbTest.SetupDB(t, [][48]byte{})
db := dbTest.SetupDB(t, [][fieldparams.BLSPubkeyLength]byte{})
v := validator{
validatorClient: client,
db: db,
@@ -253,10 +253,10 @@ func TestWaitForChainStart_StreamSetupFails(t *testing.T) {
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: make(map[[48]byte]bls.SecretKey),
keysMap: make(map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey),
}
v := validator{
validatorClient: client,
@@ -333,10 +333,10 @@ func TestWaitMultipleActivation_LogsActivationEpochOK(t *testing.T) {
client := mock2.NewMockBeaconNodeValidatorClient(ctrl)
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -370,10 +370,10 @@ func TestWaitActivation_NotAllValidatorsActivatedOK(t *testing.T) {
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -492,10 +492,10 @@ func TestUpdateDuties_ReturnsError(t *testing.T) {
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -530,10 +530,10 @@ func TestUpdateDuties_OK(t *testing.T) {
slot := params.BeaconConfig().SlotsPerEpoch
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -587,12 +587,12 @@ func TestUpdateDuties_OK_FilterBlacklistedPublicKeys(t *testing.T) {
slot := params.BeaconConfig().SlotsPerEpoch
numValidators := 10
keysMap := make(map[[48]byte]bls.SecretKey)
blacklistedPublicKeys := make(map[[48]byte]bool)
keysMap := make(map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey)
blacklistedPublicKeys := make(map[[fieldparams.BLSPubkeyLength]byte]bool)
for i := 0; i < numValidators; i++ {
priv, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], priv.PublicKey().Marshal())
keysMap[pubKey] = priv
blacklistedPublicKeys[pubKey] = true
@@ -937,8 +937,8 @@ func TestAllValidatorsAreExited_CorrectRequest(t *testing.T) {
client := mock2.NewMockBeaconNodeValidatorClient(ctrl)
// Create two different public keys
pubKey0 := [48]byte{1, 2, 3, 4}
pubKey1 := [48]byte{6, 7, 8, 9}
pubKey0 := [fieldparams.BLSPubkeyLength]byte{1, 2, 3, 4}
pubKey1 := [fieldparams.BLSPubkeyLength]byte{6, 7, 8, 9}
// This is the request expected from AllValidatorsAreExited()
request := &ethpb.MultipleValidatorStatusRequest{
PublicKeys: [][]byte{
@@ -956,7 +956,7 @@ func TestAllValidatorsAreExited_CorrectRequest(t *testing.T) {
request, // request
).Return(&ethpb.MultipleValidatorStatusResponse{Statuses: statuses}, nil /*err*/)
keysMap := make(map[[48]byte]bls.SecretKey)
keysMap := make(map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey)
// secretKey below is just filler and is used multiple times
secretKeyBytes := [32]byte{1}
secretKey, err := bls.SecretKeyFromBytes(secretKeyBytes[:])

View File

@@ -6,6 +6,7 @@ import (
"time"
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/math"
@@ -22,10 +23,10 @@ import (
// from the gRPC server.
//
// If the channel parameter is nil, WaitForActivation creates and manages its own channel.
func (v *validator) WaitForActivation(ctx context.Context, accountsChangedChan chan [][48]byte) error {
func (v *validator) WaitForActivation(ctx context.Context, accountsChangedChan chan [][fieldparams.BLSPubkeyLength]byte) error {
// Monitor the key manager for updates.
if accountsChangedChan == nil {
accountsChangedChan = make(chan [][48]byte, 1)
accountsChangedChan = make(chan [][fieldparams.BLSPubkeyLength]byte, 1)
sub := v.GetKeymanager().SubscribeAccountChanges(accountsChangedChan)
defer func() {
sub.Unsubscribe()
@@ -43,7 +44,7 @@ func (v *validator) WaitForActivation(ctx context.Context, accountsChangedChan c
// the accountsChangedChan. When an event signal is received, restart the waitForActivation routine.
// 4) If the stream is reset in error, restart the routine.
// 5) If the stream returns a response indicating one or more validators are active, exit the routine.
func (v *validator) waitForActivation(ctx context.Context, accountsChangedChan <-chan [][48]byte) error {
func (v *validator) waitForActivation(ctx context.Context, accountsChangedChan <-chan [][fieldparams.BLSPubkeyLength]byte) error {
ctx, span := trace.StartSpan(ctx, "validator.WaitForActivation")
defer span.End()

View File

@@ -9,6 +9,7 @@ import (
"github.com/golang/mock/gomock"
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -32,10 +33,10 @@ func TestWaitActivation_ContextCanceled(t *testing.T) {
client := mock.NewMockBeaconNodeValidatorClient(ctrl)
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -66,10 +67,10 @@ func TestWaitActivation_StreamSetupFails_AttemptsToReconnect(t *testing.T) {
client := mock.NewMockBeaconNodeValidatorClient(ctrl)
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -98,10 +99,10 @@ func TestWaitForActivation_ReceiveErrorFromStream_AttemptsReconnection(t *testin
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -133,10 +134,10 @@ func TestWaitActivation_LogsActivationEpochOK(t *testing.T) {
client := mock.NewMockBeaconNodeValidatorClient(ctrl)
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -168,10 +169,10 @@ func TestWaitForActivation_Exiting(t *testing.T) {
client := mock.NewMockBeaconNodeValidatorClient(ctrl)
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
}
@@ -209,10 +210,10 @@ func TestWaitForActivation_RefetchKeys(t *testing.T) {
client := mock.NewMockBeaconNodeValidatorClient(ctrl)
privKey, err := bls.RandKey()
require.NoError(t, err)
pubKey := [48]byte{}
pubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKey[:], privKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
pubKey: privKey,
},
fetchNoKeys: true,
@@ -235,7 +236,7 @@ func TestWaitForActivation_RefetchKeys(t *testing.T) {
resp,
nil,
)
assert.NoError(t, v.waitForActivation(context.Background(), make(chan [][48]byte)), "Could not wait for activation")
assert.NoError(t, v.waitForActivation(context.Background(), make(chan [][fieldparams.BLSPubkeyLength]byte)), "Could not wait for activation")
assert.LogsContain(t, hook, msgNoKeysFetched)
assert.LogsContain(t, hook, "Validator activated")
}
@@ -249,14 +250,14 @@ func TestWaitForActivation_AccountsChanged(t *testing.T) {
t.Run("Imported keymanager", func(t *testing.T) {
inactivePrivKey, err := bls.RandKey()
require.NoError(t, err)
inactivePubKey := [48]byte{}
inactivePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(inactivePubKey[:], inactivePrivKey.PublicKey().Marshal())
activePrivKey, err := bls.RandKey()
require.NoError(t, err)
activePubKey := [48]byte{}
activePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(activePubKey[:], activePrivKey.PublicKey().Marshal())
km := &mockKeymanager{
keysMap: map[[48]byte]bls.SecretKey{
keysMap: map[[fieldparams.BLSPubkeyLength]byte]bls.SecretKey{
inactivePubKey: inactivePrivKey,
},
}
@@ -300,7 +301,7 @@ func TestWaitForActivation_AccountsChanged(t *testing.T) {
// We add the active key into the keymanager and simulate a key refresh.
time.Sleep(time.Second * 1)
km.keysMap[activePubKey] = activePrivKey
km.SimulateAccountChanges(make([][48]byte, 0))
km.SimulateAccountChanges(make([][fieldparams.BLSPubkeyLength]byte, 0))
}()
assert.NoError(t, v.WaitForActivation(context.Background(), nil))
@@ -313,12 +314,12 @@ func TestWaitForActivation_AccountsChanged(t *testing.T) {
inactivePrivKey, err :=
util.PrivateKeyFromSeedAndPath(seed, fmt.Sprintf(derived.ValidatingKeyDerivationPathTemplate, 0))
require.NoError(t, err)
inactivePubKey := [48]byte{}
inactivePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(inactivePubKey[:], inactivePrivKey.PublicKey().Marshal())
activePrivKey, err :=
util.PrivateKeyFromSeedAndPath(seed, fmt.Sprintf(derived.ValidatingKeyDerivationPathTemplate, 1))
require.NoError(t, err)
activePubKey := [48]byte{}
activePubKey := [fieldparams.BLSPubkeyLength]byte{}
copy(activePubKey[:], activePrivKey.PublicKey().Marshal())
wallet := &walletMock.Wallet{
Files: make(map[string]map[string][]byte),
@@ -369,13 +370,13 @@ func TestWaitForActivation_AccountsChanged(t *testing.T) {
nil,
)
channel := make(chan [][48]byte)
channel := make(chan [][fieldparams.BLSPubkeyLength]byte)
go func() {
// We add the active key into the keymanager and simulate a key refresh.
time.Sleep(time.Second * 1)
err = km.RecoverAccountsFromMnemonic(ctx, constant.TestMnemonic, "", 2)
require.NoError(t, err)
channel <- [][48]byte{}
channel <- [][fieldparams.BLSPubkeyLength]byte{}
}()
assert.NoError(t, v.waitForActivation(context.Background(), channel))
@@ -398,7 +399,7 @@ func TestWaitForActivation_RemoteKeymanager(t *testing.T) {
inactiveKey := bytesutil.ToBytes48([]byte("inactive"))
activeKey := bytesutil.ToBytes48([]byte("active"))
km := remote.NewMock()
km.PublicKeys = [][48]byte{inactiveKey, activeKey}
km.PublicKeys = [][fieldparams.BLSPubkeyLength]byte{inactiveKey, activeKey}
slot := types.Slot(0)
t.Run("activated", func(t *testing.T) {
@@ -461,7 +462,7 @@ func TestWaitForActivation_RemoteKeymanager(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
hook := logTest.NewGlobal()
remoteKm := remote.NewMock()
remoteKm.PublicKeys = [][48]byte{inactiveKey}
remoteKm.PublicKeys = [][fieldparams.BLSPubkeyLength]byte{inactiveKey}
tickerChan := make(chan types.Slot)
ticker := &slotutilmock.MockTicker{
@@ -475,7 +476,7 @@ func TestWaitForActivation_RemoteKeymanager(t *testing.T) {
go func() {
tickerChan <- slot
time.Sleep(time.Second)
remoteKm.PublicKeys = [][48]byte{inactiveKey, activeKey}
remoteKm.PublicKeys = [][fieldparams.BLSPubkeyLength]byte{inactiveKey, activeKey}
tickerChan <- slot
// Cancel after timeout to avoid waiting on channel forever in case test goes wrong.
time.Sleep(time.Second)

View File

@@ -7,6 +7,7 @@ go_library(
# Other packages must use github.com/prysmaticlabs/prysm/validator/db.Database alias.
visibility = ["//validator/db:__subpackages__"],
deps = [
"//config/fieldparams:go_default_library",
"//monitoring/backup:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//validator/db/kv:go_default_library",

View File

@@ -6,6 +6,7 @@ import (
"io"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/monitoring/backup"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/validator/db/kv"
@@ -22,40 +23,40 @@ type ValidatorDB interface {
ClearDB() error
RunUpMigrations(ctx context.Context) error
RunDownMigrations(ctx context.Context) error
UpdatePublicKeysBuckets(publicKeys [][48]byte) error
UpdatePublicKeysBuckets(publicKeys [][fieldparams.BLSPubkeyLength]byte) error
// Genesis information related methods.
GenesisValidatorsRoot(ctx context.Context) ([]byte, error)
SaveGenesisValidatorsRoot(ctx context.Context, genValRoot []byte) error
// Proposer protection related methods.
HighestSignedProposal(ctx context.Context, publicKey [48]byte) (types.Slot, bool, error)
LowestSignedProposal(ctx context.Context, publicKey [48]byte) (types.Slot, bool, error)
ProposalHistoryForPubKey(ctx context.Context, publicKey [48]byte) ([]*kv.Proposal, error)
ProposalHistoryForSlot(ctx context.Context, publicKey [48]byte, slot types.Slot) ([32]byte, bool, error)
SaveProposalHistoryForSlot(ctx context.Context, pubKey [48]byte, slot types.Slot, signingRoot []byte) error
ProposedPublicKeys(ctx context.Context) ([][48]byte, error)
HighestSignedProposal(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) (types.Slot, bool, error)
LowestSignedProposal(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) (types.Slot, bool, error)
ProposalHistoryForPubKey(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) ([]*kv.Proposal, error)
ProposalHistoryForSlot(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte, slot types.Slot) ([32]byte, bool, error)
SaveProposalHistoryForSlot(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, slot types.Slot, signingRoot []byte) error
ProposedPublicKeys(ctx context.Context) ([][fieldparams.BLSPubkeyLength]byte, error)
// Attester protection related methods.
// Methods to store and read blacklisted public keys from EIP-3076
// slashing protection imports.
EIPImportBlacklistedPublicKeys(ctx context.Context) ([][48]byte, error)
SaveEIPImportBlacklistedPublicKeys(ctx context.Context, publicKeys [][48]byte) error
SigningRootAtTargetEpoch(ctx context.Context, publicKey [48]byte, target types.Epoch) ([32]byte, error)
LowestSignedTargetEpoch(ctx context.Context, publicKey [48]byte) (types.Epoch, bool, error)
LowestSignedSourceEpoch(ctx context.Context, publicKey [48]byte) (types.Epoch, bool, error)
AttestedPublicKeys(ctx context.Context) ([][48]byte, error)
EIPImportBlacklistedPublicKeys(ctx context.Context) ([][fieldparams.BLSPubkeyLength]byte, error)
SaveEIPImportBlacklistedPublicKeys(ctx context.Context, publicKeys [][fieldparams.BLSPubkeyLength]byte) error
SigningRootAtTargetEpoch(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte, target types.Epoch) ([32]byte, error)
LowestSignedTargetEpoch(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) (types.Epoch, bool, error)
LowestSignedSourceEpoch(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) (types.Epoch, bool, error)
AttestedPublicKeys(ctx context.Context) ([][fieldparams.BLSPubkeyLength]byte, error)
CheckSlashableAttestation(
ctx context.Context, pubKey [48]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
) (kv.SlashingKind, error)
SaveAttestationForPubKey(
ctx context.Context, pubKey [48]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
) error
SaveAttestationsForPubKey(
ctx context.Context, pubKey [48]byte, signingRoots [][32]byte, atts []*ethpb.IndexedAttestation,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signingRoots [][32]byte, atts []*ethpb.IndexedAttestation,
) error
AttestationHistoryForPubKey(
ctx context.Context, pubKey [48]byte,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte,
) ([]*kv.AttestationRecord, error)
// Graffiti ordered index related methods

View File

@@ -63,6 +63,7 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//crypto/hash:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -8,6 +8,7 @@ import (
"github.com/pkg/errors"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/monitoring/tracing"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
@@ -22,7 +23,7 @@ type SlashingKind int
// AttestationRecord which can be represented by these simple values
// for manipulation by database methods.
type AttestationRecord struct {
PubKey [48]byte
PubKey [fieldparams.BLSPubkeyLength]byte
Source types.Epoch
Target types.Epoch
SigningRoot [32]byte
@@ -92,7 +93,7 @@ var (
// AttestationHistoryForPubKey retrieves a list of attestation records for data
// we have stored in the database for the given validator public key.
func (s *Store) AttestationHistoryForPubKey(ctx context.Context, pubKey [48]byte) ([]*AttestationRecord, error) {
func (s *Store) AttestationHistoryForPubKey(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte) ([]*AttestationRecord, error) {
records := make([]*AttestationRecord, 0)
_, span := trace.StartSpan(ctx, "Validator.AttestationHistoryForPubKey")
defer span.End()
@@ -133,7 +134,7 @@ func (s *Store) AttestationHistoryForPubKey(ctx context.Context, pubKey [48]byte
// CheckSlashableAttestation verifies an incoming attestation is
// not a double vote for a validator public key nor a surround vote.
func (s *Store) CheckSlashableAttestation(
ctx context.Context, pubKey [48]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
) (SlashingKind, error) {
ctx, span := trace.StartSpan(ctx, "Validator.CheckSlashableAttestation")
defer span.End()
@@ -273,7 +274,7 @@ func (_ *Store) checkSurroundingVote(
// SaveAttestationsForPubKey stores a batch of attestations all at once.
func (s *Store) SaveAttestationsForPubKey(
ctx context.Context, pubKey [48]byte, signingRoots [][32]byte, atts []*ethpb.IndexedAttestation,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signingRoots [][32]byte, atts []*ethpb.IndexedAttestation,
) error {
ctx, span := trace.StartSpan(ctx, "Validator.SaveAttestationsForPubKey")
defer span.End()
@@ -299,7 +300,7 @@ func (s *Store) SaveAttestationsForPubKey(
// SaveAttestationForPubKey saves an attestation for a validator public
// key for local validator slashing protection.
func (s *Store) SaveAttestationForPubKey(
ctx context.Context, pubKey [48]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, signingRoot [32]byte, att *ethpb.IndexedAttestation,
) error {
_, span := trace.StartSpan(ctx, "Validator.SaveAttestationForPubKey")
defer span.End()
@@ -491,15 +492,15 @@ func (s *Store) saveAttestationRecords(ctx context.Context, atts []*AttestationR
}
// AttestedPublicKeys retrieves all public keys that have attested.
func (s *Store) AttestedPublicKeys(ctx context.Context) ([][48]byte, error) {
func (s *Store) AttestedPublicKeys(ctx context.Context) ([][fieldparams.BLSPubkeyLength]byte, error) {
_, span := trace.StartSpan(ctx, "Validator.AttestedPublicKeys")
defer span.End()
var err error
attestedPublicKeys := make([][48]byte, 0)
attestedPublicKeys := make([][fieldparams.BLSPubkeyLength]byte, 0)
err = s.view(func(tx *bolt.Tx) error {
bucket := tx.Bucket(pubKeysBucket)
return bucket.ForEach(func(pubKey []byte, _ []byte) error {
var pk [48]byte
var pk [fieldparams.BLSPubkeyLength]byte
copy(pk[:], pubKey)
attestedPublicKeys = append(attestedPublicKeys, pk)
return nil
@@ -510,7 +511,7 @@ func (s *Store) AttestedPublicKeys(ctx context.Context) ([][48]byte, error) {
// SigningRootAtTargetEpoch checks for an existing signing root at a specified
// target epoch for a given validator public key.
func (s *Store) SigningRootAtTargetEpoch(ctx context.Context, pubKey [48]byte, target types.Epoch) ([32]byte, error) {
func (s *Store) SigningRootAtTargetEpoch(ctx context.Context, pubKey [fieldparams.BLSPubkeyLength]byte, target types.Epoch) ([32]byte, error) {
_, span := trace.StartSpan(ctx, "Validator.SigningRootAtTargetEpoch")
defer span.End()
var signingRoot [32]byte
@@ -533,7 +534,7 @@ func (s *Store) SigningRootAtTargetEpoch(ctx context.Context, pubKey [48]byte, t
// LowestSignedSourceEpoch returns the lowest signed source epoch for a validator public key.
// If no data exists, returning 0 is a sensible default.
func (s *Store) LowestSignedSourceEpoch(ctx context.Context, publicKey [48]byte) (types.Epoch, bool, error) {
func (s *Store) LowestSignedSourceEpoch(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) (types.Epoch, bool, error) {
_, span := trace.StartSpan(ctx, "Validator.LowestSignedSourceEpoch")
defer span.End()
@@ -556,7 +557,7 @@ func (s *Store) LowestSignedSourceEpoch(ctx context.Context, publicKey [48]byte)
// LowestSignedTargetEpoch returns the lowest signed target epoch for a validator public key.
// If no data exists, returning 0 is a sensible default.
func (s *Store) LowestSignedTargetEpoch(ctx context.Context, publicKey [48]byte) (types.Epoch, bool, error) {
func (s *Store) LowestSignedTargetEpoch(ctx context.Context, publicKey [fieldparams.BLSPubkeyLength]byte) (types.Epoch, bool, error) {
_, span := trace.StartSpan(ctx, "Validator.LowestSignedTargetEpoch")
defer span.End()

View File

@@ -9,6 +9,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/assert"
@@ -45,7 +46,7 @@ func TestPendingAttestationRecords_Len(t *testing.T) {
func TestStore_CheckSlashableAttestation_DoubleVote(t *testing.T) {
ctx := context.Background()
numValidators := 1
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
validatorDB := setupDB(t, pubKeys)
tests := []struct {
name string
@@ -116,7 +117,7 @@ func TestStore_CheckSlashableAttestation_DoubleVote(t *testing.T) {
func TestStore_CheckSlashableAttestation_SurroundVote_MultipleTargetsPerSource(t *testing.T) {
ctx := context.Background()
numValidators := 1
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
validatorDB := setupDB(t, pubKeys)
// Create an attestation with source 1 and target 50, save it.
@@ -142,7 +143,7 @@ func TestStore_CheckSlashableAttestation_SurroundVote_54kEpochs(t *testing.T) {
ctx := context.Background()
numValidators := 1
numEpochs := types.Epoch(54000)
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
validatorDB := setupDB(t, pubKeys)
// Attest to every (source = epoch, target = epoch + 1) sequential pair
@@ -219,8 +220,8 @@ func TestLowestSignedSourceEpoch_SaveRetrieve(t *testing.T) {
require.NoError(t, validatorDB.Close(), "Failed to close database")
require.NoError(t, validatorDB.ClearDB(), "Failed to clear database")
})
p0 := [48]byte{0}
p1 := [48]byte{1}
p0 := [fieldparams.BLSPubkeyLength]byte{0}
p1 := [fieldparams.BLSPubkeyLength]byte{1}
// Can save.
require.NoError(
t,
@@ -278,8 +279,8 @@ func TestLowestSignedTargetEpoch_SaveRetrieveReplace(t *testing.T) {
require.NoError(t, validatorDB.Close(), "Failed to close database")
require.NoError(t, validatorDB.ClearDB(), "Failed to clear database")
})
p0 := [48]byte{0}
p1 := [48]byte{1}
p0 := [fieldparams.BLSPubkeyLength]byte{0}
p1 := [fieldparams.BLSPubkeyLength]byte{1}
// Can save.
require.NoError(
t,
@@ -332,7 +333,7 @@ func TestLowestSignedTargetEpoch_SaveRetrieveReplace(t *testing.T) {
func TestStore_SaveAttestationsForPubKey(t *testing.T) {
ctx := context.Background()
numValidators := 1
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
validatorDB := setupDB(t, pubKeys)
atts := make([]*ethpb.IndexedAttestation, 0)
signingRoots := make([][32]byte, 0)
@@ -374,14 +375,14 @@ func TestSaveAttestationForPubKey_BatchWrites_FullCapacity(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
numValidators := attestationBatchCapacity
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
validatorDB := setupDB(t, pubKeys)
// For each public key, we attempt to save an attestation with signing root.
var wg sync.WaitGroup
for i, pubKey := range pubKeys {
wg.Add(1)
go func(j types.Epoch, pk [48]byte, w *sync.WaitGroup) {
go func(j types.Epoch, pk [fieldparams.BLSPubkeyLength]byte, w *sync.WaitGroup) {
defer w.Done()
var signingRoot [32]byte
copy(signingRoot[:], fmt.Sprintf("%d", j))
@@ -431,14 +432,14 @@ func TestSaveAttestationForPubKey_BatchWrites_LowCapacity_TimerReached(t *testin
// test force flushing to the DB based on a timer instead
// of the max capacity being reached.
numValidators := attestationBatchCapacity / 2
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
validatorDB := setupDB(t, pubKeys)
// For each public key, we attempt to save an attestation with signing root.
var wg sync.WaitGroup
for i, pubKey := range pubKeys {
wg.Add(1)
go func(j types.Epoch, pk [48]byte, w *sync.WaitGroup) {
go func(j types.Epoch, pk [fieldparams.BLSPubkeyLength]byte, w *sync.WaitGroup) {
defer w.Done()
var signingRoot [32]byte
copy(signingRoot[:], fmt.Sprintf("%d", j))
@@ -482,20 +483,20 @@ func TestSaveAttestationForPubKey_BatchWrites_LowCapacity_TimerReached(t *testin
func BenchmarkStore_CheckSlashableAttestation_Surround_SafeAttestation_54kEpochs(b *testing.B) {
numValidators := 1
numEpochs := types.Epoch(54000)
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
benchCheckSurroundVote(b, pubKeys, numEpochs, false /* surround */)
}
func BenchmarkStore_CheckSurroundVote_Surround_Slashable_54kEpochs(b *testing.B) {
numValidators := 1
numEpochs := types.Epoch(54000)
pubKeys := make([][48]byte, numValidators)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
benchCheckSurroundVote(b, pubKeys, numEpochs, true /* surround */)
}
func benchCheckSurroundVote(
b *testing.B,
pubKeys [][48]byte,
pubKeys [][fieldparams.BLSPubkeyLength]byte,
numEpochs types.Epoch,
shouldSurround bool,
) {

View File

@@ -7,6 +7,7 @@ import (
"path/filepath"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/require"
)
@@ -41,7 +42,7 @@ func TestStore_Backup(t *testing.T) {
}
func TestStore_NestedBackup(t *testing.T) {
keys := [][48]byte{{'A'}, {'B'}}
keys := [][fieldparams.BLSPubkeyLength]byte{{'A'}, {'B'}}
db := setupDB(t, keys)
ctx := context.Background()
root := [32]byte{1}

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/prysm/async/abool"
"github.com/prysmaticlabs/prysm/async/event"
"github.com/prysmaticlabs/prysm/config/features"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/io/file"
bolt "go.etcd.io/bbolt"
@@ -52,7 +53,7 @@ var blockedBuckets = [][]byte{
// Config represents store's config object.
type Config struct {
PubKeys [][48]byte
PubKeys [][fieldparams.BLSPubkeyLength]byte
InitialMMapSize int
}
@@ -177,7 +178,7 @@ func NewKVStore(ctx context.Context, dirPath string, config *Config) (*Store, er
}
// UpdatePublicKeysBuckets for a specified list of keys.
func (s *Store) UpdatePublicKeysBuckets(pubKeys [][48]byte) error {
func (s *Store) UpdatePublicKeysBuckets(pubKeys [][fieldparams.BLSPubkeyLength]byte) error {
return s.update(func(tx *bolt.Tx) error {
bucket := tx.Bucket(historicProposalsBucket)
for _, pubKey := range pubKeys {

View File

@@ -3,22 +3,23 @@ package kv
import (
"context"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
bolt "go.etcd.io/bbolt"
"go.opencensus.io/trace"
)
// EIPImportBlacklistedPublicKeys returns keys that were marked as blacklisted during EIP-3076 slashing
// protection imports, ensuring that we can prevent these keys from having duties at runtime.
func (s *Store) EIPImportBlacklistedPublicKeys(ctx context.Context) ([][48]byte, error) {
func (s *Store) EIPImportBlacklistedPublicKeys(ctx context.Context) ([][fieldparams.BLSPubkeyLength]byte, error) {
_, span := trace.StartSpan(ctx, "Validator.EIPImportBlacklistedPublicKeys")
defer span.End()
var err error
publicKeys := make([][48]byte, 0)
publicKeys := make([][fieldparams.BLSPubkeyLength]byte, 0)
err = s.db.View(func(tx *bolt.Tx) error {
bucket := tx.Bucket(slashablePublicKeysBucket)
return bucket.ForEach(func(key []byte, _ []byte) error {
if key != nil {
pubKeyBytes := [48]byte{}
pubKeyBytes := [fieldparams.BLSPubkeyLength]byte{}
copy(pubKeyBytes[:], key)
publicKeys = append(publicKeys, pubKeyBytes)
}
@@ -30,7 +31,7 @@ func (s *Store) EIPImportBlacklistedPublicKeys(ctx context.Context) ([][48]byte,
// SaveEIPImportBlacklistedPublicKeys stores a list of blacklisted public keys that
// were determined during EIP-3076 slashing protection imports.
func (s *Store) SaveEIPImportBlacklistedPublicKeys(ctx context.Context, publicKeys [][48]byte) error {
func (s *Store) SaveEIPImportBlacklistedPublicKeys(ctx context.Context, publicKeys [][fieldparams.BLSPubkeyLength]byte) error {
_, span := trace.StartSpan(ctx, "Validator.SaveEIPImportBlacklistedPublicKeys")
defer span.End()
return s.db.Update(func(tx *bolt.Tx) error {

View File

@@ -5,6 +5,7 @@ import (
"fmt"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/testing/assert"
"github.com/prysmaticlabs/prysm/testing/require"
)
@@ -12,9 +13,9 @@ import (
func TestStore_EIPBlacklistedPublicKeys(t *testing.T) {
ctx := context.Background()
numValidators := 100
publicKeys := make([][48]byte, numValidators)
publicKeys := make([][fieldparams.BLSPubkeyLength]byte, numValidators)
for i := 0; i < numValidators; i++ {
key := [48]byte{}
key := [fieldparams.BLSPubkeyLength]byte{}
copy(key[:], fmt.Sprintf("%d", i))
publicKeys[i] = key
}
@@ -32,7 +33,7 @@ func TestStore_EIPBlacklistedPublicKeys(t *testing.T) {
require.NoError(t, err)
// Keys are not guaranteed to be ordered, so we create a map for comparisons.
want := make(map[[48]byte]bool)
want := make(map[[fieldparams.BLSPubkeyLength]byte]bool)
for _, pubKey := range publicKeys[:50] {
want[pubKey] = true
}

View File

@@ -4,13 +4,14 @@ import (
"context"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/testing/require"
)
func TestStore_GenesisValidatorsRoot_ReadAndWrite(t *testing.T) {
ctx := context.Background()
db := setupDB(t, [][48]byte{})
db := setupDB(t, [][fieldparams.BLSPubkeyLength]byte{})
tests := []struct {
name string
want []byte

View File

@@ -4,13 +4,14 @@ import (
"context"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/crypto/hash"
"github.com/prysmaticlabs/prysm/testing/require"
)
func TestStore_GraffitiOrderedIndex_ReadAndWrite(t *testing.T) {
ctx := context.Background()
db := setupDB(t, [][48]byte{})
db := setupDB(t, [][fieldparams.BLSPubkeyLength]byte{})
tests := []struct {
name string
want uint64

View File

@@ -5,6 +5,7 @@ import (
"io/ioutil"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/testing/require"
"github.com/sirupsen/logrus"
)
@@ -17,7 +18,7 @@ func TestMain(m *testing.M) {
}
// setupDB instantiates and returns a DB instance for the validator client.
func setupDB(t testing.TB, pubkeys [][48]byte) *Store {
func setupDB(t testing.TB, pubkeys [][fieldparams.BLSPubkeyLength]byte) *Store {
db, err := NewKVStore(context.Background(), t.TempDir(), &Config{
PubKeys: pubkeys,
})

View File

@@ -5,6 +5,7 @@ import (
"context"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/monitoring/progress"
@@ -114,7 +115,7 @@ func (s *Store) migrateOptimalAttesterProtectionUp(ctx context.Context) error {
// Migrate attester protection from the more optimal format to the old format in the DB.
func (s *Store) migrateOptimalAttesterProtectionDown(ctx context.Context) error {
// First we extract the public keys we are migrating down for.
pubKeys := make([][48]byte, 0)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, 0)
err := s.view(func(tx *bolt.Tx) error {
mb := tx.Bucket(migrationsBucket)
if b := mb.Get(migrationOptimalAttesterProtectionKey); b == nil {

View File

@@ -6,6 +6,7 @@ import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/testing/require"
bolt "go.etcd.io/bbolt"
@@ -38,7 +39,7 @@ func Test_migrateOptimalAttesterProtectionUp(t *testing.T) {
name: "populates optimized schema buckets",
setup: func(t *testing.T, validatorDB *Store) {
ctx := context.Background()
pubKey := [48]byte{1}
pubKey := [fieldparams.BLSPubkeyLength]byte{1}
history := newDeprecatedAttestingHistory(0)
// Attest all epochs from genesis to 50.
numEpochs := types.Epoch(50)
@@ -65,7 +66,7 @@ func Test_migrateOptimalAttesterProtectionUp(t *testing.T) {
// Verify we indeed have the data for all epochs
// since genesis to epoch 50 under the new schema.
err := validatorDB.view(func(tx *bolt.Tx) error {
pubKey := [48]byte{1}
pubKey := [fieldparams.BLSPubkeyLength]byte{1}
bucket := tx.Bucket(pubKeysBucket)
pkBucket := bucket.Bucket(pubKey[:])
signingRootsBucket := pkBucket.Bucket(attestationSigningRootsBucket)
@@ -97,7 +98,7 @@ func Test_migrateOptimalAttesterProtectionUp(t *testing.T) {
name: "partial data saved for both types still completes the migration successfully",
setup: func(t *testing.T, validatorDB *Store) {
ctx := context.Background()
pubKey := [48]byte{1}
pubKey := [fieldparams.BLSPubkeyLength]byte{1}
history := newDeprecatedAttestingHistory(0)
// Attest all epochs from genesis to 50.
numEpochs := types.Epoch(50)
@@ -151,7 +152,7 @@ func Test_migrateOptimalAttesterProtectionUp(t *testing.T) {
// Verify we indeed have the data for all epochs
// since genesis to epoch 50+1 under the new schema.
err := validatorDB.view(func(tx *bolt.Tx) error {
pubKey := [48]byte{1}
pubKey := [fieldparams.BLSPubkeyLength]byte{1}
bucket := tx.Bucket(pubKeysBucket)
pkBucket := bucket.Bucket(pubKey[:])
signingRootsBucket := pkBucket.Bucket(attestationSigningRootsBucket)
@@ -229,7 +230,7 @@ func Test_migrateOptimalAttesterProtectionDown(t *testing.T) {
{
name: "populates old format from data using the new schema",
setup: func(t *testing.T, validatorDB *Store) {
pubKeys := [][48]byte{{1}, {2}}
pubKeys := [][fieldparams.BLSPubkeyLength]byte{{1}, {2}}
// Create attesting history for two public keys
err := validatorDB.update(func(tx *bolt.Tx) error {
bkt := tx.Bucket(pubKeysBucket)
@@ -270,7 +271,7 @@ func Test_migrateOptimalAttesterProtectionDown(t *testing.T) {
},
eval: func(t *testing.T, validatorDB *Store) {
ctx := context.Background()
pubKeys := [][48]byte{{1}, {2}}
pubKeys := [][fieldparams.BLSPubkeyLength]byte{{1}, {2}}
// Next up, we validate that we have indeed rolled back our data
// into the old format for attesting history.
err := validatorDB.view(func(tx *bolt.Tx) error {

View File

@@ -7,6 +7,7 @@ import (
"reflect"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/config/fieldparams"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/testing/require"
bolt "go.etcd.io/bbolt"
@@ -17,9 +18,9 @@ func TestStore_migrateSourceTargetEpochsBucketUp(t *testing.T) {
// numKeys should be more than batch size for testing.
// See: https://github.com/prysmaticlabs/prysm/issues/8509
numKeys := 2*publicKeyMigrationBatchSize + 1
pubKeys := make([][48]byte, numKeys)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numKeys)
for i := 0; i < numKeys; i++ {
var pk [48]byte
var pk [fieldparams.BLSPubkeyLength]byte
copy(pk[:], fmt.Sprintf("%d", i))
pubKeys[i] = pk
}
@@ -118,9 +119,9 @@ func TestStore_migrateSourceTargetEpochsBucketDown(t *testing.T) {
// numKeys should be more than batch size for testing.
// See: https://github.com/prysmaticlabs/prysm/issues/8509
numKeys := 2*publicKeyMigrationBatchSize + 1
pubKeys := make([][48]byte, numKeys)
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, numKeys)
for i := 0; i < numKeys; i++ {
var pk [48]byte
var pk [fieldparams.BLSPubkeyLength]byte
copy(pk[:], fmt.Sprintf("%d", i))
pubKeys[i] = pk
}

Some files were not shown because too many files have changed in this diff Show More