Add overflow check for StartSlot (#7149)

* Add overflow checks
* More fixes
* Typos
* Merge branch 'master' into fix-overflow
* One more regression test
* Merge branch 'fix-overflow' of github.com:prysmaticlabs/prysm into fix-overflow
* Gazelle
* Merge branch 'master' into fix-overflow
* Nishant's feedback
* Merge refs/heads/master into fix-overflow
This commit is contained in:
terence tsao
2020-09-01 19:52:36 -07:00
committed by GitHub
parent afce363e00
commit 7588e491ab
36 changed files with 259 additions and 94 deletions

View File

@@ -32,14 +32,18 @@ func (s *Service) getAttPreState(ctx context.Context, c *ethpb.Checkpoint) (*sta
baseState, err := s.stateGen.StateByRoot(ctx, bytesutil.ToBytes32(c.Root))
if err != nil {
return nil, errors.Wrapf(err, "could not get pre state for slot %d", helpers.StartSlot(c.Epoch))
return nil, errors.Wrapf(err, "could not get pre state for epoch %d", c.Epoch)
}
if helpers.StartSlot(c.Epoch) > baseState.Slot() {
epochStartSlot, err := helpers.StartSlot(c.Epoch)
if err != nil {
return nil, err
}
if epochStartSlot > baseState.Slot() {
baseState = baseState.Copy()
baseState, err = state.ProcessSlots(ctx, baseState, helpers.StartSlot(c.Epoch))
baseState, err = state.ProcessSlots(ctx, baseState, epochStartSlot)
if err != nil {
return nil, errors.Wrapf(err, "could not process slots up to %d", helpers.StartSlot(c.Epoch))
return nil, errors.Wrapf(err, "could not process slots up to epoch %d", c.Epoch)
}
if err := s.checkpointState.AddCheckpointState(c, baseState); err != nil {
return nil, errors.Wrap(err, "could not saved checkpoint state to cache")
@@ -76,13 +80,17 @@ func (s *Service) getAttCheckPtInfo(ctx context.Context, c *ethpb.Checkpoint, e
// Retrieve checkpoint state to compute checkpoint info.
baseState, err := s.stateGen.StateByRoot(ctx, bytesutil.ToBytes32(c.Root))
if err != nil {
return nil, errors.Wrapf(err, "could not get pre state for slot %d", helpers.StartSlot(c.Epoch))
return nil, errors.Wrapf(err, "could not get pre state for epoch %d", c.Epoch)
}
if helpers.StartSlot(c.Epoch) > baseState.Slot() {
epochStartSlot, err := helpers.StartSlot(c.Epoch)
if err != nil {
return nil, err
}
if epochStartSlot > baseState.Slot() {
baseState = baseState.Copy()
baseState, err = state.ProcessSlots(ctx, baseState, helpers.StartSlot(c.Epoch))
baseState, err = state.ProcessSlots(ctx, baseState, epochStartSlot)
if err != nil {
return nil, errors.Wrapf(err, "could not process slots up to %d", helpers.StartSlot(c.Epoch))
return nil, errors.Wrapf(err, "could not process slots up to epoch %d", c.Epoch)
}
}
f := baseState.Fork()
@@ -155,7 +163,10 @@ func (s *Service) verifyBeaconBlock(ctx context.Context, data *ethpb.Attestation
// verifyLMDFFGConsistent verifies LMD GHOST and FFG votes are consistent with each other.
func (s *Service) verifyLMDFFGConsistent(ctx context.Context, ffgEpoch uint64, ffgRoot []byte, lmdRoot []byte) error {
ffgSlot := helpers.StartSlot(ffgEpoch)
ffgSlot, err := helpers.StartSlot(ffgEpoch)
if err != nil {
return err
}
r, err := s.ancestor(ctx, lmdRoot, ffgSlot)
if err != nil {
return err

View File

@@ -90,7 +90,7 @@ func TestStore_OnAttestation(t *testing.T) {
name: "no pre state for attestations's target block",
a: &ethpb.Attestation{Data: &ethpb.AttestationData{Target: &ethpb.Checkpoint{Root: BlkWithOutStateRoot[:]}}},
wantErr: true,
wantErrString: "could not get pre state for slot 0",
wantErrString: "could not get pre state for epoch 0",
},
{
name: "process attestation doesn't match current epoch",
@@ -212,7 +212,7 @@ func TestStore_OnAttestationUsingCheckptCache(t *testing.T) {
name: "no pre state for attestations's target block",
a: &ethpb.Attestation{Data: &ethpb.AttestationData{Target: &ethpb.Checkpoint{Root: BlkWithOutStateRoot[:]}}},
wantErr: true,
wantErrString: "could not get pre state for slot 0",
wantErrString: "could not get pre state for epoch 0",
},
{
name: "process attestation doesn't match current epoch",
@@ -363,7 +363,9 @@ func TestStore_UpdateCheckpointState(t *testing.T) {
require.NoError(t, service.beaconDB.SaveState(ctx, baseState, bytesutil.ToBytes32(newCheckpoint.Root)))
returned, err = service.getAttPreState(ctx, newCheckpoint)
require.NoError(t, err)
baseState, err = state.ProcessSlots(ctx, baseState, helpers.StartSlot(newCheckpoint.Epoch))
s, err := helpers.StartSlot(newCheckpoint.Epoch)
require.NoError(t, err)
baseState, err = state.ProcessSlots(ctx, baseState, s)
require.NoError(t, err)
assert.Equal(t, returned.Slot(), baseState.Slot(), "Incorrectly returned base state")

View File

@@ -316,8 +316,11 @@ func (s *Service) handleBlockAfterBatchVerify(ctx context.Context, signed *ethpb
func (s *Service) handleEpochBoundary(postState *stateTrie.BeaconState) error {
if postState.Slot() >= s.nextEpochBoundarySlot {
reportEpochMetrics(postState)
s.nextEpochBoundarySlot = helpers.StartSlot(helpers.NextEpoch(postState))
var err error
s.nextEpochBoundarySlot, err = helpers.StartSlot(helpers.NextEpoch(postState))
if err != nil {
return err
}
// Update committees cache at epoch boundary slot.
if err := helpers.UpdateCommitteeCache(postState, helpers.CurrentEpoch(postState)); err != nil {
return err

View File

@@ -119,7 +119,10 @@ func (s *Service) VerifyBlkDescendant(ctx context.Context, root [32]byte) error
// verifyBlkFinalizedSlot validates input block is not less than or equal
// to current finalized slot.
func (s *Service) verifyBlkFinalizedSlot(b *ethpb.BeaconBlock) error {
finalizedSlot := helpers.StartSlot(s.finalizedCheckpt.Epoch)
finalizedSlot, err := helpers.StartSlot(s.finalizedCheckpt.Epoch)
if err != nil {
return err
}
if finalizedSlot >= b.Slot {
return fmt.Errorf("block is equal or earlier than finalized block, slot %d < slot %d", b.Slot, finalizedSlot)
}
@@ -150,7 +153,11 @@ func (s *Service) shouldUpdateCurrentJustified(ctx context.Context, newJustified
}
newJustifiedBlock := newJustifiedBlockSigned.Block
if newJustifiedBlock.Slot <= helpers.StartSlot(s.justifiedCheckpt.Epoch) {
jSlot, err := helpers.StartSlot(s.justifiedCheckpt.Epoch)
if err != nil {
return false, err
}
if newJustifiedBlock.Slot <= jSlot {
return false, nil
}
var justifiedBlockSigned *ethpb.SignedBeaconBlock
@@ -310,7 +317,10 @@ func (s *Service) finalizedImpliesNewJustified(ctx context.Context, state *state
}
// Update justified if store justified is not in chain with finalized check point.
finalizedSlot := helpers.StartSlot(s.finalizedCheckpt.Epoch)
finalizedSlot, err := helpers.StartSlot(s.finalizedCheckpt.Epoch)
if err != nil {
return err
}
justifiedRoot := s.ensureRootNotZeros(bytesutil.ToBytes32(s.justifiedCheckpt.Root))
anc, err := s.ancestor(ctx, justifiedRoot[:], finalizedSlot)
if err != nil {
@@ -335,7 +345,11 @@ func (s *Service) fillInForkChoiceMissingBlocks(ctx context.Context, blk *ethpb.
parentRoot := bytesutil.ToBytes32(blk.ParentRoot)
slot := blk.Slot
// Fork choice only matters from last finalized slot.
higherThanFinalized := slot > helpers.StartSlot(s.finalizedCheckpt.Epoch)
fSlot, err := helpers.StartSlot(s.finalizedCheckpt.Epoch)
if err != nil {
return err
}
higherThanFinalized := slot > fSlot
// As long as parent node is not in fork choice store, and parent node is in DB.
for !s.forkChoiceStore.HasNode(parentRoot) && s.beaconDB.HasBlock(ctx, parentRoot) && higherThanFinalized {
b, err := s.beaconDB.Block(ctx, parentRoot)
@@ -346,7 +360,7 @@ func (s *Service) fillInForkChoiceMissingBlocks(ctx context.Context, blk *ethpb.
pendingNodes = append(pendingNodes, b.Block)
parentRoot = bytesutil.ToBytes32(b.Block.ParentRoot)
slot = b.Block.Slot
higherThanFinalized = slot > helpers.StartSlot(s.finalizedCheckpt.Epoch)
higherThanFinalized = slot > fSlot
}
// Insert parent nodes to fork choice store in reverse order.

View File

@@ -49,7 +49,11 @@ import (
// if all(bits[0:2]) and old_current_justified_checkpoint.epoch + 1 == current_epoch:
// state.finalized_checkpoint = old_current_justified_checkpoint
func ProcessJustificationAndFinalizationPreCompute(state *stateTrie.BeaconState, pBal *Balance) (*stateTrie.BeaconState, error) {
if state.Slot() <= helpers.StartSlot(2) {
canProcessSlot, err := helpers.StartSlot(2 /*epoch*/)
if err != nil {
return nil, err
}
if state.Slot() <= canProcessSlot {
return state, nil
}

View File

@@ -32,5 +32,9 @@ func BlockRootAtSlot(state *stateTrie.BeaconState, slot uint64) ([]byte, error)
// """
// return get_block_root_at_slot(state, compute_start_slot_at_epoch(epoch))
func BlockRoot(state *stateTrie.BeaconState, epoch uint64) ([]byte, error) {
return BlockRootAtSlot(state, StartSlot(epoch))
s, err := StartSlot(epoch)
if err != nil {
return nil, err
}
return BlockRootAtSlot(state, s)
}

View File

@@ -200,7 +200,10 @@ func CommitteeAssignments(
// We determine the slots in which proposers are supposed to act.
// Some validators may need to propose multiple times per epoch, so
// we use a map of proposer idx -> []slot to keep track of this possibility.
startSlot := StartSlot(epoch)
startSlot, err := StartSlot(epoch)
if err != nil {
return nil, nil, err
}
proposerIndexToSlots := make(map[uint64][]uint64, params.BeaconConfig().SlotsPerEpoch)
for slot := startSlot; slot < startSlot+params.BeaconConfig().SlotsPerEpoch; slot++ {
// Skip proposer assignment for genesis slot.
@@ -380,7 +383,10 @@ func precomputeProposerIndices(state *stateTrie.BeaconState, activeIndices []uin
if err != nil {
return nil, errors.Wrap(err, "could not generate seed")
}
slot := StartSlot(e)
slot, err := StartSlot(e)
if err != nil {
return nil, err
}
for i := uint64(0); i < params.BeaconConfig().SlotsPerEpoch; i++ {
seedWithSlot := append(seed[:], bytesutil.Bytes8(slot+i)...)
seedWithSlotHash := hashFunc(seedWithSlot)

View File

@@ -230,8 +230,10 @@ func TestCommitteeAssignments_EverySlotHasMin1Proposer(t *testing.T) {
}
}
assert.Equal(t, params.BeaconConfig().SlotsPerEpoch, uint64(len(slotsWithProposers)), "Unexpected slots")
startSlot := StartSlot(epoch)
endSlot := StartSlot(epoch + 1)
startSlot, err := StartSlot(epoch)
require.NoError(t, err)
endSlot, err := StartSlot(epoch + 1)
require.NoError(t, err)
for i := startSlot; i < endSlot; i++ {
hasProposer := slotsWithProposers[i]
assert.Equal(t, true, hasProposer, "Expected every slot in epoch 1 to have a proposer, slot %d did not", i)
@@ -390,7 +392,7 @@ func TestUpdateCommitteeCache_CanUpdate(t *testing.T) {
seed, err := Seed(state, epoch, params.BeaconConfig().DomainBeaconAttester)
require.NoError(t, err)
indices, err = committeeCache.Committee(StartSlot(epoch), seed, idx)
indices, err = committeeCache.Committee(epoch*params.BeaconConfig().SlotsPerEpoch, seed, idx)
require.NoError(t, err)
assert.Equal(t, params.BeaconConfig().TargetCommitteeSize, uint64(len(indices)), "Did not save correct indices lengths")
}

View File

@@ -1,8 +1,10 @@
package helpers
import (
"errors"
"fmt"
"math"
"math/bits"
"time"
stateTrie "github.com/prysmaticlabs/prysm/beacon-chain/state"
@@ -72,9 +74,13 @@ func NextEpoch(state *stateTrie.BeaconState) uint64 {
// """
// Return the start slot of ``epoch``.
// """
// return Slot(epoch * SLOTS_PER_EPOCH
func StartSlot(epoch uint64) uint64 {
return epoch * params.BeaconConfig().SlotsPerEpoch
// return Slot(epoch * SLOTS_PER_EPOCH)
func StartSlot(epoch uint64) (uint64, error) {
overflows, slot := bits.Mul64(epoch, params.BeaconConfig().SlotsPerEpoch)
if overflows > 0 {
return slot, errors.New("start slot calculation overflows")
}
return slot, nil
}
// IsEpochStart returns true if the given slot number is an epoch starting slot
@@ -91,7 +97,7 @@ func IsEpochEnd(slot uint64) bool {
// SlotsSinceEpochStarts returns number of slots since the start of the epoch.
func SlotsSinceEpochStarts(slot uint64) uint64 {
return slot - StartSlot(SlotToEpoch(slot))
return slot % params.BeaconConfig().SlotsPerEpoch
}
// VerifySlotTime validates the input slot is not from the future.

View File

@@ -86,14 +86,24 @@ func TestEpochStartSlot_OK(t *testing.T) {
tests := []struct {
epoch uint64
startSlot uint64
error bool
}{
{epoch: 0, startSlot: 0 * params.BeaconConfig().SlotsPerEpoch},
{epoch: 1, startSlot: 1 * params.BeaconConfig().SlotsPerEpoch},
{epoch: 10, startSlot: 10 * params.BeaconConfig().SlotsPerEpoch},
{epoch: 0, startSlot: 0 * params.BeaconConfig().SlotsPerEpoch, error: false},
{epoch: 1, startSlot: 1 * params.BeaconConfig().SlotsPerEpoch, error: false},
{epoch: 10, startSlot: 10 * params.BeaconConfig().SlotsPerEpoch, error: false},
{epoch: 1 << 58, startSlot: 1 << 63, error: false},
{epoch: 1 << 59, startSlot: 1 << 63, error: true},
{epoch: 1 << 60, startSlot: 1 << 63, error: true},
}
for _, tt := range tests {
state := &pb.BeaconState{Slot: tt.epoch}
assert.Equal(t, tt.startSlot, StartSlot(tt.epoch), "StartSlot(%d)", state.Slot)
ss, err := StartSlot(tt.epoch)
if !tt.error {
require.NoError(t, err)
assert.Equal(t, tt.startSlot, ss, "StartSlot(%d)", state.Slot)
} else {
require.ErrorContains(t, "start slot calculation overflow", err)
}
}
}

View File

@@ -333,7 +333,7 @@ func getBlockRootsByFilter(ctx context.Context, tx *bolt.Tx, f *filters.QueryFil
// We retrieve block roots that match a filter criteria of slot ranges, if specified.
filtersMap := f.Filters()
rootsBySlotRange := fetchBlockRootsBySlotRange(
rootsBySlotRange, err := fetchBlockRootsBySlotRange(
ctx,
tx.Bucket(blockSlotIndicesBucket),
filtersMap[filters.StartSlot],
@@ -342,6 +342,9 @@ func getBlockRootsByFilter(ctx context.Context, tx *bolt.Tx, f *filters.QueryFil
filtersMap[filters.EndEpoch],
filtersMap[filters.SlotStep],
)
if err != nil {
return nil, err
}
// Once we have a list of block roots that correspond to each
// lookup index, we find the intersection across all of them and use
@@ -378,7 +381,7 @@ func fetchBlockRootsBySlotRange(
startEpochEncoded interface{},
endEpochEncoded interface{},
slotStepEncoded interface{},
) [][]byte {
) ([][]byte, error) {
ctx, span := trace.StartSpan(ctx, "BeaconDB.fetchBlockRootsBySlotRange")
defer span.End()
@@ -395,9 +398,17 @@ func fetchBlockRootsBySlotRange(
}
startEpoch, startEpochOk := startEpochEncoded.(uint64)
endEpoch, endEpochOk := endEpochEncoded.(uint64)
var err error
if startEpochOk && endEpochOk {
startSlot = helpers.StartSlot(startEpoch)
endSlot = helpers.StartSlot(endEpoch) + params.BeaconConfig().SlotsPerEpoch - 1
startSlot, err = helpers.StartSlot(startEpoch)
if err != nil {
return nil, err
}
endSlot, err = helpers.StartSlot(endEpoch)
if err != nil {
return nil, err
}
endSlot = endSlot + params.BeaconConfig().SlotsPerEpoch - 1
}
min := bytesutil.Uint64ToBytesBigEndian(startSlot)
max := bytesutil.Uint64ToBytesBigEndian(endSlot)
@@ -431,7 +442,7 @@ func fetchBlockRootsBySlotRange(
}
roots = append(roots, splitRoots...)
}
return roots
return roots, nil
}
// createBlockIndicesFromBlock takes in a beacon block and returns

View File

@@ -37,7 +37,6 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/state:go_default_library",
"//shared/bls:go_default_library",
"//shared/params:go_default_library",

View File

@@ -9,7 +9,6 @@ import (
"github.com/prysmaticlabs/prysm/shared/testutil/assert"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/testutil"
"github.com/prysmaticlabs/prysm/shared/testutil/require"
@@ -88,7 +87,7 @@ func TestPool_InsertAttesterSlashing(t *testing.T) {
}
slashings[i] = sl
}
require.NoError(t, beaconState.SetSlot(helpers.StartSlot(1)))
require.NoError(t, beaconState.SetSlot(params.BeaconConfig().SlotsPerEpoch))
// We mark the following validators with some preconditions.
exitedVal, err := beaconState.ValidatorAtIndex(uint64(2))

View File

@@ -5,7 +5,6 @@ import (
"testing"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/testutil"
"github.com/prysmaticlabs/prysm/shared/testutil/assert"
@@ -41,7 +40,7 @@ func TestPool_InsertProposerSlashing(t *testing.T) {
slashings[i] = sl
}
require.NoError(t, beaconState.SetSlot(helpers.StartSlot(1)))
require.NoError(t, beaconState.SetSlot(params.BeaconConfig().SlotsPerEpoch))
// We mark the following validators with some preconditions.
exitedVal, err := beaconState.ValidatorAtIndex(uint64(2))

View File

@@ -17,6 +17,7 @@ go_library(
"//beacon-chain/flags:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/featureconfig:go_default_library",
"//shared/params:go_default_library",
"//shared/rand:go_default_library",
"//shared/roughtime:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enr:go_default_library",

View File

@@ -34,6 +34,7 @@ import (
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/roughtime"
)
@@ -489,7 +490,7 @@ func (p *Status) BestNonFinalized(minPeers int, ourFinalizedEpoch uint64) (uint6
pidHead := make(map[peer.ID]uint64, len(connected))
potentialPIDs := make([]peer.ID, 0, len(connected))
ourFinalizedSlot := helpers.StartSlot(ourFinalizedEpoch)
ourFinalizedSlot := ourFinalizedEpoch * params.BeaconConfig().SlotsPerEpoch
for _, pid := range connected {
peerChainState, err := p.ChainState(pid)
if err == nil && peerChainState != nil && peerChainState.HeadSlot > ourFinalizedSlot {

View File

@@ -50,7 +50,11 @@ func (bs *Server) ListValidatorAssignments(
)
}
requestedState, err := bs.StateGen.StateBySlot(ctx, helpers.StartSlot(requestedEpoch))
startSlot, err := helpers.StartSlot(requestedEpoch)
if err != nil {
return nil, err
}
requestedState, err := bs.StateGen.StateBySlot(ctx, startSlot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not retrieve archived state for epoch %d: %v", requestedEpoch, err)
}

View File

@@ -295,7 +295,11 @@ func (bs *Server) StreamIndexedAttestations(
}
// We use the retrieved committees for the epoch to convert all attestations
// into indexed form effectively.
startSlot := helpers.StartSlot(targetEpoch)
startSlot, err := helpers.StartSlot(targetEpoch)
if err != nil {
log.Error(err)
continue
}
endSlot := startSlot + params.BeaconConfig().SlotsPerEpoch
for _, att := range aggAtts {
// Out of range check, the attestation slot cannot be greater

View File

@@ -658,7 +658,8 @@ func TestServer_ListIndexedAttestations_OldEpoch(t *testing.T) {
count := params.BeaconConfig().SlotsPerEpoch
atts := make([]*ethpb.Attestation, 0, count)
epoch := uint64(50)
startSlot := helpers.StartSlot(epoch)
startSlot, err := helpers.StartSlot(epoch)
require.NoError(t, err)
for i := startSlot; i < count; i++ {
blockExample := &ethpb.SignedBeaconBlock{
@@ -714,9 +715,9 @@ func TestServer_ListIndexedAttestations_OldEpoch(t *testing.T) {
},
StateGen: stategen.New(db, sc),
}
err := db.SaveStateSummary(ctx, &pbp2p.StateSummary{
err = db.SaveStateSummary(ctx, &pbp2p.StateSummary{
Root: blockRoot[:],
Slot: helpers.StartSlot(epoch),
Slot: epoch * params.BeaconConfig().SlotsPerEpoch,
})
require.NoError(t, err)
require.NoError(t, db.SaveState(ctx, state, bytesutil.ToBytes32([]byte("root"))))
@@ -916,7 +917,7 @@ func TestServer_StreamIndexedAttestations_OK(t *testing.T) {
epoch := uint64(0)
attesterSeed, err := helpers.Seed(headState, epoch, params.BeaconConfig().DomainBeaconAttester)
require.NoError(t, err)
committees, err := computeCommittees(helpers.StartSlot(epoch), activeIndices, attesterSeed)
committees, err := computeCommittees(epoch*params.BeaconConfig().SlotsPerEpoch, activeIndices, attesterSeed)
require.NoError(t, err)
count := params.BeaconConfig().SlotsPerEpoch

View File

@@ -298,17 +298,29 @@ func (bs *Server) chainHeadRetrieval(ctx context.Context) (*ethpb.ChainHead, err
}
}
fSlot, err := helpers.StartSlot(finalizedCheckpoint.Epoch)
if err != nil {
return nil, err
}
jSlot, err := helpers.StartSlot(justifiedCheckpoint.Epoch)
if err != nil {
return nil, err
}
pjSlot, err := helpers.StartSlot(prevJustifiedCheckpoint.Epoch)
if err != nil {
return nil, err
}
return &ethpb.ChainHead{
HeadSlot: headBlock.Block.Slot,
HeadEpoch: helpers.SlotToEpoch(headBlock.Block.Slot),
HeadBlockRoot: headBlockRoot[:],
FinalizedSlot: helpers.StartSlot(finalizedCheckpoint.Epoch),
FinalizedSlot: fSlot,
FinalizedEpoch: finalizedCheckpoint.Epoch,
FinalizedBlockRoot: finalizedCheckpoint.Root,
JustifiedSlot: helpers.StartSlot(justifiedCheckpoint.Epoch),
JustifiedSlot: jSlot,
JustifiedEpoch: justifiedCheckpoint.Epoch,
JustifiedBlockRoot: justifiedCheckpoint.Root,
PreviousJustifiedSlot: helpers.StartSlot(prevJustifiedCheckpoint.Epoch),
PreviousJustifiedSlot: pjSlot,
PreviousJustifiedEpoch: prevJustifiedCheckpoint.Epoch,
PreviousJustifiedBlockRoot: prevJustifiedCheckpoint.Root,
}, nil

View File

@@ -404,7 +404,9 @@ func TestServer_GetChainHead(t *testing.T) {
require.NoError(t, err)
b := testutil.NewBeaconBlock()
b.Block.Slot = helpers.StartSlot(s.PreviousJustifiedCheckpoint().Epoch) + 1
b.Block.Slot, err = helpers.StartSlot(s.PreviousJustifiedCheckpoint().Epoch)
require.NoError(t, err)
b.Block.Slot++
bs := &Server{
BeaconDB: db,
HeadFetcher: &chainMock.ChainService{Block: b, State: s},
@@ -492,7 +494,8 @@ func TestServer_StreamChainHead_OnHeadUpdated(t *testing.T) {
require.NoError(t, err)
b := testutil.NewBeaconBlock()
b.Block.Slot = helpers.StartSlot(s.PreviousJustifiedCheckpoint().Epoch) + 1
b.Block.Slot, err = helpers.StartSlot(s.PreviousJustifiedCheckpoint().Epoch)
hRoot, err := b.Block.HashTreeRoot()
require.NoError(t, err)

View File

@@ -20,12 +20,15 @@ func (bs *Server) ListBeaconCommittees(
ctx context.Context,
req *ethpb.ListCommitteesRequest,
) (*ethpb.BeaconCommittees, error) {
currentSlot := bs.GenesisTimeFetcher.CurrentSlot()
var requestedSlot uint64
switch q := req.QueryFilter.(type) {
case *ethpb.ListCommitteesRequest_Epoch:
requestedSlot = helpers.StartSlot(q.Epoch)
startSlot, err := helpers.StartSlot(q.Epoch)
if err != nil {
return nil, err
}
requestedSlot = startSlot
case *ethpb.ListCommitteesRequest_Genesis:
requestedSlot = 0
default:
@@ -64,7 +67,10 @@ func (bs *Server) retrieveCommitteesForEpoch(
ctx context.Context,
epoch uint64,
) (map[uint64]*ethpb.BeaconCommittees_CommitteesList, []uint64, error) {
startSlot := helpers.StartSlot(epoch)
startSlot, err := helpers.StartSlot(epoch)
if err != nil {
return nil, nil, err
}
requestedState, err := bs.StateGen.StateBySlot(ctx, startSlot)
if err != nil {
return nil, nil, status.Error(codes.Internal, "Could not get state")
@@ -111,7 +117,10 @@ func (bs *Server) retrieveCommitteesForRoot(
return nil, nil, status.Error(codes.Internal, "Could not get active indices")
}
startSlot := helpers.StartSlot(epoch)
startSlot, err := helpers.StartSlot(epoch)
if err != nil {
return nil, nil, err
}
committeesListsBySlot, err := computeCommittees(startSlot, activeIndices, seed)
if err != nil {
return nil, nil, status.Errorf(

View File

@@ -106,7 +106,8 @@ func TestServer_ListBeaconCommittees_PreviousEpoch(t *testing.T) {
require.NoError(t, err)
attesterSeed, err := helpers.Seed(headState, 1, params.BeaconConfig().DomainBeaconAttester)
require.NoError(t, err)
startSlot := helpers.StartSlot(1)
startSlot, err := helpers.StartSlot(1)
require.NoError(t, err)
wanted, err := computeCommittees(startSlot, activeIndices, attesterSeed)
require.NoError(t, err)

View File

@@ -56,7 +56,11 @@ func (bs *Server) ListValidatorBalances(
res := make([]*ethpb.ValidatorBalances_Balance, 0)
filtered := map[uint64]bool{} // Track filtered validators to prevent duplication in the response.
requestedState, err := bs.StateGen.StateBySlot(ctx, helpers.StartSlot(requestedEpoch))
startSlot, err := helpers.StartSlot(requestedEpoch)
if err != nil {
return nil, err
}
requestedState, err := bs.StateGen.StateBySlot(ctx, startSlot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get state")
}
@@ -189,7 +193,11 @@ func (bs *Server) ListValidators(
var reqState *statetrie.BeaconState
var err error
if requestedEpoch != currentEpoch {
reqState, err = bs.StateGen.StateBySlot(ctx, helpers.StartSlot(requestedEpoch))
s, err := helpers.StartSlot(requestedEpoch)
if err != nil {
return nil, err
}
reqState, err = bs.StateGen.StateBySlot(ctx, s)
} else {
reqState, err = bs.HeadFetcher.HeadState(ctx)
}
@@ -198,14 +206,18 @@ func (bs *Server) ListValidators(
return nil, status.Error(codes.Internal, "Could not get requested state")
}
if helpers.StartSlot(requestedEpoch) > reqState.Slot() {
s, err := helpers.StartSlot(requestedEpoch)
if err != nil {
return nil, err
}
if s > reqState.Slot() {
reqState = reqState.Copy()
reqState, err = state.ProcessSlots(ctx, reqState, helpers.StartSlot(requestedEpoch))
reqState, err = state.ProcessSlots(ctx, reqState, s)
if err != nil {
return nil, status.Errorf(
codes.Internal,
"Could not process slots up to %d: %v",
helpers.StartSlot(requestedEpoch),
"Could not process slots up to epoch %d: %v",
requestedEpoch,
err,
)
}
@@ -389,7 +401,11 @@ func (bs *Server) GetValidatorActiveSetChanges(
slashedIndices := make([]uint64, 0)
ejectedIndices := make([]uint64, 0)
requestedState, err := bs.StateGen.StateBySlot(ctx, helpers.StartSlot(requestedEpoch))
s, err := helpers.StartSlot(requestedEpoch)
if err != nil {
return nil, err
}
requestedState, err := bs.StateGen.StateBySlot(ctx, s)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get state: %v", err)
}
@@ -475,7 +491,11 @@ func (bs *Server) GetValidatorParticipation(
}
// Calculate the end slot of the next epoch.
// Ex: requested epoch 1, this gets slot 95.
nextEpochEndSlot := helpers.StartSlot(requestedEpoch+2) - 1
nextEpochEndSlot, err := helpers.StartSlot(requestedEpoch + 2)
if err != nil {
return nil, err
}
nextEpochEndSlot--
requestedState, err := bs.StateGen.StateBySlot(ctx, nextEpochEndSlot)
if err != nil {
return nil, status.Error(codes.Internal, "Could not get state")
@@ -748,7 +768,11 @@ func (bs *Server) GetIndividualVotes(
)
}
requestedState, err := bs.StateGen.StateBySlot(ctx, helpers.StartSlot(req.Epoch))
s, err := helpers.StartSlot(req.Epoch)
if err != nil {
return nil, err
}
requestedState, err := bs.StateGen.StateBySlot(ctx, s)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not retrieve archived state for epoch %d: %v", req.Epoch, err)
}

View File

@@ -883,7 +883,7 @@ func TestServer_ListValidators_FromOldEpoch(t *testing.T) {
}
st := testutil.NewBeaconState()
require.NoError(t, st.SetSlot(helpers.StartSlot(30)))
require.NoError(t, st.SetSlot(30*params.BeaconConfig().SlotsPerEpoch))
require.NoError(t, st.SetValidators(validators))
b := testutil.NewBeaconBlock()
require.NoError(t, db.SaveBlock(ctx, b))
@@ -1249,7 +1249,7 @@ func TestServer_GetValidatorQueue_ExitedValidatorLeavesQueue(t *testing.T) {
// Now, we move the state.slot past the exit epoch of the validator, and now
// the validator should no longer exist in the queue.
require.NoError(t, headState.SetSlot(helpers.StartSlot(validators[1].ExitEpoch+1)))
require.NoError(t, headState.SetSlot((validators[1].ExitEpoch+1)*params.BeaconConfig().SlotsPerEpoch))
res, err = bs.GetValidatorQueue(context.Background(), &ptypes.Empty{})
require.NoError(t, err)
assert.Equal(t, 0, len(res.ExitPublicKeys))
@@ -1450,7 +1450,7 @@ func TestGetValidatorPerformance_OK(t *testing.T) {
ctx := context.Background()
epoch := uint64(1)
headState := testutil.NewBeaconState()
require.NoError(t, headState.SetSlot(helpers.StartSlot(epoch+1)))
require.NoError(t, headState.SetSlot((epoch+1)*params.BeaconConfig().SlotsPerEpoch))
atts := make([]*pb.PendingAttestation, 3)
for i := 0; i < len(atts); i++ {
atts[i] = &pb.PendingAttestation{
@@ -1527,7 +1527,7 @@ func TestGetValidatorPerformance_Indices(t *testing.T) {
defaultBal := params.BeaconConfig().MaxEffectiveBalance
extraBal := params.BeaconConfig().MaxEffectiveBalance + params.BeaconConfig().GweiPerEth
headState := testutil.NewBeaconState()
require.NoError(t, headState.SetSlot(helpers.StartSlot(epoch+1)))
require.NoError(t, headState.SetSlot((epoch+1)*params.BeaconConfig().SlotsPerEpoch))
balances := []uint64{defaultBal, extraBal, extraBal + params.BeaconConfig().GweiPerEth}
require.NoError(t, headState.SetBalances(balances))
publicKey1 := bytesutil.ToBytes48([]byte{1})
@@ -1597,7 +1597,7 @@ func TestGetValidatorPerformance_IndicesPubkeys(t *testing.T) {
defaultBal := params.BeaconConfig().MaxEffectiveBalance
extraBal := params.BeaconConfig().MaxEffectiveBalance + params.BeaconConfig().GweiPerEth
headState := testutil.NewBeaconState()
require.NoError(t, headState.SetSlot(helpers.StartSlot(epoch+1)))
require.NoError(t, headState.SetSlot((epoch+1)*params.BeaconConfig().SlotsPerEpoch))
balances := []uint64{defaultBal, extraBal, extraBal + params.BeaconConfig().GweiPerEth}
require.NoError(t, headState.SetBalances(balances))
publicKey1 := bytesutil.ToBytes48([]byte{1})

View File

@@ -113,7 +113,11 @@ func (vs *Server) duties(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.
}
// Advance state with empty transitions up to the requested epoch start slot.
if epochStartSlot := helpers.StartSlot(req.Epoch); s.Slot() < epochStartSlot {
epochStartSlot, err := helpers.StartSlot(req.Epoch)
if err != nil {
return nil, err
}
if s.Slot() < epochStartSlot {
s, err = state.ProcessSlots(ctx, s, epochStartSlot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not process slots up to %d: %v", epochStartSlot, err)

View File

@@ -14,7 +14,6 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/cache/depositcache"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
dbutil "github.com/prysmaticlabs/prysm/beacon-chain/db/testing"
mockPOW "github.com/prysmaticlabs/prysm/beacon-chain/powchain/testing"
@@ -380,7 +379,7 @@ func TestStreamDuties_OK_ChainReorg(t *testing.T) {
for sent := 0; sent == 0; {
sent = vs.StateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.Reorg,
Data: &statefeed.ReorgData{OldSlot: helpers.StartSlot(1), NewSlot: 0},
Data: &statefeed.ReorgData{OldSlot: params.BeaconConfig().SlotsPerEpoch, NewSlot: 0},
})
}
<-exitRoutine

View File

@@ -97,14 +97,17 @@ func (vs *Server) GetAttestationData(ctx context.Context, req *ethpb.Attestation
}
if helpers.CurrentEpoch(headState) < helpers.SlotToEpoch(req.Slot) {
headState, err = state.ProcessSlots(ctx, headState, helpers.StartSlot(helpers.SlotToEpoch(req.Slot)))
headState, err = state.ProcessSlots(ctx, headState, req.Slot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not process slots up to %d: %v", req.Slot, err)
}
}
targetEpoch := helpers.CurrentEpoch(headState)
epochStartSlot := helpers.StartSlot(targetEpoch)
epochStartSlot, err := helpers.StartSlot(targetEpoch)
if err != nil {
return nil, err
}
targetRoot := make([]byte, 32)
if epochStartSlot == headState.Slot() {
targetRoot = headRoot[:]

View File

@@ -208,9 +208,13 @@ func TestAttestationDataAtSlot_HandlesFarAwayJustifiedEpoch(t *testing.T) {
block := testutil.NewBeaconBlock()
block.Block.Slot = 10000
epochBoundaryBlock := testutil.NewBeaconBlock()
epochBoundaryBlock.Block.Slot = helpers.StartSlot(helpers.SlotToEpoch(10000))
var err error
epochBoundaryBlock.Block.Slot, err = helpers.StartSlot(helpers.SlotToEpoch(10000))
require.NoError(t, err)
justifiedBlock := testutil.NewBeaconBlock()
justifiedBlock.Block.Slot = helpers.StartSlot(helpers.SlotToEpoch(1500)) - 2 // Imagine two skip block
justifiedBlock.Block.Slot, err = helpers.StartSlot(helpers.SlotToEpoch(1500))
require.NoError(t, err)
justifiedBlock.Block.Slot = justifiedBlock.Block.Slot - 2 // Imagine two skip block
blockRoot, err := block.Block.HashTreeRoot()
require.NoError(t, err, "Could not hash beacon block")
justifiedBlockRoot, err := justifiedBlock.Block.HashTreeRoot()

View File

@@ -33,6 +33,7 @@ func (vs *Server) ProposeExit(ctx context.Context, req *ethpb.SignedVoluntaryExi
if err != nil {
return nil, status.Error(codes.InvalidArgument, "validator index exceeds validator set length")
}
if err := blocks.VerifyExitAndSignature(val, s.Slot(), s.Fork(), req, s.GenesisValidatorRoot()); err != nil {
return nil, status.Error(codes.InvalidArgument, err.Error())
}

View File

@@ -265,7 +265,7 @@ func (f *blocksFetcher) handleRequest(ctx context.Context, start, count uint64)
// Short circuit start far exceeding the highest finalized epoch in some infinite loop.
if f.mode == modeStopOnFinalizedEpoch {
highestFinalizedSlot := helpers.StartSlot(targetEpoch + 1)
highestFinalizedSlot := (targetEpoch + 1) * params.BeaconConfig().SlotsPerEpoch
if start > highestFinalizedSlot {
response.err = fmt.Errorf("%v, slot: %d, highest finalized slot: %d",
errSlotIsTooHigh, start, highestFinalizedSlot)

View File

@@ -94,7 +94,10 @@ func (f *blocksFetcher) nonSkippedSlotAfter(ctx context.Context, slot uint64) (u
// Quickly find the close enough epoch where a non-empty slot definitely exists.
// Only single random slot per epoch is checked - allowing to move forward relatively quickly.
slot = slot + nonSkippedSlotsFullSearchEpochs*slotsPerEpoch
upperBoundSlot := helpers.StartSlot(targetEpoch + 1)
upperBoundSlot, err := helpers.StartSlot(targetEpoch + 1)
if err != nil {
return 0, err
}
for ind := slot + 1; ind < upperBoundSlot; ind += (slotsPerEpoch * slotsPerEpoch) / 2 {
start := ind + uint64(f.rand.Intn(int(slotsPerEpoch)))
nextSlot, err := fetch(peers[pidInd%len(peers)], start, slotsPerEpoch/2, slotsPerEpoch)
@@ -112,12 +115,19 @@ func (f *blocksFetcher) nonSkippedSlotAfter(ctx context.Context, slot uint64) (u
if upperBoundSlot > slotsPerEpoch {
upperBoundSlot -= slotsPerEpoch
}
upperBoundSlot = helpers.StartSlot(helpers.SlotToEpoch(upperBoundSlot))
upperBoundSlot, err = helpers.StartSlot(helpers.SlotToEpoch(upperBoundSlot))
if err != nil {
return 0, err
}
nextSlot, err := fetch(peers[pidInd%len(peers)], upperBoundSlot, slotsPerEpoch*2, 1)
if err != nil {
return 0, err
}
if nextSlot < slot || helpers.StartSlot(targetEpoch+1) < nextSlot {
s, err := helpers.StartSlot(targetEpoch + 1)
if err != nil {
return 0, err
}
if nextSlot < slot || s < nextSlot {
return 0, errors.New("invalid range for non-skipped slot")
}
return nextSlot, nil
@@ -126,12 +136,12 @@ func (f *blocksFetcher) nonSkippedSlotAfter(ctx context.Context, slot uint64) (u
// bestFinalizedSlot returns the highest finalized slot of the majority of connected peers.
func (f *blocksFetcher) bestFinalizedSlot() uint64 {
finalizedEpoch, _ := f.p2p.Peers().BestFinalized(params.BeaconConfig().MaxPeersToSync, f.finalizationFetcher.FinalizedCheckpt().Epoch)
return helpers.StartSlot(finalizedEpoch)
return finalizedEpoch * params.BeaconConfig().SlotsPerEpoch
}
// bestNonFinalizedSlot returns the highest non-finalized slot of enough number of connected peers.
func (f *blocksFetcher) bestNonFinalizedSlot() uint64 {
headEpoch := helpers.SlotToEpoch(f.headFetcher.HeadSlot())
targetEpoch, _ := f.p2p.Peers().BestNonFinalized(flags.Get().MinimumSyncPeers*2, headEpoch)
return helpers.StartSlot(targetEpoch)
return targetEpoch * params.BeaconConfig().SlotsPerEpoch
}

View File

@@ -49,7 +49,10 @@ func (s *Service) roundRobinSync(genesis time.Time) error {
s.counter = ratecounter.NewRateCounter(counterSeconds * time.Second)
s.lastProcessedSlot = s.chain.HeadSlot()
highestFinalizedSlot := helpers.StartSlot(s.highestFinalizedEpoch() + 1)
highestFinalizedSlot, err := helpers.StartSlot(s.highestFinalizedEpoch() + 1)
if err != nil {
return err
}
queue := newBlocksQueue(ctx, &blocksQueueConfig{
p2p: s.p2p,
headFetcher: s.chain,

View File

@@ -158,7 +158,11 @@ func (s *Service) validateAggregatedAtt(ctx context.Context, signed *ethpb.Signe
// Only advance state if different epoch as the committee can only change on an epoch transition.
if helpers.SlotToEpoch(attSlot) > helpers.SlotToEpoch(bs.Slot()) {
bs, err = state.ProcessSlots(ctx, bs, helpers.StartSlot(helpers.SlotToEpoch(attSlot)))
startSlot, err := helpers.StartSlot(helpers.SlotToEpoch(attSlot))
if err != nil {
return pubsub.ValidationIgnore
}
bs, err = state.ProcessSlots(ctx, bs, startSlot)
if err != nil {
traceutil.AnnotateError(span, err)
return pubsub.ValidationIgnore

View File

@@ -101,7 +101,11 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
return pubsub.ValidationIgnore
}
if helpers.StartSlot(s.chain.FinalizedCheckpt().Epoch) >= blk.Block.Slot {
startSlot, err := helpers.StartSlot(s.chain.FinalizedCheckpt().Epoch)
if err != nil {
return pubsub.ValidationIgnore
}
if startSlot >= blk.Block.Slot {
log.Debug("Block slot older/equal than last finalized epoch start slot, rejecting it")
return pubsub.ValidationIgnore
}

View File

@@ -352,13 +352,16 @@ func (v *validator) UpdateDuties(ctx context.Context, slot uint64) error {
return nil
}
// Set deadline to end of epoch.
ctx, cancel := context.WithDeadline(ctx, v.SlotDeadline(helpers.StartSlot(helpers.SlotToEpoch(slot)+1)))
ss, err := helpers.StartSlot(helpers.SlotToEpoch(slot) + 1)
if err != nil {
return err
}
ctx, cancel := context.WithDeadline(ctx, v.SlotDeadline(ss))
defer cancel()
ctx, span := trace.StartSpan(ctx, "validator.UpdateAssignments")
defer span.End()
var validatingKeys [][48]byte
var err error
if featureconfig.Get().EnableAccountsV2 {
validatingKeys, err = v.keyManagerV2.FetchValidatingPublicKeys(ctx)
} else {
@@ -644,7 +647,7 @@ func (v *validator) logDuties(slot uint64, duties []*ethpb.DutiesResponse_Duty)
attesterKeys[i] = make([]string, 0)
}
proposerKeys := make([]string, params.BeaconConfig().SlotsPerEpoch)
slotOffset := helpers.StartSlot(helpers.SlotToEpoch(slot))
slotOffset := slot - (slot % params.BeaconConfig().SlotsPerEpoch)
for _, duty := range duties {
if v.emitAccountMetrics {