mirror of
https://gitlab.com/pulsechaincom/prysm-pulse.git
synced 2024-12-22 03:30:35 +00:00
Block backfilling (#12968)
* backfill service * fix bug where origin state is never unlocked * support mvslice states * use renamed interface * refactor db code to skip block cache for backfill * lint * add test for verifier.verify * enable service in service init test * cancellation cleanup * adding nil checks to configset juggling * assume blocks are available by default As long as we're sure the AvailableBlocker is initialized correctly during node startup, defaulting to assuming we aren't in a checkpoint sync simplifies things greatly for tests. * block saving path refactor and bugfix * fix fillback test * fix BackfillStatus init tests --------- Co-authored-by: Kasey Kirkham <kasey@users.noreply.github.com>
This commit is contained in:
parent
3187a05a76
commit
1df173e701
@ -202,3 +202,14 @@ func ParseWeakSubjectivityInputString(wsCheckpointString string) (*v1alpha1.Chec
|
||||
Root: bRoot,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// MinEpochsForBlockRequests computes the number of epochs of block history that we need to maintain,
|
||||
// relative to the current epoch, per the p2p specs. This is used to compute the slot where backfill is complete.
|
||||
// value defined:
|
||||
// https://github.com/ethereum/consensus-specs/blob/dev/specs/phase0/p2p-interface.md#configuration
|
||||
// MIN_VALIDATOR_WITHDRAWABILITY_DELAY + CHURN_LIMIT_QUOTIENT // 2 (= 33024, ~5 months)
|
||||
// detailed rationale: https://github.com/ethereum/consensus-specs/blob/dev/specs/phase0/p2p-interface.md#why-are-blocksbyrange-requests-only-required-to-be-served-for-the-latest-min_epochs_for_block_requests-epochs
|
||||
func MinEpochsForBlockRequests() primitives.Epoch {
|
||||
return params.BeaconConfig().MinValidatorWithdrawabilityDelay +
|
||||
primitives.Epoch(params.BeaconConfig().ChurnLimitQuotient/2)
|
||||
}
|
||||
|
@ -281,3 +281,19 @@ func genState(t *testing.T, valCount, avgBalance uint64) state.BeaconState {
|
||||
|
||||
return beaconState
|
||||
}
|
||||
|
||||
func TestMinEpochsForBlockRequests(t *testing.T) {
|
||||
params.SetActiveTestCleanup(t, params.MainnetConfig())
|
||||
var expected primitives.Epoch = 33024
|
||||
// expected value of 33024 via spec commentary:
|
||||
// https://github.com/ethereum/consensus-specs/blob/dev/specs/phase0/p2p-interface.md#why-are-blocksbyrange-requests-only-required-to-be-served-for-the-latest-min_epochs_for_block_requests-epochs
|
||||
// MIN_EPOCHS_FOR_BLOCK_REQUESTS is calculated using the arithmetic from compute_weak_subjectivity_period found in the weak subjectivity guide. Specifically to find this max epoch range, we use the worst case event of a very large validator size (>= MIN_PER_EPOCH_CHURN_LIMIT * CHURN_LIMIT_QUOTIENT).
|
||||
//
|
||||
// MIN_EPOCHS_FOR_BLOCK_REQUESTS = (
|
||||
// MIN_VALIDATOR_WITHDRAWABILITY_DELAY
|
||||
// + MAX_SAFETY_DECAY * CHURN_LIMIT_QUOTIENT // (2 * 100)
|
||||
// )
|
||||
//
|
||||
// Where MAX_SAFETY_DECAY = 100 and thus MIN_EPOCHS_FOR_BLOCK_REQUESTS = 33024 (~5 months).
|
||||
require.Equal(t, expected, helpers.MinEpochsForBlockRequests())
|
||||
}
|
||||
|
@ -22,9 +22,6 @@ var ErrNotFoundOriginBlockRoot = kv.ErrNotFoundOriginBlockRoot
|
||||
// ErrNotFoundBackfillBlockRoot wraps ErrNotFound for an error specific to the backfill block root.
|
||||
var ErrNotFoundBackfillBlockRoot = kv.ErrNotFoundBackfillBlockRoot
|
||||
|
||||
// ErrNotFoundGenesisBlockRoot means no genesis block root was found, indicating the db was not initialized with genesis
|
||||
var ErrNotFoundGenesisBlockRoot = kv.ErrNotFoundGenesisBlockRoot
|
||||
|
||||
// IsNotFound allows callers to treat errors from a flat-file database, where the file record is missing,
|
||||
// as equivalent to db.ErrNotFound.
|
||||
func IsNotFound(err error) bool {
|
||||
|
@ -13,9 +13,11 @@ go_library(
|
||||
"//beacon-chain/db/filters:go_default_library",
|
||||
"//beacon-chain/slasher/types:go_default_library",
|
||||
"//beacon-chain/state:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//monitoring/backup:go_default_library",
|
||||
"//proto/dbval:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common:go_default_library",
|
||||
],
|
||||
|
@ -11,9 +11,11 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filters"
|
||||
slashertypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/slasher/types"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/monitoring/backup"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
|
||||
)
|
||||
|
||||
@ -57,7 +59,7 @@ type ReadOnlyDatabase interface {
|
||||
|
||||
// origin checkpoint sync support
|
||||
OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error)
|
||||
BackfillBlockRoot(ctx context.Context) ([32]byte, error)
|
||||
BackfillStatus(context.Context) (*dbval.BackfillStatus, error)
|
||||
}
|
||||
|
||||
// NoHeadAccessDatabase defines a struct without access to chain head data.
|
||||
@ -68,6 +70,7 @@ type NoHeadAccessDatabase interface {
|
||||
DeleteBlock(ctx context.Context, root [32]byte) error
|
||||
SaveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock) error
|
||||
SaveBlocks(ctx context.Context, blocks []interfaces.ReadOnlySignedBeaconBlock) error
|
||||
SaveROBlocks(ctx context.Context, blks []blocks.ROBlock, cache bool) error
|
||||
SaveGenesisBlockRoot(ctx context.Context, blockRoot [32]byte) error
|
||||
// State related methods.
|
||||
SaveState(ctx context.Context, state state.ReadOnlyBeaconState, blockRoot [32]byte) error
|
||||
@ -106,9 +109,10 @@ type HeadAccessDatabase interface {
|
||||
SaveGenesisData(ctx context.Context, state state.BeaconState) error
|
||||
EnsureEmbeddedGenesis(ctx context.Context) error
|
||||
|
||||
// initialization method needed for origin checkpoint sync
|
||||
// Support for checkpoint sync and backfill.
|
||||
SaveOrigin(ctx context.Context, serState, serBlock []byte) error
|
||||
SaveBackfillBlockRoot(ctx context.Context, blockRoot [32]byte) error
|
||||
SaveBackfillStatus(context.Context, *dbval.BackfillStatus) error
|
||||
BackfillFinalizedIndex(ctx context.Context, blocks []blocks.ROBlock, finalizedChildRoot [32]byte) error
|
||||
}
|
||||
|
||||
// SlasherDatabase interface for persisting data related to detecting slashable offenses on Ethereum.
|
||||
|
@ -4,6 +4,7 @@ go_library(
|
||||
name = "go_default_library",
|
||||
srcs = [
|
||||
"archived_point.go",
|
||||
"backfill.go",
|
||||
"backup.go",
|
||||
"blocks.go",
|
||||
"checkpoint.go",
|
||||
@ -48,6 +49,7 @@ go_library(
|
||||
"//io/file:go_default_library",
|
||||
"//monitoring/progress:go_default_library",
|
||||
"//monitoring/tracing:go_default_library",
|
||||
"//proto/dbval:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//runtime/version:go_default_library",
|
||||
"//time:go_default_library",
|
||||
@ -73,6 +75,7 @@ go_test(
|
||||
name = "go_default_test",
|
||||
srcs = [
|
||||
"archived_point_test.go",
|
||||
"backfill_test.go",
|
||||
"backup_test.go",
|
||||
"blocks_test.go",
|
||||
"checkpoint_test.go",
|
||||
@ -107,6 +110,7 @@ go_test(
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//proto/dbval:go_default_library",
|
||||
"//proto/engine/v1:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//proto/testing:go_default_library",
|
||||
|
44
beacon-chain/db/kv/backfill.go
Normal file
44
beacon-chain/db/kv/backfill.go
Normal file
@ -0,0 +1,44 @@
|
||||
package kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
bolt "go.etcd.io/bbolt"
|
||||
"go.opencensus.io/trace"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
// SaveBackfillStatus encodes the given BackfillStatus protobuf struct and writes it to a single key in the db.
|
||||
// This value is used by the backfill service to keep track of the range of blocks that need to be synced. It is also used by the
|
||||
// code that serves blocks or regenerates states to keep track of what range of blocks are available.
|
||||
func (s *Store) SaveBackfillStatus(ctx context.Context, bf *dbval.BackfillStatus) error {
|
||||
_, span := trace.StartSpan(ctx, "BeaconDB.SaveBackfillStatus")
|
||||
defer span.End()
|
||||
bfb, err := proto.Marshal(bf)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.db.Update(func(tx *bolt.Tx) error {
|
||||
bucket := tx.Bucket(blocksBucket)
|
||||
return bucket.Put(backfillStatusKey, bfb)
|
||||
})
|
||||
}
|
||||
|
||||
// BackfillStatus retrieves the most recently saved version of the BackfillStatus protobuf struct.
|
||||
// This is used to persist information about backfill status across restarts.
|
||||
func (s *Store) BackfillStatus(ctx context.Context) (*dbval.BackfillStatus, error) {
|
||||
_, span := trace.StartSpan(ctx, "BeaconDB.BackfillStatus")
|
||||
defer span.End()
|
||||
bf := &dbval.BackfillStatus{}
|
||||
err := s.db.View(func(tx *bolt.Tx) error {
|
||||
bucket := tx.Bucket(blocksBucket)
|
||||
bs := bucket.Get(backfillStatusKey)
|
||||
if len(bs) == 0 {
|
||||
return errors.Wrap(ErrNotFound, "BackfillStatus not found")
|
||||
}
|
||||
return proto.Unmarshal(bs, bf)
|
||||
})
|
||||
return bf, err
|
||||
}
|
35
beacon-chain/db/kv/backfill_test.go
Normal file
35
beacon-chain/db/kv/backfill_test.go
Normal file
@ -0,0 +1,35 @@
|
||||
package kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
func TestBackfillRoundtrip(t *testing.T) {
|
||||
db := setupDB(t)
|
||||
b := &dbval.BackfillStatus{}
|
||||
b.LowSlot = 23
|
||||
b.LowRoot = bytesutil.PadTo([]byte("low"), 32)
|
||||
b.LowParentRoot = bytesutil.PadTo([]byte("parent"), 32)
|
||||
m, err := proto.Marshal(b)
|
||||
require.NoError(t, err)
|
||||
ub := &dbval.BackfillStatus{}
|
||||
require.NoError(t, proto.Unmarshal(m, ub))
|
||||
require.Equal(t, b.LowSlot, ub.LowSlot)
|
||||
require.DeepEqual(t, b.LowRoot, ub.LowRoot)
|
||||
require.DeepEqual(t, b.LowParentRoot, ub.LowParentRoot)
|
||||
|
||||
ctx := context.Background()
|
||||
require.NoError(t, db.SaveBackfillStatus(ctx, b))
|
||||
dbub, err := db.BackfillStatus(ctx)
|
||||
require.NoError(t, err)
|
||||
|
||||
require.Equal(t, b.LowSlot, dbub.LowSlot)
|
||||
require.DeepEqual(t, b.LowRoot, dbub.LowRoot)
|
||||
require.DeepEqual(t, b.LowParentRoot, dbub.LowParentRoot)
|
||||
}
|
@ -70,25 +70,6 @@ func (s *Store) OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error)
|
||||
return root, err
|
||||
}
|
||||
|
||||
// BackfillBlockRoot keeps track of the highest block available before the OriginCheckpointBlockRoot
|
||||
func (s *Store) BackfillBlockRoot(ctx context.Context) ([32]byte, error) {
|
||||
_, span := trace.StartSpan(ctx, "BeaconDB.BackfillBlockRoot")
|
||||
defer span.End()
|
||||
|
||||
var root [32]byte
|
||||
err := s.db.View(func(tx *bolt.Tx) error {
|
||||
bkt := tx.Bucket(blocksBucket)
|
||||
rootSlice := bkt.Get(backfillBlockRootKey)
|
||||
if len(rootSlice) == 0 {
|
||||
return ErrNotFoundBackfillBlockRoot
|
||||
}
|
||||
root = bytesutil.ToBytes32(rootSlice)
|
||||
return nil
|
||||
})
|
||||
|
||||
return root, err
|
||||
}
|
||||
|
||||
// HeadBlock returns the latest canonical block in the Ethereum Beacon Chain.
|
||||
func (s *Store) HeadBlock(ctx context.Context) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
ctx, span := trace.StartSpan(ctx, "BeaconDB.HeadBlock")
|
||||
@ -292,55 +273,95 @@ func (s *Store) SaveBlocks(ctx context.Context, blks []interfaces.ReadOnlySigned
|
||||
ctx, span := trace.StartSpan(ctx, "BeaconDB.SaveBlocks")
|
||||
defer span.End()
|
||||
|
||||
// Performing marshaling, hashing, and indexing outside the bolt transaction
|
||||
// to minimize the time we hold the DB lock.
|
||||
blockRoots := make([][]byte, len(blks))
|
||||
encodedBlocks := make([][]byte, len(blks))
|
||||
indicesForBlocks := make([]map[string][]byte, len(blks))
|
||||
for i, blk := range blks {
|
||||
blockRoot, err := blk.Block().HashTreeRoot()
|
||||
robs := make([]blocks.ROBlock, len(blks))
|
||||
for i := range blks {
|
||||
rb, err := blocks.NewROBlock(blks[i])
|
||||
if err != nil {
|
||||
return err
|
||||
return errors.Wrapf(err, "failed to make an ROBlock for a block in SaveBlocks")
|
||||
}
|
||||
enc, err := s.marshalBlock(ctx, blk)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
blockRoots[i] = blockRoot[:]
|
||||
encodedBlocks[i] = enc
|
||||
indicesByBucket := createBlockIndicesFromBlock(ctx, blk.Block())
|
||||
indicesForBlocks[i] = indicesByBucket
|
||||
robs[i] = rb
|
||||
}
|
||||
saveBlinded, err := s.shouldSaveBlinded(ctx)
|
||||
return s.SaveROBlocks(ctx, robs, true)
|
||||
}
|
||||
|
||||
type blockBatchEntry struct {
|
||||
root []byte
|
||||
block interfaces.ReadOnlySignedBeaconBlock
|
||||
enc []byte
|
||||
updated bool
|
||||
indices map[string][]byte
|
||||
}
|
||||
|
||||
func prepareBlockBatch(blks []blocks.ROBlock, shouldBlind bool) ([]blockBatchEntry, error) {
|
||||
batch := make([]blockBatchEntry, len(blks))
|
||||
for i := range blks {
|
||||
batch[i].root, batch[i].block = blks[i].RootSlice(), blks[i].ReadOnlySignedBeaconBlock
|
||||
batch[i].indices = blockIndices(batch[i].block.Block().Slot(), batch[i].block.Block().ParentRoot())
|
||||
if shouldBlind {
|
||||
blinded, err := batch[i].block.ToBlinded()
|
||||
if err != nil {
|
||||
if !errors.Is(err, blocks.ErrUnsupportedVersion) {
|
||||
return nil, errors.Wrapf(err, "could not convert block to blinded format for root %#x", batch[i].root)
|
||||
}
|
||||
// Pre-deneb blocks give ErrUnsupportedVersion; use the full block already in the batch entry.
|
||||
} else {
|
||||
batch[i].block = blinded
|
||||
}
|
||||
}
|
||||
enc, err := encodeBlock(batch[i].block)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to encode block for root %#x", batch[i].root)
|
||||
}
|
||||
batch[i].enc = enc
|
||||
}
|
||||
return batch, nil
|
||||
}
|
||||
|
||||
func (s *Store) SaveROBlocks(ctx context.Context, blks []blocks.ROBlock, cache bool) error {
|
||||
shouldBlind, err := s.shouldSaveBlinded(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.db.Update(func(tx *bolt.Tx) error {
|
||||
// Precompute expensive values outside the db transaction.
|
||||
batch, err := prepareBlockBatch(blks, shouldBlind)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "failed to encode all blocks in batch for saving to the db")
|
||||
}
|
||||
err = s.db.Update(func(tx *bolt.Tx) error {
|
||||
bkt := tx.Bucket(blocksBucket)
|
||||
for i, blk := range blks {
|
||||
if existingBlock := bkt.Get(blockRoots[i]); existingBlock != nil {
|
||||
for i := range batch {
|
||||
if exists := bkt.Get(batch[i].root); exists != nil {
|
||||
continue
|
||||
}
|
||||
if err := updateValueForIndices(ctx, indicesForBlocks[i], blockRoots[i], tx); err != nil {
|
||||
return errors.Wrap(err, "could not update DB indices")
|
||||
if err := bkt.Put(batch[i].root, batch[i].enc); err != nil {
|
||||
return errors.Wrapf(err, "could write block to db with root %#x", batch[i].root)
|
||||
}
|
||||
if saveBlinded {
|
||||
blindedBlock, err := blk.ToBlinded()
|
||||
if err != nil {
|
||||
if !errors.Is(err, blocks.ErrUnsupportedVersion) {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
blk = blindedBlock
|
||||
}
|
||||
}
|
||||
s.blockCache.Set(string(blockRoots[i]), blk, int64(len(encodedBlocks[i])))
|
||||
if err := bkt.Put(blockRoots[i], encodedBlocks[i]); err != nil {
|
||||
return err
|
||||
if err := updateValueForIndices(ctx, batch[i].indices, batch[i].root, tx); err != nil {
|
||||
return errors.Wrapf(err, "could not update DB indices for root %#x", batch[i].root)
|
||||
}
|
||||
batch[i].updated = true
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if !cache {
|
||||
return err
|
||||
}
|
||||
for i := range batch {
|
||||
if batch[i].updated {
|
||||
s.blockCache.Set(string(batch[i].root), batch[i].block, int64(len(batch[i].enc)))
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// blockIndices takes in a beacon block and returns
|
||||
// a map of bolt DB index buckets corresponding to each particular key for indices for
|
||||
// data, such as (shard indices bucket -> shard 5).
|
||||
func blockIndices(slot primitives.Slot, parentRoot [32]byte) map[string][]byte {
|
||||
return map[string][]byte{
|
||||
string(blockSlotIndicesBucket): bytesutil.SlotToBytesBigEndian(slot),
|
||||
string(blockParentRootIndicesBucket): parentRoot[:],
|
||||
}
|
||||
}
|
||||
|
||||
// SaveHeadBlockRoot to the db.
|
||||
@ -417,17 +438,6 @@ func (s *Store) SaveOriginCheckpointBlockRoot(ctx context.Context, blockRoot [32
|
||||
})
|
||||
}
|
||||
|
||||
// SaveBackfillBlockRoot is used to keep track of the most recently backfilled block root when
|
||||
// the node was initialized via checkpoint sync.
|
||||
func (s *Store) SaveBackfillBlockRoot(ctx context.Context, blockRoot [32]byte) error {
|
||||
_, span := trace.StartSpan(ctx, "BeaconDB.SaveBackfillBlockRoot")
|
||||
defer span.End()
|
||||
return s.db.Update(func(tx *bolt.Tx) error {
|
||||
bucket := tx.Bucket(blocksBucket)
|
||||
return bucket.Put(backfillBlockRootKey, blockRoot[:])
|
||||
})
|
||||
}
|
||||
|
||||
// HighestRootsBelowSlot returns roots from the database slot index from the highest slot below the input slot.
|
||||
// The slot value at the beginning of the return list is the slot where the roots were found. This is helpful so that
|
||||
// calling code can make decisions based on the slot without resolving the blocks to discover their slot (for instance
|
||||
@ -726,31 +736,6 @@ func blockRootsBySlot(ctx context.Context, tx *bolt.Tx, slot primitives.Slot) ([
|
||||
return [][32]byte{}, nil
|
||||
}
|
||||
|
||||
// createBlockIndicesFromBlock takes in a beacon block and returns
|
||||
// a map of bolt DB index buckets corresponding to each particular key for indices for
|
||||
// data, such as (shard indices bucket -> shard 5).
|
||||
func createBlockIndicesFromBlock(ctx context.Context, block interfaces.ReadOnlyBeaconBlock) map[string][]byte {
|
||||
_, span := trace.StartSpan(ctx, "BeaconDB.createBlockIndicesFromBlock")
|
||||
defer span.End()
|
||||
indicesByBucket := make(map[string][]byte)
|
||||
// Every index has a unique bucket for fast, binary-search
|
||||
// range scans for filtering across keys.
|
||||
buckets := [][]byte{
|
||||
blockSlotIndicesBucket,
|
||||
}
|
||||
indices := [][]byte{
|
||||
bytesutil.SlotToBytesBigEndian(block.Slot()),
|
||||
}
|
||||
buckets = append(buckets, blockParentRootIndicesBucket)
|
||||
parentRoot := block.ParentRoot()
|
||||
indices = append(indices, parentRoot[:])
|
||||
|
||||
for i := 0; i < len(buckets); i++ {
|
||||
indicesByBucket[string(buckets[i])] = indices[i]
|
||||
}
|
||||
return indicesByBucket
|
||||
}
|
||||
|
||||
// createBlockFiltersFromIndices takes in filter criteria and returns
|
||||
// a map with a single key-value pair: "block-parent-root-indices” -> parentRoot (array of bytes).
|
||||
//
|
||||
@ -838,74 +823,44 @@ func unmarshalBlock(_ context.Context, enc []byte) (interfaces.ReadOnlySignedBea
|
||||
return blocks.NewSignedBeaconBlock(rawBlock)
|
||||
}
|
||||
|
||||
func (s *Store) marshalBlock(
|
||||
ctx context.Context,
|
||||
blk interfaces.ReadOnlySignedBeaconBlock,
|
||||
) ([]byte, error) {
|
||||
shouldBlind, err := s.shouldSaveBlinded(ctx)
|
||||
func encodeBlock(blk interfaces.ReadOnlySignedBeaconBlock) ([]byte, error) {
|
||||
key, err := keyForBlock(blk)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, errors.Wrap(err, "could not determine version encoding key for block")
|
||||
}
|
||||
if shouldBlind {
|
||||
return marshalBlockBlinded(ctx, blk)
|
||||
enc, err := blk.MarshalSSZ()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not marshal block")
|
||||
}
|
||||
return marshalBlockFull(ctx, blk)
|
||||
dbfmt := make([]byte, len(key)+len(enc))
|
||||
if len(key) > 0 {
|
||||
copy(dbfmt, key)
|
||||
}
|
||||
copy(dbfmt[len(key):], enc)
|
||||
return snappy.Encode(nil, dbfmt), nil
|
||||
}
|
||||
|
||||
// Encodes a full beacon block to the DB with its associated key.
|
||||
func marshalBlockFull(
|
||||
_ context.Context,
|
||||
blk interfaces.ReadOnlySignedBeaconBlock,
|
||||
) ([]byte, error) {
|
||||
var encodedBlock []byte
|
||||
var err error
|
||||
encodedBlock, err = blk.MarshalSSZ()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
func keyForBlock(blk interfaces.ReadOnlySignedBeaconBlock) ([]byte, error) {
|
||||
switch blk.Version() {
|
||||
case version.Deneb:
|
||||
return snappy.Encode(nil, append(denebKey, encodedBlock...)), nil
|
||||
case version.Capella:
|
||||
return snappy.Encode(nil, append(capellaKey, encodedBlock...)), nil
|
||||
case version.Bellatrix:
|
||||
return snappy.Encode(nil, append(bellatrixKey, encodedBlock...)), nil
|
||||
case version.Altair:
|
||||
return snappy.Encode(nil, append(altairKey, encodedBlock...)), nil
|
||||
case version.Phase0:
|
||||
return snappy.Encode(nil, encodedBlock), nil
|
||||
default:
|
||||
return nil, errors.New("unknown block version")
|
||||
}
|
||||
}
|
||||
|
||||
// Encodes a blinded beacon block with its associated key.
|
||||
// If the block does not support blinding, we then encode it as a full
|
||||
// block with its associated key by calling marshalBlockFull.
|
||||
func marshalBlockBlinded(
|
||||
ctx context.Context,
|
||||
blk interfaces.ReadOnlySignedBeaconBlock,
|
||||
) ([]byte, error) {
|
||||
blindedBlock, err := blk.ToBlinded()
|
||||
if err != nil {
|
||||
switch {
|
||||
case errors.Is(err, blocks.ErrUnsupportedVersion):
|
||||
return marshalBlockFull(ctx, blk)
|
||||
default:
|
||||
return nil, errors.Wrap(err, "could not convert block to blinded format")
|
||||
if blk.IsBlinded() {
|
||||
return denebBlindKey, nil
|
||||
}
|
||||
}
|
||||
encodedBlock, err := blindedBlock.MarshalSSZ()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not marshal blinded block")
|
||||
}
|
||||
switch blk.Version() {
|
||||
case version.Deneb:
|
||||
return snappy.Encode(nil, append(denebBlindKey, encodedBlock...)), nil
|
||||
return denebKey, nil
|
||||
case version.Capella:
|
||||
return snappy.Encode(nil, append(capellaBlindKey, encodedBlock...)), nil
|
||||
if blk.IsBlinded() {
|
||||
return capellaBlindKey, nil
|
||||
}
|
||||
return capellaKey, nil
|
||||
case version.Bellatrix:
|
||||
return snappy.Encode(nil, append(bellatrixBlindKey, encodedBlock...)), nil
|
||||
if blk.IsBlinded() {
|
||||
return bellatrixBlindKey, nil
|
||||
}
|
||||
return bellatrixKey, nil
|
||||
case version.Altair:
|
||||
return altairKey, nil
|
||||
case version.Phase0:
|
||||
return nil, nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported block version: %v", blk.Version())
|
||||
}
|
||||
|
@ -126,23 +126,6 @@ var blockTests = []struct {
|
||||
},
|
||||
}
|
||||
|
||||
func TestStore_SaveBackfillBlockRoot(t *testing.T) {
|
||||
db := setupDB(t)
|
||||
ctx := context.Background()
|
||||
|
||||
_, err := db.BackfillBlockRoot(ctx)
|
||||
require.ErrorIs(t, err, ErrNotFoundBackfillBlockRoot)
|
||||
|
||||
var expected [32]byte
|
||||
copy(expected[:], []byte{0x23})
|
||||
err = db.SaveBackfillBlockRoot(ctx, expected)
|
||||
require.NoError(t, err)
|
||||
actual, err := db.BackfillBlockRoot(ctx)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expected, actual)
|
||||
|
||||
}
|
||||
|
||||
func TestStore_SaveBlock_NoDuplicates(t *testing.T) {
|
||||
BlockCacheSize = 1
|
||||
slot := primitives.Slot(20)
|
||||
|
@ -21,3 +21,8 @@ var ErrNotFoundBackfillBlockRoot = errors.Wrap(ErrNotFound, "BackfillBlockRoot")
|
||||
|
||||
// ErrNotFoundFeeRecipient is a not found error specifically for the fee recipient getter
|
||||
var ErrNotFoundFeeRecipient = errors.Wrap(ErrNotFound, "fee recipient")
|
||||
|
||||
var errEmptyBlockSlice = errors.New("[]blocks.ROBlock is empty")
|
||||
var errIncorrectBlockParent = errors.New("unexpected missing or forked blocks in a []ROBlock")
|
||||
var errFinalizedChildNotFound = errors.New("unable to find finalized root descending from backfill batch")
|
||||
var errNotConnectedToFinalized = errors.New("unable to finalize backfill blocks, finalized parent_root does not match")
|
||||
|
@ -4,6 +4,7 @@ import (
|
||||
"bytes"
|
||||
"context"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filters"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
|
||||
@ -163,6 +164,83 @@ func (s *Store) updateFinalizedBlockRoots(ctx context.Context, tx *bolt.Tx, chec
|
||||
return bkt.Put(previousFinalizedCheckpointKey, enc)
|
||||
}
|
||||
|
||||
// BackfillFinalizedIndex updates the finalized index for a contiguous chain of blocks that are the ancestors of the
|
||||
// given finalized child root. This is needed to update the finalized index during backfill, because the usual
|
||||
// updateFinalizedBlockRoots has assumptions that are incompatible with backfill processing.
|
||||
func (s *Store) BackfillFinalizedIndex(ctx context.Context, blocks []blocks.ROBlock, finalizedChildRoot [32]byte) error {
|
||||
ctx, span := trace.StartSpan(ctx, "BeaconDB.BackfillFinalizedIndex")
|
||||
defer span.End()
|
||||
if len(blocks) == 0 {
|
||||
return errEmptyBlockSlice
|
||||
}
|
||||
|
||||
fbrs := make([]*ethpb.FinalizedBlockRootContainer, len(blocks))
|
||||
encs := make([][]byte, len(blocks))
|
||||
for i := range blocks {
|
||||
pr := blocks[i].Block().ParentRoot()
|
||||
fbrs[i] = ðpb.FinalizedBlockRootContainer{
|
||||
ParentRoot: pr[:],
|
||||
// ChildRoot: will be filled in on the next iteration when we look at the descendent block.
|
||||
}
|
||||
if i == 0 {
|
||||
continue
|
||||
}
|
||||
if blocks[i-1].Root() != blocks[i].Block().ParentRoot() {
|
||||
return errors.Wrapf(errIncorrectBlockParent, "previous root=%#x, slot=%d; child parent_root=%#x, root=%#x, slot=%d",
|
||||
blocks[i-1].Root(), blocks[i-1].Block().Slot(), blocks[i].Block().ParentRoot(), blocks[i].Root(), blocks[i].Block().Slot())
|
||||
}
|
||||
|
||||
// We know the previous index is the parent of this one thanks to the assertion above,
|
||||
// so we can set the ChildRoot of the previous value to the root of the current value.
|
||||
fbrs[i-1].ChildRoot = blocks[i].RootSlice()
|
||||
// Now that the value for fbrs[i-1] is complete, perform encoding here to minimize time in Update,
|
||||
// which holds the global db lock.
|
||||
penc, err := encode(ctx, fbrs[i-1])
|
||||
if err != nil {
|
||||
tracing.AnnotateError(span, err)
|
||||
return err
|
||||
}
|
||||
encs[i-1] = penc
|
||||
|
||||
// The final element is the parent of finalizedChildRoot. This is checked inside the db transaction using
|
||||
// the parent_root value stored in the index data for finalizedChildRoot.
|
||||
if i == len(blocks)-1 {
|
||||
fbrs[i].ChildRoot = finalizedChildRoot[:]
|
||||
// Final element is complete, so it is pre-encoded like the others.
|
||||
enc, err := encode(ctx, fbrs[i])
|
||||
if err != nil {
|
||||
tracing.AnnotateError(span, err)
|
||||
return err
|
||||
}
|
||||
encs[i] = enc
|
||||
}
|
||||
}
|
||||
|
||||
return s.db.Update(func(tx *bolt.Tx) error {
|
||||
bkt := tx.Bucket(finalizedBlockRootsIndexBucket)
|
||||
child := bkt.Get(finalizedChildRoot[:])
|
||||
if len(child) == 0 {
|
||||
return errFinalizedChildNotFound
|
||||
}
|
||||
fcc := ðpb.FinalizedBlockRootContainer{}
|
||||
if err := decode(ctx, child, fcc); err != nil {
|
||||
return errors.Wrapf(err, "unable to decode finalized block root container for root=%#x", finalizedChildRoot)
|
||||
}
|
||||
// Ensure that the existing finalized chain descends from the new segment.
|
||||
if !bytes.Equal(fcc.ParentRoot, blocks[len(blocks)-1].RootSlice()) {
|
||||
return errors.Wrapf(errNotConnectedToFinalized, "finalized block root container for root=%#x has parent_root=%#x, not %#x",
|
||||
finalizedChildRoot, fcc.ParentRoot, blocks[len(blocks)-1].RootSlice())
|
||||
}
|
||||
// Update the finalized index with entries for each block in the new segment.
|
||||
for i := range fbrs {
|
||||
if err := bkt.Put(blocks[i].RootSlice(), encs[i]); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
// IsFinalizedBlock returns true if the block root is present in the finalized block root index.
|
||||
// A beacon block root contained exists in this index if it is considered finalized and canonical.
|
||||
// Note: beacon blocks from the latest finalized epoch return true, whether or not they are
|
||||
|
@ -1,6 +1,7 @@
|
||||
package kv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
@ -14,6 +15,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/assert"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/util"
|
||||
bolt "go.etcd.io/bbolt"
|
||||
)
|
||||
|
||||
var genesisBlockRoot = bytesutil.ToBytes32([]byte{'G', 'E', 'N', 'E', 'S', 'I', 'S'})
|
||||
@ -234,3 +236,64 @@ func makeBlocksAltair(t *testing.T, startIdx, num uint64, previousRoot [32]byte)
|
||||
}
|
||||
return ifaceBlocks
|
||||
}
|
||||
|
||||
func TestStore_BackfillFinalizedIndex(t *testing.T) {
|
||||
db := setupDB(t)
|
||||
ctx := context.Background()
|
||||
require.ErrorIs(t, db.BackfillFinalizedIndex(ctx, []consensusblocks.ROBlock{}, [32]byte{}), errEmptyBlockSlice)
|
||||
blks, err := consensusblocks.NewROBlockSlice(makeBlocks(t, 0, 66, [32]byte{}))
|
||||
require.NoError(t, err)
|
||||
|
||||
// set up existing finalized block
|
||||
ebpr := blks[64].Block().ParentRoot()
|
||||
ebr := blks[64].Root()
|
||||
chldr := blks[65].Root()
|
||||
ebf := ðpb.FinalizedBlockRootContainer{
|
||||
ParentRoot: ebpr[:],
|
||||
ChildRoot: chldr[:],
|
||||
}
|
||||
disjoint := []consensusblocks.ROBlock{
|
||||
blks[0],
|
||||
blks[2],
|
||||
}
|
||||
enc, err := encode(ctx, ebf)
|
||||
require.NoError(t, err)
|
||||
err = db.db.Update(func(tx *bolt.Tx) error {
|
||||
bkt := tx.Bucket(finalizedBlockRootsIndexBucket)
|
||||
return bkt.Put(ebr[:], enc)
|
||||
})
|
||||
|
||||
// reslice to remove the existing blocks
|
||||
blks = blks[0:64]
|
||||
// check the other error conditions with a descendent root that really doesn't exist
|
||||
require.NoError(t, err)
|
||||
require.ErrorIs(t, db.BackfillFinalizedIndex(ctx, disjoint, [32]byte{}), errIncorrectBlockParent)
|
||||
require.NoError(t, err)
|
||||
require.ErrorIs(t, errFinalizedChildNotFound, db.BackfillFinalizedIndex(ctx, blks, [32]byte{}))
|
||||
|
||||
// use the real root so that it succeeds
|
||||
require.NoError(t, db.BackfillFinalizedIndex(ctx, blks, ebr))
|
||||
for i := range blks {
|
||||
require.NoError(t, db.db.View(func(tx *bolt.Tx) error {
|
||||
bkt := tx.Bucket(finalizedBlockRootsIndexBucket)
|
||||
encfr := bkt.Get(blks[i].RootSlice())
|
||||
require.Equal(t, true, len(encfr) > 0)
|
||||
fr := ðpb.FinalizedBlockRootContainer{}
|
||||
require.NoError(t, decode(ctx, encfr, fr))
|
||||
require.Equal(t, 32, len(fr.ParentRoot))
|
||||
require.Equal(t, 32, len(fr.ChildRoot))
|
||||
pr := blks[i].Block().ParentRoot()
|
||||
require.Equal(t, true, bytes.Equal(fr.ParentRoot, pr[:]))
|
||||
if i > 0 {
|
||||
require.Equal(t, true, bytes.Equal(fr.ParentRoot, blks[i-1].RootSlice()))
|
||||
}
|
||||
if i < len(blks)-1 {
|
||||
require.DeepEqual(t, fr.ChildRoot, blks[i+1].RootSlice())
|
||||
}
|
||||
if i == len(blks)-1 {
|
||||
require.DeepEqual(t, fr.ChildRoot, ebr[:])
|
||||
}
|
||||
return nil
|
||||
}))
|
||||
}
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
package kv
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
@ -116,13 +117,25 @@ func Test_setupBlockStorageType(t *testing.T) {
|
||||
root, err = wrappedBlock.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, store.SaveBlock(ctx, wrappedBlock))
|
||||
retrievedBlk, err = store.Block(ctx, root)
|
||||
require.NoError(t, err)
|
||||
|
||||
require.Equal(t, true, retrievedBlk.IsBlinded())
|
||||
wrappedBlinded, err := wrappedBlock.ToBlinded()
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, wrappedBlinded, retrievedBlk)
|
||||
|
||||
retrievedBlk, err = store.Block(ctx, root)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, true, retrievedBlk.IsBlinded())
|
||||
|
||||
// Compare retrieved value by root, and marshaled bytes.
|
||||
mSrc, err := wrappedBlinded.MarshalSSZ()
|
||||
require.NoError(t, err)
|
||||
mTgt, err := retrievedBlk.MarshalSSZ()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, true, bytes.Equal(mSrc, mTgt))
|
||||
|
||||
rSrc, err := wrappedBlinded.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
rTgt, err := retrievedBlk.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, rSrc, rTgt)
|
||||
})
|
||||
t.Run("existing database with full blocks type should continue storing full blocks", func(t *testing.T) {
|
||||
store := setupDB(t)
|
||||
@ -155,10 +168,21 @@ func Test_setupBlockStorageType(t *testing.T) {
|
||||
root, err = wrappedBlock.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, store.SaveBlock(ctx, wrappedBlock))
|
||||
|
||||
retrievedBlk, err = store.Block(ctx, root)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, false, retrievedBlk.IsBlinded())
|
||||
require.DeepEqual(t, wrappedBlock, retrievedBlk)
|
||||
|
||||
// Compare retrieved value by root, and marshaled bytes.
|
||||
mSrc, err := wrappedBlock.MarshalSSZ()
|
||||
require.NoError(t, err)
|
||||
mTgt, err := retrievedBlk.MarshalSSZ()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, true, bytes.Equal(mSrc, mTgt))
|
||||
|
||||
rTgt, err := retrievedBlk.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, root, rTgt)
|
||||
})
|
||||
t.Run("existing database with blinded blocks type should error if user enables full blocks feature flag", func(t *testing.T) {
|
||||
store := setupDB(t)
|
||||
|
@ -61,8 +61,8 @@ var (
|
||||
|
||||
// block root included in the beacon state used by weak subjectivity initial sync
|
||||
originCheckpointBlockRootKey = []byte("origin-checkpoint-block-root")
|
||||
// block root tracking the progress of backfill, or pointing at genesis if backfill has not been initiated
|
||||
backfillBlockRootKey = []byte("backfill-block-root")
|
||||
// tracking data about an ongoing backfill
|
||||
backfillStatusKey = []byte("backfill-status")
|
||||
|
||||
// Deprecated: This index key was migrated in PR 6461. Do not use, except for migrations.
|
||||
lastArchivedIndexKey = []byte("last-archived")
|
||||
|
@ -8,6 +8,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/ssz/detect"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v4/runtime/version"
|
||||
)
|
||||
@ -17,18 +18,6 @@ import (
|
||||
// syncing, using the provided values as their point of origin. This is an alternative
|
||||
// to syncing from genesis, and should only be run on an empty database.
|
||||
func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error {
|
||||
genesisRoot, err := s.GenesisBlockRoot(ctx)
|
||||
if err != nil {
|
||||
if errors.Is(err, ErrNotFoundGenesisBlockRoot) {
|
||||
return errors.Wrap(err, "genesis block root not found: genesis must be provided for checkpoint sync")
|
||||
}
|
||||
return errors.Wrap(err, "genesis block root query error: checkpoint sync must verify genesis to proceed")
|
||||
}
|
||||
err = s.SaveBackfillBlockRoot(ctx, genesisRoot)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "unable to save genesis root as initial backfill starting point for checkpoint sync")
|
||||
}
|
||||
|
||||
cf, err := detect.FromState(serState)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not sniff config+fork for origin state bytes")
|
||||
@ -50,11 +39,24 @@ func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error
|
||||
}
|
||||
blk := wblk.Block()
|
||||
|
||||
// save block
|
||||
blockRoot, err := blk.HashTreeRoot()
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not compute HashTreeRoot of checkpoint block")
|
||||
}
|
||||
|
||||
pr := blk.ParentRoot()
|
||||
bf := &dbval.BackfillStatus{
|
||||
LowSlot: uint64(wblk.Block().Slot()),
|
||||
LowRoot: blockRoot[:],
|
||||
LowParentRoot: pr[:],
|
||||
OriginRoot: blockRoot[:],
|
||||
OriginSlot: uint64(wblk.Block().Slot()),
|
||||
}
|
||||
|
||||
if err = s.SaveBackfillStatus(ctx, bf); err != nil {
|
||||
return errors.Wrap(err, "unable to save backfill status data to db for checkpoint sync")
|
||||
}
|
||||
|
||||
log.Infof("saving checkpoint block to db, w/ root=%#x", blockRoot)
|
||||
if err := s.SaveBlock(ctx, wblk); err != nil {
|
||||
return errors.Wrap(err, "could not save checkpoint block")
|
||||
|
@ -40,6 +40,7 @@ go_library(
|
||||
"//beacon-chain/operations/synccommittee:go_default_library",
|
||||
"//beacon-chain/operations/voluntaryexits:go_default_library",
|
||||
"//beacon-chain/p2p:go_default_library",
|
||||
"//beacon-chain/p2p/peers:go_default_library",
|
||||
"//beacon-chain/rpc:go_default_library",
|
||||
"//beacon-chain/slasher:go_default_library",
|
||||
"//beacon-chain/startup:go_default_library",
|
||||
@ -47,6 +48,7 @@ go_library(
|
||||
"//beacon-chain/state/stategen:go_default_library",
|
||||
"//beacon-chain/sync:go_default_library",
|
||||
"//beacon-chain/sync/backfill:go_default_library",
|
||||
"//beacon-chain/sync/backfill/coverage:go_default_library",
|
||||
"//beacon-chain/sync/checkpoint:go_default_library",
|
||||
"//beacon-chain/sync/genesis:go_default_library",
|
||||
"//beacon-chain/sync/initial-sync:go_default_library",
|
||||
|
@ -42,6 +42,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/synccommittee"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/voluntaryexits"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/peers"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/rpc"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/slasher"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
@ -49,6 +50,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state/stategen"
|
||||
regularsync "github.com/prysmaticlabs/prysm/v4/beacon-chain/sync"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill/coverage"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/checkpoint"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/genesis"
|
||||
initialsync "github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/initial-sync"
|
||||
@ -113,6 +115,7 @@ type BeaconNode struct {
|
||||
CheckpointInitializer checkpoint.Initializer
|
||||
forkChoicer forkchoice.ForkChoicer
|
||||
clockWaiter startup.ClockWaiter
|
||||
BackfillOpts []backfill.ServiceOption
|
||||
initialSyncComplete chan struct{}
|
||||
BlobStorage *filesystem.BlobStorage
|
||||
blobRetentionEpochs primitives.Epoch
|
||||
@ -215,10 +218,23 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
|
||||
return nil, err
|
||||
}
|
||||
|
||||
bfs := backfill.NewStatus(beacon.db)
|
||||
if err := bfs.Reload(ctx); err != nil {
|
||||
log.Debugln("Registering P2P Service")
|
||||
if err := beacon.registerP2P(cliCtx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
bfs, err := backfill.NewUpdater(ctx, beacon.db)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "backfill status initialization error")
|
||||
}
|
||||
pa := peers.NewAssigner(beacon.fetchP2P().Peers(), beacon.forkChoicer)
|
||||
bf, err := backfill.NewService(ctx, bfs, beacon.clockWaiter, beacon.fetchP2P(), pa, beacon.BackfillOpts...)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "error initializing backfill service")
|
||||
}
|
||||
if err := beacon.services.RegisterService(bf); err != nil {
|
||||
return nil, errors.Wrap(err, "error registering backfill service")
|
||||
}
|
||||
|
||||
log.Debugln("Starting State Gen")
|
||||
if err := beacon.startStateGen(ctx, bfs, beacon.forkChoicer); err != nil {
|
||||
@ -233,11 +249,6 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
|
||||
beacon.verifyInitWaiter = verification.NewInitializerWaiter(
|
||||
beacon.clockWaiter, forkchoice.NewROForkChoice(beacon.forkChoicer), beacon.stateGen)
|
||||
|
||||
log.Debugln("Registering P2P Service")
|
||||
if err := beacon.registerP2P(cliCtx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
log.Debugln("Registering POW Chain Service")
|
||||
if err := beacon.registerPOWChainService(); err != nil {
|
||||
return nil, err
|
||||
@ -534,8 +545,8 @@ func (b *BeaconNode) startSlasherDB(cliCtx *cli.Context) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *BeaconNode) startStateGen(ctx context.Context, bfs *backfill.Status, fc forkchoice.ForkChoicer) error {
|
||||
opts := []stategen.Option{stategen.WithBackfillStatus(bfs)}
|
||||
func (b *BeaconNode) startStateGen(ctx context.Context, bfs coverage.AvailableBlocker, fc forkchoice.ForkChoicer) error {
|
||||
opts := []stategen.Option{stategen.WithAvailableBlocker(bfs)}
|
||||
sg := stategen.New(b.db, fc, opts...)
|
||||
|
||||
cp, err := b.db.FinalizedCheckpoint(ctx)
|
||||
|
@ -3,6 +3,7 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = [
|
||||
"assigner.go",
|
||||
"log.go",
|
||||
"status.go",
|
||||
],
|
||||
@ -12,8 +13,10 @@ go_library(
|
||||
"//cmd:__subpackages__",
|
||||
],
|
||||
deps = [
|
||||
"//beacon-chain/forkchoice/types:go_default_library",
|
||||
"//beacon-chain/p2p/peers/peerdata:go_default_library",
|
||||
"//beacon-chain/p2p/peers/scorers:go_default_library",
|
||||
"//cmd/beacon-chain/flags:go_default_library",
|
||||
"//config/features:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
@ -28,6 +31,7 @@ go_library(
|
||||
"@com_github_libp2p_go_libp2p//core/peer:go_default_library",
|
||||
"@com_github_multiformats_go_multiaddr//:go_default_library",
|
||||
"@com_github_multiformats_go_multiaddr//net:go_default_library",
|
||||
"@com_github_pkg_errors//:go_default_library",
|
||||
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
|
||||
"@com_github_sirupsen_logrus//:go_default_library",
|
||||
],
|
||||
@ -36,6 +40,7 @@ go_library(
|
||||
go_test(
|
||||
name = "go_default_test",
|
||||
srcs = [
|
||||
"assigner_test.go",
|
||||
"benchmark_test.go",
|
||||
"peers_test.go",
|
||||
"status_test.go",
|
||||
|
78
beacon-chain/p2p/peers/assigner.go
Normal file
78
beacon-chain/p2p/peers/assigner.go
Normal file
@ -0,0 +1,78 @@
|
||||
package peers
|
||||
|
||||
import (
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
forkchoicetypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice/types"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// FinalizedCheckpointer describes the minimum capability that Assigner needs from forkchoice.
|
||||
// That is, the ability to retrieve the latest finalized checkpoint to help with peer evaluation.
|
||||
type FinalizedCheckpointer interface {
|
||||
FinalizedCheckpoint() *forkchoicetypes.Checkpoint
|
||||
}
|
||||
|
||||
// NewAssigner assists in the correct construction of an Assigner by code in other packages,
|
||||
// assuring all the important private member fields are given values.
|
||||
// The FinalizedCheckpointer is used to retrieve the latest finalized checkpoint each time peers are requested.
|
||||
// Peers that report an older finalized checkpoint are filtered out.
|
||||
func NewAssigner(s *Status, fc FinalizedCheckpointer) *Assigner {
|
||||
return &Assigner{
|
||||
ps: s,
|
||||
fc: fc,
|
||||
}
|
||||
}
|
||||
|
||||
// Assigner uses the "BestFinalized" peer scoring method to pick the next-best peer to receive rpc requests.
|
||||
type Assigner struct {
|
||||
ps *Status
|
||||
fc FinalizedCheckpointer
|
||||
}
|
||||
|
||||
// ErrInsufficientSuitable is a sentinel error, signaling that a peer couldn't be assigned because there are currently
|
||||
// not enough peers that match our selection criteria to serve rpc requests. It is the responsibility of the caller to
|
||||
// look for this error and continue to try calling Assign with appropriate backoff logic.
|
||||
var ErrInsufficientSuitable = errors.New("no suitable peers")
|
||||
|
||||
func (a *Assigner) freshPeers() ([]peer.ID, error) {
|
||||
required := params.BeaconConfig().MaxPeersToSync
|
||||
if flags.Get().MinimumSyncPeers < required {
|
||||
required = flags.Get().MinimumSyncPeers
|
||||
}
|
||||
_, peers := a.ps.BestFinalized(params.BeaconConfig().MaxPeersToSync, a.fc.FinalizedCheckpoint().Epoch)
|
||||
if len(peers) < required {
|
||||
log.WithFields(logrus.Fields{
|
||||
"suitable": len(peers),
|
||||
"required": required}).Warn("Unable to assign peer while suitable peers < required ")
|
||||
return nil, ErrInsufficientSuitable
|
||||
}
|
||||
return peers, nil
|
||||
}
|
||||
|
||||
// Assign uses the "BestFinalized" method to select the best peers that agree on a canonical block
|
||||
// for the configured finalized epoch. At most `n` peers will be returned. The `busy` param can be used
|
||||
// to filter out peers that we know we don't want to connect to, for instance if we are trying to limit
|
||||
// the number of outbound requests to each peer from a given component.
|
||||
func (a *Assigner) Assign(busy map[peer.ID]bool, n int) ([]peer.ID, error) {
|
||||
best, err := a.freshPeers()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return pickBest(busy, n, best), nil
|
||||
}
|
||||
|
||||
func pickBest(busy map[peer.ID]bool, n int, best []peer.ID) []peer.ID {
|
||||
ps := make([]peer.ID, 0, n)
|
||||
for _, p := range best {
|
||||
if len(ps) == n {
|
||||
return ps
|
||||
}
|
||||
if !busy[p] {
|
||||
ps = append(ps, p)
|
||||
}
|
||||
}
|
||||
return ps
|
||||
}
|
114
beacon-chain/p2p/peers/assigner_test.go
Normal file
114
beacon-chain/p2p/peers/assigner_test.go
Normal file
@ -0,0 +1,114 @@
|
||||
package peers
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
)
|
||||
|
||||
func TestPickBest(t *testing.T) {
|
||||
best := testPeerIds(10)
|
||||
cases := []struct {
|
||||
name string
|
||||
busy map[peer.ID]bool
|
||||
n int
|
||||
best []peer.ID
|
||||
expected []peer.ID
|
||||
}{
|
||||
{
|
||||
name: "",
|
||||
n: 0,
|
||||
},
|
||||
{
|
||||
name: "none busy",
|
||||
n: 1,
|
||||
expected: best[0:1],
|
||||
},
|
||||
{
|
||||
name: "all busy except last",
|
||||
n: 1,
|
||||
busy: testBusyMap(best[0 : len(best)-1]),
|
||||
expected: best[len(best)-1:],
|
||||
},
|
||||
{
|
||||
name: "all busy except i=5",
|
||||
n: 1,
|
||||
busy: testBusyMap(append(append([]peer.ID{}, best[0:5]...), best[6:]...)),
|
||||
expected: []peer.ID{best[5]},
|
||||
},
|
||||
{
|
||||
name: "all busy - 0 results",
|
||||
n: 1,
|
||||
busy: testBusyMap(best),
|
||||
},
|
||||
{
|
||||
name: "first half busy",
|
||||
n: 5,
|
||||
busy: testBusyMap(best[0:5]),
|
||||
expected: best[5:],
|
||||
},
|
||||
{
|
||||
name: "back half busy",
|
||||
n: 5,
|
||||
busy: testBusyMap(best[5:]),
|
||||
expected: best[0:5],
|
||||
},
|
||||
{
|
||||
name: "pick all ",
|
||||
n: 10,
|
||||
expected: best,
|
||||
},
|
||||
{
|
||||
name: "none available",
|
||||
n: 10,
|
||||
best: []peer.ID{},
|
||||
},
|
||||
{
|
||||
name: "not enough",
|
||||
n: 10,
|
||||
best: best[0:1],
|
||||
expected: best[0:1],
|
||||
},
|
||||
{
|
||||
name: "not enough, some busy",
|
||||
n: 10,
|
||||
best: best[0:6],
|
||||
busy: testBusyMap(best[0:5]),
|
||||
expected: best[5:6],
|
||||
},
|
||||
}
|
||||
for _, c := range cases {
|
||||
name := fmt.Sprintf("n=%d", c.n)
|
||||
if c.name != "" {
|
||||
name += " " + c.name
|
||||
}
|
||||
t.Run(name, func(t *testing.T) {
|
||||
if c.best == nil {
|
||||
c.best = best
|
||||
}
|
||||
pb := pickBest(c.busy, c.n, c.best)
|
||||
require.Equal(t, len(c.expected), len(pb))
|
||||
for i := range c.expected {
|
||||
require.Equal(t, c.expected[i], pb[i])
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func testBusyMap(b []peer.ID) map[peer.ID]bool {
|
||||
m := make(map[peer.ID]bool)
|
||||
for i := range b {
|
||||
m[b[i]] = true
|
||||
}
|
||||
return m
|
||||
}
|
||||
|
||||
func testPeerIds(n int) []peer.ID {
|
||||
pids := make([]peer.ID, n)
|
||||
for i := range pids {
|
||||
pids[i] = peer.ID(fmt.Sprintf("%d", i))
|
||||
}
|
||||
return pids
|
||||
}
|
@ -121,6 +121,7 @@ type ReadOnlyValidators interface {
|
||||
ValidatorAtIndex(idx primitives.ValidatorIndex) (*ethpb.Validator, error)
|
||||
ValidatorAtIndexReadOnly(idx primitives.ValidatorIndex) (ReadOnlyValidator, error)
|
||||
ValidatorIndexByPubkey(key [fieldparams.BLSPubkeyLength]byte) (primitives.ValidatorIndex, bool)
|
||||
PublicKeys() ([][fieldparams.BLSPubkeyLength]byte, error)
|
||||
PubkeyAtIndex(idx primitives.ValidatorIndex) [fieldparams.BLSPubkeyLength]byte
|
||||
NumValidators() int
|
||||
ReadFromEveryValidator(f func(idx int, val ReadOnlyValidator) error) error
|
||||
|
@ -181,6 +181,27 @@ func (b *BeaconState) PubkeyAtIndex(idx primitives.ValidatorIndex) [fieldparams.
|
||||
return bytesutil.ToBytes48(v.PublicKey)
|
||||
}
|
||||
|
||||
// PublicKeys builds a list of all validator public keys, with each key's index aligned to its validator index.
|
||||
func (b *BeaconState) PublicKeys() ([][fieldparams.BLSPubkeyLength]byte, error) {
|
||||
b.lock.RLock()
|
||||
defer b.lock.RUnlock()
|
||||
|
||||
l := b.validatorsLen()
|
||||
res := make([][fieldparams.BLSPubkeyLength]byte, l)
|
||||
for i := 0; i < l; i++ {
|
||||
if features.Get().EnableExperimentalState {
|
||||
val, err := b.validatorsMultiValue.At(b, uint64(i))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
copy(res[i][:], val.PublicKey)
|
||||
} else {
|
||||
copy(res[i][:], b.validators[i].PublicKey)
|
||||
}
|
||||
}
|
||||
return res, nil
|
||||
}
|
||||
|
||||
// NumValidators returns the size of the validator registry.
|
||||
func (b *BeaconState) NumValidators() int {
|
||||
b.lock.RLock()
|
||||
|
@ -31,7 +31,7 @@ go_library(
|
||||
"//beacon-chain/db/filters:go_default_library",
|
||||
"//beacon-chain/forkchoice:go_default_library",
|
||||
"//beacon-chain/state:go_default_library",
|
||||
"//beacon-chain/sync/backfill:go_default_library",
|
||||
"//beacon-chain/sync/backfill/coverage:go_default_library",
|
||||
"//cache/lru:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
|
@ -332,9 +332,8 @@ func (s *State) CombinedCache() *CombinedCache {
|
||||
}
|
||||
|
||||
func (s *State) slotAvailable(slot primitives.Slot) bool {
|
||||
// default to assuming node was initialized from genesis - backfill only needs to be specified for checkpoint sync
|
||||
if s.backfillStatus == nil {
|
||||
if s.avb == nil {
|
||||
return true
|
||||
}
|
||||
return s.backfillStatus.SlotCovered(slot)
|
||||
return s.avb.AvailableBlock(slot)
|
||||
}
|
||||
|
@ -13,7 +13,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill/coverage"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/crypto/bls"
|
||||
@ -51,7 +51,7 @@ type State struct {
|
||||
finalizedInfo *finalizedInfo
|
||||
epochBoundaryStateCache *epochBoundaryState
|
||||
saveHotStateDB *saveHotStateDbConfig
|
||||
backfillStatus *backfill.Status
|
||||
avb coverage.AvailableBlocker
|
||||
migrationLock *sync.Mutex
|
||||
fc forkchoice.ForkChoicer
|
||||
}
|
||||
@ -78,9 +78,11 @@ type finalizedInfo struct {
|
||||
// Option is a functional option for controlling the initialization of a *State value
|
||||
type Option func(*State)
|
||||
|
||||
func WithBackfillStatus(bfs *backfill.Status) Option {
|
||||
// WithAvailableBlocker gives stategen an AvailableBlocker, which is used to determine if a given
|
||||
// block is available. This is necessary because backfill creates a hole in the block history.
|
||||
func WithAvailableBlocker(avb coverage.AvailableBlocker) Option {
|
||||
return func(sg *State) {
|
||||
sg.backfillStatus = bfs
|
||||
sg.avb = avb
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2,30 +2,78 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
|
||||
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = ["status.go"],
|
||||
srcs = [
|
||||
"batch.go",
|
||||
"batcher.go",
|
||||
"metrics.go",
|
||||
"pool.go",
|
||||
"service.go",
|
||||
"status.go",
|
||||
"verify.go",
|
||||
"worker.go",
|
||||
],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//beacon-chain/core/helpers:go_default_library",
|
||||
"//beacon-chain/core/signing:go_default_library",
|
||||
"//beacon-chain/db:go_default_library",
|
||||
"//beacon-chain/p2p:go_default_library",
|
||||
"//beacon-chain/p2p/peers:go_default_library",
|
||||
"//beacon-chain/startup:go_default_library",
|
||||
"//beacon-chain/state:go_default_library",
|
||||
"//beacon-chain/sync:go_default_library",
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//crypto/bls:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//network/forks:go_default_library",
|
||||
"//proto/dbval:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//runtime:go_default_library",
|
||||
"//time/slots:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p//core/peer:go_default_library",
|
||||
"@com_github_pkg_errors//:go_default_library",
|
||||
"@com_github_prometheus_client_golang//prometheus:go_default_library",
|
||||
"@com_github_prometheus_client_golang//prometheus/promauto:go_default_library",
|
||||
"@com_github_sirupsen_logrus//:go_default_library",
|
||||
],
|
||||
)
|
||||
|
||||
go_test(
|
||||
name = "go_default_test",
|
||||
srcs = ["status_test.go"],
|
||||
srcs = [
|
||||
"batcher_test.go",
|
||||
"pool_test.go",
|
||||
"service_test.go",
|
||||
"status_test.go",
|
||||
"verify_test.go",
|
||||
],
|
||||
embed = [":go_default_library"],
|
||||
deps = [
|
||||
"//beacon-chain/core/signing:go_default_library",
|
||||
"//beacon-chain/db:go_default_library",
|
||||
"//beacon-chain/p2p/testing:go_default_library",
|
||||
"//beacon-chain/startup:go_default_library",
|
||||
"//beacon-chain/state:go_default_library",
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
"//consensus-types/blocks/testing:go_default_library",
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//crypto/bls:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//network/forks:go_default_library",
|
||||
"//proto/dbval:go_default_library",
|
||||
"//runtime/interop:go_default_library",
|
||||
"//testing/require:go_default_library",
|
||||
"//testing/util:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
|
||||
"@com_github_libp2p_go_libp2p//core/peer:go_default_library",
|
||||
"@com_github_pkg_errors//:go_default_library",
|
||||
],
|
||||
)
|
||||
|
141
beacon-chain/sync/backfill/batch.go
Normal file
141
beacon-chain/sync/backfill/batch.go
Normal file
@ -0,0 +1,141 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// ErrChainBroken indicates a backfill batch can't be imported to the db because it is not known to be the ancestor
|
||||
// of the canonical chain.
|
||||
var ErrChainBroken = errors.New("batch is not the ancestor of a known finalized root")
|
||||
|
||||
type batchState int
|
||||
|
||||
func (s batchState) String() string {
|
||||
switch s {
|
||||
case batchNil:
|
||||
return "nil"
|
||||
case batchInit:
|
||||
return "init"
|
||||
case batchSequenced:
|
||||
return "sequenced"
|
||||
case batchErrRetryable:
|
||||
return "error_retryable"
|
||||
case batchImportable:
|
||||
return "importable"
|
||||
case batchImportComplete:
|
||||
return "import_complete"
|
||||
case batchEndSequence:
|
||||
return "end_sequence"
|
||||
default:
|
||||
return "unknown"
|
||||
}
|
||||
}
|
||||
|
||||
const (
|
||||
batchNil batchState = iota
|
||||
batchInit
|
||||
batchSequenced
|
||||
batchErrRetryable
|
||||
batchImportable
|
||||
batchImportComplete
|
||||
batchEndSequence
|
||||
)
|
||||
|
||||
type batchId string
|
||||
|
||||
type batch struct {
|
||||
firstScheduled time.Time
|
||||
scheduled time.Time
|
||||
seq int // sequence identifier, ie how many times has the sequence() method served this batch
|
||||
retries int
|
||||
begin primitives.Slot
|
||||
end primitives.Slot // half-open interval, [begin, end), ie >= start, < end.
|
||||
results VerifiedROBlocks
|
||||
err error
|
||||
state batchState
|
||||
pid peer.ID
|
||||
}
|
||||
|
||||
func (b batch) logFields() log.Fields {
|
||||
return map[string]interface{}{
|
||||
"batch_id": b.id(),
|
||||
"state": b.state.String(),
|
||||
"scheduled": b.scheduled.String(),
|
||||
"seq": b.seq,
|
||||
"retries": b.retries,
|
||||
"begin": b.begin,
|
||||
"end": b.end,
|
||||
"pid": b.pid,
|
||||
}
|
||||
}
|
||||
|
||||
func (b batch) replaces(r batch) bool {
|
||||
if r.state == batchImportComplete {
|
||||
return false
|
||||
}
|
||||
if b.begin != r.begin {
|
||||
return false
|
||||
}
|
||||
if b.end != r.end {
|
||||
return false
|
||||
}
|
||||
return b.seq >= r.seq
|
||||
}
|
||||
|
||||
func (b batch) id() batchId {
|
||||
return batchId(fmt.Sprintf("%d:%d", b.begin, b.end))
|
||||
}
|
||||
|
||||
func (b batch) ensureParent(expected [32]byte) error {
|
||||
tail := b.results[len(b.results)-1]
|
||||
if tail.Root() != expected {
|
||||
return errors.Wrapf(ErrChainBroken, "last parent_root=%#x, tail root=%#x", expected, tail.Root())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b batch) request() *eth.BeaconBlocksByRangeRequest {
|
||||
return ð.BeaconBlocksByRangeRequest{
|
||||
StartSlot: b.begin,
|
||||
Count: uint64(b.end - b.begin),
|
||||
Step: 1,
|
||||
}
|
||||
}
|
||||
|
||||
func (b batch) withState(s batchState) batch {
|
||||
if s == batchSequenced {
|
||||
b.scheduled = time.Now()
|
||||
switch b.state {
|
||||
case batchErrRetryable:
|
||||
b.retries += 1
|
||||
log.WithFields(b.logFields()).Info("sequencing batch for retry")
|
||||
case batchInit, batchNil:
|
||||
b.firstScheduled = b.scheduled
|
||||
}
|
||||
}
|
||||
if s == batchImportComplete {
|
||||
backfillBatchTimeRoundtrip.Observe(float64(time.Since(b.firstScheduled).Milliseconds()))
|
||||
log.WithFields(b.logFields()).Debug("Backfill batch imported.")
|
||||
}
|
||||
b.state = s
|
||||
b.seq += 1
|
||||
return b
|
||||
}
|
||||
|
||||
func (b batch) withPeer(p peer.ID) batch {
|
||||
b.pid = p
|
||||
backfillBatchTimeWaiting.Observe(float64(time.Since(b.scheduled).Milliseconds()))
|
||||
return b
|
||||
}
|
||||
|
||||
func (b batch) withRetryableError(err error) batch {
|
||||
b.err = err
|
||||
return b.withState(batchErrRetryable)
|
||||
}
|
197
beacon-chain/sync/backfill/batcher.go
Normal file
197
beacon-chain/sync/backfill/batcher.go
Normal file
@ -0,0 +1,197 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
)
|
||||
|
||||
var errMaxBatches = errors.New("backfill batch requested in excess of max outstanding batches")
|
||||
var errEndSequence = errors.New("sequence has terminated, no more backfill batches will be produced")
|
||||
var errCannotDecreaseMinimum = errors.New("the minimum backfill slot can only be increased, not decreased")
|
||||
|
||||
type batchSequencer struct {
|
||||
batcher batcher
|
||||
seq []batch
|
||||
}
|
||||
|
||||
// sequence() is meant as a verb "arrange in a particular order".
|
||||
// sequence determines the next set of batches that should be worked on based on the state of the batches
|
||||
// in its internal view. sequence relies on update() for updates to its view of the
|
||||
// batches it has previously sequenced.
|
||||
func (c *batchSequencer) sequence() ([]batch, error) {
|
||||
s := make([]batch, 0)
|
||||
// batch start slots are in descending order, c.seq[n].begin == c.seq[n+1].end
|
||||
for i := range c.seq {
|
||||
switch c.seq[i].state {
|
||||
case batchInit, batchErrRetryable:
|
||||
c.seq[i] = c.seq[i].withState(batchSequenced)
|
||||
s = append(s, c.seq[i])
|
||||
case batchNil:
|
||||
// batchNil is the zero value of the batch type.
|
||||
// This case means that we are initializing a batch that was created by the
|
||||
// initial allocation of the seq slice, so batcher need to compute its bounds.
|
||||
var b batch
|
||||
if i == 0 {
|
||||
// The first item in the list is a special case, subsequent items are initialized
|
||||
// relative to the preceding batches.
|
||||
b = c.batcher.before(c.batcher.max)
|
||||
} else {
|
||||
b = c.batcher.beforeBatch(c.seq[i-1])
|
||||
}
|
||||
c.seq[i] = b.withState(batchSequenced)
|
||||
s = append(s, c.seq[i])
|
||||
case batchEndSequence:
|
||||
if len(s) == 0 {
|
||||
s = append(s, c.seq[i])
|
||||
}
|
||||
default:
|
||||
continue
|
||||
}
|
||||
}
|
||||
if len(s) == 0 {
|
||||
return nil, errMaxBatches
|
||||
}
|
||||
|
||||
return s, nil
|
||||
}
|
||||
|
||||
// update serves 2 roles.
|
||||
// - updating batchSequencer's copy of the given batch.
|
||||
// - removing batches that are completely imported from the sequence,
|
||||
// so that they are not returned the next time import() is called, and populating
|
||||
// seq with new batches that are ready to be worked on.
|
||||
func (c *batchSequencer) update(b batch) {
|
||||
done := 0
|
||||
for i := 0; i < len(c.seq); i++ {
|
||||
if b.replaces(c.seq[i]) {
|
||||
c.seq[i] = b
|
||||
}
|
||||
// Assumes invariant that batches complete and update is called in order.
|
||||
// This should be true because the code using the sequencer doesn't know the expected parent
|
||||
// for a batch until it imports the previous batch.
|
||||
if c.seq[i].state == batchImportComplete {
|
||||
done += 1
|
||||
continue
|
||||
}
|
||||
// Move the unfinished batches to overwrite the finished ones.
|
||||
// eg consider [a,b,c,d,e] where a,b are done
|
||||
// when i==2, done==2 (since done was incremented for a and b)
|
||||
// so we want to copy c to a, then on i=3, d to b, then on i=4 e to c.
|
||||
c.seq[i-done] = c.seq[i]
|
||||
}
|
||||
// Overwrite the moved batches with the next ones in the sequence.
|
||||
// Continuing the example in the comment above, len(c.seq)==5, done=2, so i=3.
|
||||
// We want to replace index 3 with the batch that should be processed after index 2,
|
||||
// which was previously the earliest known batch, and index 4 with the batch that should
|
||||
// be processed after index 3, the new earliest batch.
|
||||
for i := len(c.seq) - done; i < len(c.seq); i++ {
|
||||
c.seq[i] = c.batcher.beforeBatch(c.seq[i-1])
|
||||
}
|
||||
}
|
||||
|
||||
// importable returns all batches that are ready to be imported. This means they satisfy 2 conditions:
|
||||
// - They are in state batchImportable, which means their data has been downloaded and proposer signatures have been verified.
|
||||
// - There are no batches that are not in state batchImportable between them and the start of the slice. This ensures that they
|
||||
// can be connected to the canonical chain, either because the root of the last block in the batch matches the parent_root of
|
||||
// the oldest block in the canonical chain, or because the root of the last block in the batch matches the parent_root of the
|
||||
// new block preceding them in the slice (which must connect to the batch before it, or to the canonical chain if it is first).
|
||||
func (c *batchSequencer) importable() []batch {
|
||||
imp := make([]batch, 0)
|
||||
for i := range c.seq {
|
||||
if c.seq[i].state == batchImportable {
|
||||
imp = append(imp, c.seq[i])
|
||||
continue
|
||||
}
|
||||
// as soon as we hit a batch with a different state, we return everything leading to it.
|
||||
// If the first element isn't importable, we'll return an empty slice.
|
||||
break
|
||||
}
|
||||
return imp
|
||||
}
|
||||
|
||||
// moveMinimum enables the backfill service to change the slot where the batcher will start replying with
|
||||
// batch state batchEndSequence (signaling that no new batches will be produced). This is done in response to
|
||||
// epochs advancing, which shrinks the gap between <checkpoint slot> and <current slot>-MIN_EPOCHS_FOR_BLOCK_REQUESTS,
|
||||
// allowing the node to download a smaller number of blocks.
|
||||
func (c *batchSequencer) moveMinimum(min primitives.Slot) error {
|
||||
if min < c.batcher.min {
|
||||
return errCannotDecreaseMinimum
|
||||
}
|
||||
c.batcher.min = min
|
||||
return nil
|
||||
}
|
||||
|
||||
// countWithState provides a view into how many batches are in a particular state
|
||||
// to be used for logging or metrics purposes.
|
||||
func (c *batchSequencer) countWithState(s batchState) int {
|
||||
n := 0
|
||||
for i := 0; i < len(c.seq); i++ {
|
||||
if c.seq[i].state == s {
|
||||
n += 1
|
||||
}
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
// numTodo computes the number of remaining batches for metrics and logging purposes.
|
||||
func (c *batchSequencer) numTodo() int {
|
||||
if len(c.seq) == 0 {
|
||||
return 0
|
||||
}
|
||||
lowest := c.seq[len(c.seq)-1]
|
||||
todo := 0
|
||||
if lowest.state != batchEndSequence {
|
||||
todo = c.batcher.remaining(lowest.begin)
|
||||
}
|
||||
for _, b := range c.seq {
|
||||
switch b.state {
|
||||
case batchEndSequence, batchImportComplete, batchNil:
|
||||
continue
|
||||
default:
|
||||
todo += 1
|
||||
}
|
||||
}
|
||||
return todo
|
||||
}
|
||||
|
||||
func newBatchSequencer(seqLen int, min, max, size primitives.Slot) *batchSequencer {
|
||||
b := batcher{min: min, max: max, size: size}
|
||||
seq := make([]batch, seqLen)
|
||||
return &batchSequencer{batcher: b, seq: seq}
|
||||
}
|
||||
|
||||
type batcher struct {
|
||||
min primitives.Slot
|
||||
max primitives.Slot
|
||||
size primitives.Slot
|
||||
}
|
||||
|
||||
func (r batcher) remaining(upTo primitives.Slot) int {
|
||||
if r.min >= upTo {
|
||||
return 0
|
||||
}
|
||||
delta := upTo - r.min
|
||||
if delta%r.size != 0 {
|
||||
return int(delta/r.size) + 1
|
||||
}
|
||||
return int(delta / r.size)
|
||||
}
|
||||
|
||||
func (r batcher) beforeBatch(upTo batch) batch {
|
||||
return r.before(upTo.begin)
|
||||
}
|
||||
|
||||
func (r batcher) before(upTo primitives.Slot) batch {
|
||||
// upTo is an exclusive upper bound. Requesting a batch before the lower bound of backfill signals the end of the
|
||||
// backfill process.
|
||||
if upTo <= r.min {
|
||||
return batch{begin: upTo, end: upTo, state: batchEndSequence}
|
||||
}
|
||||
begin := r.min
|
||||
if upTo > r.size+r.min {
|
||||
begin = upTo - r.size
|
||||
}
|
||||
|
||||
// batch.end is exclusive, .begin is inclusive, so the prev.end = next.begin
|
||||
return batch{begin: begin, end: upTo, state: batchInit}
|
||||
}
|
208
beacon-chain/sync/backfill/batcher_test.go
Normal file
208
beacon-chain/sync/backfill/batcher_test.go
Normal file
@ -0,0 +1,208 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
)
|
||||
|
||||
func TestBatcherBefore(t *testing.T) {
|
||||
cases := []struct {
|
||||
name string
|
||||
b batcher
|
||||
upTo []primitives.Slot
|
||||
expect []batch
|
||||
}{
|
||||
{
|
||||
name: "size 10",
|
||||
b: batcher{min: 0, size: 10},
|
||||
upTo: []primitives.Slot{33, 30, 10, 6},
|
||||
expect: []batch{
|
||||
{begin: 23, end: 33, state: batchInit},
|
||||
{begin: 20, end: 30, state: batchInit},
|
||||
{begin: 0, end: 10, state: batchInit},
|
||||
{begin: 0, end: 6, state: batchInit},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "size 4",
|
||||
b: batcher{min: 0, size: 4},
|
||||
upTo: []primitives.Slot{33, 6, 4},
|
||||
expect: []batch{
|
||||
{begin: 29, end: 33, state: batchInit},
|
||||
{begin: 2, end: 6, state: batchInit},
|
||||
{begin: 0, end: 4, state: batchInit},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "trigger end",
|
||||
b: batcher{min: 20, size: 10},
|
||||
upTo: []primitives.Slot{33, 30, 25, 21, 20, 19},
|
||||
expect: []batch{
|
||||
{begin: 23, end: 33, state: batchInit},
|
||||
{begin: 20, end: 30, state: batchInit},
|
||||
{begin: 20, end: 25, state: batchInit},
|
||||
{begin: 20, end: 21, state: batchInit},
|
||||
{begin: 20, end: 20, state: batchEndSequence},
|
||||
{begin: 19, end: 19, state: batchEndSequence},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, c := range cases {
|
||||
for i := range c.upTo {
|
||||
upTo := c.upTo[i]
|
||||
expect := c.expect[i]
|
||||
t.Run(fmt.Sprintf("%s upTo %d", c.name, upTo), func(t *testing.T) {
|
||||
got := c.b.before(upTo)
|
||||
require.Equal(t, expect.begin, got.begin)
|
||||
require.Equal(t, expect.end, got.end)
|
||||
require.Equal(t, expect.state, got.state)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestBatchSequencer(t *testing.T) {
|
||||
var min, max, size primitives.Slot
|
||||
seqLen := 8
|
||||
min = 0
|
||||
max = 11235
|
||||
size = 64
|
||||
seq := newBatchSequencer(seqLen, min, max, size)
|
||||
expected := []batch{
|
||||
{begin: 11171, end: 11235},
|
||||
{begin: 11107, end: 11171},
|
||||
{begin: 11043, end: 11107},
|
||||
{begin: 10979, end: 11043},
|
||||
{begin: 10915, end: 10979},
|
||||
{begin: 10851, end: 10915},
|
||||
{begin: 10787, end: 10851},
|
||||
{begin: 10723, end: 10787},
|
||||
}
|
||||
got, err := seq.sequence()
|
||||
require.Equal(t, seqLen, len(got))
|
||||
for i := 0; i < seqLen; i++ {
|
||||
g := got[i]
|
||||
exp := expected[i]
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, exp.begin, g.begin)
|
||||
require.Equal(t, exp.end, g.end)
|
||||
require.Equal(t, batchSequenced, g.state)
|
||||
}
|
||||
// This should give us the error indicating there are too many outstanding batches.
|
||||
_, err = seq.sequence()
|
||||
require.ErrorIs(t, err, errMaxBatches)
|
||||
|
||||
// mark the last batch completed so we can call sequence again.
|
||||
last := seq.seq[len(seq.seq)-1]
|
||||
// With this state, the batch should get served back to us as the next batch.
|
||||
last.state = batchErrRetryable
|
||||
seq.update(last)
|
||||
nextS, err := seq.sequence()
|
||||
require.Equal(t, 1, len(nextS))
|
||||
next := nextS[0]
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, last.begin, next.begin)
|
||||
require.Equal(t, last.end, next.end)
|
||||
// sequence() should replace the batchErrRetryable state with batchSequenced.
|
||||
require.Equal(t, batchSequenced, next.state)
|
||||
|
||||
// No batches have been marked importable.
|
||||
require.Equal(t, 0, len(seq.importable()))
|
||||
|
||||
// Mark our batch importable and make sure it shows up in the list of importable batches.
|
||||
next.state = batchImportable
|
||||
seq.update(next)
|
||||
require.Equal(t, 0, len(seq.importable()))
|
||||
first := seq.seq[0]
|
||||
first.state = batchImportable
|
||||
seq.update(first)
|
||||
require.Equal(t, 1, len(seq.importable()))
|
||||
require.Equal(t, len(seq.seq), seqLen)
|
||||
// change the last element back to batchInit so that the importable test stays simple
|
||||
last = seq.seq[len(seq.seq)-1]
|
||||
last.state = batchInit
|
||||
seq.update(last)
|
||||
// ensure that the number of importable elements grows as the list is marked importable
|
||||
for i := 0; i < len(seq.seq); i++ {
|
||||
seq.seq[i].state = batchImportable
|
||||
require.Equal(t, i+1, len(seq.importable()))
|
||||
}
|
||||
// reset everything to init
|
||||
for i := 0; i < len(seq.seq); i++ {
|
||||
seq.seq[i].state = batchInit
|
||||
require.Equal(t, 0, len(seq.importable()))
|
||||
}
|
||||
// loop backwards and make sure importable is zero until the first element is importable
|
||||
for i := len(seq.seq) - 1; i > 0; i-- {
|
||||
seq.seq[i].state = batchImportable
|
||||
require.Equal(t, 0, len(seq.importable()))
|
||||
}
|
||||
seq.seq[0].state = batchImportable
|
||||
require.Equal(t, len(seq.seq), len(seq.importable()))
|
||||
|
||||
// reset everything to init again
|
||||
for i := 0; i < len(seq.seq); i++ {
|
||||
seq.seq[i].state = batchInit
|
||||
require.Equal(t, 0, len(seq.importable()))
|
||||
}
|
||||
// set first 3 elements to importable. we should see them in the result for importable()
|
||||
// and be able to use update to cycle them away.
|
||||
seq.seq[0].state, seq.seq[1].state, seq.seq[2].state = batchImportable, batchImportable, batchImportable
|
||||
require.Equal(t, 3, len(seq.importable()))
|
||||
a, b, c, z := seq.seq[0], seq.seq[1], seq.seq[2], seq.seq[3]
|
||||
require.NotEqual(t, z.begin, seq.seq[2].begin)
|
||||
require.NotEqual(t, z.begin, seq.seq[1].begin)
|
||||
require.NotEqual(t, z.begin, seq.seq[0].begin)
|
||||
a.state, b.state, c.state = batchImportComplete, batchImportComplete, batchImportComplete
|
||||
seq.update(a)
|
||||
|
||||
// follow z as it moves down the chain to the first spot
|
||||
require.Equal(t, z.begin, seq.seq[2].begin)
|
||||
require.NotEqual(t, z.begin, seq.seq[1].begin)
|
||||
require.NotEqual(t, z.begin, seq.seq[0].begin)
|
||||
seq.update(b)
|
||||
require.NotEqual(t, z.begin, seq.seq[2].begin)
|
||||
require.Equal(t, z.begin, seq.seq[1].begin)
|
||||
require.NotEqual(t, z.begin, seq.seq[0].begin)
|
||||
seq.update(c)
|
||||
require.NotEqual(t, z.begin, seq.seq[2].begin)
|
||||
require.NotEqual(t, z.begin, seq.seq[1].begin)
|
||||
require.Equal(t, z.begin, seq.seq[0].begin)
|
||||
|
||||
// Check integrity of begin/end alignment across the sequence.
|
||||
// Also update all the states to sequenced for the convenience of the next test.
|
||||
for i := 1; i < len(seq.seq); i++ {
|
||||
require.Equal(t, seq.seq[i].end, seq.seq[i-1].begin)
|
||||
// won't touch the first element, which is fine because it is marked complete below.
|
||||
seq.seq[i].state = batchSequenced
|
||||
}
|
||||
|
||||
// set the min for the batcher close to the lowest slot. This will force the next batch to be partial and the batch
|
||||
// after that to be the final batch.
|
||||
newMin := seq.seq[len(seq.seq)-1].begin - 30
|
||||
seq.batcher.min = newMin
|
||||
first = seq.seq[0]
|
||||
first.state = batchImportComplete
|
||||
// update() with a complete state will cause the sequence to be extended with an additional batch
|
||||
seq.update(first)
|
||||
lastS, err := seq.sequence()
|
||||
last = lastS[0]
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, newMin, last.begin)
|
||||
require.Equal(t, seq.seq[len(seq.seq)-2].begin, last.end)
|
||||
|
||||
// Mark first batch done again, this time check that sequence() gives errEndSequence.
|
||||
first = seq.seq[0]
|
||||
first.state = batchImportComplete
|
||||
// update() with a complete state will cause the sequence to be extended with an additional batch
|
||||
seq.update(first)
|
||||
endExp, err := seq.sequence()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, 1, len(endExp))
|
||||
end := endExp[0]
|
||||
//require.ErrorIs(t, err, errEndSequence)
|
||||
require.Equal(t, batchEndSequence, end.state)
|
||||
}
|
9
beacon-chain/sync/backfill/coverage/BUILD.bazel
Normal file
9
beacon-chain/sync/backfill/coverage/BUILD.bazel
Normal file
@ -0,0 +1,9 @@
|
||||
load("@prysm//tools/go:def.bzl", "go_library")
|
||||
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = ["coverage.go"],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill/coverage",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = ["//consensus-types/primitives:go_default_library"],
|
||||
)
|
9
beacon-chain/sync/backfill/coverage/coverage.go
Normal file
9
beacon-chain/sync/backfill/coverage/coverage.go
Normal file
@ -0,0 +1,9 @@
|
||||
package coverage
|
||||
|
||||
import "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
|
||||
// AvailableBlocker can be used to check whether there is a finalized block in the db for the given slot.
|
||||
// This interface is typically fulfilled by backfill.Store.
|
||||
type AvailableBlocker interface {
|
||||
AvailableBlock(primitives.Slot) bool
|
||||
}
|
67
beacon-chain/sync/backfill/metrics.go
Normal file
67
beacon-chain/sync/backfill/metrics.go
Normal file
@ -0,0 +1,67 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
)
|
||||
|
||||
var (
|
||||
oldestBatch = promauto.NewGauge(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "backfill_earliest_wip_slot",
|
||||
Help: "Earliest slot that has been assigned to a worker.",
|
||||
},
|
||||
)
|
||||
batchesWaiting = promauto.NewGauge(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "backfill_importable_batches_waiting",
|
||||
Help: "Number of batches that are ready to be imported once they can be connected to the existing chain.",
|
||||
},
|
||||
)
|
||||
backfillRemainingBatches = promauto.NewGauge(
|
||||
prometheus.GaugeOpts{
|
||||
Name: "backfill_remaining_batches",
|
||||
Help: "Backfill remaining batches.",
|
||||
},
|
||||
)
|
||||
backfillBatchesImported = promauto.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "backfill_batches_imported",
|
||||
Help: "Number of backfill batches downloaded and imported.",
|
||||
},
|
||||
)
|
||||
backfillBatchApproximateBytes = promauto.NewCounter(
|
||||
prometheus.CounterOpts{
|
||||
Name: "backfill_batch_bytes_downloaded",
|
||||
Help: "Count of bytes downloaded from peers",
|
||||
},
|
||||
)
|
||||
backfillBatchTimeRoundtrip = promauto.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Name: "backfill_batch_time_roundtrip",
|
||||
Help: "Total time to import batch, from first scheduled to imported.",
|
||||
Buckets: []float64{400, 800, 1600, 3200, 6400, 12800},
|
||||
},
|
||||
)
|
||||
backfillBatchTimeWaiting = promauto.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Name: "backfill_batch_time_waiting",
|
||||
Help: "Time batch waited for a suitable peer.",
|
||||
Buckets: []float64{50, 100, 300, 1000, 2000},
|
||||
},
|
||||
)
|
||||
backfillBatchTimeDownloading = promauto.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Name: "backfill_batch_time_download",
|
||||
Help: "Time batch spent downloading blocks from peer.",
|
||||
Buckets: []float64{100, 300, 1000, 2000, 4000, 8000},
|
||||
},
|
||||
)
|
||||
backfillBatchTimeVerifying = promauto.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Name: "backfill_batch_time_verify",
|
||||
Help: "Time batch spent downloading blocks from peer.",
|
||||
Buckets: []float64{100, 300, 1000, 2000, 4000, 8000},
|
||||
},
|
||||
)
|
||||
)
|
152
beacon-chain/sync/backfill/pool.go
Normal file
152
beacon-chain/sync/backfill/pool.go
Normal file
@ -0,0 +1,152 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/peers"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type batchWorkerPool interface {
|
||||
spawn(ctx context.Context, n int, clock *startup.Clock, a PeerAssigner, v *verifier)
|
||||
todo(b batch)
|
||||
complete() (batch, error)
|
||||
}
|
||||
|
||||
type worker interface {
|
||||
run(context.Context)
|
||||
}
|
||||
|
||||
type newWorker func(id workerId, in, out chan batch, c *startup.Clock, v *verifier) worker
|
||||
|
||||
func defaultNewWorker(p p2p.P2P) newWorker {
|
||||
return func(id workerId, in, out chan batch, c *startup.Clock, v *verifier) worker {
|
||||
return newP2pWorker(id, p, in, out, c, v)
|
||||
}
|
||||
}
|
||||
|
||||
type p2pBatchWorkerPool struct {
|
||||
maxBatches int
|
||||
newWorker newWorker
|
||||
toWorkers chan batch
|
||||
fromWorkers chan batch
|
||||
toRouter chan batch
|
||||
fromRouter chan batch
|
||||
shutdownErr chan error
|
||||
endSeq []batch
|
||||
ctx context.Context
|
||||
cancel func()
|
||||
}
|
||||
|
||||
var _ batchWorkerPool = &p2pBatchWorkerPool{}
|
||||
|
||||
func newP2PBatchWorkerPool(p p2p.P2P, maxBatches int) *p2pBatchWorkerPool {
|
||||
nw := defaultNewWorker(p)
|
||||
return &p2pBatchWorkerPool{
|
||||
newWorker: nw,
|
||||
toRouter: make(chan batch, maxBatches),
|
||||
fromRouter: make(chan batch, maxBatches),
|
||||
toWorkers: make(chan batch),
|
||||
fromWorkers: make(chan batch),
|
||||
maxBatches: maxBatches,
|
||||
shutdownErr: make(chan error),
|
||||
}
|
||||
}
|
||||
|
||||
func (p *p2pBatchWorkerPool) spawn(ctx context.Context, n int, c *startup.Clock, a PeerAssigner, v *verifier) {
|
||||
p.ctx, p.cancel = context.WithCancel(ctx)
|
||||
go p.batchRouter(a)
|
||||
for i := 0; i < n; i++ {
|
||||
go p.newWorker(workerId(i), p.toWorkers, p.fromWorkers, c, v).run(p.ctx)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *p2pBatchWorkerPool) todo(b batch) {
|
||||
// Intercept batchEndSequence batches so workers can remain unaware of this state.
|
||||
// Workers don't know what to do with batchEndSequence batches. They are a signal to the pool that the batcher
|
||||
// has stopped producing things for the workers to do and the pool is close to winding down. See complete()
|
||||
// to understand how the pool manages the state where all workers are idle
|
||||
// and all incoming batches signal end of sequence.
|
||||
if b.state == batchEndSequence {
|
||||
p.endSeq = append(p.endSeq, b)
|
||||
return
|
||||
}
|
||||
p.toRouter <- b
|
||||
}
|
||||
|
||||
func (p *p2pBatchWorkerPool) complete() (batch, error) {
|
||||
if len(p.endSeq) == p.maxBatches {
|
||||
return p.endSeq[0], errEndSequence
|
||||
}
|
||||
|
||||
select {
|
||||
case b := <-p.fromRouter:
|
||||
return b, nil
|
||||
case err := <-p.shutdownErr:
|
||||
return batch{}, errors.Wrap(err, "fatal error from backfill worker pool")
|
||||
case <-p.ctx.Done():
|
||||
log.WithError(p.ctx.Err()).Info("p2pBatchWorkerPool context canceled, shutting down")
|
||||
return batch{}, p.ctx.Err()
|
||||
}
|
||||
}
|
||||
|
||||
func (p *p2pBatchWorkerPool) batchRouter(pa PeerAssigner) {
|
||||
busy := make(map[peer.ID]bool)
|
||||
todo := make([]batch, 0)
|
||||
rt := time.NewTicker(time.Second)
|
||||
earliest := primitives.Slot(math.MaxUint64)
|
||||
for {
|
||||
select {
|
||||
case b := <-p.toRouter:
|
||||
todo = append(todo, b)
|
||||
case <-rt.C:
|
||||
// Worker assignments can fail if assignBatch can't find a suitable peer.
|
||||
// This ticker exists to periodically break out of the channel select
|
||||
// to retry failed assignments.
|
||||
case b := <-p.fromWorkers:
|
||||
pid := b.pid
|
||||
busy[pid] = false
|
||||
p.fromRouter <- b
|
||||
case <-p.ctx.Done():
|
||||
log.WithError(p.ctx.Err()).Info("p2pBatchWorkerPool context canceled, shutting down")
|
||||
p.shutdown(p.ctx.Err())
|
||||
return
|
||||
}
|
||||
if len(todo) == 0 {
|
||||
continue
|
||||
}
|
||||
// Try to assign as many outstanding batches as possible to peers and feed the assigned batches to workers.
|
||||
assigned, err := pa.Assign(busy, len(todo))
|
||||
if err != nil {
|
||||
if errors.Is(err, peers.ErrInsufficientSuitable) {
|
||||
// Transient error resulting from insufficient number of connected peers. Leave batches in
|
||||
// queue and get to them whenever the peer situation is resolved.
|
||||
continue
|
||||
}
|
||||
p.shutdown(err)
|
||||
return
|
||||
}
|
||||
for _, pid := range assigned {
|
||||
busy[pid] = true
|
||||
todo[0].pid = pid
|
||||
p.toWorkers <- todo[0].withPeer(pid)
|
||||
if todo[0].begin < earliest {
|
||||
earliest = todo[0].begin
|
||||
oldestBatch.Set(float64(earliest))
|
||||
}
|
||||
todo = todo[1:]
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (p *p2pBatchWorkerPool) shutdown(err error) {
|
||||
p.cancel()
|
||||
p.shutdownErr <- err
|
||||
}
|
78
beacon-chain/sync/backfill/pool_test.go
Normal file
78
beacon-chain/sync/backfill/pool_test.go
Normal file
@ -0,0 +1,78 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
p2ptest "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/testing"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/util"
|
||||
)
|
||||
|
||||
type mockAssigner struct {
|
||||
err error
|
||||
assign []peer.ID
|
||||
}
|
||||
|
||||
// Assign satisfies the PeerAssigner interface so that mockAssigner can be used in tests
|
||||
// in place of the concrete p2p implementation of PeerAssigner.
|
||||
func (m mockAssigner) Assign(busy map[peer.ID]bool, n int) ([]peer.ID, error) {
|
||||
if m.err != nil {
|
||||
return nil, m.err
|
||||
}
|
||||
return m.assign, nil
|
||||
}
|
||||
|
||||
var _ PeerAssigner = &mockAssigner{}
|
||||
|
||||
func TestPoolDetectAllEnded(t *testing.T) {
|
||||
nw := 5
|
||||
p2p := p2ptest.NewTestP2P(t)
|
||||
ctx := context.Background()
|
||||
ma := &mockAssigner{}
|
||||
pool := newP2PBatchWorkerPool(p2p, nw)
|
||||
st, err := util.NewBeaconState()
|
||||
require.NoError(t, err)
|
||||
keys, err := st.PublicKeys()
|
||||
require.NoError(t, err)
|
||||
v, err := newBackfillVerifier(st.GenesisValidatorsRoot(), keys)
|
||||
require.NoError(t, err)
|
||||
pool.spawn(ctx, nw, startup.NewClock(time.Now(), [32]byte{}), ma, v)
|
||||
br := batcher{min: 10, size: 10}
|
||||
endSeq := br.before(0)
|
||||
require.Equal(t, batchEndSequence, endSeq.state)
|
||||
for i := 0; i < nw; i++ {
|
||||
pool.todo(endSeq)
|
||||
}
|
||||
b, err := pool.complete()
|
||||
require.ErrorIs(t, err, errEndSequence)
|
||||
require.Equal(t, b.end, endSeq.end)
|
||||
}
|
||||
|
||||
type mockPool struct {
|
||||
spawnCalled []int
|
||||
finishedChan chan batch
|
||||
finishedErr chan error
|
||||
todoChan chan batch
|
||||
}
|
||||
|
||||
func (m *mockPool) spawn(_ context.Context, _ int, _ *startup.Clock, _ PeerAssigner, _ *verifier) {
|
||||
}
|
||||
|
||||
func (m *mockPool) todo(b batch) {
|
||||
m.todoChan <- b
|
||||
}
|
||||
|
||||
func (m *mockPool) complete() (batch, error) {
|
||||
select {
|
||||
case b := <-m.finishedChan:
|
||||
return b, nil
|
||||
case err := <-m.finishedErr:
|
||||
return batch{}, err
|
||||
}
|
||||
}
|
||||
|
||||
var _ batchWorkerPool = &mockPool{}
|
300
beacon-chain/sync/backfill/service.go
Normal file
300
beacon-chain/sync/backfill/service.go
Normal file
@ -0,0 +1,300 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/helpers"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
"github.com/prysmaticlabs/prysm/v4/runtime"
|
||||
"github.com/prysmaticlabs/prysm/v4/time/slots"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type Service struct {
|
||||
ctx context.Context
|
||||
store *Store
|
||||
ms minimumSlotter
|
||||
cw startup.ClockWaiter
|
||||
enabled bool // service is disabled by default while feature is experimental
|
||||
nWorkers int
|
||||
batchSeq *batchSequencer
|
||||
batchSize uint64
|
||||
pool batchWorkerPool
|
||||
verifier *verifier
|
||||
p2p p2p.P2P
|
||||
pa PeerAssigner
|
||||
batchImporter batchImporter
|
||||
}
|
||||
|
||||
var _ runtime.Service = (*Service)(nil)
|
||||
|
||||
type ServiceOption func(*Service) error
|
||||
|
||||
func WithEnableBackfill(enabled bool) ServiceOption {
|
||||
return func(s *Service) error {
|
||||
s.enabled = enabled
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithWorkerCount(n int) ServiceOption {
|
||||
return func(s *Service) error {
|
||||
s.nWorkers = n
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithBatchSize(n uint64) ServiceOption {
|
||||
return func(s *Service) error {
|
||||
s.batchSize = n
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
type minimumSlotter interface {
|
||||
minimumSlot() primitives.Slot
|
||||
setClock(*startup.Clock)
|
||||
}
|
||||
|
||||
type defaultMinimumSlotter struct {
|
||||
clock *startup.Clock
|
||||
cw startup.ClockWaiter
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
func (d defaultMinimumSlotter) minimumSlot() primitives.Slot {
|
||||
if d.clock == nil {
|
||||
var err error
|
||||
d.clock, err = d.cw.WaitForClock(d.ctx)
|
||||
if err != nil {
|
||||
log.WithError(err).Fatal("failed to obtain system/genesis clock, unable to start backfill service")
|
||||
}
|
||||
}
|
||||
return minimumBackfillSlot(d.clock.CurrentSlot())
|
||||
}
|
||||
|
||||
func (d defaultMinimumSlotter) setClock(c *startup.Clock) {
|
||||
//nolint:all
|
||||
d.clock = c
|
||||
}
|
||||
|
||||
var _ minimumSlotter = &defaultMinimumSlotter{}
|
||||
|
||||
type batchImporter func(ctx context.Context, b batch, su *Store) (*dbval.BackfillStatus, error)
|
||||
|
||||
func defaultBatchImporter(ctx context.Context, b batch, su *Store) (*dbval.BackfillStatus, error) {
|
||||
status := su.status()
|
||||
if err := b.ensureParent(bytesutil.ToBytes32(status.LowParentRoot)); err != nil {
|
||||
return status, err
|
||||
}
|
||||
// Import blocks to db and update db state to reflect the newly imported blocks.
|
||||
// Other parts of the beacon node may use the same StatusUpdater instance
|
||||
// via the coverage.AvailableBlocker interface to safely determine if a given slot has been backfilled.
|
||||
status, err := su.fillBack(ctx, b.results)
|
||||
if err != nil {
|
||||
log.WithError(err).Fatal("Non-recoverable db error in backfill service, quitting.")
|
||||
}
|
||||
return status, nil
|
||||
}
|
||||
|
||||
// PeerAssigner describes a type that provides an Assign method, which can assign the best peer
|
||||
// to service an RPC request. The Assign method takes a map of peers that should be excluded,
|
||||
// allowing the caller to avoid making multiple concurrent requests to the same peer.
|
||||
type PeerAssigner interface {
|
||||
Assign(busy map[peer.ID]bool, n int) ([]peer.ID, error)
|
||||
}
|
||||
|
||||
// NewService initializes the backfill Service. Like all implementations of the Service interface,
|
||||
// the service won't begin its runloop until Start() is called.
|
||||
func NewService(ctx context.Context, su *Store, cw startup.ClockWaiter, p p2p.P2P, pa PeerAssigner, opts ...ServiceOption) (*Service, error) {
|
||||
s := &Service{
|
||||
ctx: ctx,
|
||||
store: su,
|
||||
cw: cw,
|
||||
ms: &defaultMinimumSlotter{cw: cw, ctx: ctx},
|
||||
p2p: p,
|
||||
pa: pa,
|
||||
batchImporter: defaultBatchImporter,
|
||||
}
|
||||
for _, o := range opts {
|
||||
if err := o(s); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
s.pool = newP2PBatchWorkerPool(p, s.nWorkers)
|
||||
|
||||
return s, nil
|
||||
}
|
||||
|
||||
func (s *Service) initVerifier(ctx context.Context) (*verifier, error) {
|
||||
cps, err := s.store.originState(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
keys, err := cps.PublicKeys()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "Unable to retrieve public keys for all validators in the origin state")
|
||||
}
|
||||
return newBackfillVerifier(cps.GenesisValidatorsRoot(), keys)
|
||||
}
|
||||
|
||||
func (s *Service) updateComplete() bool {
|
||||
b, err := s.pool.complete()
|
||||
if err != nil {
|
||||
if errors.Is(err, errEndSequence) {
|
||||
log.WithField("backfill_slot", b.begin).Info("Backfill is complete")
|
||||
return true
|
||||
}
|
||||
log.WithError(err).Fatal("Non-recoverable error in backfill service, quitting.")
|
||||
}
|
||||
s.batchSeq.update(b)
|
||||
return false
|
||||
}
|
||||
|
||||
func (s *Service) importBatches(ctx context.Context) {
|
||||
importable := s.batchSeq.importable()
|
||||
imported := 0
|
||||
defer func() {
|
||||
if imported == 0 {
|
||||
return
|
||||
}
|
||||
backfillBatchesImported.Add(float64(imported))
|
||||
}()
|
||||
for i := range importable {
|
||||
ib := importable[i]
|
||||
if len(ib.results) == 0 {
|
||||
log.WithFields(ib.logFields()).Error("Batch with no results, skipping importer.")
|
||||
}
|
||||
_, err := s.batchImporter(ctx, ib, s.store)
|
||||
if err != nil {
|
||||
log.WithError(err).WithFields(ib.logFields()).Debug("Backfill batch failed to import.")
|
||||
s.downscore(ib)
|
||||
s.batchSeq.update(ib.withState(batchErrRetryable))
|
||||
// If a batch fails, the subsequent batches are no longer considered importable.
|
||||
break
|
||||
}
|
||||
s.batchSeq.update(ib.withState(batchImportComplete))
|
||||
imported += 1
|
||||
// Calling update with state=batchImportComplete will advance the batch list.
|
||||
}
|
||||
|
||||
nt := s.batchSeq.numTodo()
|
||||
log.WithField("imported", imported).WithField("importable", len(importable)).
|
||||
WithField("batches_remaining", nt).
|
||||
Info("Backfill batches processed.")
|
||||
|
||||
backfillRemainingBatches.Set(float64(nt))
|
||||
}
|
||||
|
||||
func (s *Service) scheduleTodos() {
|
||||
batches, err := s.batchSeq.sequence()
|
||||
if err != nil {
|
||||
// This typically means we have several importable batches, but they are stuck behind a batch that needs
|
||||
// to complete first so that we can chain parent roots across batches.
|
||||
// ie backfilling [[90..100), [80..90), [70..80)], if we complete [70..80) and [80..90) but not [90..100),
|
||||
// we can't move forward until [90..100) completes, because we need to confirm 99 connects to 100,
|
||||
// and then we'll have the parent_root expected by 90 to ensure it matches the root for 89,
|
||||
// at which point we know we can process [80..90).
|
||||
if errors.Is(err, errMaxBatches) {
|
||||
log.Debug("Backfill batches waiting for descendent batch to complete.")
|
||||
return
|
||||
}
|
||||
}
|
||||
for _, b := range batches {
|
||||
s.pool.todo(b)
|
||||
}
|
||||
}
|
||||
|
||||
// Start begins the runloop of backfill.Service in the current goroutine.
|
||||
func (s *Service) Start() {
|
||||
if !s.enabled {
|
||||
log.Info("Exiting backfill service; not enabled.")
|
||||
return
|
||||
}
|
||||
ctx, cancel := context.WithCancel(s.ctx)
|
||||
defer func() {
|
||||
cancel()
|
||||
}()
|
||||
clock, err := s.cw.WaitForClock(ctx)
|
||||
if err != nil {
|
||||
log.WithError(err).Fatal("Backfill service failed to start while waiting for genesis data.")
|
||||
}
|
||||
s.ms.setClock(clock)
|
||||
|
||||
status := s.store.status()
|
||||
// Exit early if there aren't going to be any batches to backfill.
|
||||
if primitives.Slot(status.LowSlot) <= s.ms.minimumSlot() {
|
||||
log.WithField("minimum_required_slot", s.ms.minimumSlot()).
|
||||
WithField("backfill_lowest_slot", status.LowSlot).
|
||||
Info("Exiting backfill service; minimum block retention slot > lowest backfilled block.")
|
||||
return
|
||||
}
|
||||
s.verifier, err = s.initVerifier(ctx)
|
||||
if err != nil {
|
||||
log.WithError(err).Fatal("Unable to initialize backfill verifier, quitting.")
|
||||
}
|
||||
s.pool.spawn(ctx, s.nWorkers, clock, s.pa, s.verifier)
|
||||
|
||||
s.batchSeq = newBatchSequencer(s.nWorkers, s.ms.minimumSlot(), primitives.Slot(status.LowSlot), primitives.Slot(s.batchSize))
|
||||
if err = s.initBatches(); err != nil {
|
||||
log.WithError(err).Fatal("Non-recoverable error in backfill service, quitting.")
|
||||
}
|
||||
|
||||
for {
|
||||
if ctx.Err() != nil {
|
||||
return
|
||||
}
|
||||
if allComplete := s.updateComplete(); allComplete {
|
||||
return
|
||||
}
|
||||
s.importBatches(ctx)
|
||||
batchesWaiting.Set(float64(s.batchSeq.countWithState(batchImportable)))
|
||||
if err := s.batchSeq.moveMinimum(s.ms.minimumSlot()); err != nil {
|
||||
log.WithError(err).Fatal("Non-recoverable error in backfill service, quitting.")
|
||||
}
|
||||
s.scheduleTodos()
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Service) initBatches() error {
|
||||
batches, err := s.batchSeq.sequence()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, b := range batches {
|
||||
s.pool.todo(b)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Service) downscore(b batch) {
|
||||
s.p2p.Peers().Scorers().BadResponsesScorer().Increment(b.pid)
|
||||
}
|
||||
|
||||
func (s *Service) Stop() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Service) Status() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// minimumBackfillSlot determines the lowest slot that backfill needs to download based on looking back
|
||||
// MIN_EPOCHS_FOR_BLOCK_REQUESTS from the current slot.
|
||||
func minimumBackfillSlot(current primitives.Slot) primitives.Slot {
|
||||
oe := helpers.MinEpochsForBlockRequests()
|
||||
if oe > slots.MaxSafeEpoch() {
|
||||
oe = slots.MaxSafeEpoch()
|
||||
}
|
||||
offset := slots.UnsafeEpochStart(oe)
|
||||
if offset > current {
|
||||
return 0
|
||||
}
|
||||
return current - offset
|
||||
}
|
89
beacon-chain/sync/backfill/service_test.go
Normal file
89
beacon-chain/sync/backfill/service_test.go
Normal file
@ -0,0 +1,89 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
p2ptest "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/testing"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/util"
|
||||
)
|
||||
|
||||
type mockMinimumSlotter struct {
|
||||
min primitives.Slot
|
||||
}
|
||||
|
||||
var _ minimumSlotter = &mockMinimumSlotter{}
|
||||
|
||||
func (m mockMinimumSlotter) minimumSlot() primitives.Slot {
|
||||
return m.min
|
||||
}
|
||||
|
||||
func (m mockMinimumSlotter) setClock(*startup.Clock) {
|
||||
}
|
||||
|
||||
func TestServiceInit(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second*300)
|
||||
defer cancel()
|
||||
db := &mockBackfillDB{}
|
||||
su, err := NewUpdater(ctx, db)
|
||||
require.NoError(t, err)
|
||||
nWorkers := 5
|
||||
var batchSize uint64 = 100
|
||||
nBatches := nWorkers * 2
|
||||
var high uint64 = 11235
|
||||
originRoot := [32]byte{}
|
||||
origin, err := util.NewBeaconState()
|
||||
require.NoError(t, err)
|
||||
db.states = map[[32]byte]state.BeaconState{originRoot: origin}
|
||||
su.bs = &dbval.BackfillStatus{
|
||||
LowSlot: high,
|
||||
OriginRoot: originRoot[:],
|
||||
}
|
||||
remaining := nBatches
|
||||
cw := startup.NewClockSynchronizer()
|
||||
require.NoError(t, cw.SetClock(startup.NewClock(time.Now(), [32]byte{})))
|
||||
pool := &mockPool{todoChan: make(chan batch, nWorkers), finishedChan: make(chan batch, nWorkers)}
|
||||
p2pt := p2ptest.NewTestP2P(t)
|
||||
srv, err := NewService(ctx, su, cw, p2pt, &mockAssigner{}, WithBatchSize(batchSize), WithWorkerCount(nWorkers), WithEnableBackfill(true))
|
||||
require.NoError(t, err)
|
||||
srv.ms = mockMinimumSlotter{min: primitives.Slot(high - batchSize*uint64(nBatches))}
|
||||
srv.pool = pool
|
||||
srv.batchImporter = func(context.Context, batch, *Store) (*dbval.BackfillStatus, error) {
|
||||
return &dbval.BackfillStatus{}, nil
|
||||
}
|
||||
go srv.Start()
|
||||
todo := make([]batch, 0)
|
||||
todo = testReadN(t, ctx, pool.todoChan, nWorkers, todo)
|
||||
require.Equal(t, nWorkers, len(todo))
|
||||
for i := 0; i < remaining; i++ {
|
||||
b := todo[i]
|
||||
if b.state == batchSequenced {
|
||||
b.state = batchImportable
|
||||
}
|
||||
pool.finishedChan <- b
|
||||
todo = testReadN(t, ctx, pool.todoChan, 1, todo)
|
||||
}
|
||||
require.Equal(t, remaining+nWorkers, len(todo))
|
||||
for i := remaining; i < remaining+nWorkers; i++ {
|
||||
require.Equal(t, batchEndSequence, todo[i].state)
|
||||
}
|
||||
}
|
||||
|
||||
func testReadN(t *testing.T, ctx context.Context, c chan batch, n int, into []batch) []batch {
|
||||
for i := 0; i < n; i++ {
|
||||
select {
|
||||
case b := <-c:
|
||||
into = append(into, b)
|
||||
case <-ctx.Done():
|
||||
// this means we hit the timeout, so something went wrong.
|
||||
require.Equal(t, true, false)
|
||||
}
|
||||
}
|
||||
return into
|
||||
}
|
@ -2,121 +2,167 @@ package backfill
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
)
|
||||
|
||||
// NewStatus correctly initializes a Status value with the required database value.
|
||||
func NewStatus(store DB) *Status {
|
||||
return &Status{
|
||||
var errBatchDisconnected = errors.New("Highest block root in backfill batch doesn't match next parent_root")
|
||||
|
||||
// NewUpdater correctly initializes a StatusUpdater value with the required database value.
|
||||
func NewUpdater(ctx context.Context, store BeaconDB) (*Store, error) {
|
||||
s := &Store{
|
||||
store: store,
|
||||
}
|
||||
status, err := s.store.BackfillStatus(ctx)
|
||||
if err != nil {
|
||||
if errors.Is(err, db.ErrNotFound) {
|
||||
return s, s.recoverLegacy(ctx)
|
||||
}
|
||||
return nil, errors.Wrap(err, "db error while reading status of previous backfill")
|
||||
}
|
||||
s.swapStatus(status)
|
||||
return s, nil
|
||||
}
|
||||
|
||||
// Status provides a way to update and query the status of a backfill process that may be necessary to track when
|
||||
// Store provides a way to update and query the status of a backfill process that may be necessary to track when
|
||||
// a node was initialized via checkpoint sync. With checkpoint sync, there will be a gap in node history from genesis
|
||||
// until the checkpoint sync origin block. Status provides the means to update the value keeping track of the lower
|
||||
// end of the missing block range via the Advance() method, to check whether a Slot is missing from the database
|
||||
// via the SlotCovered() method, and to see the current StartGap() and EndGap().
|
||||
type Status struct {
|
||||
start primitives.Slot
|
||||
end primitives.Slot
|
||||
store DB
|
||||
// until the checkpoint sync origin block. Store provides the means to update the value keeping track of the lower
|
||||
// end of the missing block range via the FillFwd() method, to check whether a Slot is missing from the database
|
||||
// via the AvailableBlock() method, and to see the current StartGap() and EndGap().
|
||||
type Store struct {
|
||||
sync.RWMutex
|
||||
store BeaconDB
|
||||
genesisSync bool
|
||||
bs *dbval.BackfillStatus
|
||||
}
|
||||
|
||||
// SlotCovered uses StartGap() and EndGap() to determine if the given slot is covered by the current chain history.
|
||||
// If the slot is <= StartGap(), or >= EndGap(), the result is true.
|
||||
// If the slot is between StartGap() and EndGap(), the result is false.
|
||||
func (s *Status) SlotCovered(sl primitives.Slot) bool {
|
||||
// AvailableBlock determines if the given slot is covered by the current chain history.
|
||||
// If the slot is <= backfill low slot, or >= backfill high slot, the result is true.
|
||||
// If the slot is between the backfill low and high slots, the result is false.
|
||||
func (s *Store) AvailableBlock(sl primitives.Slot) bool {
|
||||
s.RLock()
|
||||
defer s.RUnlock()
|
||||
// short circuit if the node was synced from genesis
|
||||
if s.genesisSync {
|
||||
if s.genesisSync || sl == 0 || s.bs.LowSlot <= uint64(sl) {
|
||||
return true
|
||||
}
|
||||
if s.StartGap() < sl && sl < s.EndGap() {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
return false
|
||||
}
|
||||
|
||||
// StartGap returns the slot at the beginning of the range that needs to be backfilled.
|
||||
func (s *Status) StartGap() primitives.Slot {
|
||||
return s.start
|
||||
// Status is a threadsafe method to access a copy of the BackfillStatus value.
|
||||
func (s *Store) status() *dbval.BackfillStatus {
|
||||
s.RLock()
|
||||
defer s.RUnlock()
|
||||
return &dbval.BackfillStatus{
|
||||
LowSlot: s.bs.LowSlot,
|
||||
LowRoot: s.bs.LowRoot,
|
||||
LowParentRoot: s.bs.LowParentRoot,
|
||||
OriginSlot: s.bs.OriginSlot,
|
||||
OriginRoot: s.bs.OriginRoot,
|
||||
}
|
||||
}
|
||||
|
||||
// EndGap returns the slot at the end of the range that needs to be backfilled.
|
||||
func (s *Status) EndGap() primitives.Slot {
|
||||
return s.end
|
||||
// fillBack saves the slice of blocks and updates the BackfillStatus LowSlot/Root/ParentRoot tracker to the values
|
||||
// from the first block in the slice. This method assumes that the block slice has been fully validated and
|
||||
// sorted in slot order by the calling function.
|
||||
func (s *Store) fillBack(ctx context.Context, blocks []blocks.ROBlock) (*dbval.BackfillStatus, error) {
|
||||
status := s.status()
|
||||
if len(blocks) == 0 {
|
||||
return status, nil
|
||||
}
|
||||
|
||||
highest := blocks[len(blocks)-1]
|
||||
// The root of the highest block needs to match the parent root of the previous status. The backfill service will do
|
||||
// the same check, but this is an extra defensive layer in front of the db index.
|
||||
if highest.Root() != bytesutil.ToBytes32(status.LowParentRoot) {
|
||||
return nil, errors.Wrapf(errBatchDisconnected, "prev parent_root=%#x, root=%#x, prev slot=%d, slot=%d",
|
||||
status.LowParentRoot, highest.Root(), status.LowSlot, highest.Block().Slot())
|
||||
}
|
||||
|
||||
if err := s.store.SaveROBlocks(ctx, blocks, false); err != nil {
|
||||
return nil, errors.Wrapf(err, "error saving backfill blocks")
|
||||
}
|
||||
|
||||
// Update finalized block index.
|
||||
if err := s.store.BackfillFinalizedIndex(ctx, blocks, bytesutil.ToBytes32(status.LowRoot)); err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to update finalized index for batch, connecting root %#x to previously finalized block %#x",
|
||||
highest.Root(), status.LowRoot)
|
||||
}
|
||||
|
||||
// Update backfill status based on the block with the lowest slot in the batch.
|
||||
lowest := blocks[0]
|
||||
pr := lowest.Block().ParentRoot()
|
||||
status.LowSlot = uint64(lowest.Block().Slot())
|
||||
status.LowRoot = lowest.RootSlice()
|
||||
status.LowParentRoot = pr[:]
|
||||
return status, s.saveStatus(ctx, status)
|
||||
}
|
||||
|
||||
var ErrAdvancePastOrigin = errors.New("cannot advance backfill Status beyond the origin checkpoint slot")
|
||||
|
||||
// Advance advances the backfill position to the given slot & root.
|
||||
// It updates the backfill block root entry in the database,
|
||||
// and also updates the Status value's copy of the backfill position slot.
|
||||
func (s *Status) Advance(ctx context.Context, upTo primitives.Slot, root [32]byte) error {
|
||||
if upTo > s.end {
|
||||
return errors.Wrapf(ErrAdvancePastOrigin, "advance slot=%d, origin slot=%d", upTo, s.end)
|
||||
// recoverLegacy will check to see if the db is from a legacy checkpoint sync, and either build a new BackfillStatus
|
||||
// or label the node as synced from genesis.
|
||||
func (s *Store) recoverLegacy(ctx context.Context) error {
|
||||
cpr, err := s.store.OriginCheckpointBlockRoot(ctx)
|
||||
if errors.Is(err, db.ErrNotFoundOriginBlockRoot) {
|
||||
s.genesisSync = true
|
||||
return nil
|
||||
}
|
||||
s.start = upTo
|
||||
return s.store.SaveBackfillBlockRoot(ctx, root)
|
||||
|
||||
cpb, err := s.store.Block(ctx, cpr)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "error retrieving block for origin checkpoint root=%#x", cpr)
|
||||
}
|
||||
if err := blocks.BeaconBlockIsNil(cpb); err != nil {
|
||||
return errors.Wrapf(err, "nil block found for origin checkpoint root=%#x", cpr)
|
||||
}
|
||||
os := uint64(cpb.Block().Slot())
|
||||
lpr := cpb.Block().ParentRoot()
|
||||
bs := &dbval.BackfillStatus{
|
||||
LowSlot: os,
|
||||
LowRoot: cpr[:],
|
||||
LowParentRoot: lpr[:],
|
||||
OriginSlot: os,
|
||||
OriginRoot: cpr[:],
|
||||
}
|
||||
return s.saveStatus(ctx, bs)
|
||||
}
|
||||
|
||||
// Reload queries the database for backfill status, initializing the internal data and validating the database state.
|
||||
func (s *Status) Reload(ctx context.Context) error {
|
||||
cpRoot, err := s.store.OriginCheckpointBlockRoot(ctx)
|
||||
if err != nil {
|
||||
// mark genesis sync and short circuit further lookups
|
||||
if errors.Is(err, db.ErrNotFoundOriginBlockRoot) {
|
||||
s.genesisSync = true
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
cpBlock, err := s.store.Block(ctx, cpRoot)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "error retrieving block for origin checkpoint root=%#x", cpRoot)
|
||||
}
|
||||
if err := blocks.BeaconBlockIsNil(cpBlock); err != nil {
|
||||
return err
|
||||
}
|
||||
s.end = cpBlock.Block().Slot()
|
||||
|
||||
_, err = s.store.GenesisBlockRoot(ctx)
|
||||
if err != nil {
|
||||
if errors.Is(err, db.ErrNotFoundGenesisBlockRoot) {
|
||||
return errors.Wrap(err, "genesis block root required for checkpoint sync")
|
||||
}
|
||||
func (s *Store) saveStatus(ctx context.Context, bs *dbval.BackfillStatus) error {
|
||||
if err := s.store.SaveBackfillStatus(ctx, bs); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
bfRoot, err := s.store.BackfillBlockRoot(ctx)
|
||||
if err != nil {
|
||||
if errors.Is(err, db.ErrNotFoundBackfillBlockRoot) {
|
||||
return errors.Wrap(err, "found origin checkpoint block root, but no backfill block root")
|
||||
}
|
||||
return err
|
||||
}
|
||||
bfBlock, err := s.store.Block(ctx, bfRoot)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "error retrieving block for backfill root=%#x", bfRoot)
|
||||
}
|
||||
if err := blocks.BeaconBlockIsNil(bfBlock); err != nil {
|
||||
return err
|
||||
}
|
||||
s.start = bfBlock.Block().Slot()
|
||||
s.swapStatus(bs)
|
||||
return nil
|
||||
}
|
||||
|
||||
// DB describes the set of DB methods that the Status type needs to function.
|
||||
type DB interface {
|
||||
SaveBackfillBlockRoot(ctx context.Context, blockRoot [32]byte) error
|
||||
GenesisBlockRoot(ctx context.Context) ([32]byte, error)
|
||||
OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error)
|
||||
BackfillBlockRoot(ctx context.Context) ([32]byte, error)
|
||||
Block(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error)
|
||||
func (s *Store) swapStatus(bs *dbval.BackfillStatus) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
s.bs = bs
|
||||
}
|
||||
|
||||
// originState looks up the state for the checkpoint sync origin. This is a hack, because StatusUpdater is the only
|
||||
// thing that needs db access and it has the origin root handy, so it's convenient to look it up here. The state is
|
||||
// needed by the verifier.
|
||||
func (s *Store) originState(ctx context.Context) (state.BeaconState, error) {
|
||||
return s.store.StateOrError(ctx, bytesutil.ToBytes32(s.status().OriginRoot))
|
||||
}
|
||||
|
||||
// BeaconDB describes the set of DB methods that the StatusUpdater type needs to function.
|
||||
type BeaconDB interface {
|
||||
SaveBackfillStatus(context.Context, *dbval.BackfillStatus) error
|
||||
BackfillStatus(context.Context) (*dbval.BackfillStatus, error)
|
||||
BackfillFinalizedIndex(ctx context.Context, blocks []blocks.ROBlock, finalizedChildRoot [32]byte) error
|
||||
OriginCheckpointBlockRoot(context.Context) ([32]byte, error)
|
||||
Block(context.Context, [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error)
|
||||
SaveROBlocks(ctx context.Context, blks []blocks.ROBlock, cache bool) error
|
||||
StateOrError(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error)
|
||||
}
|
||||
|
@ -1,15 +1,18 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
blocktest "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks/testing"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v4/proto/dbval"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/util"
|
||||
@ -19,121 +22,125 @@ var errEmptyMockDBMethod = errors.New("uninitialized mock db method called")
|
||||
|
||||
type mockBackfillDB struct {
|
||||
saveBackfillBlockRoot func(ctx context.Context, blockRoot [32]byte) error
|
||||
genesisBlockRoot func(ctx context.Context) ([32]byte, error)
|
||||
originCheckpointBlockRoot func(ctx context.Context) ([32]byte, error)
|
||||
backfillBlockRoot func(ctx context.Context) ([32]byte, error)
|
||||
block func(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error)
|
||||
saveBackfillStatus func(ctx context.Context, status *dbval.BackfillStatus) error
|
||||
backfillStatus func(context.Context) (*dbval.BackfillStatus, error)
|
||||
status *dbval.BackfillStatus
|
||||
err error
|
||||
states map[[32]byte]state.BeaconState
|
||||
blocks map[[32]byte]blocks.ROBlock
|
||||
}
|
||||
|
||||
var _ DB = &mockBackfillDB{}
|
||||
var _ BeaconDB = &mockBackfillDB{}
|
||||
|
||||
func (db *mockBackfillDB) SaveBackfillBlockRoot(ctx context.Context, blockRoot [32]byte) error {
|
||||
if db.saveBackfillBlockRoot != nil {
|
||||
return db.saveBackfillBlockRoot(ctx, blockRoot)
|
||||
func (d *mockBackfillDB) StateOrError(_ context.Context, blockRoot [32]byte) (state.BeaconState, error) {
|
||||
st, ok := d.states[blockRoot]
|
||||
if !ok {
|
||||
return nil, db.ErrNotFoundState
|
||||
}
|
||||
return errEmptyMockDBMethod
|
||||
return st, nil
|
||||
}
|
||||
|
||||
func (db *mockBackfillDB) GenesisBlockRoot(ctx context.Context) ([32]byte, error) {
|
||||
if db.genesisBlockRoot != nil {
|
||||
return db.genesisBlockRoot(ctx)
|
||||
func (d *mockBackfillDB) SaveBackfillStatus(ctx context.Context, status *dbval.BackfillStatus) error {
|
||||
if d.saveBackfillStatus != nil {
|
||||
return d.saveBackfillStatus(ctx, status)
|
||||
}
|
||||
d.status = status
|
||||
return nil
|
||||
}
|
||||
|
||||
func (d *mockBackfillDB) BackfillStatus(ctx context.Context) (*dbval.BackfillStatus, error) {
|
||||
if d.backfillStatus != nil {
|
||||
return d.backfillStatus(ctx)
|
||||
}
|
||||
return d.status, nil
|
||||
}
|
||||
|
||||
func (d *mockBackfillDB) OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error) {
|
||||
if d.originCheckpointBlockRoot != nil {
|
||||
return d.originCheckpointBlockRoot(ctx)
|
||||
}
|
||||
return [32]byte{}, errEmptyMockDBMethod
|
||||
}
|
||||
|
||||
func (db *mockBackfillDB) OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error) {
|
||||
if db.originCheckpointBlockRoot != nil {
|
||||
return db.originCheckpointBlockRoot(ctx)
|
||||
func (d *mockBackfillDB) Block(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
if d.block != nil {
|
||||
return d.block(ctx, blockRoot)
|
||||
}
|
||||
return [32]byte{}, errEmptyMockDBMethod
|
||||
b, ok := d.blocks[blockRoot]
|
||||
if !ok {
|
||||
return nil, db.ErrNotFound
|
||||
}
|
||||
return b, nil
|
||||
}
|
||||
|
||||
func (db *mockBackfillDB) BackfillBlockRoot(ctx context.Context) ([32]byte, error) {
|
||||
if db.backfillBlockRoot != nil {
|
||||
return db.backfillBlockRoot(ctx)
|
||||
func (d *mockBackfillDB) SaveROBlocks(ctx context.Context, blks []blocks.ROBlock, cache bool) error {
|
||||
if d.blocks == nil {
|
||||
d.blocks = make(map[[32]byte]blocks.ROBlock)
|
||||
}
|
||||
return [32]byte{}, errEmptyMockDBMethod
|
||||
for i := range blks {
|
||||
d.blocks[blks[i].Root()] = blks[i]
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (db *mockBackfillDB) Block(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
if db.block != nil {
|
||||
return db.block(ctx, blockRoot)
|
||||
}
|
||||
return nil, errEmptyMockDBMethod
|
||||
func (d *mockBackfillDB) BackfillFinalizedIndex(ctx context.Context, blocks []blocks.ROBlock, finalizedChildRoot [32]byte) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestSlotCovered(t *testing.T) {
|
||||
cases := []struct {
|
||||
name string
|
||||
slot primitives.Slot
|
||||
status *Status
|
||||
status *Store
|
||||
result bool
|
||||
}{
|
||||
{
|
||||
name: "below start true",
|
||||
status: &Status{start: 1},
|
||||
name: "genesis true",
|
||||
status: &Store{bs: &dbval.BackfillStatus{LowSlot: 10}},
|
||||
slot: 0,
|
||||
result: true,
|
||||
},
|
||||
{
|
||||
name: "above end true",
|
||||
status: &Status{end: 1},
|
||||
status: &Store{bs: &dbval.BackfillStatus{LowSlot: 1}},
|
||||
slot: 2,
|
||||
result: true,
|
||||
},
|
||||
{
|
||||
name: "equal end true",
|
||||
status: &Status{end: 1},
|
||||
status: &Store{bs: &dbval.BackfillStatus{LowSlot: 1}},
|
||||
slot: 1,
|
||||
result: true,
|
||||
},
|
||||
{
|
||||
name: "equal start true",
|
||||
status: &Status{start: 2},
|
||||
slot: 2,
|
||||
result: true,
|
||||
},
|
||||
{
|
||||
name: "between false",
|
||||
status: &Status{start: 1, end: 3},
|
||||
slot: 2,
|
||||
result: false,
|
||||
},
|
||||
{
|
||||
name: "genesisSync always true",
|
||||
status: &Status{genesisSync: true},
|
||||
status: &Store{genesisSync: true},
|
||||
slot: 100,
|
||||
result: true,
|
||||
},
|
||||
}
|
||||
for _, c := range cases {
|
||||
result := c.status.SlotCovered(c.slot)
|
||||
require.Equal(t, c.result, result)
|
||||
t.Run(c.name, func(t *testing.T) {
|
||||
result := c.status.AvailableBlock(c.slot)
|
||||
require.Equal(t, c.result, result)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestAdvance(t *testing.T) {
|
||||
func TestStatusUpdater_FillBack(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
saveBackfillBuf := make([][32]byte, 0)
|
||||
mdb := &mockBackfillDB{
|
||||
saveBackfillBlockRoot: func(ctx context.Context, root [32]byte) error {
|
||||
saveBackfillBuf = append(saveBackfillBuf, root)
|
||||
return nil
|
||||
},
|
||||
}
|
||||
s := &Status{end: 100, store: mdb}
|
||||
var root [32]byte
|
||||
copy(root[:], []byte{0x23, 0x23})
|
||||
require.NoError(t, s.Advance(ctx, 90, root))
|
||||
require.Equal(t, root, saveBackfillBuf[0])
|
||||
not := s.SlotCovered(95)
|
||||
require.Equal(t, false, not)
|
||||
|
||||
// this should still be len 1 after failing to advance
|
||||
require.Equal(t, 1, len(saveBackfillBuf))
|
||||
require.ErrorIs(t, s.Advance(ctx, s.end+1, root), ErrAdvancePastOrigin)
|
||||
// this has an element in it from the previous test, there shouldn't be an additional one
|
||||
require.Equal(t, 1, len(saveBackfillBuf))
|
||||
mdb := &mockBackfillDB{}
|
||||
b, err := setupTestBlock(90)
|
||||
require.NoError(t, err)
|
||||
rob, err := blocks.NewROBlock(b)
|
||||
require.NoError(t, err)
|
||||
s := &Store{bs: &dbval.BackfillStatus{LowSlot: 100, LowParentRoot: rob.RootSlice()}, store: mdb}
|
||||
require.Equal(t, false, s.AvailableBlock(95))
|
||||
_, err = s.fillBack(ctx, []blocks.ROBlock{rob})
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, true, s.AvailableBlock(95))
|
||||
}
|
||||
|
||||
func goodBlockRoot(root [32]byte) func(ctx context.Context) ([32]byte, error) {
|
||||
@ -151,9 +158,8 @@ func setupTestBlock(slot primitives.Slot) (interfaces.ReadOnlySignedBeaconBlock,
|
||||
return blocktest.SetBlockSlot(b, slot)
|
||||
}
|
||||
|
||||
func TestReload(t *testing.T) {
|
||||
func TestNewUpdater(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
derp := errors.New("derp")
|
||||
|
||||
originSlot := primitives.Slot(100)
|
||||
var originRoot [32]byte
|
||||
@ -163,164 +169,39 @@ func TestReload(t *testing.T) {
|
||||
|
||||
backfillSlot := primitives.Slot(50)
|
||||
var backfillRoot [32]byte
|
||||
copy(originRoot[:], []byte{0x02})
|
||||
copy(backfillRoot[:], []byte{0x02})
|
||||
backfillBlock, err := setupTestBlock(backfillSlot)
|
||||
require.NoError(t, err)
|
||||
|
||||
var parentRoot [32]byte
|
||||
copy(parentRoot[:], []byte{0x03})
|
||||
var rootSlice = func(r [32]byte) []byte { return r[:] }
|
||||
typicalBackfillStatus := &dbval.BackfillStatus{
|
||||
LowSlot: 23,
|
||||
LowRoot: backfillRoot[:],
|
||||
LowParentRoot: parentRoot[:],
|
||||
OriginSlot: 1123,
|
||||
OriginRoot: originRoot[:],
|
||||
}
|
||||
cases := []struct {
|
||||
name string
|
||||
db DB
|
||||
db BeaconDB
|
||||
err error
|
||||
expected *Status
|
||||
expected *Store
|
||||
}{
|
||||
/*{
|
||||
{
|
||||
name: "origin not found, implying genesis sync ",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
backfillStatus: func(context.Context) (*dbval.BackfillStatus, error) {
|
||||
return nil, db.ErrNotFound
|
||||
},
|
||||
originCheckpointBlockRoot: func(ctx context.Context) ([32]byte, error) {
|
||||
return [32]byte{}, db.ErrNotFoundOriginBlockRoot
|
||||
}},
|
||||
expected: &Status{genesisSync: true},
|
||||
expected: &Store{genesisSync: true},
|
||||
},
|
||||
{
|
||||
name: "genesis not found error",
|
||||
err: db.ErrNotFoundGenesisBlockRoot,
|
||||
name: "legacy recovery",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: func(ctx context.Context) ([32]byte, error) {
|
||||
return [32]byte{}, db.ErrNotFoundGenesisBlockRoot
|
||||
},
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
switch root {
|
||||
case originRoot:
|
||||
return originBlock, nil
|
||||
}
|
||||
return nil, nil
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "other genesis error",
|
||||
err: derp,
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: func(ctx context.Context) ([32]byte, error) {
|
||||
return [32]byte{}, derp
|
||||
},
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
switch root {
|
||||
case originRoot:
|
||||
return originBlock, nil
|
||||
}
|
||||
return nil, nil
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "origin other error",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: func(ctx context.Context) ([32]byte, error) {
|
||||
return [32]byte{}, derp
|
||||
}},
|
||||
err: derp,
|
||||
},
|
||||
{
|
||||
name: "origin root found, block missing",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
return nil, nil
|
||||
},
|
||||
},
|
||||
err: blocks.ErrNilSignedBeaconBlock,
|
||||
},
|
||||
{
|
||||
name: "origin root found, block error",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
return nil, derp
|
||||
},
|
||||
},
|
||||
err: derp,
|
||||
},
|
||||
{
|
||||
name: "origin root found, block found, backfill root not found",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
return originBlock, nil
|
||||
},
|
||||
backfillBlockRoot: func(ctx context.Context) ([32]byte, error) {
|
||||
return [32]byte{}, db.ErrNotFoundBackfillBlockRoot
|
||||
},
|
||||
},
|
||||
err: db.ErrNotFoundBackfillBlockRoot,
|
||||
},
|
||||
{
|
||||
name: "origin root found, block found, random backfill root err",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
switch root {
|
||||
case originRoot:
|
||||
return originBlock, nil
|
||||
case backfillRoot:
|
||||
return nil, nil
|
||||
}
|
||||
return nil, derp
|
||||
},
|
||||
backfillBlockRoot: func(ctx context.Context) ([32]byte, error) {
|
||||
return [32]byte{}, derp
|
||||
},
|
||||
},
|
||||
err: derp,
|
||||
},
|
||||
{
|
||||
name: "origin root found, block found, backfill root found, backfill block not found",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
switch root {
|
||||
case originRoot:
|
||||
return originBlock, nil
|
||||
case backfillRoot:
|
||||
return nil, nil
|
||||
}
|
||||
return nil, derp
|
||||
},
|
||||
backfillBlockRoot: goodBlockRoot(backfillRoot),
|
||||
},
|
||||
err: blocks.ErrNilSignedBeaconBlock,
|
||||
},
|
||||
{
|
||||
name: "origin root found, block found, backfill root found, backfill block random err",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
switch root {
|
||||
case originRoot:
|
||||
return originBlock, nil
|
||||
case backfillRoot:
|
||||
return nil, derp
|
||||
}
|
||||
return nil, errors.New("not derp")
|
||||
},
|
||||
backfillBlockRoot: goodBlockRoot(backfillRoot),
|
||||
},
|
||||
err: derp,
|
||||
},*/
|
||||
{
|
||||
name: "complete happy path",
|
||||
db: &mockBackfillDB{
|
||||
genesisBlockRoot: goodBlockRoot(params.BeaconConfig().ZeroHash),
|
||||
originCheckpointBlockRoot: goodBlockRoot(originRoot),
|
||||
block: func(ctx context.Context, root [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
|
||||
switch root {
|
||||
@ -331,28 +212,44 @@ func TestReload(t *testing.T) {
|
||||
}
|
||||
return nil, errors.New("not derp")
|
||||
},
|
||||
backfillBlockRoot: goodBlockRoot(backfillRoot),
|
||||
backfillStatus: func(context.Context) (*dbval.BackfillStatus, error) { return nil, db.ErrNotFound },
|
||||
},
|
||||
err: derp,
|
||||
expected: &Status{genesisSync: false, start: backfillSlot, end: originSlot},
|
||||
expected: &Store{bs: &dbval.BackfillStatus{
|
||||
LowSlot: uint64(originSlot), OriginSlot: uint64(originSlot),
|
||||
LowRoot: originRoot[:], OriginRoot: originRoot[:], LowParentRoot: rootSlice(originBlock.Block().ParentRoot()),
|
||||
}},
|
||||
},
|
||||
{
|
||||
name: "backfill found",
|
||||
db: &mockBackfillDB{backfillStatus: func(ctx context.Context) (*dbval.BackfillStatus, error) {
|
||||
return typicalBackfillStatus, nil
|
||||
}},
|
||||
expected: &Store{bs: typicalBackfillStatus},
|
||||
},
|
||||
}
|
||||
|
||||
for _, c := range cases {
|
||||
s := &Status{
|
||||
store: c.db,
|
||||
}
|
||||
err := s.Reload(ctx)
|
||||
if err != nil {
|
||||
require.ErrorIs(t, err, c.err)
|
||||
continue
|
||||
}
|
||||
require.NoError(t, err)
|
||||
if c.expected == nil {
|
||||
continue
|
||||
}
|
||||
require.Equal(t, c.expected.genesisSync, s.genesisSync)
|
||||
require.Equal(t, c.expected.start, s.start)
|
||||
require.Equal(t, c.expected.end, s.end)
|
||||
t.Run(c.name, func(t *testing.T) {
|
||||
s, err := NewUpdater(ctx, c.db)
|
||||
if c.err != nil {
|
||||
require.ErrorIs(t, err, c.err)
|
||||
return
|
||||
}
|
||||
require.NoError(t, err)
|
||||
if c.expected == nil {
|
||||
return
|
||||
}
|
||||
require.Equal(t, c.expected.genesisSync, s.genesisSync)
|
||||
if c.expected.genesisSync {
|
||||
require.IsNil(t, s.bs)
|
||||
return
|
||||
}
|
||||
require.Equal(t, c.expected.bs.LowSlot, s.bs.LowSlot)
|
||||
require.Equal(t, c.expected.bs.OriginSlot, s.bs.OriginSlot)
|
||||
require.Equal(t, true, bytes.Equal(c.expected.bs.OriginRoot, s.bs.OriginRoot))
|
||||
require.Equal(t, true, bytes.Equal(c.expected.bs.LowRoot, s.bs.LowRoot))
|
||||
require.Equal(t, true, bytes.Equal(c.expected.bs.LowParentRoot, s.bs.LowParentRoot))
|
||||
})
|
||||
}
|
||||
|
||||
}
|
||||
|
125
beacon-chain/sync/backfill/verify.go
Normal file
125
beacon-chain/sync/backfill/verify.go
Normal file
@ -0,0 +1,125 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/signing"
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/crypto/bls"
|
||||
"github.com/prysmaticlabs/prysm/v4/network/forks"
|
||||
"github.com/prysmaticlabs/prysm/v4/time/slots"
|
||||
)
|
||||
|
||||
var errInvalidBatchChain = errors.New("parent_root of block does not match the previous block's root")
|
||||
var errProposerIndexTooHigh = errors.New("proposer index not present in origin state")
|
||||
var errUnknownDomain = errors.New("runtime error looking up signing domain for fork")
|
||||
|
||||
// VerifiedROBlocks represents a slice of blocks that have passed signature verification.
|
||||
type VerifiedROBlocks []blocks.ROBlock
|
||||
|
||||
type verifier struct {
|
||||
keys [][fieldparams.BLSPubkeyLength]byte
|
||||
maxVal primitives.ValidatorIndex
|
||||
domain *domainCache
|
||||
}
|
||||
|
||||
// TODO: rewrite this to use ROBlock.
|
||||
func (vr verifier) verify(blks []interfaces.ReadOnlySignedBeaconBlock) (VerifiedROBlocks, error) {
|
||||
var err error
|
||||
result := make([]blocks.ROBlock, len(blks))
|
||||
sigSet := bls.NewSet()
|
||||
for i := range blks {
|
||||
result[i], err = blocks.NewROBlock(blks[i])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if i > 0 && result[i-1].Root() != result[i].Block().ParentRoot() {
|
||||
p, b := result[i-1], result[i]
|
||||
return nil, errors.Wrapf(errInvalidBatchChain,
|
||||
"slot %d parent_root=%#x, slot %d root=%#x",
|
||||
b.Block().Slot(), b.Block().ParentRoot(),
|
||||
p.Block().Slot(), p.Root())
|
||||
}
|
||||
set, err := vr.blockSignatureBatch(result[i])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sigSet.Join(set)
|
||||
}
|
||||
v, err := sigSet.Verify()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "block signature verification error")
|
||||
}
|
||||
if !v {
|
||||
return nil, errors.New("batch block signature verification failed")
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func (vr verifier) blockSignatureBatch(b blocks.ROBlock) (*bls.SignatureBatch, error) {
|
||||
pidx := b.Block().ProposerIndex()
|
||||
if pidx > vr.maxVal {
|
||||
return nil, errProposerIndexTooHigh
|
||||
}
|
||||
dom, err := vr.domain.forEpoch(slots.ToEpoch(b.Block().Slot()))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sig := b.Signature()
|
||||
pk := vr.keys[pidx][:]
|
||||
root := b.Root()
|
||||
rootF := func() ([32]byte, error) { return root, nil }
|
||||
return signing.BlockSignatureBatch(pk, sig[:], dom, rootF)
|
||||
}
|
||||
|
||||
func newBackfillVerifier(vr []byte, keys [][fieldparams.BLSPubkeyLength]byte) (*verifier, error) {
|
||||
dc, err := newDomainCache(vr, params.BeaconConfig().DomainBeaconProposer,
|
||||
forks.NewOrderedSchedule(params.BeaconConfig()))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
v := &verifier{
|
||||
keys: keys,
|
||||
domain: dc,
|
||||
}
|
||||
v.maxVal = primitives.ValidatorIndex(len(v.keys) - 1)
|
||||
return v, nil
|
||||
}
|
||||
|
||||
// domainCache provides a fast signing domain lookup by epoch.
|
||||
type domainCache struct {
|
||||
fsched forks.OrderedSchedule
|
||||
forkDomains map[[4]byte][]byte
|
||||
dType [bls.DomainByteLength]byte
|
||||
}
|
||||
|
||||
func newDomainCache(vRoot []byte, dType [bls.DomainByteLength]byte, fsched forks.OrderedSchedule) (*domainCache, error) {
|
||||
dc := &domainCache{
|
||||
fsched: fsched,
|
||||
forkDomains: make(map[[4]byte][]byte),
|
||||
dType: dType,
|
||||
}
|
||||
for _, entry := range fsched {
|
||||
d, err := signing.ComputeDomain(dc.dType, entry.Version[:], vRoot)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to pre-compute signing domain for fork version=%#x", entry.Version)
|
||||
}
|
||||
dc.forkDomains[entry.Version] = d
|
||||
}
|
||||
return dc, nil
|
||||
}
|
||||
|
||||
func (dc *domainCache) forEpoch(e primitives.Epoch) ([]byte, error) {
|
||||
fork, err := dc.fsched.VersionForEpoch(e)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
d, ok := dc.forkDomains[fork]
|
||||
if !ok {
|
||||
return nil, errors.Wrapf(errUnknownDomain, "fork version=%#x, epoch=%d", fork, e)
|
||||
}
|
||||
return d, nil
|
||||
}
|
75
beacon-chain/sync/backfill/verify_test.go
Normal file
75
beacon-chain/sync/backfill/verify_test.go
Normal file
@ -0,0 +1,75 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/signing"
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v4/crypto/bls"
|
||||
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v4/network/forks"
|
||||
"github.com/prysmaticlabs/prysm/v4/runtime/interop"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v4/testing/util"
|
||||
)
|
||||
|
||||
func TestDomainCache(t *testing.T) {
|
||||
cfg := params.MainnetConfig()
|
||||
vRoot, err := hexutil.Decode("0x0011223344556677889900112233445566778899001122334455667788990011")
|
||||
dType := cfg.DomainBeaconProposer
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, 32, len(vRoot))
|
||||
fsched := forks.NewOrderedSchedule(cfg)
|
||||
dc, err := newDomainCache(vRoot,
|
||||
dType, fsched)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, len(fsched), len(dc.forkDomains))
|
||||
for i := range fsched {
|
||||
e := fsched[i].Epoch
|
||||
ad, err := dc.forEpoch(e)
|
||||
require.NoError(t, err)
|
||||
ed, err := signing.ComputeDomain(dType, fsched[i].Version[:], vRoot)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, ed, ad)
|
||||
}
|
||||
}
|
||||
|
||||
func testBlocksWithKeys(t *testing.T, nBlocks uint64, nBlobs int, vr []byte) ([]blocks.ROBlock, [][]blocks.ROBlob, []bls.SecretKey, []bls.PublicKey) {
|
||||
blks := make([]blocks.ROBlock, nBlocks)
|
||||
blbs := make([][]blocks.ROBlob, nBlocks)
|
||||
sks, pks, err := interop.DeterministicallyGenerateKeys(0, nBlocks)
|
||||
require.NoError(t, err)
|
||||
prevRoot := [32]byte{}
|
||||
for i := uint64(0); i < nBlocks; i++ {
|
||||
block, blobs := util.GenerateTestDenebBlockWithSidecar(t, prevRoot, primitives.Slot(i), nBlobs, util.WithProposerSigning(primitives.ValidatorIndex(i), sks[i], vr))
|
||||
prevRoot = block.Root()
|
||||
blks[i] = block
|
||||
blbs[i] = blobs
|
||||
}
|
||||
return blks, blbs, sks, pks
|
||||
}
|
||||
|
||||
func TestVerify(t *testing.T) {
|
||||
vr := make([]byte, 32)
|
||||
copy(vr, []byte("yooooo"))
|
||||
blks, _, _, pks := testBlocksWithKeys(t, 2, 0, vr)
|
||||
pubkeys := make([][fieldparams.BLSPubkeyLength]byte, len(pks))
|
||||
for i := range pks {
|
||||
pubkeys[i] = bytesutil.ToBytes48(pks[i].Marshal())
|
||||
}
|
||||
v, err := newBackfillVerifier(vr, pubkeys)
|
||||
require.NoError(t, err)
|
||||
notrob := make([]interfaces.ReadOnlySignedBeaconBlock, len(blks))
|
||||
// We have to unwrap the ROBlocks for this code because that's what it expects (for now).
|
||||
for i := range blks {
|
||||
notrob[i] = blks[i].ReadOnlySignedBeaconBlock
|
||||
}
|
||||
vbs, err := v.verify(notrob)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, len(blks), len(vbs))
|
||||
}
|
72
beacon-chain/sync/backfill/worker.go
Normal file
72
beacon-chain/sync/backfill/worker.go
Normal file
@ -0,0 +1,72 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type workerId int
|
||||
|
||||
type p2pWorker struct {
|
||||
id workerId
|
||||
todo chan batch
|
||||
done chan batch
|
||||
p2p p2p.P2P
|
||||
v *verifier
|
||||
c *startup.Clock
|
||||
}
|
||||
|
||||
func (w *p2pWorker) run(ctx context.Context) {
|
||||
for {
|
||||
select {
|
||||
case b := <-w.todo:
|
||||
log.WithFields(b.logFields()).WithField("backfill_worker", w.id).Debug("Backfill worker received batch.")
|
||||
w.done <- w.handle(ctx, b)
|
||||
case <-ctx.Done():
|
||||
log.WithField("backfill_worker", w.id).Info("Backfill worker exiting after context canceled.")
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (w *p2pWorker) handle(ctx context.Context, b batch) batch {
|
||||
start := time.Now()
|
||||
results, err := sync.SendBeaconBlocksByRangeRequest(ctx, w.c, w.p2p, b.pid, b.request(), nil)
|
||||
dlt := time.Now()
|
||||
backfillBatchTimeDownloading.Observe(float64(dlt.Sub(start).Milliseconds()))
|
||||
if err != nil {
|
||||
return b.withRetryableError(err)
|
||||
}
|
||||
vb, err := w.v.verify(results)
|
||||
backfillBatchTimeVerifying.Observe(float64(time.Since(dlt).Milliseconds()))
|
||||
if err != nil {
|
||||
return b.withRetryableError(err)
|
||||
}
|
||||
// This is a hack to get the rough size of the batch. This helps us approximate the amount of memory needed
|
||||
// to hold batches and relative sizes between batches, but will be inaccurate when it comes to measuring actual
|
||||
// bytes downloaded from peers, mainly because the p2p messages are snappy compressed.
|
||||
bdl := 0
|
||||
for i := range vb {
|
||||
bdl += vb[i].SizeSSZ()
|
||||
}
|
||||
backfillBatchApproximateBytes.Add(float64(bdl))
|
||||
log.WithField("dlbytes", bdl).Debug("backfill batch bytes downloaded")
|
||||
b.results = vb
|
||||
return b.withState(batchImportable)
|
||||
}
|
||||
|
||||
func newP2pWorker(id workerId, p p2p.P2P, todo, done chan batch, c *startup.Clock, v *verifier) *p2pWorker {
|
||||
return &p2pWorker{
|
||||
id: id,
|
||||
todo: todo,
|
||||
done: done,
|
||||
p2p: p,
|
||||
v: v,
|
||||
c: c,
|
||||
}
|
||||
}
|
@ -18,7 +18,7 @@ import (
|
||||
func testSignedBlockBlobKeys(t *testing.T, valRoot []byte, slot primitives.Slot, nblobs int) (blocks.ROBlock, []blocks.ROBlob, bls.SecretKey, bls.PublicKey) {
|
||||
sks, pks, err := interop.DeterministicallyGenerateKeys(0, 1)
|
||||
require.NoError(t, err)
|
||||
block, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, slot, nblobs, util.WithProposerSigning(0, sks[0], pks[0], valRoot))
|
||||
block, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, slot, nblobs, util.WithProposerSigning(0, sks[0], valRoot))
|
||||
return block, blobs, sks[0], pks[0]
|
||||
}
|
||||
|
||||
|
@ -21,6 +21,7 @@ go_library(
|
||||
"//cmd/beacon-chain/flags:go_default_library",
|
||||
"//cmd/beacon-chain/jwt:go_default_library",
|
||||
"//cmd/beacon-chain/storage:go_default_library",
|
||||
"//cmd/beacon-chain/sync/backfill:go_default_library",
|
||||
"//cmd/beacon-chain/sync/checkpoint:go_default_library",
|
||||
"//cmd/beacon-chain/sync/genesis:go_default_library",
|
||||
"//config/features:go_default_library",
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
|
||||
jwtcommands "github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/jwt"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/storage"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/backfill"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/checkpoint"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/genesis"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/features"
|
||||
@ -138,6 +139,9 @@ var appFlags = []cli.Flag{
|
||||
flags.JwtId,
|
||||
storage.BlobStoragePathFlag,
|
||||
storage.BlobRetentionEpochFlag,
|
||||
backfill.EnableExperimentalBackfill,
|
||||
backfill.BackfillBatchSize,
|
||||
backfill.BackfillWorkerCount,
|
||||
}
|
||||
|
||||
func init() {
|
||||
@ -287,6 +291,7 @@ func startNode(ctx *cli.Context, cancel context.CancelFunc) error {
|
||||
genesis.BeaconNodeOptions,
|
||||
checkpoint.BeaconNodeOptions,
|
||||
storage.BeaconNodeOptions,
|
||||
backfill.BeaconNodeOptions,
|
||||
}
|
||||
for _, of := range optFuncs {
|
||||
ofo, err := of(ctx)
|
||||
|
13
cmd/beacon-chain/sync/backfill/BUILD.bazel
Normal file
13
cmd/beacon-chain/sync/backfill/BUILD.bazel
Normal file
@ -0,0 +1,13 @@
|
||||
load("@prysm//tools/go:def.bzl", "go_library")
|
||||
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = ["options.go"],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/backfill",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//beacon-chain/node:go_default_library",
|
||||
"//beacon-chain/sync/backfill:go_default_library",
|
||||
"@com_github_urfave_cli_v2//:go_default_library",
|
||||
],
|
||||
)
|
53
cmd/beacon-chain/sync/backfill/options.go
Normal file
53
cmd/beacon-chain/sync/backfill/options.go
Normal file
@ -0,0 +1,53 @@
|
||||
package backfill
|
||||
|
||||
import (
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/node"
|
||||
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/backfill"
|
||||
"github.com/urfave/cli/v2"
|
||||
)
|
||||
|
||||
var (
|
||||
backfillBatchSizeName = "backfill-batch-size"
|
||||
backfillWorkerCountName = "backfill-worker-count"
|
||||
// EnableExperimentalBackfill enables backfill for checkpoint synced nodes.
|
||||
// This flag will be removed onced backfill is enabled by default.
|
||||
EnableExperimentalBackfill = &cli.BoolFlag{
|
||||
Name: "enable-experimental-backfill",
|
||||
Usage: "Backfill is still experimental at this time." +
|
||||
"It will only be enabled if this flag is specified and the node was started using checkpoint sync.",
|
||||
}
|
||||
// BackfillBatchSize allows users to tune block backfill request sizes to maximize network utilization
|
||||
// at the cost of higher memory.
|
||||
BackfillBatchSize = &cli.Uint64Flag{
|
||||
Name: backfillBatchSizeName,
|
||||
Usage: "Number of blocks per backfill batch. " +
|
||||
"A larger number will request more blocks at once from peers, but also consume more system memory to " +
|
||||
"hold batches in memory during processing. This has a multiplicative effect with " + backfillWorkerCountName,
|
||||
Value: 64,
|
||||
}
|
||||
// BackfillWorkerCount allows users to tune the number of concurrent backfill batches to download, to maximize
|
||||
// network utilization at the cost of higher memory.
|
||||
BackfillWorkerCount = &cli.IntFlag{
|
||||
Name: backfillWorkerCountName,
|
||||
Usage: "Number of concurrent backfill batch requests. " +
|
||||
"A larger number will better utilize network resources, up to a system-dependent limit, but will also " +
|
||||
"consume more system memory to hold batches in memory during processing. Multiply by backfill-batch-size and " +
|
||||
"average block size (~2MB before deneb) to find the right number for your system. " +
|
||||
"This has a multiplicatice effect with " + backfillBatchSizeName,
|
||||
Value: 2,
|
||||
}
|
||||
)
|
||||
|
||||
// BeaconNodeOptions sets the appropriate functional opts on the *node.BeaconNode value, to decouple options
|
||||
// from flag parsing.
|
||||
func BeaconNodeOptions(c *cli.Context) ([]node.Option, error) {
|
||||
opt := func(node *node.BeaconNode) (err error) {
|
||||
node.BackfillOpts = []backfill.ServiceOption{
|
||||
backfill.WithBatchSize(c.Uint64(BackfillBatchSize.Name)),
|
||||
backfill.WithWorkerCount(c.Int(BackfillWorkerCount.Name)),
|
||||
backfill.WithEnableBackfill(c.Bool(EnableExperimentalBackfill.Name)),
|
||||
}
|
||||
return nil
|
||||
}
|
||||
return []node.Option{opt}, nil
|
||||
}
|
@ -8,6 +8,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/storage"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/backfill"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/checkpoint"
|
||||
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/genesis"
|
||||
"github.com/prysmaticlabs/prysm/v4/config/features"
|
||||
@ -136,6 +137,9 @@ var appHelpFlagGroups = []flagGroup{
|
||||
genesis.BeaconAPIURL,
|
||||
storage.BlobStoragePathFlag,
|
||||
storage.BlobRetentionEpochFlag,
|
||||
backfill.EnableExperimentalBackfill,
|
||||
backfill.BackfillWorkerCount,
|
||||
backfill.BackfillBatchSize,
|
||||
},
|
||||
},
|
||||
{
|
||||
|
@ -114,6 +114,9 @@ func (r *configset) replace(cfg *BeaconChainConfig) error {
|
||||
func (r *configset) replaceWithUndo(cfg *BeaconChainConfig) (func() error, error) {
|
||||
name := cfg.ConfigName
|
||||
prev := r.nameToConfig[name]
|
||||
if prev != nil {
|
||||
prev = prev.Copy()
|
||||
}
|
||||
if err := r.replace(cfg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -142,7 +145,11 @@ func (r *configset) setActive(c *BeaconChainConfig) error {
|
||||
}
|
||||
|
||||
func (r *configset) setActiveWithUndo(c *BeaconChainConfig) (func() error, error) {
|
||||
active := r.active
|
||||
if r.active == nil {
|
||||
return nil, errors.Wrap(errCannotNullifyActive,
|
||||
"active config is currently nil, refusing to construct undo method that will leave it nil again")
|
||||
}
|
||||
active := r.active.Copy()
|
||||
r.active = c
|
||||
undo, err := r.replaceWithUndo(c)
|
||||
if err != nil {
|
||||
|
@ -11,15 +11,30 @@ func SetupTestConfigCleanup(t testing.TB) {
|
||||
temp := configs.getActive().Copy()
|
||||
undo, err := SetActiveWithUndo(temp)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
t.Fatal(err)
|
||||
}
|
||||
prevNetworkCfg := networkConfig.Copy()
|
||||
t.Cleanup(func() {
|
||||
mainnetBeaconConfig = prevDefaultBeaconConfig
|
||||
err = undo()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
t.Fatal(err)
|
||||
}
|
||||
networkConfig = prevNetworkCfg
|
||||
})
|
||||
}
|
||||
|
||||
// SetActiveTestCleanup sets an active config,
|
||||
// and adds a test cleanup hook to revert to the default config after the test completes.
|
||||
func SetActiveTestCleanup(t *testing.T, cfg *BeaconChainConfig) {
|
||||
undo, err := SetActiveWithUndo(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
t.Cleanup(func() {
|
||||
err = undo()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
@ -20,6 +20,14 @@ func (b ROBlock) Root() [32]byte {
|
||||
return b.root
|
||||
}
|
||||
|
||||
// RootSlice returns a slice of the value returned by Root(). This is convenient because slicing the result of a func
|
||||
// is not allowed, so only offering a fixed-length array version results in boilerplate code to
|
||||
func (b ROBlock) RootSlice() []byte {
|
||||
r := make([]byte, 32)
|
||||
copy(r, b.root[:])
|
||||
return r
|
||||
}
|
||||
|
||||
// NewROBlockWithRoot creates an ROBlock embedding the given block with its root. It accepts the root as parameter rather than
|
||||
// computing it internally, because in some cases a block is retrieved by its root and recomputing it is a waste.
|
||||
func NewROBlockWithRoot(b interfaces.ReadOnlySignedBeaconBlock, root [32]byte) (ROBlock, error) {
|
||||
@ -42,6 +50,20 @@ func NewROBlock(b interfaces.ReadOnlySignedBeaconBlock) (ROBlock, error) {
|
||||
return ROBlock{ReadOnlySignedBeaconBlock: b, root: root}, nil
|
||||
}
|
||||
|
||||
// NewROBlockSlice is a helper method for converting a slice of the ReadOnlySignedBeaconBlock interface
|
||||
// to a slice of ROBlock.
|
||||
func NewROBlockSlice(blks []interfaces.ReadOnlySignedBeaconBlock) ([]ROBlock, error) {
|
||||
robs := make([]ROBlock, len(blks))
|
||||
var err error
|
||||
for i := range blks {
|
||||
robs[i], err = NewROBlock(blks[i])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return robs, nil
|
||||
}
|
||||
|
||||
// ROBlockSlice implements sort.Interface so that slices of ROBlocks can be easily sorted.
|
||||
// A slice of ROBlock is sorted first by slot, with ties broken by cached block roots.
|
||||
type ROBlockSlice []ROBlock
|
||||
|
@ -18,8 +18,8 @@ searchstring="prysmaticlabs/prysm/v4/"
|
||||
for ((i = 0; i < arraylength; i++)); do
|
||||
color "34" "$destination"
|
||||
destination=${file_list[i]#*$searchstring}
|
||||
chmod 755 "$destination"
|
||||
cp -R -L "${file_list[i]}" "$destination"
|
||||
chmod 755 "$destination"
|
||||
done
|
||||
|
||||
# Run goimports on newly generated protos
|
||||
|
23
proto/dbval/BUILD.bazel
Normal file
23
proto/dbval/BUILD.bazel
Normal file
@ -0,0 +1,23 @@
|
||||
load("@prysm//tools/go:def.bzl", "go_library")
|
||||
load("@rules_proto//proto:defs.bzl", "proto_library")
|
||||
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
|
||||
|
||||
proto_library(
|
||||
name = "dbval_proto",
|
||||
srcs = ["dbval.proto"],
|
||||
visibility = ["//visibility:public"],
|
||||
)
|
||||
|
||||
go_proto_library(
|
||||
name = "dbval_go_proto",
|
||||
importpath = "github.com/prysmaticlabs/prysm/v4/proto/dbval",
|
||||
proto = ":dbval_proto",
|
||||
visibility = ["//visibility:public"],
|
||||
)
|
||||
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
embed = [":dbval_go_proto"],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v4/proto/dbval",
|
||||
visibility = ["//visibility:public"],
|
||||
)
|
187
proto/dbval/dbval.pb.go
generated
Executable file
187
proto/dbval/dbval.pb.go
generated
Executable file
@ -0,0 +1,187 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.30.0
|
||||
// protoc v4.23.3
|
||||
// source: proto/dbval/dbval.proto
|
||||
|
||||
package dbval
|
||||
|
||||
import (
|
||||
reflect "reflect"
|
||||
sync "sync"
|
||||
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
)
|
||||
|
||||
const (
|
||||
// Verify that this generated code is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
|
||||
// Verify that runtime/protoimpl is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
||||
)
|
||||
|
||||
type BackfillStatus struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
LowSlot uint64 `protobuf:"varint,1,opt,name=low_slot,json=lowSlot,proto3" json:"low_slot,omitempty"`
|
||||
LowRoot []byte `protobuf:"bytes,2,opt,name=low_root,json=lowRoot,proto3" json:"low_root,omitempty"`
|
||||
LowParentRoot []byte `protobuf:"bytes,3,opt,name=low_parent_root,json=lowParentRoot,proto3" json:"low_parent_root,omitempty"`
|
||||
OriginSlot uint64 `protobuf:"varint,4,opt,name=origin_slot,json=originSlot,proto3" json:"origin_slot,omitempty"`
|
||||
OriginRoot []byte `protobuf:"bytes,6,opt,name=origin_root,json=originRoot,proto3" json:"origin_root,omitempty"`
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) Reset() {
|
||||
*x = BackfillStatus{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_dbval_dbval_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*BackfillStatus) ProtoMessage() {}
|
||||
|
||||
func (x *BackfillStatus) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_dbval_dbval_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use BackfillStatus.ProtoReflect.Descriptor instead.
|
||||
func (*BackfillStatus) Descriptor() ([]byte, []int) {
|
||||
return file_proto_dbval_dbval_proto_rawDescGZIP(), []int{0}
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) GetLowSlot() uint64 {
|
||||
if x != nil {
|
||||
return x.LowSlot
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) GetLowRoot() []byte {
|
||||
if x != nil {
|
||||
return x.LowRoot
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) GetLowParentRoot() []byte {
|
||||
if x != nil {
|
||||
return x.LowParentRoot
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) GetOriginSlot() uint64 {
|
||||
if x != nil {
|
||||
return x.OriginSlot
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *BackfillStatus) GetOriginRoot() []byte {
|
||||
if x != nil {
|
||||
return x.OriginRoot
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
var File_proto_dbval_dbval_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_proto_dbval_dbval_proto_rawDesc = []byte{
|
||||
0x0a, 0x17, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x62, 0x76, 0x61, 0x6c, 0x2f, 0x64, 0x62,
|
||||
0x76, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x65, 0x74, 0x68, 0x65, 0x72,
|
||||
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x64, 0x62, 0x76, 0x61, 0x6c, 0x22, 0xb0, 0x01,
|
||||
0x0a, 0x0e, 0x42, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
|
||||
0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x04, 0x52, 0x07, 0x6c, 0x6f, 0x77, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x6c,
|
||||
0x6f, 0x77, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x6c,
|
||||
0x6f, 0x77, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x61,
|
||||
0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52,
|
||||
0x0d, 0x6c, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x1f,
|
||||
0x0a, 0x0b, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x04, 0x20,
|
||||
0x01, 0x28, 0x04, 0x52, 0x0a, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x12,
|
||||
0x1f, 0x0a, 0x0b, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x06,
|
||||
0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x52, 0x6f, 0x6f, 0x74,
|
||||
0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70,
|
||||
0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
|
||||
0x73, 0x6d, 0x2f, 0x76, 0x34, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x62, 0x76, 0x61,
|
||||
0x6c, 0x3b, 0x64, 0x62, 0x76, 0x61, 0x6c, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
file_proto_dbval_dbval_proto_rawDescOnce sync.Once
|
||||
file_proto_dbval_dbval_proto_rawDescData = file_proto_dbval_dbval_proto_rawDesc
|
||||
)
|
||||
|
||||
func file_proto_dbval_dbval_proto_rawDescGZIP() []byte {
|
||||
file_proto_dbval_dbval_proto_rawDescOnce.Do(func() {
|
||||
file_proto_dbval_dbval_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_dbval_dbval_proto_rawDescData)
|
||||
})
|
||||
return file_proto_dbval_dbval_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_proto_dbval_dbval_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
|
||||
var file_proto_dbval_dbval_proto_goTypes = []interface{}{
|
||||
(*BackfillStatus)(nil), // 0: ethereum.eth.dbval.BackfillStatus
|
||||
}
|
||||
var file_proto_dbval_dbval_proto_depIdxs = []int32{
|
||||
0, // [0:0] is the sub-list for method output_type
|
||||
0, // [0:0] is the sub-list for method input_type
|
||||
0, // [0:0] is the sub-list for extension type_name
|
||||
0, // [0:0] is the sub-list for extension extendee
|
||||
0, // [0:0] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_proto_dbval_dbval_proto_init() }
|
||||
func file_proto_dbval_dbval_proto_init() {
|
||||
if File_proto_dbval_dbval_proto != nil {
|
||||
return
|
||||
}
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_proto_dbval_dbval_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*BackfillStatus); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_proto_dbval_dbval_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 1,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
GoTypes: file_proto_dbval_dbval_proto_goTypes,
|
||||
DependencyIndexes: file_proto_dbval_dbval_proto_depIdxs,
|
||||
MessageInfos: file_proto_dbval_dbval_proto_msgTypes,
|
||||
}.Build()
|
||||
File_proto_dbval_dbval_proto = out.File
|
||||
file_proto_dbval_dbval_proto_rawDesc = nil
|
||||
file_proto_dbval_dbval_proto_goTypes = nil
|
||||
file_proto_dbval_dbval_proto_depIdxs = nil
|
||||
}
|
23
proto/dbval/dbval.proto
Normal file
23
proto/dbval/dbval.proto
Normal file
@ -0,0 +1,23 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package ethereum.eth.dbval;
|
||||
|
||||
option go_package = "github.com/prysmaticlabs/prysm/v4/proto/dbval;dbval";
|
||||
|
||||
// BackfillStatus is a value used to keep track of the progress of the process of backfilling blocks leading up to
|
||||
// the origin block used to checkpoint sync a node. There is only one BackfillStatus value in the database.
|
||||
message BackfillStatus {
|
||||
// low_slot is the slot of the last block that backfill will attempt to download and import.
|
||||
// This is determined by MIN_EPOCHS_FOR_BLOCK_REQUESTS, or by a user-specified override.
|
||||
uint64 low_slot = 1;
|
||||
// low_root is the root of the last block that backfill will attempt to download and import.
|
||||
bytes low_root = 2;
|
||||
// low_parent_root is the parent_root of the block at low_root. This enables the backfill service to check
|
||||
// that a block is the direct ancestor of the block for low_root without an additional db lookup.
|
||||
bytes low_parent_root = 3;
|
||||
// origin_slot is the slot of the origin block, which is the block used to initiate a checkpoint sync.
|
||||
// Backfill uses the origin block as the reference point to determine canonical ancestors.
|
||||
uint64 origin_slot = 4;
|
||||
// origin_root is the root of the origin block.
|
||||
bytes origin_root = 6;
|
||||
}
|
@ -169,19 +169,28 @@ func ErrorContains(loggerFn assertionLoggerFn, want string, err error, msg ...in
|
||||
|
||||
// NotNil asserts that passed value is not nil.
|
||||
func NotNil(loggerFn assertionLoggerFn, obj interface{}, msg ...interface{}) {
|
||||
if isNil(obj) {
|
||||
if deepNil(obj) {
|
||||
errMsg := parseMsg("Unexpected nil value", msg...)
|
||||
_, file, line, _ := runtime.Caller(2)
|
||||
loggerFn("%s:%d %s", filepath.Base(file), line, errMsg)
|
||||
}
|
||||
}
|
||||
|
||||
// isNil checks that underlying value of obj is nil.
|
||||
func isNil(obj interface{}) bool {
|
||||
if obj == nil {
|
||||
// IsNil asserts that observed value is nil.
|
||||
func IsNil(loggerFn assertionLoggerFn, got interface{}, msg ...interface{}) {
|
||||
if !deepNil(got) {
|
||||
errMsg := parseMsg("Value is unexpectedly not nil", msg...)
|
||||
_, file, line, _ := runtime.Caller(2)
|
||||
loggerFn("%s:%d %s", filepath.Base(file), line, errMsg)
|
||||
}
|
||||
}
|
||||
|
||||
// deepNil checks that underlying value of obj is nil.
|
||||
func deepNil(got interface{}) bool {
|
||||
if got == nil {
|
||||
return true
|
||||
}
|
||||
value := reflect.ValueOf(obj)
|
||||
value := reflect.ValueOf(got)
|
||||
switch value.Kind() {
|
||||
case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice, reflect.UnsafePointer:
|
||||
return value.IsNil()
|
||||
|
@ -51,6 +51,11 @@ func ErrorContains(tb assertions.AssertionTestingTB, want string, err error, msg
|
||||
assertions.ErrorContains(tb.Fatalf, want, err, msg...)
|
||||
}
|
||||
|
||||
// IsNil asserts that the observed value is nil.
|
||||
func IsNil(tb assertions.AssertionTestingTB, got interface{}, msg ...interface{}) {
|
||||
assertions.IsNil(tb.Fatalf, got, msg...)
|
||||
}
|
||||
|
||||
// NotNil asserts that passed value is not nil.
|
||||
func NotNil(tb assertions.AssertionTestingTB, obj interface{}, msg ...interface{}) {
|
||||
assertions.NotNil(tb.Fatalf, obj, msg...)
|
||||
|
@ -30,17 +30,15 @@ type denebBlockGenerator struct {
|
||||
nblobs int
|
||||
sign bool
|
||||
sk bls.SecretKey
|
||||
pk bls.PublicKey
|
||||
proposer primitives.ValidatorIndex
|
||||
valRoot []byte
|
||||
}
|
||||
|
||||
func WithProposerSigning(idx primitives.ValidatorIndex, sk bls.SecretKey, pk bls.PublicKey, valRoot []byte) DenebBlockGeneratorOption {
|
||||
func WithProposerSigning(idx primitives.ValidatorIndex, sk bls.SecretKey, valRoot []byte) DenebBlockGeneratorOption {
|
||||
return func(g *denebBlockGenerator) {
|
||||
g.sign = true
|
||||
g.proposer = idx
|
||||
g.sk = sk
|
||||
g.pk = pk
|
||||
g.valRoot = valRoot
|
||||
}
|
||||
}
|
||||
@ -97,6 +95,7 @@ func GenerateTestDenebBlockWithSidecar(t *testing.T, parent [32]byte, slot primi
|
||||
block.Block.Body.ExecutionPayload = payload
|
||||
block.Block.Slot = g.slot
|
||||
block.Block.ParentRoot = g.parent[:]
|
||||
block.Block.ProposerIndex = g.proposer
|
||||
commitments := make([][48]byte, g.nblobs)
|
||||
block.Block.Body.BlobKzgCommitments = make([][]byte, g.nblobs)
|
||||
for i := range commitments {
|
||||
|
@ -98,6 +98,16 @@ func EpochStart(epoch primitives.Epoch) (primitives.Slot, error) {
|
||||
return slot, nil
|
||||
}
|
||||
|
||||
// UnsafeEpochStart is a version of EpochStart that panics if there is an overflow. It can be safely used by code
|
||||
// that first guarantees epoch <= MaxSafeEpoch.
|
||||
func UnsafeEpochStart(epoch primitives.Epoch) primitives.Slot {
|
||||
es, err := EpochStart(epoch)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return es
|
||||
}
|
||||
|
||||
// EpochEnd returns the last slot number of the
|
||||
// current epoch.
|
||||
func EpochEnd(epoch primitives.Epoch) (primitives.Slot, error) {
|
||||
@ -271,3 +281,8 @@ func WithinVotingWindow(genesisTime uint64, slot primitives.Slot) bool {
|
||||
votingWindow := params.BeaconConfig().SecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
|
||||
return time.Since(StartTime(genesisTime, slot)) < time.Duration(votingWindow)*time.Second
|
||||
}
|
||||
|
||||
// MaxSafeEpoch gives the largest epoch value that can be safely converted to a slot.
|
||||
func MaxSafeEpoch() primitives.Epoch {
|
||||
return primitives.Epoch(math.MaxUint64 / uint64(params.BeaconConfig().SlotsPerEpoch))
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user