Backfill data columns (#15580)

**What type of PR is this?**

Feature

**What does this PR do? Why is it needed?**

Adds data column support to backfill.

**Acknowledgements**

- [x] I have read
[CONTRIBUTING.md](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md).
- [x] I have included a uniquely named [changelog fragment
file](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md#maintaining-changelogmd).
- [x] I have added a description to this PR with sufficient context for
reviewers to understand this PR.

---------

Co-authored-by: Kasey <kasey@users.noreply.github.com>
Co-authored-by: Claude <noreply@anthropic.com>
Co-authored-by: Preston Van Loon <preston@pvl.dev>
This commit is contained in:
kasey
2025-12-02 10:19:32 -05:00
committed by GitHub
parent 2773bdef89
commit 61de11e2c4
84 changed files with 10944 additions and 964 deletions

View File

@@ -134,7 +134,7 @@ func getStateVersionAndPayload(st state.BeaconState) (int, interfaces.ExecutionD
return preStateVersion, preStateHeader, nil
}
func (s *Service) onBlockBatch(ctx context.Context, blks []consensusblocks.ROBlock, avs das.AvailabilityStore) error {
func (s *Service) onBlockBatch(ctx context.Context, blks []consensusblocks.ROBlock, avs das.AvailabilityChecker) error {
ctx, span := trace.StartSpan(ctx, "blockChain.onBlockBatch")
defer span.End()
@@ -306,7 +306,7 @@ func (s *Service) onBlockBatch(ctx context.Context, blks []consensusblocks.ROBlo
return s.saveHeadNoDB(ctx, lastB, lastBR, preState, !isValidPayload)
}
func (s *Service) areSidecarsAvailable(ctx context.Context, avs das.AvailabilityStore, roBlock consensusblocks.ROBlock) error {
func (s *Service) areSidecarsAvailable(ctx context.Context, avs das.AvailabilityChecker, roBlock consensusblocks.ROBlock) error {
blockVersion := roBlock.Version()
block := roBlock.Block()
slot := block.Slot()

View File

@@ -39,8 +39,8 @@ var epochsSinceFinalityExpandCache = primitives.Epoch(4)
// BlockReceiver interface defines the methods of chain service for receiving and processing new blocks.
type BlockReceiver interface {
ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityStore) error
ReceiveBlockBatch(ctx context.Context, blocks []blocks.ROBlock, avs das.AvailabilityStore) error
ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityChecker) error
ReceiveBlockBatch(ctx context.Context, blocks []blocks.ROBlock, avs das.AvailabilityChecker) error
HasBlock(ctx context.Context, root [32]byte) bool
RecentBlockSlot(root [32]byte) (primitives.Slot, error)
BlockBeingSynced([32]byte) bool
@@ -69,7 +69,7 @@ type SlashingReceiver interface {
// 1. Validate block, apply state transition and update checkpoints
// 2. Apply fork choice to the processed block
// 3. Save latest head info
func (s *Service) ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityStore) error {
func (s *Service) ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityChecker) error {
ctx, span := trace.StartSpan(ctx, "blockChain.ReceiveBlock")
defer span.End()
// Return early if the block is blacklisted
@@ -242,7 +242,7 @@ func (s *Service) validateExecutionAndConsensus(
return postState, isValidPayload, nil
}
func (s *Service) handleDA(ctx context.Context, avs das.AvailabilityStore, block blocks.ROBlock) (time.Duration, error) {
func (s *Service) handleDA(ctx context.Context, avs das.AvailabilityChecker, block blocks.ROBlock) (time.Duration, error) {
var err error
start := time.Now()
if avs != nil {
@@ -332,7 +332,7 @@ func (s *Service) executePostFinalizationTasks(ctx context.Context, finalizedSta
// ReceiveBlockBatch processes the whole block batch at once, assuming the block batch is linear ,transitioning
// the state, performing batch verification of all collected signatures and then performing the appropriate
// actions for a block post-transition.
func (s *Service) ReceiveBlockBatch(ctx context.Context, blocks []blocks.ROBlock, avs das.AvailabilityStore) error {
func (s *Service) ReceiveBlockBatch(ctx context.Context, blocks []blocks.ROBlock, avs das.AvailabilityChecker) error {
ctx, span := trace.StartSpan(ctx, "blockChain.ReceiveBlockBatch")
defer span.End()

View File

@@ -275,7 +275,7 @@ func (s *ChainService) ReceiveBlockInitialSync(ctx context.Context, block interf
}
// ReceiveBlockBatch processes blocks in batches from initial-sync.
func (s *ChainService) ReceiveBlockBatch(ctx context.Context, blks []blocks.ROBlock, _ das.AvailabilityStore) error {
func (s *ChainService) ReceiveBlockBatch(ctx context.Context, blks []blocks.ROBlock, _ das.AvailabilityChecker) error {
if s.State == nil {
return ErrNilState
}
@@ -305,7 +305,7 @@ func (s *ChainService) ReceiveBlockBatch(ctx context.Context, blks []blocks.ROBl
}
// ReceiveBlock mocks ReceiveBlock method in chain service.
func (s *ChainService) ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, _ [32]byte, _ das.AvailabilityStore) error {
func (s *ChainService) ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, _ [32]byte, _ das.AvailabilityChecker) error {
if s.ReceiveBlockMockErr != nil {
return s.ReceiveBlockMockErr
}

View File

@@ -2,6 +2,7 @@ package peerdas
import (
"encoding/binary"
"maps"
"sync"
"github.com/ethereum/go-ethereum/p2p/enode"
@@ -107,3 +108,102 @@ func computeInfoCacheKey(nodeID enode.ID, custodyGroupCount uint64) [nodeInfoCac
return key
}
// ColumnIndices represents as a set of ColumnIndices. This could be the set of indices that a node is required to custody,
// the set that a peer custodies, missing indices for a given block, indices that are present on disk, etc.
type ColumnIndices map[uint64]struct{}
// Has returns true if the index is present in the ColumnIndices.
func (ci ColumnIndices) Has(index uint64) bool {
_, ok := ci[index]
return ok
}
// Count returns the number of indices present in the ColumnIndices.
func (ci ColumnIndices) Count() int {
return len(ci)
}
// Set sets the index in the ColumnIndices.
func (ci ColumnIndices) Set(index uint64) {
ci[index] = struct{}{}
}
// Unset removes the index from the ColumnIndices.
func (ci ColumnIndices) Unset(index uint64) {
delete(ci, index)
}
// Copy creates a copy of the ColumnIndices.
func (ci ColumnIndices) Copy() ColumnIndices {
newCi := make(ColumnIndices, len(ci))
maps.Copy(newCi, ci)
return newCi
}
// Intersection returns a new ColumnIndices that contains only the indices that are present in both ColumnIndices.
func (ci ColumnIndices) Intersection(other ColumnIndices) ColumnIndices {
result := make(ColumnIndices)
for index := range ci {
if other.Has(index) {
result.Set(index)
}
}
return result
}
// Merge mutates the receiver so that any index that is set in either of
// the two ColumnIndices is set in the receiver after the function finishes.
// It does not mutate the other ColumnIndices given as a function argument.
func (ci ColumnIndices) Merge(other ColumnIndices) {
for index := range other {
ci.Set(index)
}
}
// ToMap converts a ColumnIndices into a map[uint64]struct{}.
// In the future ColumnIndices may be changed to a bit map, so using
// ToMap will ensure forwards-compatibility.
func (ci ColumnIndices) ToMap() map[uint64]struct{} {
return ci.Copy()
}
// ToSlice converts a ColumnIndices into a slice of uint64 indices.
func (ci ColumnIndices) ToSlice() []uint64 {
indices := make([]uint64, 0, len(ci))
for index := range ci {
indices = append(indices, index)
}
return indices
}
// NewColumnIndicesFromSlice creates a ColumnIndices from a slice of uint64.
func NewColumnIndicesFromSlice(indices []uint64) ColumnIndices {
ci := make(ColumnIndices, len(indices))
for _, index := range indices {
ci[index] = struct{}{}
}
return ci
}
// NewColumnIndicesFromMap creates a ColumnIndices from a map[uint64]bool. This kind of map
// is used in several places in peerdas code. Converting from this map type to ColumnIndices
// will allow us to move ColumnIndices underlying type to a bitmap in the future and avoid
// lots of loops for things like intersections/unions or copies.
func NewColumnIndicesFromMap(indices map[uint64]bool) ColumnIndices {
ci := make(ColumnIndices, len(indices))
for index, set := range indices {
if !set {
continue
}
ci[index] = struct{}{}
}
return ci
}
// NewColumnIndices creates an empty ColumnIndices.
// In the future ColumnIndices may change from a reference type to a value type,
// so using this constructor will ensure forwards-compatibility.
func NewColumnIndices() ColumnIndices {
return make(ColumnIndices)
}

View File

@@ -25,3 +25,10 @@ func TestInfo(t *testing.T) {
require.DeepEqual(t, expectedDataColumnsSubnets, actual.DataColumnsSubnets)
}
}
func TestNewColumnIndicesFromMap(t *testing.T) {
t.Run("nil map", func(t *testing.T) {
ci := peerdas.NewColumnIndicesFromMap(nil)
require.Equal(t, 0, ci.Count())
})
}

View File

@@ -4,14 +4,19 @@ go_library(
name = "go_default_library",
srcs = [
"availability_blobs.go",
"availability_columns.go",
"bisect.go",
"blob_cache.go",
"data_column_cache.go",
"iface.go",
"log.go",
"mock.go",
"needs.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/das",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
@@ -21,6 +26,7 @@ go_library(
"//runtime/logging:go_default_library",
"//runtime/version:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enode:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
@@ -30,11 +36,14 @@ go_test(
name = "go_default_test",
srcs = [
"availability_blobs_test.go",
"availability_columns_test.go",
"blob_cache_test.go",
"data_column_cache_test.go",
"needs_test.go",
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
@@ -45,6 +54,7 @@ go_test(
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enode:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
)

View File

@@ -11,9 +11,8 @@ import (
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/runtime/logging"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
"github.com/sirupsen/logrus"
)
var (
@@ -24,12 +23,13 @@ var (
// This implementation will hold any blobs passed to Persist until the IsDataAvailable is called for their
// block, at which time they will undergo full verification and be saved to the disk.
type LazilyPersistentStoreBlob struct {
store *filesystem.BlobStorage
cache *blobCache
verifier BlobBatchVerifier
store *filesystem.BlobStorage
cache *blobCache
verifier BlobBatchVerifier
shouldRetain RetentionChecker
}
var _ AvailabilityStore = &LazilyPersistentStoreBlob{}
var _ AvailabilityChecker = &LazilyPersistentStoreBlob{}
// BlobBatchVerifier enables LazyAvailabilityStore to manage the verification process
// going from ROBlob->VerifiedROBlob, while avoiding the decision of which individual verifications
@@ -42,11 +42,12 @@ type BlobBatchVerifier interface {
// NewLazilyPersistentStore creates a new LazilyPersistentStore. This constructor should always be used
// when creating a LazilyPersistentStore because it needs to initialize the cache under the hood.
func NewLazilyPersistentStore(store *filesystem.BlobStorage, verifier BlobBatchVerifier) *LazilyPersistentStoreBlob {
func NewLazilyPersistentStore(store *filesystem.BlobStorage, verifier BlobBatchVerifier, shouldRetain RetentionChecker) *LazilyPersistentStoreBlob {
return &LazilyPersistentStoreBlob{
store: store,
cache: newBlobCache(),
verifier: verifier,
store: store,
cache: newBlobCache(),
verifier: verifier,
shouldRetain: shouldRetain,
}
}
@@ -66,9 +67,6 @@ func (s *LazilyPersistentStoreBlob) Persist(current primitives.Slot, sidecars ..
}
}
}
if !params.WithinDAPeriod(slots.ToEpoch(sidecars[0].Slot()), slots.ToEpoch(current)) {
return nil
}
key := keyFromSidecar(sidecars[0])
entry := s.cache.ensure(key)
for _, blobSidecar := range sidecars {
@@ -81,8 +79,17 @@ func (s *LazilyPersistentStoreBlob) Persist(current primitives.Slot, sidecars ..
// IsDataAvailable returns nil if all the commitments in the given block are persisted to the db and have been verified.
// BlobSidecars already in the db are assumed to have been previously verified against the block.
func (s *LazilyPersistentStoreBlob) IsDataAvailable(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error {
blockCommitments, err := commitmentsToCheck(b, current)
func (s *LazilyPersistentStoreBlob) IsDataAvailable(ctx context.Context, current primitives.Slot, blks ...blocks.ROBlock) error {
for _, b := range blks {
if err := s.checkOne(ctx, current, b); err != nil {
return err
}
}
return nil
}
func (s *LazilyPersistentStoreBlob) checkOne(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error {
blockCommitments, err := commitmentsToCheck(b, s.shouldRetain)
if err != nil {
return errors.Wrapf(err, "could not check data availability for block %#x", b.Root())
}
@@ -112,7 +119,7 @@ func (s *LazilyPersistentStoreBlob) IsDataAvailable(ctx context.Context, current
ok := errors.As(err, &me)
if ok {
fails := me.Failures()
lf := make(log.Fields, len(fails))
lf := make(logrus.Fields, len(fails))
for i := range fails {
lf[fmt.Sprintf("fail_%d", i)] = fails[i].Error()
}
@@ -131,13 +138,12 @@ func (s *LazilyPersistentStoreBlob) IsDataAvailable(ctx context.Context, current
return nil
}
func commitmentsToCheck(b blocks.ROBlock, current primitives.Slot) ([][]byte, error) {
func commitmentsToCheck(b blocks.ROBlock, shouldRetain RetentionChecker) ([][]byte, error) {
if b.Version() < version.Deneb {
return nil, nil
}
// We are only required to check within MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUEST
if !params.WithinDAPeriod(slots.ToEpoch(b.Block().Slot()), slots.ToEpoch(current)) {
if !shouldRetain(b.Block().Slot()) {
return nil, nil
}

View File

@@ -17,6 +17,10 @@ import (
errors "github.com/pkg/errors"
)
func testShouldRetainAlways(s primitives.Slot) bool {
return true
}
func Test_commitmentsToCheck(t *testing.T) {
params.SetupTestConfigCleanup(t)
params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2
@@ -30,11 +34,12 @@ func Test_commitmentsToCheck(t *testing.T) {
commits[i] = bytesutil.PadTo([]byte{byte(i)}, 48)
}
cases := []struct {
name string
commits [][]byte
block func(*testing.T) blocks.ROBlock
slot primitives.Slot
err error
name string
commits [][]byte
block func(*testing.T) blocks.ROBlock
slot primitives.Slot
err error
shouldRetain RetentionChecker
}{
{
name: "pre deneb",
@@ -60,6 +65,7 @@ func Test_commitmentsToCheck(t *testing.T) {
require.NoError(t, err)
return rb
},
shouldRetain: testShouldRetainAlways,
commits: func() [][]byte {
mb := params.GetNetworkScheduleEntry(slots.ToEpoch(fulu + 100)).MaxBlobsPerBlock
return commits[:mb]
@@ -79,7 +85,8 @@ func Test_commitmentsToCheck(t *testing.T) {
require.NoError(t, err)
return rb
},
slot: fulu + windowSlots + 1,
shouldRetain: func(s primitives.Slot) bool { return false },
slot: fulu + windowSlots + 1,
},
{
name: "excessive commitments",
@@ -97,14 +104,15 @@ func Test_commitmentsToCheck(t *testing.T) {
require.Equal(t, true, len(c) > params.BeaconConfig().MaxBlobsPerBlock(sb.Block().Slot()))
return rb
},
slot: windowSlots + 1,
err: errIndexOutOfBounds,
shouldRetain: testShouldRetainAlways,
slot: windowSlots + 1,
err: errIndexOutOfBounds,
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
b := c.block(t)
co, err := commitmentsToCheck(b, c.slot)
co, err := commitmentsToCheck(b, c.shouldRetain)
if c.err != nil {
require.ErrorIs(t, err, c.err)
} else {
@@ -126,7 +134,7 @@ func TestLazilyPersistent_Missing(t *testing.T) {
blk, blobSidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, ds, 3)
mbv := &mockBlobBatchVerifier{t: t, scs: blobSidecars}
as := NewLazilyPersistentStore(store, mbv)
as := NewLazilyPersistentStore(store, mbv, testShouldRetainAlways)
// Only one commitment persisted, should return error with other indices
require.NoError(t, as.Persist(ds, blobSidecars[2]))
@@ -153,7 +161,7 @@ func TestLazilyPersistent_Mismatch(t *testing.T) {
mbv := &mockBlobBatchVerifier{t: t, err: errors.New("kzg check should not run")}
blobSidecars[0].KzgCommitment = bytesutil.PadTo([]byte("nope"), 48)
as := NewLazilyPersistentStore(store, mbv)
as := NewLazilyPersistentStore(store, mbv, testShouldRetainAlways)
// Only one commitment persisted, should return error with other indices
require.NoError(t, as.Persist(ds, blobSidecars[0]))
@@ -166,11 +174,11 @@ func TestLazyPersistOnceCommitted(t *testing.T) {
ds := util.SlotAtEpoch(t, params.BeaconConfig().DenebForkEpoch)
_, blobSidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, ds, 6)
as := NewLazilyPersistentStore(filesystem.NewEphemeralBlobStorage(t), &mockBlobBatchVerifier{})
as := NewLazilyPersistentStore(filesystem.NewEphemeralBlobStorage(t), &mockBlobBatchVerifier{}, testShouldRetainAlways)
// stashes as expected
require.NoError(t, as.Persist(ds, blobSidecars...))
// ignores duplicates
require.ErrorIs(t, as.Persist(ds, blobSidecars...), ErrDuplicateSidecar)
require.ErrorIs(t, as.Persist(ds, blobSidecars...), errDuplicateSidecar)
// ignores index out of bound
blobSidecars[0].Index = 6
@@ -183,7 +191,7 @@ func TestLazyPersistOnceCommitted(t *testing.T) {
require.NoError(t, as.Persist(slotOOB, moreBlobSidecars[0]))
// doesn't ignore new sidecars with a different block root
require.NoError(t, as.Persist(ds, moreBlobSidecars...))
require.NoError(t, as.Persist(ds, moreBlobSidecars[1:]...))
}
type mockBlobBatchVerifier struct {

View File

@@ -0,0 +1,244 @@
package das
import (
"context"
"io"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/ethereum/go-ethereum/p2p/enode"
errors "github.com/pkg/errors"
)
// LazilyPersistentStoreColumn is an implementation of AvailabilityStore to be used when batch syncing data columns.
// This implementation will hold any data columns passed to Persist until the IsDataAvailable is called for their
// block, at which time they will undergo full verification and be saved to the disk.
type LazilyPersistentStoreColumn struct {
store *filesystem.DataColumnStorage
cache *dataColumnCache
newDataColumnsVerifier verification.NewDataColumnsVerifier
custody *custodyRequirement
bisector Bisector
shouldRetain RetentionChecker
}
var _ AvailabilityChecker = &LazilyPersistentStoreColumn{}
// DataColumnsVerifier enables LazilyPersistentStoreColumn to manage the verification process
// going from RODataColumn->VerifiedRODataColumn, while avoiding the decision of which individual verifications
// to run and in what order. Since LazilyPersistentStoreColumn always tries to verify and save data columns only when
// they are all available, the interface takes a slice of data column sidecars.
type DataColumnsVerifier interface {
VerifiedRODataColumns(ctx context.Context, blk blocks.ROBlock, scs []blocks.RODataColumn) ([]blocks.VerifiedRODataColumn, error)
}
// NewLazilyPersistentStoreColumn creates a new LazilyPersistentStoreColumn.
// WARNING: The resulting LazilyPersistentStoreColumn is NOT thread-safe.
func NewLazilyPersistentStoreColumn(
store *filesystem.DataColumnStorage,
newDataColumnsVerifier verification.NewDataColumnsVerifier,
nodeID enode.ID,
cgc uint64,
bisector Bisector,
shouldRetain RetentionChecker,
) *LazilyPersistentStoreColumn {
return &LazilyPersistentStoreColumn{
store: store,
cache: newDataColumnCache(),
newDataColumnsVerifier: newDataColumnsVerifier,
custody: &custodyRequirement{nodeID: nodeID, cgc: cgc},
bisector: bisector,
shouldRetain: shouldRetain,
}
}
// PersistColumns adds columns to the working column cache. Columns stored in this cache will be persisted
// for at least as long as the node is running. Once IsDataAvailable succeeds, all columns referenced
// by the given block are guaranteed to be persisted for the remainder of the retention period.
func (s *LazilyPersistentStoreColumn) Persist(_ primitives.Slot, sidecars ...blocks.RODataColumn) error {
for _, sidecar := range sidecars {
if err := s.cache.stash(sidecar); err != nil {
return errors.Wrap(err, "stash DataColumnSidecar")
}
}
return nil
}
// IsDataAvailable returns nil if all the commitments in the given block are persisted to the db and have been verified.
// DataColumnsSidecars already in the db are assumed to have been previously verified against the block.
func (s *LazilyPersistentStoreColumn) IsDataAvailable(ctx context.Context, _ primitives.Slot, blks ...blocks.ROBlock) error {
toVerify := make([]blocks.RODataColumn, 0)
for _, block := range blks {
indices, err := s.required(block)
if err != nil {
return errors.Wrapf(err, "full commitments to check with block root `%#x`", block.Root())
}
if indices.Count() == 0 {
continue
}
key := keyFromBlock(block)
entry := s.cache.entry(key)
toVerify, err = entry.append(toVerify, IndicesNotStored(s.store.Summary(block.Root()), indices))
if err != nil {
return errors.Wrap(err, "entry filter")
}
}
if err := s.verifyAndSave(toVerify); err != nil {
log.Warn("Batch verification failed, bisecting columns by peer")
if err := s.bisectVerification(toVerify); err != nil {
return errors.Wrap(err, "bisect verification")
}
}
s.cache.cleanup(blks)
return nil
}
// required returns the set of column indices to check for a given block.
func (s *LazilyPersistentStoreColumn) required(block blocks.ROBlock) (peerdas.ColumnIndices, error) {
if !s.shouldRetain(block.Block().Slot()) {
return peerdas.NewColumnIndices(), nil
}
// If there are any commitments in the block, there are blobs,
// and if there are blobs, we need the columns bisecting those blobs.
commitments, err := block.Block().Body().BlobKzgCommitments()
if err != nil {
return nil, errors.Wrap(err, "blob KZG commitments")
}
// No DA check needed if the block has no blobs.
if len(commitments) == 0 {
return peerdas.NewColumnIndices(), nil
}
return s.custody.required()
}
// verifyAndSave calls Save on the column store if the columns pass verification.
func (s *LazilyPersistentStoreColumn) verifyAndSave(columns []blocks.RODataColumn) error {
if len(columns) == 0 {
return nil
}
verified, err := s.verifyColumns(columns)
if err != nil {
return errors.Wrap(err, "verify columns")
}
if err := s.store.Save(verified); err != nil {
return errors.Wrap(err, "save data column sidecars")
}
return nil
}
func (s *LazilyPersistentStoreColumn) verifyColumns(columns []blocks.RODataColumn) ([]blocks.VerifiedRODataColumn, error) {
if len(columns) == 0 {
return nil, nil
}
verifier := s.newDataColumnsVerifier(columns, verification.ByRangeRequestDataColumnSidecarRequirements)
if err := verifier.ValidFields(); err != nil {
return nil, errors.Wrap(err, "valid fields")
}
if err := verifier.SidecarInclusionProven(); err != nil {
return nil, errors.Wrap(err, "sidecar inclusion proven")
}
if err := verifier.SidecarKzgProofVerified(); err != nil {
return nil, errors.Wrap(err, "sidecar KZG proof verified")
}
return verifier.VerifiedRODataColumns()
}
// bisectVerification is used when verification of a batch of columns fails. Since the batch could
// span multiple blocks or have been fetched from multiple peers, this pattern enables code using the
// store to break the verification into smaller units and learn the results, in order to plan to retry
// retrieval of the unusable columns.
func (s *LazilyPersistentStoreColumn) bisectVerification(columns []blocks.RODataColumn) error {
if len(columns) == 0 {
return nil
}
if s.bisector == nil {
return errors.New("bisector not initialized")
}
iter, err := s.bisector.Bisect(columns)
if err != nil {
return errors.Wrap(err, "Bisector.Bisect")
}
// It's up to the bisector how to chunk up columns for verification,
// which could be by block, or by peer, or any other strategy.
// For the purposes of range syncing or backfill this will be by peer,
// so that the node can learn which peer is giving us bad data and downscore them.
for columns, err := iter.Next(); columns != nil; columns, err = iter.Next() {
if err != nil {
if !errors.Is(err, io.EOF) {
return errors.Wrap(err, "Bisector.Next")
}
break // io.EOF signals end of iteration
}
// We save the parts of the batch that have been verified successfully even though we don't know
// if all columns for the block will be available until the block is imported.
if err := s.verifyAndSave(s.columnsNotStored(columns)); err != nil {
iter.OnError(err)
continue
}
}
// This should give us a single error representing any unresolved errors seen via onError.
return iter.Error()
}
// columnsNotStored filters the list of ROColumnSidecars to only include those that are not found in the storage summary.
func (s *LazilyPersistentStoreColumn) columnsNotStored(sidecars []blocks.RODataColumn) []blocks.RODataColumn {
// We use this method to filter a set of sidecars that were previously seen to be unavailable on disk. So our base assumption
// is that they are still available and we don't need to copy the list. Instead we make a slice of any indices that are unexpectedly
// stored and only when we find that the storage view has changed do we need to create a new slice.
stored := make(map[int]struct{}, 0)
lastRoot := [32]byte{}
var sum filesystem.DataColumnStorageSummary
for i, sc := range sidecars {
if sc.BlockRoot() != lastRoot {
sum = s.store.Summary(sc.BlockRoot())
lastRoot = sc.BlockRoot()
}
if sum.HasIndex(sc.Index) {
stored[i] = struct{}{}
}
}
// If the view on storage hasn't changed, return the original list.
if len(stored) == 0 {
return sidecars
}
shift := 0
for i := range sidecars {
if _, ok := stored[i]; ok {
// If the index is stored, skip and overwrite it.
// Track how many spaces down to shift unseen sidecars (to overwrite the previously shifted or seen).
shift++
continue
}
if shift > 0 {
// If the index is not stored and we have seen stored indices,
// we need to shift the current index down.
sidecars[i-shift] = sidecars[i]
}
}
return sidecars[:len(sidecars)-shift]
}
type custodyRequirement struct {
nodeID enode.ID
cgc uint64 // custody group count
indices peerdas.ColumnIndices
}
func (c *custodyRequirement) required() (peerdas.ColumnIndices, error) {
peerInfo, _, err := peerdas.Info(c.nodeID, c.cgc)
if err != nil {
return peerdas.NewColumnIndices(), errors.Wrap(err, "peer info")
}
return peerdas.NewColumnIndicesFromMap(peerInfo.CustodyColumns), nil
}

View File

@@ -0,0 +1,908 @@
package das
import (
"context"
"fmt"
"io"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/pkg/errors"
)
func mockShouldRetain(current primitives.Epoch) RetentionChecker {
return func(slot primitives.Slot) bool {
return params.WithinDAPeriod(slots.ToEpoch(slot), current)
}
}
var commitments = [][]byte{
bytesutil.PadTo([]byte("a"), 48),
bytesutil.PadTo([]byte("b"), 48),
bytesutil.PadTo([]byte("c"), 48),
bytesutil.PadTo([]byte("d"), 48),
}
func TestPersist(t *testing.T) {
t.Run("no sidecars", func(t *testing.T) {
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, nil, enode.ID{}, 0, nil, mockShouldRetain(0))
err := lazilyPersistentStoreColumns.Persist(0)
require.NoError(t, err)
require.Equal(t, 0, len(lazilyPersistentStoreColumns.cache.entries))
})
t.Run("outside DA period", func(t *testing.T) {
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
dataColumnParamsByBlockRoot := []util.DataColumnParam{
{Slot: 1, Index: 1},
}
var current primitives.Slot = 1_000_000
sr := mockShouldRetain(slots.ToEpoch(current))
roSidecars, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, nil, enode.ID{}, 0, nil, sr)
err := lazilyPersistentStoreColumns.Persist(current, roSidecars...)
require.NoError(t, err)
require.Equal(t, len(roSidecars), len(lazilyPersistentStoreColumns.cache.entries))
})
t.Run("nominal", func(t *testing.T) {
const slot = 42
store := filesystem.NewEphemeralDataColumnStorage(t)
dataColumnParamsByBlockRoot := []util.DataColumnParam{
{Slot: slot, Index: 1},
{Slot: slot, Index: 5},
}
roSidecars, roDataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
avs := NewLazilyPersistentStoreColumn(store, nil, enode.ID{}, 0, nil, mockShouldRetain(slots.ToEpoch(slot)))
err := avs.Persist(slot, roSidecars...)
require.NoError(t, err)
require.Equal(t, 1, len(avs.cache.entries))
key := cacheKey{slot: slot, root: roDataColumns[0].BlockRoot()}
entry, ok := avs.cache.entries[key]
require.Equal(t, true, ok)
summary := store.Summary(key.root)
// A call to Persist does NOT save the sidecars to disk.
require.Equal(t, uint64(0), summary.Count())
require.Equal(t, len(roSidecars), len(entry.scs))
idx1 := entry.scs[1]
require.NotNil(t, idx1)
require.DeepSSZEqual(t, roDataColumns[0].BlockRoot(), idx1.BlockRoot())
idx5 := entry.scs[5]
require.NotNil(t, idx5)
require.DeepSSZEqual(t, roDataColumns[1].BlockRoot(), idx5.BlockRoot())
for i, roDataColumn := range entry.scs {
if map[uint64]bool{1: true, 5: true}[i] {
continue
}
require.IsNil(t, roDataColumn)
}
})
}
func TestIsDataAvailable(t *testing.T) {
params.SetupTestConfigCleanup(t)
params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2
newDataColumnsVerifier := func(dataColumnSidecars []blocks.RODataColumn, _ []verification.Requirement) verification.DataColumnsVerifier {
return &mockDataColumnsVerifier{t: t, dataColumnSidecars: dataColumnSidecars}
}
ctx := t.Context()
t.Run("without commitments", func(t *testing.T) {
signedBeaconBlockFulu := util.NewBeaconBlockFulu()
signedRoBlock := newSignedRoBlock(t, signedBeaconBlockFulu)
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, newDataColumnsVerifier, enode.ID{}, 0, nil, mockShouldRetain(0))
err := lazilyPersistentStoreColumns.IsDataAvailable(ctx, 0, signedRoBlock)
require.NoError(t, err)
})
t.Run("with commitments", func(t *testing.T) {
signedBeaconBlockFulu := util.NewBeaconBlockFulu()
signedBeaconBlockFulu.Block.Slot = primitives.Slot(params.BeaconConfig().FuluForkEpoch) * params.BeaconConfig().SlotsPerEpoch
signedBeaconBlockFulu.Block.Body.BlobKzgCommitments = commitments
signedRoBlock := newSignedRoBlock(t, signedBeaconBlockFulu)
block := signedRoBlock.Block()
slot := block.Slot()
proposerIndex := block.ProposerIndex()
parentRoot := block.ParentRoot()
stateRoot := block.StateRoot()
bodyRoot, err := block.Body().HashTreeRoot()
require.NoError(t, err)
root := signedRoBlock.Root()
storage := filesystem.NewEphemeralDataColumnStorage(t)
indices := []uint64{1, 17, 19, 42, 75, 87, 102, 117}
avs := NewLazilyPersistentStoreColumn(storage, newDataColumnsVerifier, enode.ID{}, uint64(len(indices)), nil, mockShouldRetain(slots.ToEpoch(slot)))
dcparams := make([]util.DataColumnParam, 0, len(indices))
for _, index := range indices {
dataColumnParams := util.DataColumnParam{
Index: index,
KzgCommitments: commitments,
Slot: slot,
ProposerIndex: proposerIndex,
ParentRoot: parentRoot[:],
StateRoot: stateRoot[:],
BodyRoot: bodyRoot[:],
}
dcparams = append(dcparams, dataColumnParams)
}
_, verifiedRoDataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, dcparams)
key := keyFromBlock(signedRoBlock)
entry := avs.cache.entry(key)
defer avs.cache.delete(key)
for _, verifiedRoDataColumn := range verifiedRoDataColumns {
err := entry.stash(verifiedRoDataColumn.RODataColumn)
require.NoError(t, err)
}
err = avs.IsDataAvailable(ctx, slot, signedRoBlock)
require.NoError(t, err)
actual, err := storage.Get(root, indices)
require.NoError(t, err)
//summary := storage.Summary(root)
require.Equal(t, len(verifiedRoDataColumns), len(actual))
//require.Equal(t, uint64(len(indices)), summary.Count())
//require.DeepSSZEqual(t, verifiedRoDataColumns, actual)
})
}
func TestRetentionWindow(t *testing.T) {
windowSlots, err := slots.EpochEnd(params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest)
require.NoError(t, err)
fuluSlot, err := slots.EpochStart(params.BeaconConfig().FuluForkEpoch)
require.NoError(t, err)
numberOfColumns := fieldparams.NumberOfColumns
testCases := []struct {
name string
commitments [][]byte
block func(*testing.T) blocks.ROBlock
slot primitives.Slot
wantedCols int
}{
{
name: "Pre-Fulu block",
block: func(t *testing.T) blocks.ROBlock {
return newSignedRoBlock(t, util.NewBeaconBlockElectra())
},
},
{
name: "Commitments outside data availability window",
block: func(t *testing.T) blocks.ROBlock {
beaconBlockElectra := util.NewBeaconBlockElectra()
// Block is from slot 0, "current slot" is window size +1 (so outside the window)
beaconBlockElectra.Block.Body.BlobKzgCommitments = commitments
return newSignedRoBlock(t, beaconBlockElectra)
},
slot: fuluSlot + windowSlots,
},
{
name: "Commitments within data availability window",
block: func(t *testing.T) blocks.ROBlock {
signedBeaconBlockFulu := util.NewBeaconBlockFulu()
signedBeaconBlockFulu.Block.Body.BlobKzgCommitments = commitments
signedBeaconBlockFulu.Block.Slot = fuluSlot + windowSlots - 1
return newSignedRoBlock(t, signedBeaconBlockFulu)
},
commitments: commitments,
slot: fuluSlot + windowSlots,
wantedCols: numberOfColumns,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
b := tc.block(t)
s := NewLazilyPersistentStoreColumn(nil, nil, enode.ID{}, uint64(numberOfColumns), nil, mockShouldRetain(slots.ToEpoch(tc.slot)))
indices, err := s.required(b)
require.NoError(t, err)
require.Equal(t, tc.wantedCols, len(indices))
})
}
}
func newSignedRoBlock(t *testing.T, signedBeaconBlock any) blocks.ROBlock {
sb, err := blocks.NewSignedBeaconBlock(signedBeaconBlock)
require.NoError(t, err)
rb, err := blocks.NewROBlock(sb)
require.NoError(t, err)
return rb
}
type mockDataColumnsVerifier struct {
t *testing.T
dataColumnSidecars []blocks.RODataColumn
validCalled, SidecarInclusionProvenCalled, SidecarKzgProofVerifiedCalled bool
}
var _ verification.DataColumnsVerifier = &mockDataColumnsVerifier{}
func (m *mockDataColumnsVerifier) VerifiedRODataColumns() ([]blocks.VerifiedRODataColumn, error) {
require.Equal(m.t, true, m.validCalled && m.SidecarInclusionProvenCalled && m.SidecarKzgProofVerifiedCalled)
verifiedDataColumnSidecars := make([]blocks.VerifiedRODataColumn, 0, len(m.dataColumnSidecars))
for _, dataColumnSidecar := range m.dataColumnSidecars {
verifiedDataColumnSidecar := blocks.NewVerifiedRODataColumn(dataColumnSidecar)
verifiedDataColumnSidecars = append(verifiedDataColumnSidecars, verifiedDataColumnSidecar)
}
return verifiedDataColumnSidecars, nil
}
func (m *mockDataColumnsVerifier) SatisfyRequirement(verification.Requirement) {}
func (m *mockDataColumnsVerifier) ValidFields() error {
m.validCalled = true
return nil
}
func (m *mockDataColumnsVerifier) CorrectSubnet(dataColumnSidecarSubTopic string, expectedTopics []string) error {
return nil
}
func (m *mockDataColumnsVerifier) NotFromFutureSlot() error { return nil }
func (m *mockDataColumnsVerifier) SlotAboveFinalized() error { return nil }
func (m *mockDataColumnsVerifier) ValidProposerSignature(ctx context.Context) error { return nil }
func (m *mockDataColumnsVerifier) SidecarParentSeen(parentSeen func([fieldparams.RootLength]byte) bool) error {
return nil
}
func (m *mockDataColumnsVerifier) SidecarParentValid(badParent func([fieldparams.RootLength]byte) bool) error {
return nil
}
func (m *mockDataColumnsVerifier) SidecarParentSlotLower() error { return nil }
func (m *mockDataColumnsVerifier) SidecarDescendsFromFinalized() error { return nil }
func (m *mockDataColumnsVerifier) SidecarInclusionProven() error {
m.SidecarInclusionProvenCalled = true
return nil
}
func (m *mockDataColumnsVerifier) SidecarKzgProofVerified() error {
m.SidecarKzgProofVerifiedCalled = true
return nil
}
func (m *mockDataColumnsVerifier) SidecarProposerExpected(ctx context.Context) error { return nil }
// Mock implementations for bisectVerification tests
// mockBisectionIterator simulates a BisectionIterator for testing.
type mockBisectionIterator struct {
chunks [][]blocks.RODataColumn
chunkErrors []error
finalError error
chunkIndex int
nextCallCount int
onErrorCallCount int
onErrorErrors []error
}
func (m *mockBisectionIterator) Next() ([]blocks.RODataColumn, error) {
if m.chunkIndex >= len(m.chunks) {
return nil, io.EOF
}
chunk := m.chunks[m.chunkIndex]
var err error
if m.chunkIndex < len(m.chunkErrors) {
err = m.chunkErrors[m.chunkIndex]
}
m.chunkIndex++
m.nextCallCount++
if err != nil {
return chunk, err
}
return chunk, nil
}
func (m *mockBisectionIterator) OnError(err error) {
m.onErrorCallCount++
m.onErrorErrors = append(m.onErrorErrors, err)
}
func (m *mockBisectionIterator) Error() error {
return m.finalError
}
// mockBisector simulates a Bisector for testing.
type mockBisector struct {
shouldError bool
bisectErr error
iterator *mockBisectionIterator
}
func (m *mockBisector) Bisect(columns []blocks.RODataColumn) (BisectionIterator, error) {
if m.shouldError {
return nil, m.bisectErr
}
return m.iterator, nil
}
// testDataColumnsVerifier implements verification.DataColumnsVerifier for testing.
type testDataColumnsVerifier struct {
t *testing.T
shouldFail bool
columns []blocks.RODataColumn
}
func (v *testDataColumnsVerifier) VerifiedRODataColumns() ([]blocks.VerifiedRODataColumn, error) {
verified := make([]blocks.VerifiedRODataColumn, len(v.columns))
for i, col := range v.columns {
verified[i] = blocks.NewVerifiedRODataColumn(col)
}
return verified, nil
}
func (v *testDataColumnsVerifier) SatisfyRequirement(verification.Requirement) {}
func (v *testDataColumnsVerifier) ValidFields() error {
if v.shouldFail {
return errors.New("verification failed")
}
return nil
}
func (v *testDataColumnsVerifier) CorrectSubnet(string, []string) error { return nil }
func (v *testDataColumnsVerifier) NotFromFutureSlot() error { return nil }
func (v *testDataColumnsVerifier) SlotAboveFinalized() error { return nil }
func (v *testDataColumnsVerifier) ValidProposerSignature(context.Context) error { return nil }
func (v *testDataColumnsVerifier) SidecarParentSeen(func([fieldparams.RootLength]byte) bool) error {
return nil
}
func (v *testDataColumnsVerifier) SidecarParentValid(func([fieldparams.RootLength]byte) bool) error {
return nil
}
func (v *testDataColumnsVerifier) SidecarParentSlotLower() error { return nil }
func (v *testDataColumnsVerifier) SidecarDescendsFromFinalized() error { return nil }
func (v *testDataColumnsVerifier) SidecarInclusionProven() error { return nil }
func (v *testDataColumnsVerifier) SidecarKzgProofVerified() error { return nil }
func (v *testDataColumnsVerifier) SidecarProposerExpected(context.Context) error { return nil }
// Helper function to create test data columns
func makeTestDataColumns(t *testing.T, count int, blockRoot [32]byte, startIndex uint64) []blocks.RODataColumn {
columns := make([]blocks.RODataColumn, 0, count)
for i := range count {
params := util.DataColumnParam{
Index: startIndex + uint64(i),
KzgCommitments: commitments,
Slot: primitives.Slot(params.BeaconConfig().FuluForkEpoch) * params.BeaconConfig().SlotsPerEpoch,
}
_, verifiedCols := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params})
if len(verifiedCols) > 0 {
columns = append(columns, verifiedCols[0].RODataColumn)
}
}
return columns
}
// Helper function to create test verifier factory with failure pattern
func makeTestVerifierFactory(failurePattern []bool) verification.NewDataColumnsVerifier {
callIndex := 0
return func(cols []blocks.RODataColumn, _ []verification.Requirement) verification.DataColumnsVerifier {
shouldFail := callIndex < len(failurePattern) && failurePattern[callIndex]
callIndex++
return &testDataColumnsVerifier{
shouldFail: shouldFail,
columns: cols,
}
}
}
// TestBisectVerification tests the bisectVerification method with comprehensive table-driven test cases.
func TestBisectVerification(t *testing.T) {
params.SetupTestConfigCleanup(t)
params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2
cases := []struct {
expectedError bool
bisectorNil bool
expectedOnErrorCallCount int
expectedNextCallCount int
inputCount int
iteratorFinalError error
bisectorError error
name string
storedColumnIndices []uint64
verificationFailurePattern []bool
chunkErrors []error
chunks [][]blocks.RODataColumn
}{
{
name: "EmptyColumns",
inputCount: 0,
expectedError: false,
expectedNextCallCount: 0,
expectedOnErrorCallCount: 0,
},
{
name: "NilBisector",
inputCount: 3,
bisectorNil: true,
expectedError: true,
expectedNextCallCount: 0,
expectedOnErrorCallCount: 0,
},
{
name: "BisectError",
inputCount: 5,
bisectorError: errors.New("bisect failed"),
expectedError: true,
expectedNextCallCount: 0,
expectedOnErrorCallCount: 0,
},
{
name: "SingleChunkSuccess",
inputCount: 4,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "SingleChunkFails",
inputCount: 4,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{true},
iteratorFinalError: errors.New("chunk failed"),
expectedError: true,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 1,
},
{
name: "TwoChunks_BothPass",
inputCount: 8,
chunks: [][]blocks.RODataColumn{{}, {}},
verificationFailurePattern: []bool{false, false},
expectedError: false,
expectedNextCallCount: 3,
expectedOnErrorCallCount: 0,
},
{
name: "TwoChunks_FirstFails",
inputCount: 8,
chunks: [][]blocks.RODataColumn{{}, {}},
verificationFailurePattern: []bool{true, false},
iteratorFinalError: errors.New("first failed"),
expectedError: true,
expectedNextCallCount: 3,
expectedOnErrorCallCount: 1,
},
{
name: "TwoChunks_SecondFails",
inputCount: 8,
chunks: [][]blocks.RODataColumn{{}, {}},
verificationFailurePattern: []bool{false, true},
iteratorFinalError: errors.New("second failed"),
expectedError: true,
expectedNextCallCount: 3,
expectedOnErrorCallCount: 1,
},
{
name: "TwoChunks_BothFail",
inputCount: 8,
chunks: [][]blocks.RODataColumn{{}, {}},
verificationFailurePattern: []bool{true, true},
iteratorFinalError: errors.New("both failed"),
expectedError: true,
expectedNextCallCount: 3,
expectedOnErrorCallCount: 2,
},
{
name: "ManyChunks_AllPass",
inputCount: 16,
chunks: [][]blocks.RODataColumn{{}, {}, {}, {}},
verificationFailurePattern: []bool{false, false, false, false},
expectedError: false,
expectedNextCallCount: 5,
expectedOnErrorCallCount: 0,
},
{
name: "ManyChunks_MixedFail",
inputCount: 16,
chunks: [][]blocks.RODataColumn{{}, {}, {}, {}},
verificationFailurePattern: []bool{false, true, false, true},
iteratorFinalError: errors.New("mixed failures"),
expectedError: true,
expectedNextCallCount: 5,
expectedOnErrorCallCount: 2,
},
{
name: "FilterStoredColumns_PartialFilter",
inputCount: 6,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
storedColumnIndices: []uint64{1, 3},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "FilterStoredColumns_AllStored",
inputCount: 6,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
storedColumnIndices: []uint64{0, 1, 2, 3, 4, 5},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "FilterStoredColumns_MixedAccess",
inputCount: 10,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
storedColumnIndices: []uint64{1, 5, 9},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "IteratorNextError",
inputCount: 4,
chunks: [][]blocks.RODataColumn{{}, {}},
chunkErrors: []error{nil, errors.New("next error")},
verificationFailurePattern: []bool{false},
expectedError: true,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "IteratorNextEOF",
inputCount: 4,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "LargeChunkSize",
inputCount: 128,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "ManySmallChunks",
inputCount: 32,
chunks: [][]blocks.RODataColumn{{}, {}, {}, {}, {}, {}, {}, {}},
verificationFailurePattern: []bool{false, false, false, false, false, false, false, false},
expectedError: false,
expectedNextCallCount: 9,
expectedOnErrorCallCount: 0,
},
{
name: "ChunkWithSomeStoredColumns",
inputCount: 6,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{false},
storedColumnIndices: []uint64{0, 2, 4},
expectedError: false,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 0,
},
{
name: "OnErrorDoesNotStopIteration",
inputCount: 8,
chunks: [][]blocks.RODataColumn{{}, {}},
verificationFailurePattern: []bool{true, false},
iteratorFinalError: errors.New("first failed"),
expectedError: true,
expectedNextCallCount: 3,
expectedOnErrorCallCount: 1,
},
{
name: "VerificationErrorWrapping",
inputCount: 4,
chunks: [][]blocks.RODataColumn{{}},
verificationFailurePattern: []bool{true},
iteratorFinalError: errors.New("verification failed"),
expectedError: true,
expectedNextCallCount: 2,
expectedOnErrorCallCount: 1,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
// Setup storage
var store *filesystem.DataColumnStorage
if len(tc.storedColumnIndices) > 0 {
mocker, s := filesystem.NewEphemeralDataColumnStorageWithMocker(t)
blockRoot := [32]byte{1, 2, 3}
slot := primitives.Slot(params.BeaconConfig().FuluForkEpoch) * params.BeaconConfig().SlotsPerEpoch
require.NoError(t, mocker.CreateFakeIndices(blockRoot, slot, tc.storedColumnIndices...))
store = s
} else {
store = filesystem.NewEphemeralDataColumnStorage(t)
}
// Create test columns
blockRoot := [32]byte{1, 2, 3}
columns := makeTestDataColumns(t, tc.inputCount, blockRoot, 0)
// Setup iterator with chunks
iterator := &mockBisectionIterator{
chunks: tc.chunks,
chunkErrors: tc.chunkErrors,
finalError: tc.iteratorFinalError,
}
// Setup bisector
var bisector Bisector
if tc.bisectorNil || tc.inputCount == 0 {
bisector = nil
} else if tc.bisectorError != nil {
bisector = &mockBisector{
shouldError: true,
bisectErr: tc.bisectorError,
}
} else {
bisector = &mockBisector{
shouldError: false,
iterator: iterator,
}
}
// Create store with verifier
verifierFactory := makeTestVerifierFactory(tc.verificationFailurePattern)
lazilyPersistentStore := &LazilyPersistentStoreColumn{
store: store,
cache: newDataColumnCache(),
newDataColumnsVerifier: verifierFactory,
custody: &custodyRequirement{},
bisector: bisector,
}
// Execute
err := lazilyPersistentStore.bisectVerification(columns)
// Assert
if tc.expectedError {
require.NotNil(t, err)
} else {
require.NoError(t, err)
}
// Verify iterator interactions for non-error cases
if tc.inputCount > 0 && bisector != nil && tc.bisectorError == nil && !tc.expectedError {
require.NotEqual(t, 0, iterator.nextCallCount, "iterator Next() should have been called")
require.Equal(t, tc.expectedOnErrorCallCount, iterator.onErrorCallCount, "OnError() call count mismatch")
}
})
}
}
func allIndicesExcept(total int, excluded []uint64) []uint64 {
excludeMap := make(map[uint64]bool)
for _, idx := range excluded {
excludeMap[idx] = true
}
var result []uint64
for i := range total {
if !excludeMap[uint64(i)] {
result = append(result, uint64(i))
}
}
return result
}
// TestColumnsNotStored tests the columnsNotStored method.
func TestColumnsNotStored(t *testing.T) {
params.SetupTestConfigCleanup(t)
params.BeaconConfig().FuluForkEpoch = params.BeaconConfig().ElectraForkEpoch + 4096*2
cases := []struct {
name string
count int
stored []uint64 // Column indices marked as stored
expected []uint64 // Expected column indices in returned result
}{
// Empty cases
{
name: "EmptyInput",
count: 0,
stored: []uint64{},
expected: []uint64{},
},
// Single element cases
{
name: "SingleElement_NotStored",
count: 1,
stored: []uint64{},
expected: []uint64{0},
},
{
name: "SingleElement_Stored",
count: 1,
stored: []uint64{0},
expected: []uint64{},
},
// All not stored cases
{
name: "AllNotStored_FiveElements",
count: 5,
stored: []uint64{},
expected: []uint64{0, 1, 2, 3, 4},
},
// All stored cases
{
name: "AllStored",
count: 5,
stored: []uint64{0, 1, 2, 3, 4},
expected: []uint64{},
},
// Partial storage - beginning
{
name: "StoredAtBeginning",
count: 5,
stored: []uint64{0, 1},
expected: []uint64{2, 3, 4},
},
// Partial storage - end
{
name: "StoredAtEnd",
count: 5,
stored: []uint64{3, 4},
expected: []uint64{0, 1, 2},
},
// Partial storage - middle
{
name: "StoredInMiddle",
count: 5,
stored: []uint64{2},
expected: []uint64{0, 1, 3, 4},
},
// Partial storage - scattered
{
name: "StoredScattered",
count: 8,
stored: []uint64{1, 3, 5},
expected: []uint64{0, 2, 4, 6, 7},
},
// Alternating pattern
{
name: "AlternatingPattern",
count: 8,
stored: []uint64{0, 2, 4, 6},
expected: []uint64{1, 3, 5, 7},
},
// Consecutive stored
{
name: "ConsecutiveStored",
count: 10,
stored: []uint64{3, 4, 5, 6},
expected: []uint64{0, 1, 2, 7, 8, 9},
},
// Large slice cases
{
name: "LargeSlice_NoStored",
count: 64,
stored: []uint64{},
expected: allIndicesExcept(64, []uint64{}),
},
{
name: "LargeSlice_SingleStored",
count: 64,
stored: []uint64{32},
expected: allIndicesExcept(64, []uint64{32}),
},
}
slot := primitives.Slot(params.BeaconConfig().FuluForkEpoch) * params.BeaconConfig().SlotsPerEpoch
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
// Create test columns first to get the actual block root
var columns []blocks.RODataColumn
if tc.count > 0 {
columns = makeTestDataColumns(t, tc.count, [32]byte{}, 0)
}
// Get the actual block root from the first column (if any)
var blockRoot [32]byte
if len(columns) > 0 {
blockRoot = columns[0].BlockRoot()
}
// Setup storage
var store *filesystem.DataColumnStorage
if len(tc.stored) > 0 {
mocker, s := filesystem.NewEphemeralDataColumnStorageWithMocker(t)
require.NoError(t, mocker.CreateFakeIndices(blockRoot, slot, tc.stored...))
store = s
} else {
store = filesystem.NewEphemeralDataColumnStorage(t)
}
// Create store instance
lazilyPersistentStore := &LazilyPersistentStoreColumn{
store: store,
}
// Execute
result := lazilyPersistentStore.columnsNotStored(columns)
// Assert count
require.Equal(t, len(tc.expected), len(result),
fmt.Sprintf("expected %d columns, got %d", len(tc.expected), len(result)))
// Verify that no stored columns are in the result
if len(tc.stored) > 0 {
resultIndices := make(map[uint64]bool)
for _, col := range result {
resultIndices[col.Index] = true
}
for _, storedIdx := range tc.stored {
require.Equal(t, false, resultIndices[storedIdx],
fmt.Sprintf("stored column index %d should not be in result", storedIdx))
}
}
// If expectedIndices is specified, verify the exact column indices in order
if len(tc.expected) > 0 && len(tc.stored) == 0 {
// Only check exact order for non-stored cases (where we know they stay in same order)
for i, expectedIdx := range tc.expected {
require.Equal(t, columns[expectedIdx].Index, result[i].Index,
fmt.Sprintf("column %d: expected index %d, got %d", i, columns[expectedIdx].Index, result[i].Index))
}
}
// Verify optimization: if nothing stored, should return original slice
if len(tc.stored) == 0 && tc.count > 0 {
require.Equal(t, &columns[0], &result[0],
"when no columns stored, should return original slice (same pointer)")
}
// Verify optimization: if some stored, result should use in-place shifting
if len(tc.stored) > 0 && len(tc.expected) > 0 && tc.count > 0 {
require.Equal(t, cap(columns), cap(result),
"result should be in-place shifted from original (same capacity)")
}
})
}
}

View File

@@ -0,0 +1,40 @@
package das
import (
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
)
// Bisector describes a type that takes a set of RODataColumns via the Bisect method
// and returns a BisectionIterator that returns batches of those columns to be
// verified together.
type Bisector interface {
// Bisect initializes the BisectionIterator and returns the result.
Bisect([]blocks.RODataColumn) (BisectionIterator, error)
}
// BisectionIterator describes an iterator that returns groups of columns to verify.
// It is up to the bisector implementation to decide how to chunk up the columns,
// whether by block, by peer, or any other strategy. For example, backfill implements
// a bisector that keeps track of the source of each sidecar by peer, and groups
// sidecars by peer in the Next method, enabling it to track which peers, out of all
// the peers contributing to a batch, gave us bad data.
// When a batch fails, the OnError method should be used so that the bisector can
// keep track of the failed groups of columns and eg apply that knowledge in peer scoring.
// The same column will be returned multiple times by Next; first as part of a larger batch,
// and again as part of a more fine grained batch if there was an error in the large batch.
// For example, first as part of a batch of all columns spanning peers, and then again
// as part of a batch of columns from a single peer if some column in the larger batch
// failed verification.
type BisectionIterator interface {
// Next returns the next group of columns to verify.
// When the iteration is complete, Next should return (nil, io.EOF).
Next() ([]blocks.RODataColumn, error)
// OnError should be called when verification of a group of columns obtained via Next() fails.
OnError(error)
// Error can be used at the end of the iteration to get a single error result. It will return
// nil if OnError was never called, or an error of the implementers choosing representing the set
// of errors seen during iteration. For instance when bisecting from columns spanning peers to columns
// from a single peer, the broader error could be dropped, and then the more specific error
// (for a single peer's response) returned after bisecting to it.
Error() error
}

View File

@@ -76,7 +76,7 @@ func (e *blobCacheEntry) stash(sc *blocks.ROBlob) error {
e.scs = make([]*blocks.ROBlob, maxBlobsPerBlock)
}
if e.scs[sc.Index] != nil {
return errors.Wrapf(ErrDuplicateSidecar, "root=%#x, index=%d, commitment=%#x", sc.BlockRoot(), sc.Index, sc.KzgCommitment)
return errors.Wrapf(errDuplicateSidecar, "root=%#x, index=%d, commitment=%#x", sc.BlockRoot(), sc.Index, sc.KzgCommitment)
}
e.scs[sc.Index] = sc
return nil

View File

@@ -34,7 +34,8 @@ type filterTestCaseSetupFunc func(t *testing.T) (*blobCacheEntry, [][]byte, []bl
func filterTestCaseSetup(slot primitives.Slot, nBlobs int, onDisk []int, numExpected int) filterTestCaseSetupFunc {
return func(t *testing.T) (*blobCacheEntry, [][]byte, []blocks.ROBlob) {
blk, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, slot, nBlobs)
commits, err := commitmentsToCheck(blk, blk.Block().Slot())
shouldRetain := func(s primitives.Slot) bool { return true }
commits, err := commitmentsToCheck(blk, shouldRetain)
require.NoError(t, err)
entry := &blobCacheEntry{}
if len(onDisk) > 0 {

View File

@@ -1,9 +1,7 @@
package das
import (
"bytes"
"slices"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
@@ -11,9 +9,9 @@ import (
)
var (
ErrDuplicateSidecar = errors.New("duplicate sidecar stashed in AvailabilityStore")
errDuplicateSidecar = errors.New("duplicate sidecar stashed in AvailabilityStore")
errColumnIndexTooHigh = errors.New("column index too high")
errCommitmentMismatch = errors.New("KzgCommitment of sidecar in cache did not match block commitment")
errCommitmentMismatch = errors.New("commitment of sidecar in cache did not match block commitment")
errMissingSidecar = errors.New("no sidecar in cache for block commitment")
)
@@ -25,107 +23,80 @@ func newDataColumnCache() *dataColumnCache {
return &dataColumnCache{entries: make(map[cacheKey]*dataColumnCacheEntry)}
}
// ensure returns the entry for the given key, creating it if it isn't already present.
func (c *dataColumnCache) ensure(key cacheKey) *dataColumnCacheEntry {
// entry returns the entry for the given key, creating it if it isn't already present.
func (c *dataColumnCache) entry(key cacheKey) *dataColumnCacheEntry {
entry, ok := c.entries[key]
if !ok {
entry = &dataColumnCacheEntry{}
entry = newDataColumnCacheEntry(key.root)
c.entries[key] = entry
}
return entry
}
func (c *dataColumnCache) cleanup(blks []blocks.ROBlock) {
for _, block := range blks {
key := cacheKey{slot: block.Block().Slot(), root: block.Root()}
c.delete(key)
}
}
// delete removes the cache entry from the cache.
func (c *dataColumnCache) delete(key cacheKey) {
delete(c.entries, key)
}
// dataColumnCacheEntry holds a fixed-length cache of BlobSidecars.
type dataColumnCacheEntry struct {
scs [fieldparams.NumberOfColumns]*blocks.RODataColumn
diskSummary filesystem.DataColumnStorageSummary
func (c *dataColumnCache) stash(sc blocks.RODataColumn) error {
key := cacheKey{slot: sc.Slot(), root: sc.BlockRoot()}
entry := c.entry(key)
return entry.stash(sc)
}
func (e *dataColumnCacheEntry) setDiskSummary(sum filesystem.DataColumnStorageSummary) {
e.diskSummary = sum
func newDataColumnCacheEntry(root [32]byte) *dataColumnCacheEntry {
return &dataColumnCacheEntry{scs: make(map[uint64]blocks.RODataColumn), root: &root}
}
// dataColumnCacheEntry is the set of RODataColumns for a given block.
type dataColumnCacheEntry struct {
root *[32]byte
scs map[uint64]blocks.RODataColumn
}
// stash adds an item to the in-memory cache of DataColumnSidecars.
// Only the first DataColumnSidecar of a given Index will be kept in the cache.
// stash will return an error if the given data colunn is already in the cache, or if the Index is out of bounds.
func (e *dataColumnCacheEntry) stash(sc *blocks.RODataColumn) error {
// stash will return an error if the given data column Index is out of bounds.
// It will overwrite any existing entry for the same index.
func (e *dataColumnCacheEntry) stash(sc blocks.RODataColumn) error {
if sc.Index >= fieldparams.NumberOfColumns {
return errors.Wrapf(errColumnIndexTooHigh, "index=%d", sc.Index)
}
if e.scs[sc.Index] != nil {
return errors.Wrapf(ErrDuplicateSidecar, "root=%#x, index=%d, commitment=%#x", sc.BlockRoot(), sc.Index, sc.KzgCommitments)
}
e.scs[sc.Index] = sc
return nil
}
func (e *dataColumnCacheEntry) filter(root [32]byte, commitmentsArray *safeCommitmentsArray) ([]blocks.RODataColumn, error) {
nonEmptyIndices := commitmentsArray.nonEmptyIndices()
if e.diskSummary.AllAvailable(nonEmptyIndices) {
return nil, nil
// append appends the requested root and indices from the cache to the given sidecars slice and returns the result.
// If any of the given indices are missing, an error will be returned and the sidecars slice will be unchanged.
func (e *dataColumnCacheEntry) append(sidecars []blocks.RODataColumn, indices peerdas.ColumnIndices) ([]blocks.RODataColumn, error) {
needed := indices.ToMap()
for col := range needed {
_, ok := e.scs[col]
if !ok {
return nil, errors.Wrapf(errMissingSidecar, "root=%#x, index=%#x", e.root, col)
}
}
commitmentsCount := commitmentsArray.count()
sidecars := make([]blocks.RODataColumn, 0, commitmentsCount)
for i := range nonEmptyIndices {
if e.diskSummary.HasIndex(i) {
continue
}
if e.scs[i] == nil {
return nil, errors.Wrapf(errMissingSidecar, "root=%#x, index=%#x", root, i)
}
if !sliceBytesEqual(commitmentsArray[i], e.scs[i].KzgCommitments) {
return nil, errors.Wrapf(errCommitmentMismatch, "root=%#x, index=%#x, commitment=%#x, block commitment=%#x", root, i, e.scs[i].KzgCommitments, commitmentsArray[i])
}
sidecars = append(sidecars, *e.scs[i])
// Loop twice so we can avoid touching the slice if any of the blobs are missing.
for col := range needed {
sidecars = append(sidecars, e.scs[col])
}
return sidecars, nil
}
// safeCommitmentsArray is a fixed size array of commitments.
// This is helpful for avoiding gratuitous bounds checks.
type safeCommitmentsArray [fieldparams.NumberOfColumns][][]byte
// count returns the number of commitments in the array.
func (s *safeCommitmentsArray) count() int {
count := 0
for i := range s {
if s[i] != nil {
count++
// IndicesNotStored filters the list of indices to only include those that are not found in the storage summary.
func IndicesNotStored(sum filesystem.DataColumnStorageSummary, indices peerdas.ColumnIndices) peerdas.ColumnIndices {
indices = indices.Copy()
for col := range indices {
if sum.HasIndex(col) {
indices.Unset(col)
}
}
return count
}
// nonEmptyIndices returns a map of indices that are non-nil in the array.
func (s *safeCommitmentsArray) nonEmptyIndices() map[uint64]bool {
columns := make(map[uint64]bool)
for i := range s {
if s[i] != nil {
columns[uint64(i)] = true
}
}
return columns
}
func sliceBytesEqual(a, b [][]byte) bool {
return slices.EqualFunc(a, b, bytes.Equal)
return indices
}

View File

@@ -1,8 +1,10 @@
package das
import (
"slices"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
@@ -13,124 +15,105 @@ import (
func TestEnsureDeleteSetDiskSummary(t *testing.T) {
c := newDataColumnCache()
key := cacheKey{}
entry := c.ensure(key)
require.DeepEqual(t, dataColumnCacheEntry{}, *entry)
entry := c.entry(key)
require.Equal(t, 0, len(entry.scs))
diskSummary := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{true})
entry.setDiskSummary(diskSummary)
entry = c.ensure(key)
require.DeepEqual(t, dataColumnCacheEntry{diskSummary: diskSummary}, *entry)
nonDupe := c.entry(key)
require.Equal(t, entry, nonDupe) // same pointer
expect, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 1}})
require.NoError(t, entry.stash(expect[0]))
require.Equal(t, 1, len(entry.scs))
cols, err := nonDupe.append([]blocks.RODataColumn{}, peerdas.NewColumnIndicesFromSlice([]uint64{expect[0].Index}))
require.NoError(t, err)
require.DeepEqual(t, expect[0], cols[0])
c.delete(key)
entry = c.ensure(key)
require.DeepEqual(t, dataColumnCacheEntry{}, *entry)
entry = c.entry(key)
require.Equal(t, 0, len(entry.scs))
require.NotEqual(t, entry, nonDupe) // different pointer
}
func TestStash(t *testing.T) {
t.Run("Index too high", func(t *testing.T) {
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 10_000}})
columns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 10_000}})
var entry dataColumnCacheEntry
err := entry.stash(&roDataColumns[0])
err := entry.stash(columns[0])
require.NotNil(t, err)
})
t.Run("Nominal and already existing", func(t *testing.T) {
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 1}})
var entry dataColumnCacheEntry
err := entry.stash(&roDataColumns[0])
entry := newDataColumnCacheEntry(roDataColumns[0].BlockRoot())
err := entry.stash(roDataColumns[0])
require.NoError(t, err)
require.DeepEqual(t, roDataColumns[0], entry.scs[1])
err = entry.stash(&roDataColumns[0])
require.NotNil(t, err)
require.NoError(t, entry.stash(roDataColumns[0]))
// stash simply replaces duplicate values now
require.DeepEqual(t, roDataColumns[0], entry.scs[1])
})
}
func TestFilterDataColumns(t *testing.T) {
func TestAppendDataColumns(t *testing.T) {
t.Run("All available", func(t *testing.T) {
commitmentsArray := safeCommitmentsArray{nil, [][]byte{[]byte{1}}, nil, [][]byte{[]byte{3}}}
diskSummary := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{false, true, false, true})
dataColumnCacheEntry := dataColumnCacheEntry{diskSummary: diskSummary}
actual, err := dataColumnCacheEntry.filter([fieldparams.RootLength]byte{}, &commitmentsArray)
sum := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{false, true, false, true})
notStored := IndicesNotStored(sum, peerdas.NewColumnIndicesFromSlice([]uint64{1, 3}))
actual, err := newDataColumnCacheEntry([32]byte{}).append([]blocks.RODataColumn{}, notStored)
require.NoError(t, err)
require.IsNil(t, actual)
require.Equal(t, 0, len(actual))
})
t.Run("Some scs missing", func(t *testing.T) {
commitmentsArray := safeCommitmentsArray{nil, [][]byte{[]byte{1}}}
sum := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{})
diskSummary := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{})
dataColumnCacheEntry := dataColumnCacheEntry{diskSummary: diskSummary}
_, err := dataColumnCacheEntry.filter([fieldparams.RootLength]byte{}, &commitmentsArray)
require.NotNil(t, err)
})
t.Run("Commitments not equal", func(t *testing.T) {
commitmentsArray := safeCommitmentsArray{nil, [][]byte{[]byte{1}}}
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 1}})
var scs [fieldparams.NumberOfColumns]*blocks.RODataColumn
scs[1] = &roDataColumns[0]
dataColumnCacheEntry := dataColumnCacheEntry{scs: scs}
_, err := dataColumnCacheEntry.filter(roDataColumns[0].BlockRoot(), &commitmentsArray)
notStored := IndicesNotStored(sum, peerdas.NewColumnIndicesFromSlice([]uint64{1}))
actual, err := newDataColumnCacheEntry([32]byte{}).append([]blocks.RODataColumn{}, notStored)
require.Equal(t, 0, len(actual))
require.NotNil(t, err)
})
t.Run("Nominal", func(t *testing.T) {
commitmentsArray := safeCommitmentsArray{nil, [][]byte{[]byte{1}}, nil, [][]byte{[]byte{3}}}
diskSummary := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{false, true})
indices := peerdas.NewColumnIndicesFromSlice([]uint64{1, 3})
expected, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 3, KzgCommitments: [][]byte{[]byte{3}}}})
var scs [fieldparams.NumberOfColumns]*blocks.RODataColumn
scs[3] = &expected[0]
scs := map[uint64]blocks.RODataColumn{
3: expected[0],
}
sum := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{false, true})
entry := dataColumnCacheEntry{scs: scs}
dataColumnCacheEntry := dataColumnCacheEntry{scs: scs, diskSummary: diskSummary}
actual, err := dataColumnCacheEntry.filter(expected[0].BlockRoot(), &commitmentsArray)
actual, err := entry.append([]blocks.RODataColumn{}, IndicesNotStored(sum, indices))
require.NoError(t, err)
require.DeepEqual(t, expected, actual)
})
}
func TestCount(t *testing.T) {
s := safeCommitmentsArray{nil, [][]byte{[]byte{1}}, nil, [][]byte{[]byte{3}}}
require.Equal(t, 2, s.count())
}
t.Run("Append does not mutate the input", func(t *testing.T) {
indices := peerdas.NewColumnIndicesFromSlice([]uint64{1, 2})
expected, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{
{Index: 0, KzgCommitments: [][]byte{[]byte{1}}},
{Index: 1, KzgCommitments: [][]byte{[]byte{2}}},
{Index: 2, KzgCommitments: [][]byte{[]byte{3}}},
})
func TestNonEmptyIndices(t *testing.T) {
s := safeCommitmentsArray{nil, [][]byte{[]byte{10}}, nil, [][]byte{[]byte{20}}}
actual := s.nonEmptyIndices()
require.DeepEqual(t, map[uint64]bool{1: true, 3: true}, actual)
}
scs := map[uint64]blocks.RODataColumn{
1: expected[1],
2: expected[2],
}
entry := dataColumnCacheEntry{scs: scs}
func TestSliceBytesEqual(t *testing.T) {
t.Run("Different lengths", func(t *testing.T) {
a := [][]byte{[]byte{1, 2, 3}}
b := [][]byte{[]byte{1, 2, 3}, []byte{4, 5, 6}}
require.Equal(t, false, sliceBytesEqual(a, b))
})
t.Run("Same length but different content", func(t *testing.T) {
a := [][]byte{[]byte{1, 2, 3}, []byte{4, 5, 6}}
b := [][]byte{[]byte{1, 2, 3}, []byte{4, 5, 7}}
require.Equal(t, false, sliceBytesEqual(a, b))
})
t.Run("Equal slices", func(t *testing.T) {
a := [][]byte{[]byte{1, 2, 3}, []byte{4, 5, 6}}
b := [][]byte{[]byte{1, 2, 3}, []byte{4, 5, 6}}
require.Equal(t, true, sliceBytesEqual(a, b))
original := []blocks.RODataColumn{expected[0]}
actual, err := entry.append(original, indices)
require.NoError(t, err)
require.Equal(t, len(expected), len(actual))
slices.SortFunc(actual, func(i, j blocks.RODataColumn) int {
return int(i.Index) - int(j.Index)
})
for i := range expected {
require.Equal(t, expected[i].Index, actual[i].Index)
}
require.Equal(t, 1, len(original))
})
}

View File

@@ -7,13 +7,13 @@ import (
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
)
// AvailabilityStore describes a component that can verify and save sidecars for a given block, and confirm previously
// verified and saved sidecars.
// Persist guarantees that the sidecar will be available to perform a DA check
// for the life of the beacon node process.
// IsDataAvailable guarantees that all blobs committed to in the block have been
// durably persisted before returning a non-error value.
type AvailabilityStore interface {
IsDataAvailable(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error
Persist(current primitives.Slot, blobSidecar ...blocks.ROBlob) error
// AvailabilityChecker is the minimum interface needed to check if data is available for a block.
// By convention there is a concept of an AvailabilityStore that implements a method to persist
// blobs or data columns to prepare for Availability checking, but since those methods are different
// for different forms of blob data, they are not included in the interface.
type AvailabilityChecker interface {
IsDataAvailable(ctx context.Context, current primitives.Slot, b ...blocks.ROBlock) error
}
// RetentionChecker is a callback that determines whether blobs at the given slot are within the retention period.
type RetentionChecker func(primitives.Slot) bool

5
beacon-chain/das/log.go Normal file
View File

@@ -0,0 +1,5 @@
package das
import "github.com/sirupsen/logrus"
var log = logrus.WithField("prefix", "das")

View File

@@ -9,16 +9,20 @@ import (
// MockAvailabilityStore is an implementation of AvailabilityStore that can be used by other packages in tests.
type MockAvailabilityStore struct {
VerifyAvailabilityCallback func(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error
VerifyAvailabilityCallback func(ctx context.Context, current primitives.Slot, b ...blocks.ROBlock) error
ErrIsDataAvailable error
PersistBlobsCallback func(current primitives.Slot, blobSidecar ...blocks.ROBlob) error
}
var _ AvailabilityStore = &MockAvailabilityStore{}
var _ AvailabilityChecker = &MockAvailabilityStore{}
// IsDataAvailable satisfies the corresponding method of the AvailabilityStore interface in a way that is useful for tests.
func (m *MockAvailabilityStore) IsDataAvailable(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error {
func (m *MockAvailabilityStore) IsDataAvailable(ctx context.Context, current primitives.Slot, b ...blocks.ROBlock) error {
if m.ErrIsDataAvailable != nil {
return m.ErrIsDataAvailable
}
if m.VerifyAvailabilityCallback != nil {
return m.VerifyAvailabilityCallback(ctx, current, b)
return m.VerifyAvailabilityCallback(ctx, current, b...)
}
return nil
}

135
beacon-chain/das/needs.go Normal file
View File

@@ -0,0 +1,135 @@
package das
import (
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
)
// NeedSpan represents the need for a resource over a span of slots.
type NeedSpan struct {
Begin primitives.Slot
End primitives.Slot
}
// At returns whether blocks/blobs/columns are needed At the given slot.
func (n NeedSpan) At(slot primitives.Slot) bool {
return slot >= n.Begin && slot < n.End
}
// CurrentNeeds fields can be used to check whether the given resource type is needed
// at a given slot. The values are based on the current slot, so this value shouldn't
// be retained / reused across slots.
type CurrentNeeds struct {
Block NeedSpan
Blob NeedSpan
Col NeedSpan
}
// SyncNeeds holds configuration and state for determining what data is needed
// at any given slot during backfill based on the current slot.
type SyncNeeds struct {
current func() primitives.Slot
deneb primitives.Slot
fulu primitives.Slot
oldestSlotFlagPtr *primitives.Slot
validOldestSlotPtr *primitives.Slot
blockRetention primitives.Epoch
blobRetentionFlag primitives.Epoch
blobRetention primitives.Epoch
colRetention primitives.Epoch
}
type CurrentSlotter func() primitives.Slot
func NewSyncNeeds(current CurrentSlotter, oldestSlotFlagPtr *primitives.Slot, blobRetentionFlag primitives.Epoch) (SyncNeeds, error) {
deneb, err := slots.EpochStart(params.BeaconConfig().DenebForkEpoch)
if err != nil {
return SyncNeeds{}, errors.Wrap(err, "deneb fork slot")
}
fuluBoundary := min(params.BeaconConfig().FuluForkEpoch, slots.MaxSafeEpoch())
fulu, err := slots.EpochStart(fuluBoundary)
if err != nil {
return SyncNeeds{}, errors.Wrap(err, "fulu fork slot")
}
sn := SyncNeeds{
current: func() primitives.Slot { return current() },
deneb: deneb,
fulu: fulu,
blobRetentionFlag: blobRetentionFlag,
}
// We apply the --blob-retention-epochs flag to both blob and column retention.
sn.blobRetention = max(sn.blobRetentionFlag, params.BeaconConfig().MinEpochsForBlobsSidecarsRequest)
sn.colRetention = max(sn.blobRetentionFlag, params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest)
// Override spec minimum block retention with user-provided flag only if it is lower than the spec minimum.
sn.blockRetention = primitives.Epoch(params.BeaconConfig().MinEpochsForBlockRequests)
if oldestSlotFlagPtr != nil {
oldestEpoch := slots.ToEpoch(*oldestSlotFlagPtr)
if oldestEpoch < sn.blockRetention {
sn.validOldestSlotPtr = oldestSlotFlagPtr
} else {
log.WithField("backfill-oldest-slot", *oldestSlotFlagPtr).
WithField("specMinSlot", syncEpochOffset(current(), sn.blockRetention)).
Warn("Ignoring user-specified slot > MIN_EPOCHS_FOR_BLOCK_REQUESTS.")
}
}
return sn, nil
}
// Currently is the main callback given to the different parts of backfill to determine
// what resources are needed at a given slot. It assumes the current instance of SyncNeeds
// is the result of calling initialize.
func (n SyncNeeds) Currently() CurrentNeeds {
current := n.current()
c := CurrentNeeds{
Block: n.blockSpan(current),
Blob: NeedSpan{Begin: syncEpochOffset(current, n.blobRetention), End: n.fulu},
Col: NeedSpan{Begin: syncEpochOffset(current, n.colRetention), End: current},
}
// Adjust the minimums forward to the slots where the sidecar types were introduced
c.Blob.Begin = max(c.Blob.Begin, n.deneb)
c.Col.Begin = max(c.Col.Begin, n.fulu)
return c
}
func (n SyncNeeds) blockSpan(current primitives.Slot) NeedSpan {
if n.validOldestSlotPtr != nil { // assumes validation done in initialize()
return NeedSpan{Begin: *n.validOldestSlotPtr, End: current}
}
return NeedSpan{Begin: syncEpochOffset(current, n.blockRetention), End: current}
}
func (n SyncNeeds) BlobRetentionChecker() RetentionChecker {
return func(slot primitives.Slot) bool {
current := n.Currently()
return current.Blob.At(slot)
}
}
func (n SyncNeeds) DataColumnRetentionChecker() RetentionChecker {
return func(slot primitives.Slot) bool {
current := n.Currently()
return current.Col.At(slot)
}
}
// syncEpochOffset subtracts a number of epochs as slots from the current slot, with underflow checks.
// It returns slot 1 if the result would be 0 or underflow. It doesn't return slot 0 because the
// genesis block needs to be specially synced (it doesn't have a valid signature).
func syncEpochOffset(current primitives.Slot, subtract primitives.Epoch) primitives.Slot {
minEpoch := min(subtract, slots.MaxSafeEpoch())
// compute slot offset - offset is a number of slots to go back from current (not an absolute slot).
offset := slots.UnsafeEpochStart(minEpoch)
// Undeflow protection: slot 0 is the genesis block, therefore the signature in it is invalid.
// To prevent us from rejecting a batch, we restrict the minimum backfill batch till only slot 1
if offset >= current {
return 1
}
return current - offset
}

View File

@@ -0,0 +1,675 @@
package das
import (
"fmt"
"testing"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/time/slots"
)
// TestNeedSpanAt tests the needSpan.at() method for range checking.
func TestNeedSpanAt(t *testing.T) {
cases := []struct {
name string
span NeedSpan
slots []primitives.Slot
expected bool
}{
{
name: "within bounds",
span: NeedSpan{Begin: 100, End: 200},
slots: []primitives.Slot{101, 150, 199},
expected: true,
},
{
name: "before begin / at end boundary (exclusive)",
span: NeedSpan{Begin: 100, End: 200},
slots: []primitives.Slot{99, 200, 201},
expected: false,
},
{
name: "empty span (begin == end)",
span: NeedSpan{Begin: 100, End: 100},
slots: []primitives.Slot{100},
expected: false,
},
{
name: "slot 0 with span starting at 0",
span: NeedSpan{Begin: 0, End: 100},
slots: []primitives.Slot{0},
expected: true,
},
}
for _, tc := range cases {
for _, sl := range tc.slots {
t.Run(fmt.Sprintf("%s at slot %d, ", tc.name, sl), func(t *testing.T) {
result := tc.span.At(sl)
require.Equal(t, tc.expected, result)
})
}
}
}
// TestSyncEpochOffset tests the syncEpochOffset helper function.
func TestSyncEpochOffset(t *testing.T) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
cases := []struct {
name string
current primitives.Slot
subtract primitives.Epoch
expected primitives.Slot
}{
{
name: "typical offset - 5 epochs back",
current: primitives.Slot(10000),
subtract: 5,
expected: primitives.Slot(10000 - 5*slotsPerEpoch),
},
{
name: "zero subtract returns current",
current: primitives.Slot(5000),
subtract: 0,
expected: primitives.Slot(5000),
},
{
name: "subtract 1 epoch from mid-range slot",
current: primitives.Slot(1000),
subtract: 1,
expected: primitives.Slot(1000 - slotsPerEpoch),
},
{
name: "offset equals current - underflow protection",
current: primitives.Slot(slotsPerEpoch),
subtract: 1,
expected: 1,
},
{
name: "offset exceeds current - underflow protection",
current: primitives.Slot(50),
subtract: 1000,
expected: 1,
},
{
name: "current very close to 0",
current: primitives.Slot(10),
subtract: 1,
expected: 1,
},
{
name: "subtract MaxSafeEpoch",
current: primitives.Slot(1000000),
subtract: slots.MaxSafeEpoch(),
expected: 1, // underflow protection
},
{
name: "result exactly at slot 1",
current: primitives.Slot(1 + slotsPerEpoch),
subtract: 1,
expected: 1,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
result := syncEpochOffset(tc.current, tc.subtract)
require.Equal(t, tc.expected, result)
})
}
}
// TestSyncNeedsInitialize tests the syncNeeds.initialize() method.
func TestSyncNeedsInitialize(t *testing.T) {
params.SetupTestConfigCleanup(t)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
minBlobEpochs := params.BeaconConfig().MinEpochsForBlobsSidecarsRequest
minColEpochs := params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest
currentSlot := primitives.Slot(10000)
currentFunc := func() primitives.Slot { return currentSlot }
cases := []struct {
invalidOldestFlag bool
expectValidOldest bool
oldestSlotFlagPtr *primitives.Slot
blobRetentionFlag primitives.Epoch
expectedBlob primitives.Epoch
expectedCol primitives.Epoch
name string
input SyncNeeds
}{
{
name: "basic initialization with no flags",
expectValidOldest: false,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
blobRetentionFlag: 0,
},
{
name: "blob retention flag less than spec minimum",
blobRetentionFlag: minBlobEpochs - 1,
expectValidOldest: false,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
},
{
name: "blob retention flag greater than spec minimum",
blobRetentionFlag: minBlobEpochs + 10,
expectValidOldest: false,
expectedBlob: minBlobEpochs + 10,
expectedCol: minBlobEpochs + 10,
},
{
name: "oldestSlotFlagPtr is nil",
blobRetentionFlag: 0,
oldestSlotFlagPtr: nil,
expectValidOldest: false,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
},
{
name: "valid oldestSlotFlagPtr (earlier than spec minimum)",
blobRetentionFlag: 0,
oldestSlotFlagPtr: func() *primitives.Slot {
slot := primitives.Slot(10)
return &slot
}(),
expectValidOldest: true,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
},
{
name: "invalid oldestSlotFlagPtr (later than spec minimum)",
blobRetentionFlag: 0,
oldestSlotFlagPtr: func() *primitives.Slot {
// Make it way past the spec minimum
slot := currentSlot - primitives.Slot(params.BeaconConfig().MinEpochsForBlockRequests-1)*slotsPerEpoch
return &slot
}(),
expectValidOldest: false,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
invalidOldestFlag: true,
},
{
name: "oldestSlotFlagPtr at boundary (exactly at spec minimum)",
blobRetentionFlag: 0,
oldestSlotFlagPtr: func() *primitives.Slot {
slot := currentSlot - primitives.Slot(params.BeaconConfig().MinEpochsForBlockRequests)*slotsPerEpoch
return &slot
}(),
expectValidOldest: false,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
invalidOldestFlag: true,
},
{
name: "both blob retention flag and oldest slot set",
blobRetentionFlag: minBlobEpochs + 5,
oldestSlotFlagPtr: func() *primitives.Slot {
slot := primitives.Slot(100)
return &slot
}(),
expectValidOldest: true,
expectedBlob: minBlobEpochs + 5,
expectedCol: minBlobEpochs + 5,
},
{
name: "zero blob retention uses spec minimum",
blobRetentionFlag: 0,
expectValidOldest: false,
expectedBlob: minBlobEpochs,
expectedCol: minColEpochs,
},
{
name: "large blob retention value",
blobRetentionFlag: 5000,
expectValidOldest: false,
expectedBlob: 5000,
expectedCol: 5000,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
result, err := NewSyncNeeds(currentFunc, tc.oldestSlotFlagPtr, tc.blobRetentionFlag)
require.NoError(t, err)
// Check that current, deneb, fulu are set correctly
require.Equal(t, currentSlot, result.current())
// Check retention calculations
require.Equal(t, tc.expectedBlob, result.blobRetention)
require.Equal(t, tc.expectedCol, result.colRetention)
if tc.invalidOldestFlag {
require.IsNil(t, result.validOldestSlotPtr)
} else {
require.Equal(t, tc.oldestSlotFlagPtr, result.validOldestSlotPtr)
}
// Check blockRetention is always spec minimum
require.Equal(t, primitives.Epoch(params.BeaconConfig().MinEpochsForBlockRequests), result.blockRetention)
})
}
}
// TestSyncNeedsBlockSpan tests the syncNeeds.blockSpan() method.
func TestSyncNeedsBlockSpan(t *testing.T) {
params.SetupTestConfigCleanup(t)
minBlockEpochs := params.BeaconConfig().MinEpochsForBlockRequests
cases := []struct {
name string
validOldest *primitives.Slot
blockRetention primitives.Epoch
current primitives.Slot
expectedBegin primitives.Slot
expectedEnd primitives.Slot
}{
{
name: "with validOldestSlotPtr set",
validOldest: func() *primitives.Slot { s := primitives.Slot(500); return &s }(),
blockRetention: primitives.Epoch(minBlockEpochs),
current: 10000,
expectedBegin: 500,
expectedEnd: 10000,
},
{
name: "without validOldestSlotPtr (nil)",
validOldest: nil,
blockRetention: primitives.Epoch(minBlockEpochs),
current: 10000,
expectedBegin: syncEpochOffset(10000, primitives.Epoch(minBlockEpochs)),
expectedEnd: 10000,
},
{
name: "very low current slot",
validOldest: nil,
blockRetention: primitives.Epoch(minBlockEpochs),
current: 100,
expectedBegin: 1, // underflow protection
expectedEnd: 100,
},
{
name: "very high current slot",
validOldest: nil,
blockRetention: primitives.Epoch(minBlockEpochs),
current: 1000000,
expectedBegin: syncEpochOffset(1000000, primitives.Epoch(minBlockEpochs)),
expectedEnd: 1000000,
},
{
name: "validOldestSlotPtr at boundary value",
validOldest: func() *primitives.Slot { s := primitives.Slot(1); return &s }(),
blockRetention: primitives.Epoch(minBlockEpochs),
current: 5000,
expectedBegin: 1,
expectedEnd: 5000,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
sn := SyncNeeds{
validOldestSlotPtr: tc.validOldest,
blockRetention: tc.blockRetention,
}
result := sn.blockSpan(tc.current)
require.Equal(t, tc.expectedBegin, result.Begin)
require.Equal(t, tc.expectedEnd, result.End)
})
}
}
// TestSyncNeedsCurrently tests the syncNeeds.currently() method.
func TestSyncNeedsCurrently(t *testing.T) {
params.SetupTestConfigCleanup(t)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
denebSlot := primitives.Slot(1000)
fuluSlot := primitives.Slot(2000)
cases := []struct {
name string
current primitives.Slot
blobRetention primitives.Epoch
colRetention primitives.Epoch
blockRetention primitives.Epoch
validOldest *primitives.Slot
// Expected block span
expectBlockBegin primitives.Slot
expectBlockEnd primitives.Slot
// Expected blob span
expectBlobBegin primitives.Slot
expectBlobEnd primitives.Slot
// Expected column span
expectColBegin primitives.Slot
expectColEnd primitives.Slot
}{
{
name: "pre-Deneb - only blocks needed",
current: 500,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(500, 5),
expectBlockEnd: 500,
expectBlobBegin: denebSlot, // adjusted to deneb
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot, // adjusted to fulu
expectColEnd: 500,
},
{
name: "between Deneb and Fulu - blocks and blobs needed",
current: 1500,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(1500, 5),
expectBlockEnd: 1500,
expectBlobBegin: max(syncEpochOffset(1500, 10), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot, // adjusted to fulu
expectColEnd: 1500,
},
{
name: "post-Fulu - all resources needed",
current: 3000,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(3000, 5),
expectBlockEnd: 3000,
expectBlobBegin: max(syncEpochOffset(3000, 10), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: max(syncEpochOffset(3000, 10), fuluSlot),
expectColEnd: 3000,
},
{
name: "exactly at Deneb boundary",
current: denebSlot,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(denebSlot, 5),
expectBlockEnd: denebSlot,
expectBlobBegin: denebSlot,
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot,
expectColEnd: denebSlot,
},
{
name: "exactly at Fulu boundary",
current: fuluSlot,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(fuluSlot, 5),
expectBlockEnd: fuluSlot,
expectBlobBegin: max(syncEpochOffset(fuluSlot, 10), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot,
expectColEnd: fuluSlot,
},
{
name: "small retention periods",
current: 5000,
blobRetention: 1,
colRetention: 2,
blockRetention: 1,
validOldest: nil,
expectBlockBegin: syncEpochOffset(5000, 1),
expectBlockEnd: 5000,
expectBlobBegin: max(syncEpochOffset(5000, 1), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: max(syncEpochOffset(5000, 2), fuluSlot),
expectColEnd: 5000,
},
{
name: "large retention periods",
current: 10000,
blobRetention: 100,
colRetention: 100,
blockRetention: 50,
validOldest: nil,
expectBlockBegin: syncEpochOffset(10000, 50),
expectBlockEnd: 10000,
expectBlobBegin: max(syncEpochOffset(10000, 100), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: max(syncEpochOffset(10000, 100), fuluSlot),
expectColEnd: 10000,
},
{
name: "with validOldestSlotPtr for blocks",
current: 8000,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: func() *primitives.Slot { s := primitives.Slot(100); return &s }(),
expectBlockBegin: 100,
expectBlockEnd: 8000,
expectBlobBegin: max(syncEpochOffset(8000, 10), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: max(syncEpochOffset(8000, 10), fuluSlot),
expectColEnd: 8000,
},
{
name: "retention approaching current slot",
current: primitives.Slot(2000 + 5*slotsPerEpoch),
blobRetention: 5,
colRetention: 5,
blockRetention: 3,
validOldest: nil,
expectBlockBegin: syncEpochOffset(primitives.Slot(2000+5*slotsPerEpoch), 3),
expectBlockEnd: primitives.Slot(2000 + 5*slotsPerEpoch),
expectBlobBegin: max(syncEpochOffset(primitives.Slot(2000+5*slotsPerEpoch), 5), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: max(syncEpochOffset(primitives.Slot(2000+5*slotsPerEpoch), 5), fuluSlot),
expectColEnd: primitives.Slot(2000 + 5*slotsPerEpoch),
},
{
name: "current just after Deneb",
current: denebSlot + 10,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(denebSlot+10, 5),
expectBlockEnd: denebSlot + 10,
expectBlobBegin: denebSlot,
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot,
expectColEnd: denebSlot + 10,
},
{
name: "current just after Fulu",
current: fuluSlot + 10,
blobRetention: 10,
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(fuluSlot+10, 5),
expectBlockEnd: fuluSlot + 10,
expectBlobBegin: max(syncEpochOffset(fuluSlot+10, 10), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot,
expectColEnd: fuluSlot + 10,
},
{
name: "blob retention would start before Deneb",
current: denebSlot + primitives.Slot(5*slotsPerEpoch),
blobRetention: 100, // very large retention
colRetention: 10,
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(denebSlot+primitives.Slot(5*slotsPerEpoch), 5),
expectBlockEnd: denebSlot + primitives.Slot(5*slotsPerEpoch),
expectBlobBegin: denebSlot, // clamped to deneb
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot,
expectColEnd: denebSlot + primitives.Slot(5*slotsPerEpoch),
},
{
name: "column retention would start before Fulu",
current: fuluSlot + primitives.Slot(5*slotsPerEpoch),
blobRetention: 10,
colRetention: 100, // very large retention
blockRetention: 5,
validOldest: nil,
expectBlockBegin: syncEpochOffset(fuluSlot+primitives.Slot(5*slotsPerEpoch), 5),
expectBlockEnd: fuluSlot + primitives.Slot(5*slotsPerEpoch),
expectBlobBegin: max(syncEpochOffset(fuluSlot+primitives.Slot(5*slotsPerEpoch), 10), denebSlot),
expectBlobEnd: fuluSlot,
expectColBegin: fuluSlot, // clamped to fulu
expectColEnd: fuluSlot + primitives.Slot(5*slotsPerEpoch),
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
sn := SyncNeeds{
current: func() primitives.Slot { return tc.current },
deneb: denebSlot,
fulu: fuluSlot,
validOldestSlotPtr: tc.validOldest,
blockRetention: tc.blockRetention,
blobRetention: tc.blobRetention,
colRetention: tc.colRetention,
}
result := sn.Currently()
// Verify block span
require.Equal(t, tc.expectBlockBegin, result.Block.Begin,
"block.begin mismatch")
require.Equal(t, tc.expectBlockEnd, result.Block.End,
"block.end mismatch")
// Verify blob span
require.Equal(t, tc.expectBlobBegin, result.Blob.Begin,
"blob.begin mismatch")
require.Equal(t, tc.expectBlobEnd, result.Blob.End,
"blob.end mismatch")
// Verify column span
require.Equal(t, tc.expectColBegin, result.Col.Begin,
"col.begin mismatch")
require.Equal(t, tc.expectColEnd, result.Col.End,
"col.end mismatch")
})
}
}
// TestCurrentNeedsIntegration verifies the complete currentNeeds workflow.
func TestCurrentNeedsIntegration(t *testing.T) {
params.SetupTestConfigCleanup(t)
denebSlot := primitives.Slot(1000)
fuluSlot := primitives.Slot(2000)
cases := []struct {
name string
current primitives.Slot
blobRetention primitives.Epoch
colRetention primitives.Epoch
testSlots []primitives.Slot
expectBlockAt []bool
expectBlobAt []bool
expectColAt []bool
}{
{
name: "pre-Deneb slot - only blocks",
current: 500,
blobRetention: 10,
colRetention: 10,
testSlots: []primitives.Slot{100, 250, 499, 500, 1000, 2000},
expectBlockAt: []bool{true, true, true, false, false, false},
expectBlobAt: []bool{false, false, false, false, true, false},
expectColAt: []bool{false, false, false, false, false, false},
},
{
name: "between Deneb and Fulu - blocks and blobs",
current: 1500,
blobRetention: 10,
colRetention: 10,
testSlots: []primitives.Slot{500, 1000, 1200, 1499, 1500, 2000},
expectBlockAt: []bool{true, true, true, true, false, false},
expectBlobAt: []bool{false, false, true, true, true, false},
expectColAt: []bool{false, false, false, false, false, false},
},
{
name: "post-Fulu - all resources",
current: 3000,
blobRetention: 10,
colRetention: 10,
testSlots: []primitives.Slot{1000, 1500, 2000, 2500, 2999, 3000},
expectBlockAt: []bool{true, true, true, true, true, false},
expectBlobAt: []bool{false, false, false, false, false, false},
expectColAt: []bool{false, false, false, false, true, false},
},
{
name: "at Deneb boundary",
current: denebSlot,
blobRetention: 5,
colRetention: 5,
testSlots: []primitives.Slot{500, 999, 1000, 1500, 2000},
expectBlockAt: []bool{true, true, false, false, false},
expectBlobAt: []bool{false, false, true, true, false},
expectColAt: []bool{false, false, false, false, false},
},
{
name: "at Fulu boundary",
current: fuluSlot,
blobRetention: 5,
colRetention: 5,
testSlots: []primitives.Slot{1000, 1500, 1999, 2000, 2001},
expectBlockAt: []bool{true, true, true, false, false},
expectBlobAt: []bool{false, false, true, false, false},
expectColAt: []bool{false, false, false, false, false},
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
sn := SyncNeeds{
current: func() primitives.Slot { return tc.current },
deneb: denebSlot,
fulu: fuluSlot,
blockRetention: 100,
blobRetention: tc.blobRetention,
colRetention: tc.colRetention,
}
cn := sn.Currently()
// Verify block.end == current
require.Equal(t, tc.current, cn.Block.End, "block.end should equal current")
// Verify blob.end == fulu
require.Equal(t, fuluSlot, cn.Blob.End, "blob.end should equal fulu")
// Verify col.end == current
require.Equal(t, tc.current, cn.Col.End, "col.end should equal current")
// Test each slot
for i, slot := range tc.testSlots {
require.Equal(t, tc.expectBlockAt[i], cn.Block.At(slot),
"block.at(%d) mismatch at index %d", slot, i)
require.Equal(t, tc.expectBlobAt[i], cn.Blob.At(slot),
"blob.at(%d) mismatch at index %d", slot, i)
require.Equal(t, tc.expectColAt[i], cn.Col.At(slot),
"col.at(%d) mismatch at index %d", slot, i)
}
})
}
}

View File

@@ -23,6 +23,7 @@ go_library(
"//beacon-chain/builder:go_default_library",
"//beacon-chain/cache:go_default_library",
"//beacon-chain/cache/depositsnapshot:go_default_library",
"//beacon-chain/das:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/kv:go_default_library",

View File

@@ -26,6 +26,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/builder"
"github.com/OffchainLabs/prysm/v7/beacon-chain/cache"
"github.com/OffchainLabs/prysm/v7/beacon-chain/cache/depositsnapshot"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/kv"
@@ -116,7 +117,7 @@ type BeaconNode struct {
GenesisProviders []genesis.Provider
CheckpointInitializer checkpoint.Initializer
forkChoicer forkchoice.ForkChoicer
clockWaiter startup.ClockWaiter
ClockWaiter startup.ClockWaiter
BackfillOpts []backfill.ServiceOption
initialSyncComplete chan struct{}
BlobStorage *filesystem.BlobStorage
@@ -129,6 +130,7 @@ type BeaconNode struct {
slasherEnabled bool
lcStore *lightclient.Store
ConfigOptions []params.Option
SyncNeedsWaiter func() (das.SyncNeeds, error)
}
// New creates a new node instance, sets up configuration options, and registers
@@ -193,7 +195,7 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
params.LogDigests(params.BeaconConfig())
synchronizer := startup.NewClockSynchronizer()
beacon.clockWaiter = synchronizer
beacon.ClockWaiter = synchronizer
beacon.forkChoicer = doublylinkedtree.New()
depositAddress, err := execution.DepositContractAddress()
@@ -233,12 +235,13 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
beacon.lhsp = &verification.LazyHeadStateProvider{}
beacon.verifyInitWaiter = verification.NewInitializerWaiter(
beacon.clockWaiter, forkchoice.NewROForkChoice(beacon.forkChoicer), beacon.stateGen, beacon.lhsp)
beacon.ClockWaiter, forkchoice.NewROForkChoice(beacon.forkChoicer), beacon.stateGen, beacon.lhsp)
beacon.BackfillOpts = append(
beacon.BackfillOpts,
backfill.WithVerifierWaiter(beacon.verifyInitWaiter),
backfill.WithInitSyncWaiter(initSyncWaiter(ctx, beacon.initialSyncComplete)),
backfill.WithSyncNeedsWaiter(beacon.SyncNeedsWaiter),
)
if err := registerServices(cliCtx, beacon, synchronizer, bfs); err != nil {
@@ -664,7 +667,7 @@ func (b *BeaconNode) registerP2P(cliCtx *cli.Context) error {
StateNotifier: b,
DB: b.db,
StateGen: b.stateGen,
ClockWaiter: b.clockWaiter,
ClockWaiter: b.ClockWaiter,
})
if err != nil {
return err
@@ -706,7 +709,7 @@ func (b *BeaconNode) registerSlashingPoolService() error {
return err
}
s := slashings.NewPoolService(b.ctx, b.slashingsPool, slashings.WithElectraTimer(b.clockWaiter, chainService.CurrentSlot))
s := slashings.NewPoolService(b.ctx, b.slashingsPool, slashings.WithElectraTimer(b.ClockWaiter, chainService.CurrentSlot))
return b.services.RegisterService(s)
}
@@ -828,7 +831,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithSlasherAttestationsFeed(b.slasherAttestationsFeed),
regularsync.WithSlasherBlockHeadersFeed(b.slasherBlockHeadersFeed),
regularsync.WithReconstructor(web3Service),
regularsync.WithClockWaiter(b.clockWaiter),
regularsync.WithClockWaiter(b.ClockWaiter),
regularsync.WithInitialSyncComplete(initialSyncComplete),
regularsync.WithStateNotifier(b),
regularsync.WithBlobStorage(b.BlobStorage),
@@ -859,7 +862,8 @@ func (b *BeaconNode) registerInitialSyncService(complete chan struct{}) error {
P2P: b.fetchP2P(),
StateNotifier: b,
BlockNotifier: b,
ClockWaiter: b.clockWaiter,
ClockWaiter: b.ClockWaiter,
SyncNeedsWaiter: b.SyncNeedsWaiter,
InitialSyncComplete: complete,
BlobStorage: b.BlobStorage,
DataColumnStorage: b.DataColumnStorage,
@@ -890,7 +894,7 @@ func (b *BeaconNode) registerSlasherService() error {
SlashingPoolInserter: b.slashingsPool,
SyncChecker: syncService,
HeadStateFetcher: chainService,
ClockWaiter: b.clockWaiter,
ClockWaiter: b.ClockWaiter,
})
if err != nil {
return err
@@ -983,7 +987,7 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error {
MaxMsgSize: maxMsgSize,
BlockBuilder: b.fetchBuilderService(),
Router: router,
ClockWaiter: b.clockWaiter,
ClockWaiter: b.ClockWaiter,
BlobStorage: b.BlobStorage,
DataColumnStorage: b.DataColumnStorage,
TrackedValidatorsCache: b.trackedValidatorsCache,
@@ -1128,7 +1132,7 @@ func (b *BeaconNode) registerPrunerService(cliCtx *cli.Context) error {
func (b *BeaconNode) RegisterBackfillService(cliCtx *cli.Context, bfs *backfill.Store) error {
pa := peers.NewAssigner(b.fetchP2P().Peers(), b.forkChoicer)
bf, err := backfill.NewService(cliCtx.Context, bfs, b.BlobStorage, b.clockWaiter, b.fetchP2P(), pa, b.BackfillOpts...)
bf, err := backfill.NewService(cliCtx.Context, bfs, b.BlobStorage, b.DataColumnStorage, b.ClockWaiter, b.fetchP2P(), pa, b.BackfillOpts...)
if err != nil {
return errors.Wrap(err, "error initializing backfill service")
}

View File

@@ -47,6 +47,7 @@ go_test(
],
embed = [":go_default_library"],
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",

View File

@@ -4,11 +4,18 @@ import (
forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
// StatusProvider describes the minimum capability that Assigner needs from peer status tracking.
// That is, the ability to retrieve the best peers by finalized checkpoint.
type StatusProvider interface {
BestFinalized(ourFinalized primitives.Epoch) (primitives.Epoch, []peer.ID)
}
// 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 {
@@ -17,9 +24,9 @@ type FinalizedCheckpointer interface {
// 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.
// The StatusProvider is used to retrieve best peers, and 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 {
func NewAssigner(s StatusProvider, fc FinalizedCheckpointer) *Assigner {
return &Assigner{
ps: s,
fc: fc,
@@ -28,7 +35,7 @@ func NewAssigner(s *Status, fc FinalizedCheckpointer) *Assigner {
// Assigner uses the "BestFinalized" peer scoring method to pick the next-best peer to receive rpc requests.
type Assigner struct {
ps *Status
ps StatusProvider
fc FinalizedCheckpointer
}
@@ -38,38 +45,42 @@ type Assigner struct {
var ErrInsufficientSuitable = errors.New("no suitable peers")
func (a *Assigner) freshPeers() ([]peer.ID, error) {
required := min(flags.Get().MinimumSyncPeers, params.BeaconConfig().MaxPeersToSync)
_, peers := a.ps.BestFinalized(params.BeaconConfig().MaxPeersToSync, a.fc.FinalizedCheckpoint().Epoch)
required := min(flags.Get().MinimumSyncPeers, min(flags.Get().MinimumSyncPeers, params.BeaconConfig().MaxPeersToSync))
_, peers := a.ps.BestFinalized(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 ")
"required": required}).Trace("Unable to assign peer while suitable peers < required")
return nil, ErrInsufficientSuitable
}
return peers, nil
}
// AssignmentFilter describes a function that takes a list of peer.IDs and returns a filtered subset.
// An example is the NotBusy filter.
type AssignmentFilter func([]peer.ID) []peer.ID
// 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) {
func (a *Assigner) Assign(filter AssignmentFilter) ([]peer.ID, error) {
best, err := a.freshPeers()
if err != nil {
return nil, err
}
return pickBest(busy, n, best), nil
return filter(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)
// NotBusy is a filter that returns the list of peer.IDs that are not in the `busy` map.
func NotBusy(busy map[peer.ID]bool) AssignmentFilter {
return func(peers []peer.ID) []peer.ID {
ps := make([]peer.ID, 0, len(peers))
for _, p := range peers {
if !busy[p] {
ps = append(ps, p)
}
}
return ps
}
return ps
}

View File

@@ -5,6 +5,8 @@ import (
"slices"
"testing"
forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/libp2p/go-libp2p/core/peer"
)
@@ -14,82 +16,68 @@ func TestPickBest(t *testing.T) {
cases := []struct {
name string
busy map[peer.ID]bool
n int
best []peer.ID
expected []peer.ID
}{
{
name: "",
n: 0,
name: "don't limit",
expected: best,
},
{
name: "none busy",
n: 1,
expected: best[0:1],
expected: best,
},
{
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(slices.Concat(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) {
t.Run(c.name, func(t *testing.T) {
if c.best == nil {
c.best = best
}
pb := pickBest(c.busy, c.n, c.best)
filt := NotBusy(c.busy)
pb := filt(c.best)
require.Equal(t, len(c.expected), len(pb))
for i := range c.expected {
require.Equal(t, c.expected[i], pb[i])
@@ -113,3 +101,310 @@ func testPeerIds(n int) []peer.ID {
}
return pids
}
// MockStatus is a test mock for the Status interface used in Assigner.
type MockStatus struct {
bestFinalizedEpoch primitives.Epoch
bestPeers []peer.ID
}
func (m *MockStatus) BestFinalized(ourFinalized primitives.Epoch) (primitives.Epoch, []peer.ID) {
return m.bestFinalizedEpoch, m.bestPeers
}
// MockFinalizedCheckpointer is a test mock for FinalizedCheckpointer interface.
type MockFinalizedCheckpointer struct {
checkpoint *forkchoicetypes.Checkpoint
}
func (m *MockFinalizedCheckpointer) FinalizedCheckpoint() *forkchoicetypes.Checkpoint {
return m.checkpoint
}
// TestAssign_HappyPath tests the Assign method with sufficient peers and various filters.
func TestAssign_HappyPath(t *testing.T) {
peers := testPeerIds(10)
cases := []struct {
name string
bestPeers []peer.ID
finalizedEpoch primitives.Epoch
filter AssignmentFilter
expectedCount int
}{
{
name: "sufficient peers with identity filter",
bestPeers: peers,
finalizedEpoch: 10,
filter: func(p []peer.ID) []peer.ID { return p },
expectedCount: 10,
},
{
name: "sufficient peers with NotBusy filter (no busy)",
bestPeers: peers,
finalizedEpoch: 10,
filter: NotBusy(make(map[peer.ID]bool)),
expectedCount: 10,
},
{
name: "sufficient peers with NotBusy filter (some busy)",
bestPeers: peers,
finalizedEpoch: 10,
filter: NotBusy(testBusyMap(peers[0:5])),
expectedCount: 5,
},
{
name: "minimum threshold exactly met",
bestPeers: peers[0:5],
finalizedEpoch: 10,
filter: func(p []peer.ID) []peer.ID { return p },
expectedCount: 5,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
mockStatus := &MockStatus{
bestFinalizedEpoch: tc.finalizedEpoch,
bestPeers: tc.bestPeers,
}
mockCheckpointer := &MockFinalizedCheckpointer{
checkpoint: &forkchoicetypes.Checkpoint{Epoch: tc.finalizedEpoch},
}
assigner := NewAssigner(mockStatus, mockCheckpointer)
result, err := assigner.Assign(tc.filter)
require.NoError(t, err)
require.Equal(t, tc.expectedCount, len(result),
fmt.Sprintf("expected %d peers, got %d", tc.expectedCount, len(result)))
})
}
}
// TestAssign_InsufficientPeers tests error handling when not enough suitable peers are available.
// Note: The actual peer threshold depends on config values MaxPeersToSync and MinimumSyncPeers.
func TestAssign_InsufficientPeers(t *testing.T) {
cases := []struct {
name string
bestPeers []peer.ID
expectedErr error
description string
}{
{
name: "exactly at minimum threshold",
bestPeers: testPeerIds(5),
expectedErr: nil,
description: "5 peers should meet the minimum threshold",
},
{
name: "well above minimum threshold",
bestPeers: testPeerIds(50),
expectedErr: nil,
description: "50 peers should easily meet requirements",
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
mockStatus := &MockStatus{
bestFinalizedEpoch: 10,
bestPeers: tc.bestPeers,
}
mockCheckpointer := &MockFinalizedCheckpointer{
checkpoint: &forkchoicetypes.Checkpoint{Epoch: 10},
}
assigner := NewAssigner(mockStatus, mockCheckpointer)
result, err := assigner.Assign(NotBusy(make(map[peer.ID]bool)))
if tc.expectedErr != nil {
require.NotNil(t, err, tc.description)
require.Equal(t, tc.expectedErr, err)
} else {
require.NoError(t, err, tc.description)
require.Equal(t, len(tc.bestPeers), len(result))
}
})
}
}
// TestAssign_FilterApplication verifies that filters are correctly applied to peer lists.
func TestAssign_FilterApplication(t *testing.T) {
peers := testPeerIds(10)
cases := []struct {
name string
bestPeers []peer.ID
filterToApply AssignmentFilter
expectedCount int
description string
}{
{
name: "identity filter returns all peers",
bestPeers: peers,
filterToApply: func(p []peer.ID) []peer.ID { return p },
expectedCount: 10,
description: "identity filter should not change peer list",
},
{
name: "filter removes all peers (all busy)",
bestPeers: peers,
filterToApply: NotBusy(testBusyMap(peers)),
expectedCount: 0,
description: "all peers busy should return empty list",
},
{
name: "filter removes first 5 peers",
bestPeers: peers,
filterToApply: NotBusy(testBusyMap(peers[0:5])),
expectedCount: 5,
description: "should only return non-busy peers",
},
{
name: "filter removes last 5 peers",
bestPeers: peers,
filterToApply: NotBusy(testBusyMap(peers[5:])),
expectedCount: 5,
description: "should only return non-busy peers from beginning",
},
{
name: "custom filter selects every other peer",
bestPeers: peers,
filterToApply: func(p []peer.ID) []peer.ID {
result := make([]peer.ID, 0)
for i := 0; i < len(p); i += 2 {
result = append(result, p[i])
}
return result
},
expectedCount: 5,
description: "custom filter selecting every other peer",
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
mockStatus := &MockStatus{
bestFinalizedEpoch: 10,
bestPeers: tc.bestPeers,
}
mockCheckpointer := &MockFinalizedCheckpointer{
checkpoint: &forkchoicetypes.Checkpoint{Epoch: 10},
}
assigner := NewAssigner(mockStatus, mockCheckpointer)
result, err := assigner.Assign(tc.filterToApply)
require.NoError(t, err, fmt.Sprintf("unexpected error: %v", err))
require.Equal(t, tc.expectedCount, len(result),
fmt.Sprintf("%s: expected %d peers, got %d", tc.description, tc.expectedCount, len(result)))
})
}
}
// TestAssign_FinalizedCheckpointUsage verifies that the finalized checkpoint is correctly used.
func TestAssign_FinalizedCheckpointUsage(t *testing.T) {
peers := testPeerIds(10)
cases := []struct {
name string
finalizedEpoch primitives.Epoch
bestPeers []peer.ID
expectedCount int
description string
}{
{
name: "epoch 0",
finalizedEpoch: 0,
bestPeers: peers,
expectedCount: 10,
description: "epoch 0 should work",
},
{
name: "epoch 100",
finalizedEpoch: 100,
bestPeers: peers,
expectedCount: 10,
description: "high epoch number should work",
},
{
name: "epoch changes between calls",
finalizedEpoch: 50,
bestPeers: testPeerIds(5),
expectedCount: 5,
description: "epoch value should be used in checkpoint",
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
mockStatus := &MockStatus{
bestFinalizedEpoch: tc.finalizedEpoch,
bestPeers: tc.bestPeers,
}
mockCheckpointer := &MockFinalizedCheckpointer{
checkpoint: &forkchoicetypes.Checkpoint{Epoch: tc.finalizedEpoch},
}
assigner := NewAssigner(mockStatus, mockCheckpointer)
result, err := assigner.Assign(NotBusy(make(map[peer.ID]bool)))
require.NoError(t, err)
require.Equal(t, tc.expectedCount, len(result),
fmt.Sprintf("%s: expected %d peers, got %d", tc.description, tc.expectedCount, len(result)))
})
}
}
// TestAssign_EdgeCases tests boundary conditions and edge cases.
func TestAssign_EdgeCases(t *testing.T) {
cases := []struct {
name string
bestPeers []peer.ID
filter AssignmentFilter
expectedCount int
description string
}{
{
name: "filter returns empty from sufficient peers",
bestPeers: testPeerIds(10),
filter: func(p []peer.ID) []peer.ID { return []peer.ID{} },
expectedCount: 0,
description: "filter can return empty list even if sufficient peers available",
},
{
name: "filter selects subset from sufficient peers",
bestPeers: testPeerIds(10),
filter: func(p []peer.ID) []peer.ID { return p[0:2] },
expectedCount: 2,
description: "filter can return subset of available peers",
},
{
name: "filter selects single peer from many",
bestPeers: testPeerIds(20),
filter: func(p []peer.ID) []peer.ID { return p[0:1] },
expectedCount: 1,
description: "filter can select single peer from many available",
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
mockStatus := &MockStatus{
bestFinalizedEpoch: 10,
bestPeers: tc.bestPeers,
}
mockCheckpointer := &MockFinalizedCheckpointer{
checkpoint: &forkchoicetypes.Checkpoint{Epoch: 10},
}
assigner := NewAssigner(mockStatus, mockCheckpointer)
result, err := assigner.Assign(tc.filter)
require.NoError(t, err, fmt.Sprintf("%s: unexpected error: %v", tc.description, err))
require.Equal(t, tc.expectedCount, len(result),
fmt.Sprintf("%s: expected %d peers, got %d", tc.description, tc.expectedCount, len(result)))
})
}
}

View File

@@ -704,76 +704,54 @@ func (p *Status) deprecatedPrune() {
p.tallyIPTracker()
}
// BestFinalized returns the highest finalized epoch equal to or higher than `ourFinalizedEpoch`
// that is agreed upon by the majority of peers, and the peers agreeing on this finalized epoch.
// This method may not return the absolute highest finalized epoch, but the finalized epoch in which
// most peers can serve blocks (plurality voting). Ideally, all peers would be reporting the same
// finalized epoch but some may be behind due to their own latency, or because of their finalized
// epoch at the time we queried them.
func (p *Status) BestFinalized(maxPeers int, ourFinalizedEpoch primitives.Epoch) (primitives.Epoch, []peer.ID) {
// Retrieve all connected peers.
// BestFinalized groups all peers by their last known finalized epoch
// and selects the epoch of the largest group as best.
// Any peer with a finalized epoch < ourFinalized is excluded from consideration.
// In the event of a tie in largest group size, the higher epoch is the tie breaker.
// The selected epoch is returned, along with a list of peers with a finalized epoch >= the selected epoch.
func (p *Status) BestFinalized(ourFinalized primitives.Epoch) (primitives.Epoch, []peer.ID) {
connected := p.Connected()
pids := make([]peer.ID, 0, len(connected))
views := make(map[peer.ID]*pb.StatusV2, len(connected))
// key: finalized epoch, value: number of peers that support this finalized epoch.
finalizedEpochVotes := make(map[primitives.Epoch]uint64)
// key: peer ID, value: finalized epoch of the peer.
pidEpoch := make(map[peer.ID]primitives.Epoch, len(connected))
// key: peer ID, value: head slot of the peer.
pidHead := make(map[peer.ID]primitives.Slot, len(connected))
potentialPIDs := make([]peer.ID, 0, len(connected))
votes := make(map[primitives.Epoch]uint64)
winner := primitives.Epoch(0)
for _, pid := range connected {
peerChainState, err := p.ChainState(pid)
// Skip if the peer's finalized epoch is not defined, or if the peer's finalized epoch is
// lower than ours.
if err != nil || peerChainState == nil || peerChainState.FinalizedEpoch < ourFinalizedEpoch {
view, err := p.ChainState(pid)
if err != nil || view == nil || view.FinalizedEpoch < ourFinalized {
continue
}
pids = append(pids, pid)
views[pid] = view
finalizedEpochVotes[peerChainState.FinalizedEpoch]++
pidEpoch[pid] = peerChainState.FinalizedEpoch
pidHead[pid] = peerChainState.HeadSlot
potentialPIDs = append(potentialPIDs, pid)
}
// Select the target epoch, which is the epoch most peers agree upon.
// If there is a tie, select the highest epoch.
targetEpoch, mostVotes := primitives.Epoch(0), uint64(0)
for epoch, count := range finalizedEpochVotes {
if count > mostVotes || (count == mostVotes && epoch > targetEpoch) {
mostVotes = count
targetEpoch = epoch
votes[view.FinalizedEpoch]++
if winner == 0 {
winner = view.FinalizedEpoch
continue
}
e, v := view.FinalizedEpoch, votes[view.FinalizedEpoch]
if v > votes[winner] || v == votes[winner] && e > winner {
winner = e
}
}
// Sort PIDs by finalized (epoch, head), in decreasing order.
sort.Slice(potentialPIDs, func(i, j int) bool {
if pidEpoch[potentialPIDs[i]] == pidEpoch[potentialPIDs[j]] {
return pidHead[potentialPIDs[i]] > pidHead[potentialPIDs[j]]
// Descending sort by (finalized, head).
sort.Slice(pids, func(i, j int) bool {
iv, jv := views[pids[i]], views[pids[j]]
if iv.FinalizedEpoch == jv.FinalizedEpoch {
return iv.HeadSlot > jv.HeadSlot
}
return pidEpoch[potentialPIDs[i]] > pidEpoch[potentialPIDs[j]]
return iv.FinalizedEpoch > jv.FinalizedEpoch
})
// Trim potential peers to those on or after target epoch.
for i, pid := range potentialPIDs {
if pidEpoch[pid] < targetEpoch {
potentialPIDs = potentialPIDs[:i]
break
}
}
// Find the first peer with finalized epoch < winner, trim and all following (lower) peers.
trim := sort.Search(len(pids), func(i int) bool {
return views[pids[i]].FinalizedEpoch < winner
})
pids = pids[:trim]
// Trim potential peers to at most maxPeers.
if len(potentialPIDs) > maxPeers {
potentialPIDs = potentialPIDs[:maxPeers]
}
return targetEpoch, potentialPIDs
return winner, pids
}
// BestNonFinalized returns the highest known epoch, higher than ours,

View File

@@ -654,9 +654,10 @@ func TestTrimmedOrderedPeers(t *testing.T) {
FinalizedRoot: mockroot2[:],
})
target, pids := p.BestFinalized(maxPeers, 0)
target, pids := p.BestFinalized(0)
assert.Equal(t, expectedTarget, target, "Incorrect target epoch retrieved")
assert.Equal(t, maxPeers, len(pids), "Incorrect number of peers retrieved")
// addPeer called 5 times above
assert.Equal(t, 5, len(pids), "Incorrect number of peers retrieved")
// Expect the returned list to be ordered by finalized epoch and trimmed to max peers.
assert.Equal(t, pid3, pids[0], "Incorrect first peer")
@@ -1017,7 +1018,10 @@ func TestStatus_BestPeer(t *testing.T) {
HeadSlot: peerConfig.headSlot,
})
}
epoch, pids := p.BestFinalized(tt.limitPeers, tt.ourFinalizedEpoch)
epoch, pids := p.BestFinalized(tt.ourFinalizedEpoch)
if len(pids) > tt.limitPeers {
pids = pids[:tt.limitPeers]
}
assert.Equal(t, tt.targetEpoch, epoch, "Unexpected epoch retrieved")
assert.Equal(t, tt.targetEpochSupport, len(pids), "Unexpected number of peers supporting retrieved epoch")
})
@@ -1044,7 +1048,10 @@ func TestBestFinalized_returnsMaxValue(t *testing.T) {
})
}
_, pids := p.BestFinalized(maxPeers, 0)
_, pids := p.BestFinalized(0)
if len(pids) > maxPeers {
pids = pids[:maxPeers]
}
assert.Equal(t, maxPeers, len(pids), "Wrong number of peers returned")
}

View File

@@ -7,6 +7,7 @@ go_library(
"block_batcher.go",
"context.go",
"custody.go",
"data_column_assignment.go",
"data_column_sidecars.go",
"data_columns_reconstruct.go",
"deadlines.go",
@@ -135,6 +136,7 @@ go_library(
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enode:go_default_library",
"@com_github_hashicorp_golang_lru//:go_default_library",
"@com_github_libp2p_go_libp2p//core:go_default_library",
"@com_github_libp2p_go_libp2p//core/host:go_default_library",
@@ -167,6 +169,7 @@ go_test(
"block_batcher_test.go",
"context_test.go",
"custody_test.go",
"data_column_assignment_test.go",
"data_column_sidecars_test.go",
"data_columns_reconstruct_test.go",
"decode_pubsub_test.go",

View File

@@ -6,17 +6,22 @@ go_library(
"batch.go",
"batcher.go",
"blobs.go",
"columns.go",
"error.go",
"fulu_transition.go",
"log.go",
"metrics.go",
"pool.go",
"service.go",
"status.go",
"verify.go",
"verify_column.go",
"worker.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/sync/backfill",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/das:go_default_library",
"//beacon-chain/db:go_default_library",
@@ -37,7 +42,6 @@ go_library(
"//proto/dbval:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime:go_default_library",
"//runtime/version: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",
@@ -51,19 +55,27 @@ go_test(
name = "go_default_test",
srcs = [
"batch_test.go",
"batcher_expiration_test.go",
"batcher_test.go",
"blobs_test.go",
"columns_test.go",
"fulu_transition_test.go",
"log_test.go",
"pool_test.go",
"service_test.go",
"status_test.go",
"verify_column_test.go",
"verify_test.go",
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/blockchain/kzg:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/das:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/p2p/peers:go_default_library",
"//beacon-chain/p2p/testing:go_default_library",
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
@@ -85,5 +97,7 @@ go_test(
"@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",
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_stretchr_testify//require:go_default_library",
],
)

View File

@@ -8,7 +8,6 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
eth "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/libp2p/go-libp2p/core/peer"
@@ -16,9 +15,13 @@ import (
"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")
var errChainBroken = errors.New("batch is not the ancestor of a known finalized root")
// retryLogMod defines how often retryable errors are logged at debug level instead of trace.
const retryLogMod = 5
// retryDelay defines the delay between retry attempts for a batch.
const retryDelay = time.Second
type batchState int
@@ -30,16 +33,20 @@ func (s batchState) String() string {
return "init"
case batchSequenced:
return "sequenced"
case batchErrRetryable:
return "error_retryable"
case batchSyncBlobs:
return "sync_blobs"
case batchSyncColumns:
return "sync_columns"
case batchImportable:
return "importable"
case batchImportComplete:
return "import_complete"
case batchEndSequence:
return "end_sequence"
case batchBlobSync:
return "blob_sync"
case batchErrRetryable:
return "error_retryable"
case batchErrFatal:
return "error_fatal"
default:
return "unknown"
}
@@ -49,15 +56,15 @@ const (
batchNil batchState = iota
batchInit
batchSequenced
batchErrRetryable
batchBlobSync
batchSyncBlobs
batchSyncColumns
batchImportable
batchImportComplete
batchErrRetryable
batchErrFatal // if this is received in the main loop, the worker pool will be shut down.
batchEndSequence
)
var retryDelay = time.Second
type batchId string
type batch struct {
@@ -67,35 +74,52 @@ type batch struct {
retries int
retryAfter time.Time
begin primitives.Slot
end primitives.Slot // half-open interval, [begin, end), ie >= start, < end.
results verifiedROBlocks
end primitives.Slot // half-open interval, [begin, end), ie >= begin, < end.
blocks verifiedROBlocks
err error
state batchState
busy peer.ID
blockPid peer.ID
blobPid peer.ID
bs *blobSync
// `assignedPeer` is used by the worker pool to assign and unassign peer.IDs to serve requests for the current batch state.
// Depending on the state it will be copied to blockPeer, columns.Peer, blobs.Peer.
assignedPeer peer.ID
blockPeer peer.ID
nextReqCols []uint64
blobs *blobSync
columns *columnSync
}
func (b batch) logFields() logrus.Fields {
f := map[string]any{
"batchId": b.id(),
"state": b.state.String(),
"scheduled": b.scheduled.String(),
"seq": b.seq,
"retries": b.retries,
"begin": b.begin,
"end": b.end,
"busyPid": b.busy,
"blockPid": b.blockPid,
"blobPid": b.blobPid,
"batchId": b.id(),
"state": b.state.String(),
"scheduled": b.scheduled.String(),
"seq": b.seq,
"retries": b.retries,
"retryAfter": b.retryAfter.String(),
"begin": b.begin,
"end": b.end,
"busyPid": b.assignedPeer,
"blockPid": b.blockPeer,
}
if b.blobs != nil {
f["blobPid"] = b.blobs.peer
}
if b.columns != nil {
f["colPid"] = b.columns.peer
}
if b.retries > 0 {
f["retryAfter"] = b.retryAfter.String()
}
if b.state == batchSyncColumns {
f["nextColumns"] = fmt.Sprintf("%v", b.nextReqCols)
}
if b.state == batchErrRetryable && b.blobs != nil {
f["blobsMissing"] = b.blobs.needed()
}
return f
}
// replaces returns true if `r` is a version of `b` that has been updated by a worker,
// meaning it should replace `b` in the batch sequencing queue.
func (b batch) replaces(r batch) bool {
if r.state == batchImportComplete {
return false
@@ -114,9 +138,9 @@ func (b batch) id() batchId {
}
func (b batch) ensureParent(expected [32]byte) error {
tail := b.results[len(b.results)-1]
tail := b.blocks[len(b.blocks)-1]
if tail.Root() != expected {
return errors.Wrapf(ErrChainBroken, "last parent_root=%#x, tail root=%#x", expected, tail.Root())
return errors.Wrapf(errChainBroken, "last parent_root=%#x, tail root=%#x", expected, tail.Root())
}
return nil
}
@@ -136,21 +160,15 @@ func (b batch) blobRequest() *eth.BlobSidecarsByRangeRequest {
}
}
func (b batch) withResults(results verifiedROBlocks, bs *blobSync) batch {
b.results = results
b.bs = bs
if bs.blobsNeeded() > 0 {
return b.withState(batchBlobSync)
func (b batch) transitionToNext() batch {
if len(b.blocks) == 0 {
return b.withState(batchSequenced)
}
return b.withState(batchImportable)
}
func (b batch) postBlobSync() batch {
if b.blobsNeeded() > 0 {
log.WithFields(b.logFields()).WithField("blobsMissing", b.blobsNeeded()).Error("Batch still missing blobs after downloading from peer")
b.bs = nil
b.results = []blocks.ROBlock{}
return b.withState(batchErrRetryable)
if len(b.columns.columnsNeeded()) > 0 {
return b.withState(batchSyncColumns)
}
if b.blobs != nil && b.blobs.needed() > 0 {
return b.withState(batchSyncBlobs)
}
return b.withState(batchImportable)
}
@@ -159,44 +177,89 @@ func (b batch) withState(s batchState) batch {
if s == batchSequenced {
b.scheduled = time.Now()
switch b.state {
case batchErrRetryable:
b.retries += 1
b.retryAfter = time.Now().Add(retryDelay)
log.WithFields(b.logFields()).Info("Sequencing batch for retry after delay")
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.blockPid = p
backfillBatchTimeWaiting.Observe(float64(time.Since(b.scheduled).Milliseconds()))
return b
}
func (b batch) withRetryableError(err error) batch {
b.err = err
b.retries += 1
b.retryAfter = time.Now().Add(retryDelay)
msg := "Could not proceed with batch processing due to error"
logBase := log.WithFields(b.logFields()).WithError(err)
// Log at trace level to limit log noise,
// but escalate to debug level every nth attempt for batches that have some peristent issue.
if b.retries&retryLogMod != 0 {
logBase.Trace(msg)
} else {
logBase.Debug(msg)
}
return b.withState(batchErrRetryable)
}
func (b batch) blobsNeeded() int {
return b.bs.blobsNeeded()
func (b batch) withFatalError(err error) batch {
log.WithFields(b.logFields()).WithError(err).Error("Fatal batch processing error")
b.err = err
return b.withState(batchErrFatal)
}
func (b batch) blobResponseValidator() sync.BlobResponseValidation {
return b.bs.validateNext
func (b batch) withError(err error) batch {
if isRetryable(err) {
return b.withRetryableError(err)
}
return b.withFatalError(err)
}
func (b batch) availabilityStore() das.AvailabilityStore {
return b.bs.store
func (b batch) validatingColumnRequest(cb *columnBisector) (*validatingColumnRequest, error) {
req, err := b.columns.request(b.nextReqCols, columnRequestLimit)
if err != nil {
return nil, errors.Wrap(err, "columns request")
}
if req == nil {
return nil, nil
}
return &validatingColumnRequest{
req: req,
columnSync: b.columns,
bisector: cb,
}, nil
}
// resetToRetryColumns is called after a partial batch failure. It adds column indices back
// to the toDownload structure for any blocks where those columns failed, and resets the bisector state.
// Note that this method will also prune any columns that have expired, meaning we no longer need them
// per spec and/or our backfill & retention settings.
func resetToRetryColumns(b batch, needs das.CurrentNeeds) batch {
// return the given batch as-is if it isn't in a state that this func should handle.
if b.columns == nil || b.columns.bisector == nil || len(b.columns.bisector.errs) == 0 {
return b.transitionToNext()
}
pruned := make(map[[32]byte]struct{})
b.columns.pruneExpired(needs, pruned)
// clear out failed column state in the bisector and add back to
bisector := b.columns.bisector
roots := bisector.failingRoots()
// Add all the failed columns back to the toDownload structure and reset the bisector state.
for _, root := range roots {
if _, rm := pruned[root]; rm {
continue
}
bc := b.columns.toDownload[root]
bc.remaining.Merge(bisector.failuresFor(root))
}
b.columns.bisector.reset()
return b.transitionToNext()
}
var batchBlockUntil = func(ctx context.Context, untilRetry time.Duration, b batch) error {
@@ -223,6 +286,26 @@ func (b batch) waitUntilReady(ctx context.Context) error {
return nil
}
func (b batch) workComplete() bool {
return b.state == batchImportable
}
func (b batch) expired(needs das.CurrentNeeds) bool {
if !needs.Block.At(b.end - 1) {
log.WithFields(b.logFields()).WithField("retentionStartSlot", needs.Block.Begin).Debug("Batch outside retention window")
return true
}
return false
}
func (b batch) selectPeer(picker *sync.PeerPicker, busy map[peer.ID]bool) (peer.ID, []uint64, error) {
if b.state == batchSyncColumns {
return picker.ForColumns(b.columns.columnsNeeded(), busy)
}
peer, err := picker.ForBlocks(busy)
return peer, nil, err
}
func sortBatchDesc(bb []batch) {
sort.Slice(bb, func(i, j int) bool {
return bb[i].end > bb[j].end

View File

@@ -24,17 +24,16 @@ func TestSortBatchDesc(t *testing.T) {
}
func TestWaitUntilReady(t *testing.T) {
b := batch{}.withState(batchErrRetryable)
require.Equal(t, time.Time{}, b.retryAfter)
var got time.Duration
wur := batchBlockUntil
var got time.Duration
var errDerp = errors.New("derp")
batchBlockUntil = func(_ context.Context, ur time.Duration, _ batch) error {
got = ur
return errDerp
}
// retries counter and timestamp are set when we mark the batch for sequencing, if it is in the retry state
b = b.withState(batchSequenced)
b := batch{}.withRetryableError(errors.New("test error"))
require.ErrorIs(t, b.waitUntilReady(t.Context()), errDerp)
require.Equal(t, true, retryDelay-time.Until(b.retryAfter) < time.Millisecond)
require.Equal(t, true, got < retryDelay && got > retryDelay-time.Millisecond)

View File

@@ -1,6 +1,7 @@
package backfill
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/pkg/errors"
)
@@ -10,8 +11,9 @@ var errEndSequence = errors.New("sequence has terminated, no more backfill batch
var errCannotDecreaseMinimum = errors.New("the minimum backfill slot can only be increased, not decreased")
type batchSequencer struct {
batcher batcher
seq []batch
batcher batcher
seq []batch
currentNeeds func() das.CurrentNeeds
}
// sequence() is meant as a verb "arrange in a particular order".
@@ -19,32 +21,38 @@ type batchSequencer struct {
// 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) {
needs := c.currentNeeds()
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:
if c.seq[i].state == 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)
c.seq[i] = c.batcher.before(c.batcher.max)
} else {
b = c.batcher.beforeBatch(c.seq[i-1])
c.seq[i] = 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 {
}
if c.seq[i].state == batchInit || c.seq[i].state == batchErrRetryable {
// This means the batch has fallen outside the retention window so we no longer need to sync it.
// Since we always create batches from high to low, we can assume we've already created the
// descendent batches from the batch we're dropping, so there won't be another batch depending on
// this one - we can stop adding batches and mark put this one in the batchEndSequence state.
// When all batches are in batchEndSequence, worker pool spins down and marks backfill complete.
if c.seq[i].expired(needs) {
c.seq[i] = c.seq[i].withState(batchEndSequence)
} else {
c.seq[i] = c.seq[i].withState(batchSequenced)
s = append(s, c.seq[i])
continue
}
default:
}
if c.seq[i].state == batchEndSequence && len(s) == 0 {
s = append(s, c.seq[i])
continue
}
}
@@ -62,6 +70,7 @@ func (c *batchSequencer) sequence() ([]batch, error) {
// seq with new batches that are ready to be worked on.
func (c *batchSequencer) update(b batch) {
done := 0
needs := c.currentNeeds()
for i := 0; i < len(c.seq); i++ {
if b.replaces(c.seq[i]) {
c.seq[i] = b
@@ -73,16 +82,23 @@ func (c *batchSequencer) update(b batch) {
done += 1
continue
}
if c.seq[i].expired(needs) {
c.seq[i] = c.seq[i].withState(batchEndSequence)
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]
}
if done == 1 && len(c.seq) == 1 {
if done == len(c.seq) {
c.seq[0] = c.batcher.beforeBatch(c.seq[0])
return
}
// 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,
@@ -113,18 +129,6 @@ func (c *batchSequencer) importable() []batch {
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 {
@@ -158,23 +162,24 @@ func (c *batchSequencer) numTodo() int {
return todo
}
func newBatchSequencer(seqLen int, min, max, size primitives.Slot) *batchSequencer {
b := batcher{min: min, max: max, size: size}
func newBatchSequencer(seqLen int, max, size primitives.Slot, needsCb func() das.CurrentNeeds) *batchSequencer {
b := batcher{currentNeeds: needsCb, max: max, size: size}
seq := make([]batch, seqLen)
return &batchSequencer{batcher: b, seq: seq}
return &batchSequencer{batcher: b, seq: seq, currentNeeds: needsCb}
}
type batcher struct {
min primitives.Slot
max primitives.Slot
size primitives.Slot
currentNeeds func() das.CurrentNeeds
max primitives.Slot
size primitives.Slot
}
func (r batcher) remaining(upTo primitives.Slot) int {
if r.min >= upTo {
needs := r.currentNeeds()
if !needs.Block.At(upTo) {
return 0
}
delta := upTo - r.min
delta := upTo - needs.Block.Begin
if delta%r.size != 0 {
return int(delta/r.size) + 1
}
@@ -186,13 +191,18 @@ func (r batcher) beforeBatch(upTo batch) batch {
}
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 {
// upTo is an exclusive upper bound. If we do not need the block at the upTo slot,
// we don't have anything left to sync, signaling the end of the backfill process.
needs := r.currentNeeds()
// The upper bound is exclusive, so we shouldn't return in this case where the previous
// batch beginning sits at the exact slot of the start of the retention window. In that case
// we've actually hit the end of the sync sequence.
if !needs.Block.At(upTo) || needs.Block.Begin == upTo {
return batch{begin: upTo, end: upTo, state: batchEndSequence}
}
begin := r.min
if upTo > r.size+r.min {
begin := needs.Block.Begin
if upTo > r.size+needs.Block.Begin {
begin = upTo - r.size
}

View File

@@ -0,0 +1,831 @@
package backfill
import (
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
// dynamicNeeds provides a mutable currentNeeds callback for testing scenarios
// where the retention window changes over time.
type dynamicNeeds struct {
blockBegin primitives.Slot
blockEnd primitives.Slot
blobBegin primitives.Slot
blobEnd primitives.Slot
colBegin primitives.Slot
colEnd primitives.Slot
}
func newDynamicNeeds(blockBegin, blockEnd primitives.Slot) *dynamicNeeds {
return &dynamicNeeds{
blockBegin: blockBegin,
blockEnd: blockEnd,
blobBegin: blockBegin,
blobEnd: blockEnd,
colBegin: blockBegin,
colEnd: blockEnd,
}
}
func (d *dynamicNeeds) get() das.CurrentNeeds {
return das.CurrentNeeds{
Block: das.NeedSpan{Begin: d.blockBegin, End: d.blockEnd},
Blob: das.NeedSpan{Begin: d.blobBegin, End: d.blobEnd},
Col: das.NeedSpan{Begin: d.colBegin, End: d.colEnd},
}
}
// advance moves the retention window forward by the given number of slots.
func (d *dynamicNeeds) advance(slots primitives.Slot) {
d.blockBegin += slots
d.blockEnd += slots
d.blobBegin += slots
d.blobEnd += slots
d.colBegin += slots
d.colEnd += slots
}
// setBlockBegin sets only the block retention start slot.
func (d *dynamicNeeds) setBlockBegin(begin primitives.Slot) {
d.blockBegin = begin
}
// ============================================================================
// Category 1: Basic Expiration During sequence()
// ============================================================================
func TestSequenceExpiration_SingleBatchExpires_Init(t *testing.T) {
// Single batch in batchInit expires when needs.block.begin moves past it
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(1, 200, 50, dn.get)
// Initialize batch: [150, 200)
seq.seq[0] = batch{begin: 150, end: 200, state: batchInit}
// Move retention window past the batch
dn.setBlockBegin(200)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchEndSequence, got[0].state)
}
func TestSequenceExpiration_SingleBatchExpires_ErrRetryable(t *testing.T) {
// Single batch in batchErrRetryable expires when needs change
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(1, 200, 50, dn.get)
seq.seq[0] = batch{begin: 150, end: 200, state: batchErrRetryable}
// Move retention window past the batch
dn.setBlockBegin(200)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchEndSequence, got[0].state)
}
func TestSequenceExpiration_MultipleBatchesExpire_Partial(t *testing.T) {
// 4 batches, 2 expire when needs change
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(4, 400, 50, dn.get)
// Batches: [350,400), [300,350), [250,300), [200,250)
seq.seq[0] = batch{begin: 350, end: 400, state: batchInit}
seq.seq[1] = batch{begin: 300, end: 350, state: batchInit}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[3] = batch{begin: 200, end: 250, state: batchInit}
// Move retention to 300 - batches [250,300) and [200,250) should expire
dn.setBlockBegin(300)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 2, len(got))
// First two batches should be sequenced (not expired)
require.Equal(t, batchSequenced, got[0].state)
require.Equal(t, primitives.Slot(350), got[0].begin)
require.Equal(t, batchSequenced, got[1].state)
require.Equal(t, primitives.Slot(300), got[1].begin)
// Verify expired batches are marked batchEndSequence in seq
require.Equal(t, batchEndSequence, seq.seq[2].state)
require.Equal(t, batchEndSequence, seq.seq[3].state)
}
func TestSequenceExpiration_AllBatchesExpire(t *testing.T) {
// All batches expire, returns one batchEndSequence
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
// Move retention past all batches
dn.setBlockBegin(350)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchEndSequence, got[0].state)
}
func TestSequenceExpiration_BatchAtExactBoundary(t *testing.T) {
// Batch with end == needs.block.begin should expire
// Because expired() checks !needs.block.at(b.end - 1)
// If batch.end = 200 and needs.block.begin = 200, then at(199) = false → expired
dn := newDynamicNeeds(200, 500)
seq := newBatchSequencer(1, 250, 50, dn.get)
// Batch [150, 200) - end is exactly at retention start
seq.seq[0] = batch{begin: 150, end: 200, state: batchInit}
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchEndSequence, got[0].state)
}
func TestSequenceExpiration_BatchJustInsideBoundary(t *testing.T) {
// Batch with end == needs.block.begin + 1 should NOT expire
// at(200) with begin=200 returns true
dn := newDynamicNeeds(200, 500)
seq := newBatchSequencer(1, 251, 50, dn.get)
// Batch [200, 251) - end-1 = 250 which is inside [200, 500)
seq.seq[0] = batch{begin: 200, end: 251, state: batchInit}
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchSequenced, got[0].state)
}
// ============================================================================
// Category 2: Expiration During update()
// ============================================================================
func TestUpdateExpiration_UpdateCausesExpiration(t *testing.T) {
// Update a batch while needs have changed, causing other batches to expire
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchSequenced}
seq.seq[1] = batch{begin: 200, end: 250, state: batchSequenced}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
// Move retention window
dn.setBlockBegin(200)
seq.batcher.currentNeeds = dn.get
// Update first batch (should still be valid)
updated := batch{begin: 250, end: 300, state: batchImportable, seq: 1}
seq.update(updated)
// First batch should be updated
require.Equal(t, batchImportable, seq.seq[0].state)
// Third batch should have expired during update
require.Equal(t, batchEndSequence, seq.seq[2].state)
}
func TestUpdateExpiration_MultipleExpireDuringUpdate(t *testing.T) {
// Several batches expire when needs advance significantly
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(4, 400, 50, dn.get)
seq.seq[0] = batch{begin: 350, end: 400, state: batchSequenced}
seq.seq[1] = batch{begin: 300, end: 350, state: batchSequenced}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[3] = batch{begin: 200, end: 250, state: batchInit}
// Move retention to expire last two batches
dn.setBlockBegin(300)
seq.batcher.currentNeeds = dn.get
// Update first batch
updated := batch{begin: 350, end: 400, state: batchImportable, seq: 1}
seq.update(updated)
// Check that expired batches are marked
require.Equal(t, batchEndSequence, seq.seq[2].state)
require.Equal(t, batchEndSequence, seq.seq[3].state)
}
func TestUpdateExpiration_UpdateCompleteWhileExpiring(t *testing.T) {
// Mark batch complete while other batches expire
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchImportable}
seq.seq[1] = batch{begin: 200, end: 250, state: batchSequenced}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
// Move retention to expire last batch
dn.setBlockBegin(200)
seq.batcher.currentNeeds = dn.get
// Mark first batch complete
completed := batch{begin: 250, end: 300, state: batchImportComplete, seq: 1}
seq.update(completed)
// Completed batch removed, third batch should have expired
// Check that we still have 3 batches (shifted + new ones added)
require.Equal(t, 3, len(seq.seq))
// The batch that was at index 2 should now be expired
foundExpired := false
for _, b := range seq.seq {
if b.state == batchEndSequence {
foundExpired = true
break
}
}
require.Equal(t, true, foundExpired, "should have an expired batch")
}
func TestUpdateExpiration_ExpiredBatchNotShiftedIncorrectly(t *testing.T) {
// Verify expired batches don't get incorrectly shifted
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchImportComplete}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
// Move retention to expire all remaining init batches
dn.setBlockBegin(250)
seq.batcher.currentNeeds = dn.get
// Update with the completed batch
completed := batch{begin: 250, end: 300, state: batchImportComplete, seq: 1}
seq.update(completed)
// Verify sequence integrity
require.Equal(t, 3, len(seq.seq))
}
func TestUpdateExpiration_NewBatchCreatedRespectsNeeds(t *testing.T) {
// When new batch is created after expiration, it should respect current needs
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(2, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchImportable}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
// Mark first batch complete to trigger new batch creation
completed := batch{begin: 250, end: 300, state: batchImportComplete, seq: 1}
seq.update(completed)
// New batch should be created - verify it respects the needs
require.Equal(t, 2, len(seq.seq))
// New batch should have proper bounds
for _, b := range seq.seq {
if b.state == batchNil {
continue
}
require.Equal(t, true, b.begin < b.end, "batch bounds should be valid")
}
}
// ============================================================================
// Category 3: Progressive Slot Advancement
// ============================================================================
func TestProgressiveAdvancement_SlotAdvancesGradually(t *testing.T) {
// Simulate gradual slot advancement with batches expiring one by one
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(4, 400, 50, dn.get)
// Initialize batches
seq.seq[0] = batch{begin: 350, end: 400, state: batchInit}
seq.seq[1] = batch{begin: 300, end: 350, state: batchInit}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[3] = batch{begin: 200, end: 250, state: batchInit}
// First sequence - all should be returned
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 4, len(got))
// Advance by 50 slots - last batch should expire
dn.setBlockBegin(250)
seq.batcher.currentNeeds = dn.get
// Mark first batch importable and update
seq.seq[0].state = batchImportable
seq.update(seq.seq[0])
// Last batch should now be expired
require.Equal(t, batchEndSequence, seq.seq[3].state)
// Advance again
dn.setBlockBegin(300)
seq.batcher.currentNeeds = dn.get
seq.seq[1].state = batchImportable
seq.update(seq.seq[1])
// Count expired batches
expiredCount := 0
for _, b := range seq.seq {
if b.state == batchEndSequence {
expiredCount++
}
}
require.Equal(t, true, expiredCount >= 2, "expected at least 2 expired batches")
}
func TestProgressiveAdvancement_SlotAdvancesInBursts(t *testing.T) {
// Large jump in slots causes multiple batches to expire at once
dn := newDynamicNeeds(100, 600)
seq := newBatchSequencer(6, 500, 50, dn.get)
// Initialize batches: [450,500), [400,450), [350,400), [300,350), [250,300), [200,250)
for i := range 6 {
seq.seq[i] = batch{
begin: primitives.Slot(500 - (i+1)*50),
end: primitives.Slot(500 - i*50),
state: batchInit,
}
}
// Large jump - expire 4 batches at once
dn.setBlockBegin(400)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
// Should have 2 non-expired batches returned
nonExpired := 0
for _, b := range got {
if b.state == batchSequenced {
nonExpired++
}
}
require.Equal(t, 2, nonExpired)
}
func TestProgressiveAdvancement_WorkerProcessingDuringAdvancement(t *testing.T) {
// Batches in various processing states while needs advance
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(4, 400, 50, dn.get)
seq.seq[0] = batch{begin: 350, end: 400, state: batchSyncBlobs}
seq.seq[1] = batch{begin: 300, end: 350, state: batchSyncColumns}
seq.seq[2] = batch{begin: 250, end: 300, state: batchSequenced}
seq.seq[3] = batch{begin: 200, end: 250, state: batchInit}
// Advance past last batch
dn.setBlockBegin(250)
seq.batcher.currentNeeds = dn.get
// Call sequence - only batchInit should transition
got, err := seq.sequence()
require.NoError(t, err)
// batchInit batch should have expired
require.Equal(t, batchEndSequence, seq.seq[3].state)
// Batches in other states should not be returned by sequence (already dispatched)
for _, b := range got {
require.NotEqual(t, batchSyncBlobs, b.state)
require.NotEqual(t, batchSyncColumns, b.state)
}
}
func TestProgressiveAdvancement_CompleteBeforeExpiration(t *testing.T) {
// Batch completes just before it would expire
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(2, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchSequenced}
seq.seq[1] = batch{begin: 200, end: 250, state: batchImportable}
// Complete the second batch BEFORE advancing needs
completed := batch{begin: 200, end: 250, state: batchImportComplete, seq: 1}
seq.update(completed)
// Now advance needs past where the batch was
dn.setBlockBegin(250)
seq.batcher.currentNeeds = dn.get
// The completed batch should have been removed successfully
// Sequence should work normally
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, true, len(got) >= 1, "expected at least 1 batch")
}
// ============================================================================
// Category 4: Batch State Transitions Under Expiration
// ============================================================================
func TestStateExpiration_NilBatchNotExpired(t *testing.T) {
// batchNil should be initialized, not expired
dn := newDynamicNeeds(200, 500)
seq := newBatchSequencer(2, 300, 50, dn.get)
// Leave seq[0] as batchNil (zero value)
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
got, err := seq.sequence()
require.NoError(t, err)
// batchNil should have been initialized and sequenced
foundSequenced := false
for _, b := range got {
if b.state == batchSequenced {
foundSequenced = true
}
}
require.Equal(t, true, foundSequenced, "expected at least one sequenced batch")
}
func TestStateExpiration_InitBatchExpires(t *testing.T) {
// batchInit batches expire when outside retention
dn := newDynamicNeeds(200, 500)
seq := newBatchSequencer(1, 250, 50, dn.get)
seq.seq[0] = batch{begin: 150, end: 200, state: batchInit}
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchEndSequence, got[0].state)
}
func TestStateExpiration_SequencedBatchNotCheckedBySequence(t *testing.T) {
// batchSequenced batches are not returned by sequence() (already dispatched)
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(2, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchSequenced}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
// Move retention past second batch
dn.setBlockBegin(250)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
// Init batch should expire, sequenced batch not returned
for _, b := range got {
require.NotEqual(t, batchSequenced, b.state)
}
}
func TestStateExpiration_SyncBlobsBatchNotCheckedBySequence(t *testing.T) {
// batchSyncBlobs not returned by sequence
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(1, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchSyncBlobs}
_, err := seq.sequence()
require.ErrorIs(t, err, errMaxBatches) // No batch to return
}
func TestStateExpiration_SyncColumnsBatchNotCheckedBySequence(t *testing.T) {
// batchSyncColumns not returned by sequence
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(1, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchSyncColumns}
_, err := seq.sequence()
require.ErrorIs(t, err, errMaxBatches)
}
func TestStateExpiration_ImportableBatchNotCheckedBySequence(t *testing.T) {
// batchImportable not returned by sequence
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(1, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchImportable}
_, err := seq.sequence()
require.ErrorIs(t, err, errMaxBatches)
}
func TestStateExpiration_RetryableBatchExpires(t *testing.T) {
// batchErrRetryable batches can expire
dn := newDynamicNeeds(200, 500)
seq := newBatchSequencer(1, 250, 50, dn.get)
seq.seq[0] = batch{begin: 150, end: 200, state: batchErrRetryable}
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
require.Equal(t, batchEndSequence, got[0].state)
}
// ============================================================================
// Category 5: Edge Cases and Boundaries
// ============================================================================
func TestEdgeCase_NeedsSpanShrinks(t *testing.T) {
// Unusual case: retention window becomes smaller
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 400, 50, dn.get)
seq.seq[0] = batch{begin: 350, end: 400, state: batchInit}
seq.seq[1] = batch{begin: 300, end: 350, state: batchInit}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
// Shrink window from both ends
dn.blockBegin = 300
dn.blockEnd = 400
seq.batcher.currentNeeds = dn.get
_, err := seq.sequence()
require.NoError(t, err)
// Third batch should have expired
require.Equal(t, batchEndSequence, seq.seq[2].state)
}
func TestEdgeCase_EmptySequenceAfterExpiration(t *testing.T) {
// All batches in non-schedulable states, none can be sequenced
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(2, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchImportable}
seq.seq[1] = batch{begin: 200, end: 250, state: batchImportable}
// No batchInit or batchErrRetryable to sequence
_, err := seq.sequence()
require.ErrorIs(t, err, errMaxBatches)
}
func TestEdgeCase_EndSequenceChainReaction(t *testing.T) {
// When batches expire, subsequent calls should handle them correctly
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
// Expire all
dn.setBlockBegin(300)
seq.batcher.currentNeeds = dn.get
got1, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got1))
require.Equal(t, batchEndSequence, got1[0].state)
// Calling sequence again should still return batchEndSequence
got2, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got2))
require.Equal(t, batchEndSequence, got2[0].state)
}
func TestEdgeCase_MixedExpirationAndCompletion(t *testing.T) {
// Some batches complete while others expire simultaneously
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(4, 400, 50, dn.get)
seq.seq[0] = batch{begin: 350, end: 400, state: batchImportComplete}
seq.seq[1] = batch{begin: 300, end: 350, state: batchImportable}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[3] = batch{begin: 200, end: 250, state: batchInit}
// Expire last two batches
dn.setBlockBegin(300)
seq.batcher.currentNeeds = dn.get
// Update with completed batch to trigger processing
completed := batch{begin: 350, end: 400, state: batchImportComplete, seq: 1}
seq.update(completed)
// Verify expired batches are marked
expiredCount := 0
for _, b := range seq.seq {
if b.state == batchEndSequence {
expiredCount++
}
}
require.Equal(t, true, expiredCount >= 2, "expected at least 2 expired batches")
}
func TestEdgeCase_BatchExpiresAtSlotZero(t *testing.T) {
// Edge case with very low slot numbers
dn := newDynamicNeeds(50, 200)
seq := newBatchSequencer(2, 100, 50, dn.get)
seq.seq[0] = batch{begin: 50, end: 100, state: batchInit}
seq.seq[1] = batch{begin: 0, end: 50, state: batchInit}
// Move past first batch
dn.setBlockBegin(100)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
// Both batches should have expired
for _, b := range got {
require.Equal(t, batchEndSequence, b.state)
}
}
// ============================================================================
// Category 6: Integration with numTodo/remaining
// ============================================================================
func TestNumTodo_AfterExpiration(t *testing.T) {
// numTodo should correctly reflect expired batches
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchSequenced}
seq.seq[1] = batch{begin: 200, end: 250, state: batchSequenced}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
todoBefore := seq.numTodo()
// Expire last batch
dn.setBlockBegin(200)
seq.batcher.currentNeeds = dn.get
// Force expiration via sequence
_, err := seq.sequence()
require.NoError(t, err)
todoAfter := seq.numTodo()
// Todo count should have decreased
require.Equal(t, true, todoAfter < todoBefore, "expected todo count to decrease after expiration")
}
func TestRemaining_AfterNeedsChange(t *testing.T) {
// batcher.remaining() should use updated needs
dn := newDynamicNeeds(100, 500)
b := batcher{currentNeeds: dn.get, size: 50}
remainingBefore := b.remaining(300)
// Move retention window
dn.setBlockBegin(250)
b.currentNeeds = dn.get
remainingAfter := b.remaining(300)
// Remaining should have decreased
require.Equal(t, true, remainingAfter < remainingBefore, "expected remaining to decrease after needs change")
}
func TestCountWithState_AfterExpiration(t *testing.T) {
// State counts should be accurate after expiration
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
require.Equal(t, 3, seq.countWithState(batchInit))
require.Equal(t, 0, seq.countWithState(batchEndSequence))
// Expire all batches
dn.setBlockBegin(300)
seq.batcher.currentNeeds = dn.get
_, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 0, seq.countWithState(batchInit))
require.Equal(t, 3, seq.countWithState(batchEndSequence))
}
// ============================================================================
// Category 7: Fork Transition Scenarios (Blob/Column specific)
// ============================================================================
func TestForkTransition_BlobNeedsChange(t *testing.T) {
// Test when blob retention is different from block retention
dn := newDynamicNeeds(100, 500)
// Set blob begin to be further ahead
dn.blobBegin = 200
seq := newBatchSequencer(3, 300, 50, dn.get)
seq.seq[0] = batch{begin: 250, end: 300, state: batchInit}
seq.seq[1] = batch{begin: 200, end: 250, state: batchInit}
seq.seq[2] = batch{begin: 150, end: 200, state: batchInit}
// Sequence should work based on block needs
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 3, len(got))
}
func TestForkTransition_ColumnNeedsChange(t *testing.T) {
// Test when column retention is different from block retention
dn := newDynamicNeeds(100, 500)
// Set column begin to be further ahead
dn.colBegin = 300
seq := newBatchSequencer(3, 400, 50, dn.get)
seq.seq[0] = batch{begin: 350, end: 400, state: batchInit}
seq.seq[1] = batch{begin: 300, end: 350, state: batchInit}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
// Batch expiration is based on block needs, not column needs
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 3, len(got))
}
func TestForkTransition_BlockNeedsVsBlobNeeds(t *testing.T) {
// Blocks still needed but blobs have shorter retention
dn := newDynamicNeeds(100, 500)
dn.blobBegin = 300 // Blobs only needed from slot 300
dn.blobEnd = 500
seq := newBatchSequencer(3, 400, 50, dn.get)
seq.seq[0] = batch{begin: 350, end: 400, state: batchInit}
seq.seq[1] = batch{begin: 300, end: 350, state: batchInit}
seq.seq[2] = batch{begin: 250, end: 300, state: batchInit}
// All batches should be returned (block expiration, not blob)
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 3, len(got))
// Now change block needs to match blob needs
dn.blockBegin = 300
seq.batcher.currentNeeds = dn.get
// Re-sequence - last batch should expire
seq.seq[0].state = batchInit
seq.seq[1].state = batchInit
seq.seq[2].state = batchInit
got2, err := seq.sequence()
require.NoError(t, err)
// Should have 2 non-expired batches
nonExpired := 0
for _, b := range got2 {
if b.state == batchSequenced {
nonExpired++
}
}
require.Equal(t, 2, nonExpired)
}
func TestForkTransition_AllResourceTypesAdvance(t *testing.T) {
// Block, blob, and column spans all advance together
dn := newDynamicNeeds(100, 500)
seq := newBatchSequencer(4, 400, 50, dn.get)
// Batches: [350,400), [300,350), [250,300), [200,250)
for i := range 4 {
seq.seq[i] = batch{
begin: primitives.Slot(400 - (i+1)*50),
end: primitives.Slot(400 - i*50),
state: batchInit,
}
}
// Advance all needs together by 200 slots
// blockBegin moves from 100 to 300
dn.advance(200)
seq.batcher.currentNeeds = dn.get
got, err := seq.sequence()
require.NoError(t, err)
// Count non-expired
nonExpired := 0
for _, b := range got {
if b.state == batchSequenced {
nonExpired++
}
}
// With begin=300, batches [200,250) and [250,300) should have expired
// Batches [350,400) and [300,350) remain valid
require.Equal(t, 2, nonExpired)
}

View File

@@ -4,6 +4,7 @@ import (
"fmt"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
@@ -17,7 +18,7 @@ func TestBatcherBefore(t *testing.T) {
}{
{
name: "size 10",
b: batcher{min: 0, size: 10},
b: batcher{currentNeeds: mockCurrentNeedsFunc(0, 100), size: 10},
upTo: []primitives.Slot{33, 30, 10, 6},
expect: []batch{
{begin: 23, end: 33, state: batchInit},
@@ -28,7 +29,7 @@ func TestBatcherBefore(t *testing.T) {
},
{
name: "size 4",
b: batcher{min: 0, size: 4},
b: batcher{currentNeeds: mockCurrentNeedsFunc(0, 100), size: 4},
upTo: []primitives.Slot{33, 6, 4},
expect: []batch{
{begin: 29, end: 33, state: batchInit},
@@ -38,7 +39,7 @@ func TestBatcherBefore(t *testing.T) {
},
{
name: "trigger end",
b: batcher{min: 20, size: 10},
b: batcher{currentNeeds: mockCurrentNeedsFunc(20, 100), size: 10},
upTo: []primitives.Slot{33, 30, 25, 21, 20, 19},
expect: []batch{
{begin: 23, end: 33, state: batchInit},
@@ -71,7 +72,7 @@ func TestBatchSingleItem(t *testing.T) {
min = 0
max = 11235
size = 64
seq := newBatchSequencer(seqLen, min, max, size)
seq := newBatchSequencer(seqLen, max, size, mockCurrentNeedsFunc(min, max+1))
got, err := seq.sequence()
require.NoError(t, err)
require.Equal(t, 1, len(got))
@@ -99,7 +100,7 @@ func TestBatchSequencer(t *testing.T) {
min = 0
max = 11235
size = 64
seq := newBatchSequencer(seqLen, min, max, size)
seq := newBatchSequencer(seqLen, max, size, mockCurrentNeedsFunc(min, max+1))
expected := []batch{
{begin: 11171, end: 11235},
{begin: 11107, end: 11171},
@@ -212,7 +213,10 @@ func TestBatchSequencer(t *testing.T) {
// 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
seq.currentNeeds = func() das.CurrentNeeds {
return das.CurrentNeeds{Block: das.NeedSpan{Begin: newMin, End: seq.batcher.max}}
}
seq.batcher.currentNeeds = seq.currentNeeds
first = seq.seq[0]
first.state = batchImportComplete
// update() with a complete state will cause the sequence to be extended with an additional batch
@@ -235,3 +239,863 @@ func TestBatchSequencer(t *testing.T) {
//require.ErrorIs(t, err, errEndSequence)
require.Equal(t, batchEndSequence, end.state)
}
// initializeBatchWithSlots sets the begin and end slot values for a batch
// in descending order (slot positions decrease as index increases)
func initializeBatchWithSlots(batches []batch, min primitives.Slot, size primitives.Slot) {
for i := range batches {
// Batches are ordered descending by slot: earliest batches have lower indices
// so batch[0] covers highest slots, batch[N] covers lowest slots
end := min + primitives.Slot((len(batches)-i)*int(size))
begin := end - size
batches[i].begin = begin
batches[i].end = end
}
}
// TestSequence tests the sequence() method with various batch states
func TestSequence(t *testing.T) {
testCases := []struct {
name string
seqLen int
min primitives.Slot
max primitives.Slot
size primitives.Slot
initialStates []batchState
expectedCount int
expectedErr error
stateTransform func([]batch) // optional: transform states before test
}{
{
name: "EmptySequence",
seqLen: 0,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{},
expectedCount: 0,
expectedErr: errMaxBatches,
},
{
name: "SingleBatchInit",
seqLen: 1,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{batchInit},
expectedCount: 1,
},
{
name: "SingleBatchErrRetryable",
seqLen: 1,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{batchErrRetryable},
expectedCount: 1,
},
{
name: "MultipleBatchesInit",
seqLen: 3,
min: 100,
max: 1000,
size: 200,
initialStates: []batchState{batchInit, batchInit, batchInit},
expectedCount: 3,
},
{
name: "MixedStates_InitAndSequenced",
seqLen: 2,
min: 100,
max: 1000,
size: 100,
initialStates: []batchState{batchInit, batchSequenced},
expectedCount: 1,
},
{
name: "MixedStates_SequencedFirst",
seqLen: 2,
min: 100,
max: 1000,
size: 100,
initialStates: []batchState{batchSequenced, batchInit},
expectedCount: 1,
},
{
name: "AllBatchesSequenced",
seqLen: 3,
min: 100,
max: 1000,
size: 200,
initialStates: []batchState{batchSequenced, batchSequenced, batchSequenced},
expectedCount: 0,
expectedErr: errMaxBatches,
},
{
name: "EndSequenceOnly",
seqLen: 1,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{batchEndSequence},
expectedCount: 1,
},
{
name: "EndSequenceWithOthers",
seqLen: 2,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{batchInit, batchEndSequence},
expectedCount: 1,
},
{
name: "ImportableNotSequenced",
seqLen: 1,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{batchImportable},
expectedCount: 0,
expectedErr: errMaxBatches,
},
{
name: "ImportCompleteNotSequenced",
seqLen: 1,
min: 100,
max: 1000,
size: 64,
initialStates: []batchState{batchImportComplete},
expectedCount: 0,
expectedErr: errMaxBatches,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seq := newBatchSequencer(tc.seqLen, tc.max, tc.size, mockCurrentNeedsFunc(tc.min, tc.max+1))
// Initialize batches with valid slot ranges
initializeBatchWithSlots(seq.seq, tc.min, tc.size)
// Set initial states
for i, state := range tc.initialStates {
seq.seq[i].state = state
}
// Apply any transformations
if tc.stateTransform != nil {
tc.stateTransform(seq.seq)
}
got, err := seq.sequence()
if tc.expectedErr != nil {
require.ErrorIs(t, err, tc.expectedErr)
} else {
require.NoError(t, err)
}
require.Equal(t, tc.expectedCount, len(got))
// Verify returned batches are in batchSequenced state
for _, b := range got {
if b.state != batchEndSequence {
require.Equal(t, batchSequenced, b.state)
}
}
})
}
}
// TestUpdate tests the update() method which: (1) updates batch state, (2) removes batchImportComplete batches,
// (3) shifts remaining batches down, and (4) adds new batches to fill vacated positions.
// NOTE: The sequence length can change! Completed batches are removed and new ones are added.
func TestUpdate(t *testing.T) {
testCases := []struct {
name string
seqLen int
batches []batchState
updateIdx int
newState batchState
expectedLen int // expected length after update
expected []batchState // expected states of first N batches after update
}{
{
name: "SingleBatchUpdate",
seqLen: 1,
batches: []batchState{batchInit},
updateIdx: 0,
newState: batchImportable,
expectedLen: 1,
expected: []batchState{batchImportable},
},
{
name: "RemoveFirstCompleted_ShiftOthers",
seqLen: 3,
batches: []batchState{batchImportComplete, batchInit, batchInit},
updateIdx: 0,
newState: batchImportComplete,
expectedLen: 3, // 1 removed + 2 new added
expected: []batchState{batchInit, batchInit}, // shifted down
},
{
name: "RemoveMultipleCompleted",
seqLen: 3,
batches: []batchState{batchImportComplete, batchImportComplete, batchInit},
updateIdx: 0,
newState: batchImportComplete,
expectedLen: 3, // 2 removed + 2 new added
expected: []batchState{batchInit}, // only 1 non-complete batch
},
{
name: "RemoveMiddleCompleted_AlsoShifts",
seqLen: 3,
batches: []batchState{batchInit, batchImportComplete, batchInit},
updateIdx: 1,
newState: batchImportComplete,
expectedLen: 3, // 1 removed + 1 new added
expected: []batchState{batchInit, batchInit}, // middle complete removed, last shifted to middle
},
{
name: "SingleBatchComplete_Replaced",
seqLen: 1,
batches: []batchState{batchInit},
updateIdx: 0,
newState: batchImportComplete,
expectedLen: 1, // special case: replaced with new batch
expected: []batchState{batchInit}, // new batch from beforeBatch
},
{
name: "UpdateNonMatchingBatch",
seqLen: 2,
batches: []batchState{batchInit, batchInit},
updateIdx: 0,
newState: batchImportable,
expectedLen: 2,
expected: []batchState{batchImportable, batchInit},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seq := newBatchSequencer(tc.seqLen, 1000, 64, mockCurrentNeedsFunc(0, 1000+1))
// Initialize batches with proper slot ranges
for i := range seq.seq {
seq.seq[i] = batch{
begin: primitives.Slot(1000 - (i+1)*64),
end: primitives.Slot(1000 - i*64),
state: tc.batches[i],
}
}
// Create batch to update (must match begin/end to be replaced)
updateBatch := seq.seq[tc.updateIdx]
updateBatch.state = tc.newState
seq.update(updateBatch)
// Verify expected length
if len(seq.seq) != tc.expectedLen {
t.Fatalf("expected length %d, got %d", tc.expectedLen, len(seq.seq))
}
// Verify expected states of first N batches
for i, expectedState := range tc.expected {
if i >= len(seq.seq) {
t.Fatalf("expected state at index %d but seq only has %d batches", i, len(seq.seq))
}
if seq.seq[i].state != expectedState {
t.Fatalf("batch[%d]: expected state %s, got %s", i, expectedState.String(), seq.seq[i].state.String())
}
}
// Verify slot contiguity for non-newly-generated batches
// (newly generated batches from beforeBatch() may not be contiguous with shifted batches)
// For this test, we just verify they're in valid slot ranges
for i := 0; i < len(seq.seq); i++ {
if seq.seq[i].begin >= seq.seq[i].end {
t.Fatalf("invalid batch[%d]: begin=%d should be < end=%d", i, seq.seq[i].begin, seq.seq[i].end)
}
}
})
}
}
// TestImportable tests the importable() method for contiguity checking
func TestImportable(t *testing.T) {
testCases := []struct {
name string
seqLen int
states []batchState
expectedCount int
expectedBreak int // index where importable chain breaks (-1 if none)
}{
{
name: "EmptySequence",
seqLen: 0,
states: []batchState{},
expectedCount: 0,
expectedBreak: -1,
},
{
name: "FirstBatchNotImportable",
seqLen: 2,
states: []batchState{batchInit, batchImportable},
expectedCount: 0,
expectedBreak: 0,
},
{
name: "FirstBatchImportable",
seqLen: 1,
states: []batchState{batchImportable},
expectedCount: 1,
expectedBreak: -1,
},
{
name: "TwoImportableConsecutive",
seqLen: 2,
states: []batchState{batchImportable, batchImportable},
expectedCount: 2,
expectedBreak: -1,
},
{
name: "ThreeImportableConsecutive",
seqLen: 3,
states: []batchState{batchImportable, batchImportable, batchImportable},
expectedCount: 3,
expectedBreak: -1,
},
{
name: "ImportsBreak_SecondNotImportable",
seqLen: 2,
states: []batchState{batchImportable, batchInit},
expectedCount: 1,
expectedBreak: 1,
},
{
name: "ImportsBreak_MiddleNotImportable",
seqLen: 4,
states: []batchState{batchImportable, batchImportable, batchInit, batchImportable},
expectedCount: 2,
expectedBreak: 2,
},
{
name: "EndSequenceAfterImportable",
seqLen: 3,
states: []batchState{batchImportable, batchImportable, batchEndSequence},
expectedCount: 2,
expectedBreak: 2,
},
{
name: "AllStatesNotImportable",
seqLen: 3,
states: []batchState{batchInit, batchSequenced, batchErrRetryable},
expectedCount: 0,
expectedBreak: 0,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seq := newBatchSequencer(tc.seqLen, 1000, 64, mockCurrentNeedsFunc(0, 1000+1))
for i, state := range tc.states {
seq.seq[i] = batch{
begin: primitives.Slot(1000 - (i+1)*64),
end: primitives.Slot(1000 - i*64),
state: state,
}
}
imp := seq.importable()
require.Equal(t, tc.expectedCount, len(imp))
})
}
}
// TestMoveMinimumWithNonImportableUpdate tests integration of moveMinimum with update()
func TestMoveMinimumWithNonImportableUpdate(t *testing.T) {
t.Run("UpdateBatchAfterMinimumChange", func(t *testing.T) {
seq := newBatchSequencer(3, 300, 50, mockCurrentNeedsFunc(100, 300+1))
// Initialize with batches
seq.seq[0] = batch{begin: 200, end: 250, state: batchInit}
seq.seq[1] = batch{begin: 150, end: 200, state: batchInit}
seq.seq[2] = batch{begin: 100, end: 150, state: batchInit}
seq.currentNeeds = mockCurrentNeedsFunc(150, 300+1)
seq.batcher.currentNeeds = seq.currentNeeds
// Update non-importable batch above new minimum
batchToUpdate := batch{begin: 200, end: 250, state: batchSequenced}
seq.update(batchToUpdate)
// Verify batch was updated
require.Equal(t, batchSequenced, seq.seq[0].state)
// Verify numTodo reflects updated minimum
todo := seq.numTodo()
require.NotEqual(t, 0, todo, "numTodo should be greater than 0 after moveMinimum and update")
})
}
// TestCountWithState tests state counting
func TestCountWithState(t *testing.T) {
testCases := []struct {
name string
seqLen int
states []batchState
queryState batchState
expectedCount int
}{
{
name: "CountInit_NoBatches",
seqLen: 0,
states: []batchState{},
queryState: batchInit,
expectedCount: 0,
},
{
name: "CountInit_OneBatch",
seqLen: 1,
states: []batchState{batchInit},
queryState: batchInit,
expectedCount: 1,
},
{
name: "CountInit_MultipleBatches",
seqLen: 3,
states: []batchState{batchInit, batchInit, batchInit},
queryState: batchInit,
expectedCount: 3,
},
{
name: "CountInit_MixedStates",
seqLen: 3,
states: []batchState{batchInit, batchSequenced, batchInit},
queryState: batchInit,
expectedCount: 2,
},
{
name: "CountSequenced",
seqLen: 3,
states: []batchState{batchInit, batchSequenced, batchImportable},
queryState: batchSequenced,
expectedCount: 1,
},
{
name: "CountImportable",
seqLen: 3,
states: []batchState{batchImportable, batchImportable, batchInit},
queryState: batchImportable,
expectedCount: 2,
},
{
name: "CountComplete",
seqLen: 3,
states: []batchState{batchImportComplete, batchImportComplete, batchInit},
queryState: batchImportComplete,
expectedCount: 2,
},
{
name: "CountEndSequence",
seqLen: 3,
states: []batchState{batchInit, batchEndSequence, batchInit},
queryState: batchEndSequence,
expectedCount: 1,
},
{
name: "CountZero_NonexistentState",
seqLen: 2,
states: []batchState{batchInit, batchInit},
queryState: batchImportable,
expectedCount: 0,
},
{
name: "CountNil",
seqLen: 3,
states: []batchState{batchNil, batchNil, batchInit},
queryState: batchNil,
expectedCount: 2,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seq := newBatchSequencer(tc.seqLen, 1000, 64, mockCurrentNeedsFunc(0, 1000+1))
for i, state := range tc.states {
seq.seq[i].state = state
}
count := seq.countWithState(tc.queryState)
require.Equal(t, tc.expectedCount, count)
})
}
}
// TestNumTodo tests remaining batch count calculation
func TestNumTodo(t *testing.T) {
testCases := []struct {
name string
seqLen int
min primitives.Slot
max primitives.Slot
size primitives.Slot
states []batchState
expectedTodo int
}{
{
name: "EmptySequence",
seqLen: 0,
min: 0,
max: 1000,
size: 64,
states: []batchState{},
expectedTodo: 0,
},
{
name: "SingleBatchComplete",
seqLen: 1,
min: 0,
max: 1000,
size: 64,
states: []batchState{batchImportComplete},
expectedTodo: 0,
},
{
name: "SingleBatchInit",
seqLen: 1,
min: 0,
max: 100,
size: 10,
states: []batchState{batchInit},
expectedTodo: 1,
},
{
name: "AllBatchesIgnored",
seqLen: 3,
min: 0,
max: 1000,
size: 64,
states: []batchState{batchImportComplete, batchImportComplete, batchNil},
expectedTodo: 0,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seq := newBatchSequencer(tc.seqLen, tc.max, tc.size, mockCurrentNeedsFunc(tc.min, tc.max+1))
for i, state := range tc.states {
seq.seq[i] = batch{
begin: primitives.Slot(tc.max - primitives.Slot((i+1)*10)),
end: primitives.Slot(tc.max - primitives.Slot(i*10)),
state: state,
}
}
// Just verify numTodo doesn't panic
_ = seq.numTodo()
})
}
}
// TestBatcherRemaining tests the remaining() calculation logic
func TestBatcherRemaining(t *testing.T) {
testCases := []struct {
name string
min primitives.Slot
upTo primitives.Slot
size primitives.Slot
expected int
}{
{
name: "UpToLessThanMin",
min: 100,
upTo: 50,
size: 10,
expected: 0,
},
{
name: "UpToEqualsMin",
min: 100,
upTo: 100,
size: 10,
expected: 0,
},
{
name: "ExactBoundary",
min: 100,
upTo: 110,
size: 10,
expected: 1,
},
{
name: "ExactBoundary_Multiple",
min: 100,
upTo: 150,
size: 10,
expected: 5,
},
{
name: "PartialBatch",
min: 100,
upTo: 115,
size: 10,
expected: 2,
},
{
name: "PartialBatch_Small",
min: 100,
upTo: 105,
size: 10,
expected: 1,
},
{
name: "LargeRange",
min: 100,
upTo: 500,
size: 10,
expected: 40,
},
{
name: "LargeRange_Partial",
min: 100,
upTo: 505,
size: 10,
expected: 41,
},
{
name: "PartialBatch_Size1",
min: 100,
upTo: 101,
size: 1,
expected: 1,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
needs := func() das.CurrentNeeds {
return das.CurrentNeeds{Block: das.NeedSpan{Begin: tc.min, End: tc.upTo + 1}}
}
b := batcher{size: tc.size, currentNeeds: needs}
result := b.remaining(tc.upTo)
require.Equal(t, tc.expected, result)
})
}
}
// assertAllBatchesAboveMinimum verifies all returned batches have end > minimum
func assertAllBatchesAboveMinimum(t *testing.T, batches []batch, min primitives.Slot) {
for _, b := range batches {
if b.state != batchEndSequence {
if b.end <= min {
t.Fatalf("batch begin=%d end=%d has end <= minimum %d", b.begin, b.end, min)
}
}
}
}
// assertBatchesContiguous verifies contiguity of returned batches
func assertBatchesContiguous(t *testing.T, batches []batch) {
for i := 0; i < len(batches)-1; i++ {
require.Equal(t, batches[i].begin, batches[i+1].end,
"batch[%d] begin=%d not contiguous with batch[%d] end=%d", i, batches[i].begin, i+1, batches[i+1].end)
}
}
// assertBatchNotReturned verifies a specific batch is not in the returned list
func assertBatchNotReturned(t *testing.T, batches []batch, shouldNotBe batch) {
for _, b := range batches {
if b.begin == shouldNotBe.begin && b.end == shouldNotBe.end {
t.Fatalf("batch begin=%d end=%d should not be returned", shouldNotBe.begin, shouldNotBe.end)
}
}
}
// TestMoveMinimumFiltersOutOfRangeBatches tests that batches below new minimum are not returned by sequence()
// after moveMinimum is called. The sequence() method marks expired batches (end <= min) as batchEndSequence
// but does not return them (unless they're the only batches left).
func TestMoveMinimumFiltersOutOfRangeBatches(t *testing.T) {
testCases := []struct {
name string
seqLen int
min primitives.Slot
max primitives.Slot
size primitives.Slot
initialStates []batchState
newMinimum primitives.Slot
expectedReturned int
expectedAllAbove primitives.Slot // all returned batches should have end > this value (except batchEndSequence)
}{
// Category 1: Single Batch Below New Minimum
{
name: "BatchBelowMinimum_Init",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit, batchInit},
newMinimum: 175,
expectedReturned: 3, // [250-300], [200-250], [150-200] are returned
expectedAllAbove: 175,
},
{
name: "BatchBelowMinimum_ErrRetryable",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchSequenced, batchSequenced, batchErrRetryable, batchErrRetryable},
newMinimum: 175,
expectedReturned: 1, // only [150-200] (ErrRetryable) is returned; [100-150] is expired and not returned
expectedAllAbove: 175,
},
// Category 2: Multiple Batches Below New Minimum
{
name: "MultipleBatchesBelowMinimum",
seqLen: 8,
min: 100,
max: 500,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit, batchInit, batchInit, batchInit, batchInit, batchInit},
newMinimum: 320,
expectedReturned: 4, // [450-500], [400-450], [350-400], [300-350] returned; rest expired/not returned
expectedAllAbove: 320,
},
// Category 3: Batches at Boundary - batch.end == minimum is expired
{
name: "BatchExactlyAtMinimum",
seqLen: 3,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit},
newMinimum: 200,
expectedReturned: 1, // [250-300] returned; [200-250] (end==200) and [100-150] are expired
expectedAllAbove: 200,
},
{
name: "BatchJustAboveMinimum",
seqLen: 3,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit},
newMinimum: 199,
expectedReturned: 2, // [250-300], [200-250] returned; [100-150] (end<=199) is expired
expectedAllAbove: 199,
},
// Category 4: No Batches Affected
{
name: "MoveMinimumNoAffect",
seqLen: 3,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit},
newMinimum: 120,
expectedReturned: 3, // all batches returned, none below minimum
expectedAllAbove: 120,
},
// Category 5: Mixed States Below Minimum
{
name: "MixedStatesBelowMinimum",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchSequenced, batchInit, batchErrRetryable, batchInit},
newMinimum: 175,
expectedReturned: 2, // [200-250] (Init) and [150-200] (ErrRetryable) returned; others not in Init/ErrRetryable or expired
expectedAllAbove: 175,
},
// Category 6: Large moveMinimum
{
name: "LargeMoveMinimumSkipsMost",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit, batchInit},
newMinimum: 290,
expectedReturned: 1, // only [250-300] (end=300 > 290) returned
expectedAllAbove: 290,
},
// Category 7: All Batches Expired
{
name: "AllBatchesExpired",
seqLen: 3,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit},
newMinimum: 300,
expectedReturned: 1, // when all expire, one batchEndSequence is returned
expectedAllAbove: 0, // batchEndSequence may have any slot value, don't check
},
// Category 8: Contiguity after filtering
{
name: "ContiguityMaintained",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
initialStates: []batchState{batchInit, batchInit, batchInit, batchInit},
newMinimum: 150,
expectedReturned: 3, // [250-300], [200-250], [150-200] returned
expectedAllAbove: 150,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
seq := newBatchSequencer(tc.seqLen, tc.max, tc.size, mockCurrentNeedsFunc(tc.min, tc.max+1))
// Initialize batches with valid slot ranges
initializeBatchWithSlots(seq.seq, tc.min, tc.size)
// Set initial states
for i, state := range tc.initialStates {
seq.seq[i].state = state
}
// move minimum and call sequence to update set of batches
seq.currentNeeds = mockCurrentNeedsFunc(tc.newMinimum, tc.max+1)
seq.batcher.currentNeeds = seq.currentNeeds
got, err := seq.sequence()
require.NoError(t, err)
// Verify count
if len(got) != tc.expectedReturned {
t.Fatalf("expected %d batches returned, got %d", tc.expectedReturned, len(got))
}
// Verify all returned non-endSequence batches have end > newMinimum
// (batchEndSequence may be returned when all batches are expired, so exclude from check)
if tc.expectedAllAbove > 0 {
for _, b := range got {
if b.state != batchEndSequence && b.end <= tc.expectedAllAbove {
t.Fatalf("batch begin=%d end=%d has end <= %d (should be filtered)",
b.begin, b.end, tc.expectedAllAbove)
}
}
}
// Verify contiguity is maintained for returned batches
if len(got) > 1 {
assertBatchesContiguous(t, got)
}
})
}
}

View File

@@ -12,6 +12,7 @@ import (
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
@@ -30,35 +31,47 @@ type blobSummary struct {
}
type blobSyncConfig struct {
retentionStart primitives.Slot
nbv verification.NewBlobVerifier
store *filesystem.BlobStorage
nbv verification.NewBlobVerifier
store *filesystem.BlobStorage
currentNeeds func() das.CurrentNeeds
}
func newBlobSync(current primitives.Slot, vbs verifiedROBlocks, cfg *blobSyncConfig) (*blobSync, error) {
expected, err := vbs.blobIdents(cfg.retentionStart)
expected, err := vbs.blobIdents(cfg.currentNeeds)
if err != nil {
return nil, err
}
bbv := newBlobBatchVerifier(cfg.nbv)
as := das.NewLazilyPersistentStore(cfg.store, bbv)
shouldRetain := func(slot primitives.Slot) bool {
needs := cfg.currentNeeds()
return needs.Blob.At(slot)
}
as := das.NewLazilyPersistentStore(cfg.store, bbv, shouldRetain)
return &blobSync{current: current, expected: expected, bbv: bbv, store: as}, nil
}
type blobVerifierMap map[[32]byte][]verification.BlobVerifier
type blobSync struct {
store das.AvailabilityStore
store *das.LazilyPersistentStoreBlob
expected []blobSummary
next int
bbv *blobBatchVerifier
current primitives.Slot
peer peer.ID
}
func (bs *blobSync) blobsNeeded() int {
func (bs *blobSync) needed() int {
return len(bs.expected) - bs.next
}
// validateNext is given to the RPC request code as one of the a validation callbacks.
// It orchestrates setting up the batch verifier (blobBatchVerifier) and calls Persist on the
// AvailabilityStore. This enables the rest of the code in between RPC and the AvailabilityStore
// to stay decoupled from each other. The AvailabilityStore holds the blobs in memory between the
// call to Persist, and the call to IsDataAvailable (where the blobs are actually written to disk
// if successfully verified).
func (bs *blobSync) validateNext(rb blocks.ROBlob) error {
if bs.next >= len(bs.expected) {
return errUnexpectedResponseSize
@@ -102,6 +115,7 @@ func newBlobBatchVerifier(nbv verification.NewBlobVerifier) *blobBatchVerifier {
return &blobBatchVerifier{newBlobVerifier: nbv, verifiers: make(blobVerifierMap)}
}
// blobBatchVerifier implements the BlobBatchVerifier interface required by the das store.
type blobBatchVerifier struct {
newBlobVerifier verification.NewBlobVerifier
verifiers blobVerifierMap
@@ -117,6 +131,7 @@ func (bbv *blobBatchVerifier) newVerifier(rb blocks.ROBlob) verification.BlobVer
return m[rb.Index]
}
// VerifiedROBlobs satisfies the BlobBatchVerifier interface expected by the AvailabilityChecker
func (bbv *blobBatchVerifier) VerifiedROBlobs(_ context.Context, blk blocks.ROBlock, _ []blocks.ROBlob) ([]blocks.VerifiedROBlob, error) {
m, ok := bbv.verifiers[blk.Root()]
if !ok {

View File

@@ -3,6 +3,7 @@ package backfill
import (
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/config/params"
@@ -11,28 +12,42 @@ import (
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/OffchainLabs/prysm/v7/time/slots"
)
const testBlobGenBlobCount = 3
func testBlobGen(t *testing.T, start primitives.Slot, n int) ([]blocks.ROBlock, [][]blocks.ROBlob) {
blks := make([]blocks.ROBlock, n)
blobs := make([][]blocks.ROBlob, n)
for i := range n {
bk, bl := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, start+primitives.Slot(i), 3)
bk, bl := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, start+primitives.Slot(i), testBlobGenBlobCount)
blks[i] = bk
blobs[i] = bl
}
return blks, blobs
}
func setupCurrentNeeds(t *testing.T, current primitives.Slot) das.SyncNeeds {
cs := func() primitives.Slot { return current }
sn, err := das.NewSyncNeeds(cs, nil, 0)
require.NoError(t, err)
return sn
}
func TestValidateNext_happy(t *testing.T) {
startSlot := util.SlotAtEpoch(t, params.BeaconConfig().DenebForkEpoch)
current := startSlot + 65
blks, blobs := testBlobGen(t, startSlot, 4)
cfg := &blobSyncConfig{
retentionStart: 0,
nbv: testNewBlobVerifier(),
store: filesystem.NewEphemeralBlobStorage(t),
nbv: testNewBlobVerifier(),
store: filesystem.NewEphemeralBlobStorage(t),
currentNeeds: mockCurrentNeedsFunc(0, current+1),
}
//expected :=
expected, err := verifiedROBlocks(blks).blobIdents(cfg.currentNeeds)
require.NoError(t, err)
require.Equal(t, len(blks)*testBlobGenBlobCount, len(expected))
bsync, err := newBlobSync(current, blks, cfg)
require.NoError(t, err)
nb := 0
@@ -49,26 +64,32 @@ func TestValidateNext_happy(t *testing.T) {
}
func TestValidateNext_cheapErrors(t *testing.T) {
denebSlot, err := slots.EpochStart(params.BeaconConfig().DenebForkEpoch)
require.NoError(t, err)
current := primitives.Slot(128)
blks, blobs := testBlobGen(t, 63, 2)
syncNeeds := setupCurrentNeeds(t, current)
cfg := &blobSyncConfig{
retentionStart: 0,
nbv: testNewBlobVerifier(),
store: filesystem.NewEphemeralBlobStorage(t),
nbv: testNewBlobVerifier(),
store: filesystem.NewEphemeralBlobStorage(t),
currentNeeds: syncNeeds.Currently,
}
blks, blobs := testBlobGen(t, denebSlot, 2)
bsync, err := newBlobSync(current, blks, cfg)
require.NoError(t, err)
require.ErrorIs(t, bsync.validateNext(blobs[len(blobs)-1][0]), errUnexpectedResponseContent)
}
func TestValidateNext_sigMatch(t *testing.T) {
denebSlot, err := slots.EpochStart(params.BeaconConfig().DenebForkEpoch)
require.NoError(t, err)
current := primitives.Slot(128)
blks, blobs := testBlobGen(t, 63, 1)
syncNeeds := setupCurrentNeeds(t, current)
cfg := &blobSyncConfig{
retentionStart: 0,
nbv: testNewBlobVerifier(),
store: filesystem.NewEphemeralBlobStorage(t),
nbv: testNewBlobVerifier(),
store: filesystem.NewEphemeralBlobStorage(t),
currentNeeds: syncNeeds.Currently,
}
blks, blobs := testBlobGen(t, denebSlot, 1)
bsync, err := newBlobSync(current, blks, cfg)
require.NoError(t, err)
blobs[0][0].SignedBlockHeader.Signature = bytesutil.PadTo([]byte("derp"), 48)
@@ -79,6 +100,8 @@ func TestValidateNext_errorsFromVerifier(t *testing.T) {
ds := util.SlotAtEpoch(t, params.BeaconConfig().DenebForkEpoch)
current := primitives.Slot(ds + 96)
blks, blobs := testBlobGen(t, ds+31, 1)
cn := mockCurrentNeedsFunc(0, current+1)
cases := []struct {
name string
err error
@@ -109,9 +132,9 @@ func TestValidateNext_errorsFromVerifier(t *testing.T) {
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
cfg := &blobSyncConfig{
retentionStart: 0,
nbv: testNewBlobVerifier(c.cb),
store: filesystem.NewEphemeralBlobStorage(t),
nbv: testNewBlobVerifier(c.cb),
store: filesystem.NewEphemeralBlobStorage(t),
currentNeeds: cn,
}
bsync, err := newBlobSync(current, blks, cfg)
require.NoError(t, err)

View File

@@ -0,0 +1,282 @@
package backfill
import (
"bytes"
"context"
"fmt"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
var (
errInvalidDataColumnResponse = errors.New("invalid DataColumnSidecar response")
errUnexpectedBlockRoot = errors.Wrap(errInvalidDataColumnResponse, "unexpected sidecar block root")
errCommitmentLengthMismatch = errors.Wrap(errInvalidDataColumnResponse, "sidecar has different commitment count than block")
errCommitmentValueMismatch = errors.Wrap(errInvalidDataColumnResponse, "sidecar commitments do not match block")
)
// tune the amount of columns we try to download from peers at once.
// The spec limit is 128 * 32, but connection errors are more likely when
// requesting so much at once.
const columnRequestLimit = 128 * 4
type columnBatch struct {
first primitives.Slot
last primitives.Slot
custodyGroups peerdas.ColumnIndices
toDownload map[[32]byte]*toDownload
}
type toDownload struct {
remaining peerdas.ColumnIndices
commitments [][]byte
slot primitives.Slot
}
func (cs *columnBatch) needed() peerdas.ColumnIndices {
// make a copy that we can modify to reduce search iterations.
search := cs.custodyGroups.ToMap()
ci := peerdas.ColumnIndices{}
for _, v := range cs.toDownload {
if len(search) == 0 {
return ci
}
for col := range search {
if v.remaining.Has(col) {
ci.Set(col)
// avoid iterating every single block+index by only searching for indices
// we haven't found yet.
delete(search, col)
}
}
}
return ci
}
// pruneExpired removes any columns from the batch that are no longer needed.
// If `pruned` is non-nil, it is populated with the roots that were removed.
func (cs *columnBatch) pruneExpired(needs das.CurrentNeeds, pruned map[[32]byte]struct{}) {
for root, td := range cs.toDownload {
if !needs.Col.At(td.slot) {
delete(cs.toDownload, root)
if pruned != nil {
pruned[root] = struct{}{}
}
}
}
}
// neededSidecarCount returns the total number of sidecars still needed to complete the batch.
func (cs *columnBatch) neededSidecarCount() int {
count := 0
for _, v := range cs.toDownload {
count += v.remaining.Count()
}
return count
}
// neededSidecarsByColumn counts how many sidecars are still needed for each column index.
func (cs *columnBatch) neededSidecarsByColumn(peerHas peerdas.ColumnIndices) map[uint64]int {
need := make(map[uint64]int, len(peerHas))
for _, v := range cs.toDownload {
for idx := range v.remaining {
if peerHas.Has(idx) {
need[idx]++
}
}
}
return need
}
type columnSync struct {
*columnBatch
store *das.LazilyPersistentStoreColumn
current primitives.Slot
peer peer.ID
bisector *columnBisector
}
func newColumnSync(ctx context.Context, b batch, blks verifiedROBlocks, current primitives.Slot, p p2p.P2P, cfg *workerCfg) (*columnSync, error) {
cgc, err := p.CustodyGroupCount(ctx)
if err != nil {
return nil, errors.Wrap(err, "custody group count")
}
cb, err := buildColumnBatch(ctx, b, blks, p, cfg.colStore, cfg.currentNeeds())
if err != nil {
return nil, err
}
if cb == nil {
return &columnSync{}, nil
}
shouldRetain := func(sl primitives.Slot) bool {
needs := cfg.currentNeeds()
return needs.Col.At(sl)
}
bisector := newColumnBisector(cfg.downscore)
return &columnSync{
columnBatch: cb,
current: current,
store: das.NewLazilyPersistentStoreColumn(cfg.colStore, cfg.newVC, p.NodeID(), cgc, bisector, shouldRetain),
bisector: bisector,
}, nil
}
func (cs *columnSync) blockColumns(root [32]byte) *toDownload {
if cs.columnBatch == nil {
return nil
}
return cs.columnBatch.toDownload[root]
}
func (cs *columnSync) columnsNeeded() peerdas.ColumnIndices {
if cs.columnBatch == nil {
return peerdas.ColumnIndices{}
}
return cs.columnBatch.needed()
}
func (cs *columnSync) request(reqCols []uint64, limit int) (*ethpb.DataColumnSidecarsByRangeRequest, error) {
if len(reqCols) == 0 {
return nil, nil
}
// Use cheaper check to avoid allocating map and counting sidecars if under limit.
if cs.neededSidecarCount() <= limit {
return sync.DataColumnSidecarsByRangeRequest(reqCols, cs.first, cs.last)
}
// Re-slice b.nextReqCols to keep the number of requested sidecars under the limit.
reqCount := 0
peerHas := peerdas.NewColumnIndicesFromSlice(reqCols)
needed := cs.neededSidecarsByColumn(peerHas)
for i := range reqCols {
addSidecars := needed[reqCols[i]]
if reqCount+addSidecars > columnRequestLimit {
reqCols = reqCols[:i]
break
}
reqCount += addSidecars
}
return sync.DataColumnSidecarsByRangeRequest(reqCols, cs.first, cs.last)
}
type validatingColumnRequest struct {
req *ethpb.DataColumnSidecarsByRangeRequest
columnSync *columnSync
bisector *columnBisector
}
func (v *validatingColumnRequest) validate(cd blocks.RODataColumn) (err error) {
defer func(validity string, start time.Time) {
dataColumnSidecarVerifyMs.Observe(float64(time.Since(start).Milliseconds()))
if err != nil {
validity = "invalid"
}
dataColumnSidecarDownloadCount.WithLabelValues(fmt.Sprintf("%d", cd.Index), validity).Inc()
dataColumnSidecarDownloadBytes.Add(float64(cd.SizeSSZ()))
}("valid", time.Now())
return v.countedValidation(cd)
}
// When we call Persist we'll get the verification checks that are provided by the availability store.
// In addition to those checks this function calls rpcValidity which maintains a state machine across
// response values to ensure that the response is valid in the context of the overall request,
// like making sure that the block roots is one of the ones we expect based on the blocks we used to
// construct the request. It also does cheap sanity checks on the DataColumnSidecar values like
// ensuring that the commitments line up with the block.
func (v *validatingColumnRequest) countedValidation(cd blocks.RODataColumn) error {
root := cd.BlockRoot()
expected := v.columnSync.blockColumns(root)
if expected == nil {
return errors.Wrapf(errUnexpectedBlockRoot, "root=%#x, slot=%d", root, cd.Slot())
}
// We don't need this column, but we trust the column state machine verified we asked for it as part of a range request.
// So we can just skip over it and not try to persist it.
if !expected.remaining.Has(cd.Index) {
return nil
}
if len(cd.KzgCommitments) != len(expected.commitments) {
return errors.Wrapf(errCommitmentLengthMismatch, "root=%#x, slot=%d, index=%d", root, cd.Slot(), cd.Index)
}
for i, cmt := range cd.KzgCommitments {
if !bytes.Equal(cmt, expected.commitments[i]) {
return errors.Wrapf(errCommitmentValueMismatch, "root=%#x, slot=%d, index=%d", root, cd.Slot(), cd.Index)
}
}
if err := v.columnSync.store.Persist(v.columnSync.current, cd); err != nil {
return errors.Wrap(err, "persisting data column")
}
v.bisector.addPeerColumns(v.columnSync.peer, cd)
expected.remaining.Unset(cd.Index)
return nil
}
func currentCustodiedColumns(ctx context.Context, p p2p.P2P) (peerdas.ColumnIndices, error) {
cgc, err := p.CustodyGroupCount(ctx)
if err != nil {
return nil, errors.Wrap(err, "custody group count")
}
peerInfo, _, err := peerdas.Info(p.NodeID(), cgc)
if err != nil {
return nil, errors.Wrap(err, "peer info")
}
return peerdas.NewColumnIndicesFromMap(peerInfo.CustodyColumns), nil
}
func buildColumnBatch(ctx context.Context, b batch, blks verifiedROBlocks, p p2p.P2P, store *filesystem.DataColumnStorage, needs das.CurrentNeeds) (*columnBatch, error) {
if len(blks) == 0 {
return nil, nil
}
if !needs.Col.At(b.begin) && !needs.Col.At(b.end-1) {
return nil, nil
}
indices, err := currentCustodiedColumns(ctx, p)
if err != nil {
return nil, errors.Wrap(err, "current custodied columns")
}
summary := &columnBatch{
custodyGroups: indices,
toDownload: make(map[[32]byte]*toDownload, len(blks)),
}
for _, b := range blks {
slot := b.Block().Slot()
if !needs.Col.At(slot) {
continue
}
cmts, err := b.Block().Body().BlobKzgCommitments()
if err != nil {
return nil, errors.Wrap(err, "failed to get blob kzg commitments")
}
if len(cmts) == 0 {
continue
}
// The last block this part of the loop sees will be the last one
// we need to download data columns for.
if len(summary.toDownload) == 0 {
// toDownload is only empty the first time through, so this is the first block with data columns.
summary.first = slot
}
summary.last = slot
summary.toDownload[b.Root()] = &toDownload{
remaining: das.IndicesNotStored(store.Summary(b.Root()), indices),
commitments: cmts,
slot: slot,
}
}
return summary, nil
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,9 @@
package backfill
import "github.com/pkg/errors"
var errUnrecoverable = errors.New("service in unrecoverable state")
func isRetryable(err error) bool {
return !errors.Is(err, errUnrecoverable)
}

View File

@@ -0,0 +1,130 @@
package backfill
import (
"context"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/pkg/errors"
)
var errMissingAvailabilityChecker = errors.Wrap(errUnrecoverable, "batch is missing required availability checker")
var errUnsafeRange = errors.Wrap(errUnrecoverable, "invalid slice indices")
type checkMultiplexer struct {
blobCheck das.AvailabilityChecker
colCheck das.AvailabilityChecker
currentNeeds das.CurrentNeeds
}
// Persist implements das.AvailabilityStore.
var _ das.AvailabilityChecker = &checkMultiplexer{}
// newCheckMultiplexer initializes an AvailabilityChecker that multiplexes to the BlobSidecar and DataColumnSidecar
// AvailabilityCheckers present in the batch.
func newCheckMultiplexer(needs das.CurrentNeeds, b batch) *checkMultiplexer {
s := &checkMultiplexer{currentNeeds: needs}
if b.blobs != nil && b.blobs.store != nil {
s.blobCheck = b.blobs.store
}
if b.columns != nil && b.columns.store != nil {
s.colCheck = b.columns.store
}
return s
}
// IsDataAvailable implements the das.AvailabilityStore interface.
func (m *checkMultiplexer) IsDataAvailable(ctx context.Context, current primitives.Slot, blks ...blocks.ROBlock) error {
needs, err := m.divideByChecker(blks)
if err != nil {
return errors.Wrap(errUnrecoverable, "failed to slice blocks by DA type")
}
if err := doAvailabilityCheck(ctx, m.blobCheck, current, needs.blobs); err != nil {
return errors.Wrap(err, "blob store availability check failed")
}
if err := doAvailabilityCheck(ctx, m.colCheck, current, needs.cols); err != nil {
return errors.Wrap(err, "column store availability check failed")
}
return nil
}
func doAvailabilityCheck(ctx context.Context, check das.AvailabilityChecker, current primitives.Slot, blks []blocks.ROBlock) error {
if len(blks) == 0 {
return nil
}
// Double check that the checker is non-nil.
if check == nil {
return errMissingAvailabilityChecker
}
return check.IsDataAvailable(ctx, current, blks...)
}
// daGroups is a helper type that groups blocks by their DA type.
type daGroups struct {
blobs []blocks.ROBlock
cols []blocks.ROBlock
}
// blocksByDaType slices the given blocks into two slices: one for deneb blocks (BlobSidecar)
// and one for fulu blocks (DataColumnSidecar). Blocks that are pre-deneb or have no
// blob commitments are skipped.
func (m *checkMultiplexer) divideByChecker(blks []blocks.ROBlock) (daGroups, error) {
needs := daGroups{}
for _, blk := range blks {
slot := blk.Block().Slot()
if !m.currentNeeds.Blob.At(slot) && !m.currentNeeds.Col.At(slot) {
continue
}
cmts, err := blk.Block().Body().BlobKzgCommitments()
if err != nil {
return needs, err
}
if len(cmts) == 0 {
continue
}
if m.currentNeeds.Col.At(slot) {
needs.cols = append(needs.cols, blk)
continue
}
if m.currentNeeds.Blob.At(slot) {
needs.blobs = append(needs.blobs, blk)
continue
}
}
return needs, nil
}
// safeRange is a helper type that enforces safe slicing.
type safeRange struct {
start uint
end uint
}
// isZero returns true if the range is zero-length.
func (r safeRange) isZero() bool {
return r.start == r.end
}
// subSlice returns the subslice of s defined by sub
// if it can be safely sliced, or an error if the range is invalid
// with respect to the slice.
func subSlice[T any](s []T, sub safeRange) ([]T, error) {
slen := uint(len(s))
if slen == 0 || sub.isZero() {
return nil, nil
}
// Check that minimum bound is safe.
if sub.end < sub.start {
return nil, errUnsafeRange
}
// Check that upper bound is safe.
if sub.start >= slen || sub.end > slen {
return nil, errUnsafeRange
}
return s[sub.start:sub.end], nil
}

View File

@@ -0,0 +1,822 @@
package backfill
import (
"context"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
)
type mockChecker struct {
}
var mockAvailabilityFailure = errors.New("fake error from IsDataAvailable")
var mockColumnFailure = errors.Wrap(mockAvailabilityFailure, "column checker failure")
var mockBlobFailure = errors.Wrap(mockAvailabilityFailure, "blob checker failure")
// trackingAvailabilityChecker wraps a das.AvailabilityChecker and tracks calls
type trackingAvailabilityChecker struct {
checker das.AvailabilityChecker
callCount int
blocksSeenPerCall [][]blocks.ROBlock // Track blocks passed in each call
}
// NewTrackingAvailabilityChecker creates a wrapper that tracks calls to the underlying checker
func NewTrackingAvailabilityChecker(checker das.AvailabilityChecker) *trackingAvailabilityChecker {
return &trackingAvailabilityChecker{
checker: checker,
callCount: 0,
blocksSeenPerCall: [][]blocks.ROBlock{},
}
}
// IsDataAvailable implements das.AvailabilityChecker
func (t *trackingAvailabilityChecker) IsDataAvailable(ctx context.Context, current primitives.Slot, blks ...blocks.ROBlock) error {
t.callCount++
// Track a copy of the blocks passed in this call
blocksCopy := make([]blocks.ROBlock, len(blks))
copy(blocksCopy, blks)
t.blocksSeenPerCall = append(t.blocksSeenPerCall, blocksCopy)
// Delegate to the underlying checker
return t.checker.IsDataAvailable(ctx, current, blks...)
}
// GetCallCount returns how many times IsDataAvailable was called
func (t *trackingAvailabilityChecker) GetCallCount() int {
return t.callCount
}
// GetBlocksInCall returns the blocks passed in a specific call (0-indexed)
func (t *trackingAvailabilityChecker) GetBlocksInCall(callIndex int) []blocks.ROBlock {
if callIndex < 0 || callIndex >= len(t.blocksSeenPerCall) {
return nil
}
return t.blocksSeenPerCall[callIndex]
}
// GetTotalBlocksSeen returns total number of blocks seen across all calls
func (t *trackingAvailabilityChecker) GetTotalBlocksSeen() int {
total := 0
for _, blkSlice := range t.blocksSeenPerCall {
total += len(blkSlice)
}
return total
}
func TestNewCheckMultiplexer(t *testing.T) {
denebSlot, fuluSlot := testDenebAndFuluSlots(t)
cases := []struct {
name string
batch func() batch
setupChecker func(*checkMultiplexer)
current primitives.Slot
err error
}{
{
name: "no availability checkers, no blocks",
batch: func() batch { return batch{} },
},
{
name: "no blob availability checkers, deneb blocks",
batch: func() batch {
blks, _ := testBlobGen(t, denebSlot, 2)
return batch{
blocks: blks,
}
},
setupChecker: func(m *checkMultiplexer) {
// Provide a column checker which should be unused in this test.
m.colCheck = &das.MockAvailabilityStore{}
},
err: errMissingAvailabilityChecker,
},
{
name: "no column availability checker, fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot, 2)
return batch{
blocks: blks,
}
},
err: errMissingAvailabilityChecker,
setupChecker: func(m *checkMultiplexer) {
// Provide a blob checker which should be unused in this test.
m.blobCheck = &das.MockAvailabilityStore{}
},
},
{
name: "has column availability checker, fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot, 2)
return batch{
blocks: blks,
}
},
setupChecker: func(m *checkMultiplexer) {
// Provide a blob checker which should be unused in this test.
m.colCheck = &das.MockAvailabilityStore{}
},
},
{
name: "has blob availability checker, deneb blocks",
batch: func() batch {
blks, _ := testBlobGen(t, denebSlot, 2)
return batch{
blocks: blks,
}
},
setupChecker: func(m *checkMultiplexer) {
// Provide a blob checker which should be unused in this test.
m.blobCheck = &das.MockAvailabilityStore{}
},
},
{
name: "has blob but not col availability checker, deneb and fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot-2, 4) // spans deneb and fulu
return batch{
blocks: blks,
}
},
err: errMissingAvailabilityChecker, // fails because column store is not present
setupChecker: func(m *checkMultiplexer) {
m.blobCheck = &das.MockAvailabilityStore{}
},
},
{
name: "has col but not blob availability checker, deneb and fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot-2, 4) // spans deneb and fulu
return batch{
blocks: blks,
}
},
err: errMissingAvailabilityChecker, // fails because column store is not present
setupChecker: func(m *checkMultiplexer) {
m.colCheck = &das.MockAvailabilityStore{}
},
},
{
name: "both checkers, deneb and fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot-2, 4) // spans deneb and fulu
return batch{
blocks: blks,
}
},
setupChecker: func(m *checkMultiplexer) {
m.blobCheck = &das.MockAvailabilityStore{}
m.colCheck = &das.MockAvailabilityStore{}
},
},
{
name: "deneb checker fails, deneb and fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot-2, 4) // spans deneb and fulu
return batch{
blocks: blks,
}
},
err: mockBlobFailure,
setupChecker: func(m *checkMultiplexer) {
m.blobCheck = &das.MockAvailabilityStore{ErrIsDataAvailable: mockBlobFailure}
m.colCheck = &das.MockAvailabilityStore{}
},
},
{
name: "fulu checker fails, deneb and fulu blocks",
batch: func() batch {
blks, _ := testBlobGen(t, fuluSlot-2, 4) // spans deneb and fulu
return batch{
blocks: blks,
}
},
err: mockBlobFailure,
setupChecker: func(m *checkMultiplexer) {
m.blobCheck = &das.MockAvailabilityStore{}
m.colCheck = &das.MockAvailabilityStore{ErrIsDataAvailable: mockBlobFailure}
},
},
}
needs := mockCurrentSpecNeeds()
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
b := tc.batch()
var checker *checkMultiplexer
checker = newCheckMultiplexer(needs, b)
if tc.setupChecker != nil {
tc.setupChecker(checker)
}
err := checker.IsDataAvailable(t.Context(), tc.current, b.blocks...)
if tc.err != nil {
require.ErrorIs(t, err, tc.err)
} else {
require.NoError(t, err)
}
})
}
}
func testBlocksWithCommitments(t *testing.T, startSlot primitives.Slot, count int) []blocks.ROBlock {
blks := make([]blocks.ROBlock, count)
for i := range count {
blk, _ := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, startSlot+primitives.Slot(i), 1)
blks[i] = blk
}
return blks
}
func TestDaNeeds(t *testing.T) {
denebSlot, fuluSlot := testDenebAndFuluSlots(t)
mux := &checkMultiplexer{currentNeeds: mockCurrentSpecNeeds()}
cases := []struct {
name string
setup func() (daGroups, []blocks.ROBlock)
expect daGroups
err error
}{
{
name: "empty range",
setup: func() (daGroups, []blocks.ROBlock) {
return daGroups{}, testBlocksWithCommitments(t, 10, 5)
},
},
{
name: "single deneb block",
setup: func() (daGroups, []blocks.ROBlock) {
blks := testBlocksWithCommitments(t, denebSlot, 1)
return daGroups{
blobs: []blocks.ROBlock{blks[0]},
}, blks
},
},
{
name: "single fulu block",
setup: func() (daGroups, []blocks.ROBlock) {
blks := testBlocksWithCommitments(t, fuluSlot, 1)
return daGroups{
cols: []blocks.ROBlock{blks[0]},
}, blks
},
},
{
name: "deneb range",
setup: func() (daGroups, []blocks.ROBlock) {
blks := testBlocksWithCommitments(t, denebSlot, 3)
return daGroups{
blobs: blks,
}, blks
},
},
{
name: "one deneb one fulu",
setup: func() (daGroups, []blocks.ROBlock) {
deneb := testBlocksWithCommitments(t, denebSlot, 1)
fulu := testBlocksWithCommitments(t, fuluSlot, 1)
return daGroups{
blobs: []blocks.ROBlock{deneb[0]},
cols: []blocks.ROBlock{fulu[0]},
}, append(deneb, fulu...)
},
},
{
name: "deneb and fulu range",
setup: func() (daGroups, []blocks.ROBlock) {
deneb := testBlocksWithCommitments(t, denebSlot, 3)
fulu := testBlocksWithCommitments(t, fuluSlot, 3)
return daGroups{
blobs: deneb,
cols: fulu,
}, append(deneb, fulu...)
},
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
expectNeeds, blks := tc.setup()
needs, err := mux.divideByChecker(blks)
if tc.err != nil {
require.ErrorIs(t, err, tc.err)
} else {
require.NoError(t, err)
}
expectBlob := make(map[[32]byte]struct{})
for _, blk := range expectNeeds.blobs {
expectBlob[blk.Root()] = struct{}{}
}
for _, blk := range needs.blobs {
_, ok := expectBlob[blk.Root()]
require.Equal(t, true, ok, "unexpected blob block root %#x", blk.Root())
delete(expectBlob, blk.Root())
}
require.Equal(t, 0, len(expectBlob), "missing blob blocks")
expectCol := make(map[[32]byte]struct{})
for _, blk := range expectNeeds.cols {
expectCol[blk.Root()] = struct{}{}
}
for _, blk := range needs.cols {
_, ok := expectCol[blk.Root()]
require.Equal(t, true, ok, "unexpected col block root %#x", blk.Root())
delete(expectCol, blk.Root())
}
require.Equal(t, 0, len(expectCol), "missing col blocks")
})
}
}
func testDenebAndFuluSlots(t *testing.T) (primitives.Slot, primitives.Slot) {
params.SetupTestConfigCleanup(t)
denebEpoch := params.BeaconConfig().DenebForkEpoch
if params.BeaconConfig().FuluForkEpoch == params.BeaconConfig().FarFutureEpoch {
params.BeaconConfig().FuluForkEpoch = denebEpoch + 4096*2
}
fuluEpoch := params.BeaconConfig().FuluForkEpoch
fuluSlot, err := slots.EpochStart(fuluEpoch)
require.NoError(t, err)
denebSlot, err := slots.EpochStart(denebEpoch)
require.NoError(t, err)
return denebSlot, fuluSlot
}
// Helper to create test blocks without blob commitments
// Uses 0 commitments instead of 1 like testBlocksWithCommitments
func testBlocksWithoutCommitments(t *testing.T, startSlot primitives.Slot, count int) []blocks.ROBlock {
blks := make([]blocks.ROBlock, count)
for i := range count {
blk, _ := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, startSlot+primitives.Slot(i), 0)
blks[i] = blk
}
return blks
}
// TestBlockDaNeedsWithoutCommitments verifies blocks without commitments are skipped
func TestBlockDaNeedsWithoutCommitments(t *testing.T) {
denebSlot, fuluSlot := testDenebAndFuluSlots(t)
mux := &checkMultiplexer{currentNeeds: mockCurrentSpecNeeds()}
cases := []struct {
name string
setup func() (daGroups, []blocks.ROBlock)
expect daGroups
err error
}{
{
name: "deneb blocks without commitments",
setup: func() (daGroups, []blocks.ROBlock) {
blks := testBlocksWithoutCommitments(t, denebSlot, 3)
return daGroups{}, blks // Expect empty daNeeds
},
},
{
name: "fulu blocks without commitments",
setup: func() (daGroups, []blocks.ROBlock) {
blks := testBlocksWithoutCommitments(t, fuluSlot, 3)
return daGroups{}, blks // Expect empty daNeeds
},
},
{
name: "mixed: some deneb with commitments, some without",
setup: func() (daGroups, []blocks.ROBlock) {
withCommit := testBlocksWithCommitments(t, denebSlot, 2)
withoutCommit := testBlocksWithoutCommitments(t, denebSlot+2, 2)
blks := append(withCommit, withoutCommit...)
return daGroups{
blobs: withCommit, // Only the ones with commitments
}, blks
},
},
{
name: "pre-deneb blocks are skipped",
setup: func() (daGroups, []blocks.ROBlock) {
blks := testBlocksWithCommitments(t, denebSlot-10, 5)
return daGroups{}, blks // All pre-deneb, expect empty
},
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
expectNeeds, blks := tc.setup()
needs, err := mux.divideByChecker(blks)
if tc.err != nil {
require.ErrorIs(t, err, tc.err)
} else {
require.NoError(t, err)
}
// Verify blob blocks
require.Equal(t, len(expectNeeds.blobs), len(needs.blobs),
"expected %d blob blocks, got %d", len(expectNeeds.blobs), len(needs.blobs))
// Verify col blocks
require.Equal(t, len(expectNeeds.cols), len(needs.cols),
"expected %d col blocks, got %d", len(expectNeeds.cols), len(needs.cols))
})
}
}
// TestBlockDaNeedsAcrossEras verifies blocks spanning pre-deneb/deneb/fulu boundaries
func TestBlockDaNeedsAcrossEras(t *testing.T) {
denebSlot, fuluSlot := testDenebAndFuluSlots(t)
mux := &checkMultiplexer{currentNeeds: mockCurrentSpecNeeds()}
cases := []struct {
name string
setup func() (daGroups, []blocks.ROBlock)
expectBlobCount int
expectColCount int
}{
{
name: "pre-deneb, deneb, fulu sequence",
setup: func() (daGroups, []blocks.ROBlock) {
preDeneb := testBlocksWithCommitments(t, denebSlot-1, 1)
deneb := testBlocksWithCommitments(t, denebSlot, 2)
fulu := testBlocksWithCommitments(t, fuluSlot, 2)
blks := append(preDeneb, append(deneb, fulu...)...)
return daGroups{}, blks
},
expectBlobCount: 2, // Only deneb blocks
expectColCount: 2, // Only fulu blocks
},
{
name: "blocks at exact deneb boundary",
setup: func() (daGroups, []blocks.ROBlock) {
atBoundary := testBlocksWithCommitments(t, denebSlot, 1)
return daGroups{
blobs: atBoundary,
}, atBoundary
},
expectBlobCount: 1,
expectColCount: 0,
},
{
name: "blocks at exact fulu boundary",
setup: func() (daGroups, []blocks.ROBlock) {
atBoundary := testBlocksWithCommitments(t, fuluSlot, 1)
return daGroups{
cols: atBoundary,
}, atBoundary
},
expectBlobCount: 0,
expectColCount: 1,
},
{
name: "many deneb blocks before fulu transition",
setup: func() (daGroups, []blocks.ROBlock) {
deneb := testBlocksWithCommitments(t, denebSlot, 10)
fulu := testBlocksWithCommitments(t, fuluSlot, 5)
blks := append(deneb, fulu...)
return daGroups{}, blks
},
expectBlobCount: 10,
expectColCount: 5,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
_, blks := tc.setup()
needs, err := mux.divideByChecker(blks)
require.NoError(t, err)
require.Equal(t, tc.expectBlobCount, len(needs.blobs),
"expected %d blob blocks, got %d", tc.expectBlobCount, len(needs.blobs))
require.Equal(t, tc.expectColCount, len(needs.cols),
"expected %d col blocks, got %d", tc.expectColCount, len(needs.cols))
})
}
}
// TestDoAvailabilityCheckEdgeCases verifies edge cases in doAvailabilityCheck
func TestDoAvailabilityCheckEdgeCases(t *testing.T) {
denebSlot, _ := testDenebAndFuluSlots(t)
checkerErr := errors.New("checker error")
cases := []struct {
name string
checker das.AvailabilityChecker
blocks []blocks.ROBlock
expectErr error
setupTestBlocks func() []blocks.ROBlock
}{
{
name: "nil checker with empty blocks",
checker: nil,
blocks: []blocks.ROBlock{},
expectErr: nil, // Should succeed with no blocks
},
{
name: "nil checker with blocks",
checker: nil,
expectErr: errMissingAvailabilityChecker,
setupTestBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, denebSlot, 1)
},
},
{
name: "valid checker with empty blocks",
checker: &das.MockAvailabilityStore{},
blocks: []blocks.ROBlock{},
expectErr: nil,
},
{
name: "valid checker with blocks succeeds",
checker: &das.MockAvailabilityStore{},
expectErr: nil,
setupTestBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, denebSlot, 3)
},
},
{
name: "valid checker error is propagated",
checker: &das.MockAvailabilityStore{ErrIsDataAvailable: checkerErr},
expectErr: checkerErr,
setupTestBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, denebSlot, 1)
},
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
blks := tc.blocks
if tc.setupTestBlocks != nil {
blks = tc.setupTestBlocks()
}
err := doAvailabilityCheck(t.Context(), tc.checker, denebSlot, blks)
if tc.expectErr != nil {
require.NotNil(t, err)
require.ErrorIs(t, err, tc.expectErr)
} else {
require.NoError(t, err)
}
})
}
}
// TestBlockDaNeedsErrorWrapping verifies error messages are properly wrapped
func TestBlockDaNeedsErrorWrapping(t *testing.T) {
denebSlot, _ := testDenebAndFuluSlots(t)
mux := &checkMultiplexer{currentNeeds: mockCurrentSpecNeeds()}
// Test with a block that has commitments but in deneb range
blks := testBlocksWithCommitments(t, denebSlot, 2)
// This should succeed without errors
needs, err := mux.divideByChecker(blks)
require.NoError(t, err)
require.Equal(t, 2, len(needs.blobs))
require.Equal(t, 0, len(needs.cols))
}
// TestIsDataAvailableCallRouting verifies that blocks are routed to the correct checker
// based on their era (pre-deneb, deneb, fulu) and tests various block combinations
func TestIsDataAvailableCallRouting(t *testing.T) {
denebSlot, fuluSlot := testDenebAndFuluSlots(t)
cases := []struct {
name string
buildBlocks func() []blocks.ROBlock
expectedBlobCalls int
expectedBlobBlocks int
expectedColCalls int
expectedColBlocks int
}{
{
name: "PreDenebOnly",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, denebSlot-10, 3)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "DenebOnly",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, denebSlot, 3)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 3,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "FuluOnly",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, fuluSlot, 3)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 1,
expectedColBlocks: 3,
},
{
name: "PreDeneb_Deneb_Mix",
buildBlocks: func() []blocks.ROBlock {
preDeneb := testBlocksWithCommitments(t, denebSlot-10, 3)
deneb := testBlocksWithCommitments(t, denebSlot, 3)
return append(preDeneb, deneb...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 3,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "PreDeneb_Fulu_Mix",
buildBlocks: func() []blocks.ROBlock {
preDeneb := testBlocksWithCommitments(t, denebSlot-10, 3)
fulu := testBlocksWithCommitments(t, fuluSlot, 3)
return append(preDeneb, fulu...)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 1,
expectedColBlocks: 3,
},
{
name: "Deneb_Fulu_Mix",
buildBlocks: func() []blocks.ROBlock {
deneb := testBlocksWithCommitments(t, denebSlot, 3)
fulu := testBlocksWithCommitments(t, fuluSlot, 3)
return append(deneb, fulu...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 3,
expectedColCalls: 1,
expectedColBlocks: 3,
},
{
name: "PreDeneb_Deneb_Fulu_Mix",
buildBlocks: func() []blocks.ROBlock {
preDeneb := testBlocksWithCommitments(t, denebSlot-10, 3)
deneb := testBlocksWithCommitments(t, denebSlot, 4)
fulu := testBlocksWithCommitments(t, fuluSlot, 3)
return append(preDeneb, append(deneb, fulu...)...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 4,
expectedColCalls: 1,
expectedColBlocks: 3,
},
{
name: "DenebNoCommitments",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithoutCommitments(t, denebSlot, 3)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "FuluNoCommitments",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithoutCommitments(t, fuluSlot, 3)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "MixedCommitments_Deneb",
buildBlocks: func() []blocks.ROBlock {
with := testBlocksWithCommitments(t, denebSlot, 3)
without := testBlocksWithoutCommitments(t, denebSlot+3, 3)
return append(with, without...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 3,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "MixedCommitments_Fulu",
buildBlocks: func() []blocks.ROBlock {
with := testBlocksWithCommitments(t, fuluSlot, 3)
without := testBlocksWithoutCommitments(t, fuluSlot+3, 3)
return append(with, without...)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 1,
expectedColBlocks: 3,
},
{
name: "MixedCommitments_All",
buildBlocks: func() []blocks.ROBlock {
denebWith := testBlocksWithCommitments(t, denebSlot, 3)
denebWithout := testBlocksWithoutCommitments(t, denebSlot+3, 2)
fuluWith := testBlocksWithCommitments(t, fuluSlot, 3)
fuluWithout := testBlocksWithoutCommitments(t, fuluSlot+3, 2)
return append(denebWith, append(denebWithout, append(fuluWith, fuluWithout...)...)...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 3,
expectedColCalls: 1,
expectedColBlocks: 3,
},
{
name: "EmptyBlocks",
buildBlocks: func() []blocks.ROBlock {
return []blocks.ROBlock{}
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "SingleDeneb",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, denebSlot, 1)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 1,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "SingleFulu",
buildBlocks: func() []blocks.ROBlock {
return testBlocksWithCommitments(t, fuluSlot, 1)
},
expectedBlobCalls: 0,
expectedBlobBlocks: 0,
expectedColCalls: 1,
expectedColBlocks: 1,
},
{
name: "DenebAtBoundary",
buildBlocks: func() []blocks.ROBlock {
preDeneb := testBlocksWithCommitments(t, denebSlot-1, 1)
atBoundary := testBlocksWithCommitments(t, denebSlot, 1)
return append(preDeneb, atBoundary...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 1,
expectedColCalls: 0,
expectedColBlocks: 0,
},
{
name: "FuluAtBoundary",
buildBlocks: func() []blocks.ROBlock {
deneb := testBlocksWithCommitments(t, fuluSlot-1, 1)
atBoundary := testBlocksWithCommitments(t, fuluSlot, 1)
return append(deneb, atBoundary...)
},
expectedBlobCalls: 1,
expectedBlobBlocks: 1,
expectedColCalls: 1,
expectedColBlocks: 1,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
// Create tracking wrappers around mock checkers
blobTracker := NewTrackingAvailabilityChecker(&das.MockAvailabilityStore{})
colTracker := NewTrackingAvailabilityChecker(&das.MockAvailabilityStore{})
// Create multiplexer with tracked checkers
mux := &checkMultiplexer{
blobCheck: blobTracker,
colCheck: colTracker,
currentNeeds: mockCurrentSpecNeeds(),
}
// Build blocks and run availability check
blocks := tc.buildBlocks()
err := mux.IsDataAvailable(t.Context(), denebSlot, blocks...)
require.NoError(t, err)
// Assert blob checker was called the expected number of times
require.Equal(t, tc.expectedBlobCalls, blobTracker.GetCallCount(),
"blob checker call count mismatch for test %s", tc.name)
// Assert blob checker saw the expected number of blocks
require.Equal(t, tc.expectedBlobBlocks, blobTracker.GetTotalBlocksSeen(),
"blob checker block count mismatch for test %s", tc.name)
// Assert column checker was called the expected number of times
require.Equal(t, tc.expectedColCalls, colTracker.GetCallCount(),
"column checker call count mismatch for test %s", tc.name)
// Assert column checker saw the expected number of blocks
require.Equal(t, tc.expectedColBlocks, colTracker.GetTotalBlocksSeen(),
"column checker block count mismatch for test %s", tc.name)
})
}
}

View File

@@ -1,5 +1,115 @@
package backfill
import "github.com/sirupsen/logrus"
import (
"sync"
"sync/atomic"
"time"
"github.com/sirupsen/logrus"
)
var log = logrus.WithField("prefix", "backfill")
// intervalLogger only logs once for each interval. It only customizes a single
// instance of the entry/logger and should just be used to control the logging rate for
// *one specific line of code*.
type intervalLogger struct {
*logrus.Entry
base *logrus.Entry
mux sync.Mutex
seconds int64 // seconds is the number of seconds per logging interval
last *atomic.Int64 // last is the quantized representation of the last time a log was emitted
now func() time.Time
}
func newIntervalLogger(base *logrus.Entry, secondsBetweenLogs int64) *intervalLogger {
return &intervalLogger{
Entry: base,
base: base,
seconds: secondsBetweenLogs,
last: new(atomic.Int64),
now: time.Now,
}
}
// intervalNumber is a separate pure function because this helps tests determine
// proposer timestamp alignment.
func intervalNumber(t time.Time, seconds int64) int64 {
return t.Unix() / seconds
}
// intervalNumber is the integer division of the current unix timestamp
// divided by the number of seconds per interval.
func (l *intervalLogger) intervalNumber() int64 {
return intervalNumber(l.now(), l.seconds)
}
func (l *intervalLogger) copy() *intervalLogger {
return &intervalLogger{
Entry: l.Entry,
base: l.base,
seconds: l.seconds,
last: l.last,
now: l.now,
}
}
// Log overloads the Log() method of logrus.Entry, which is called under the hood
// when a log-level specific method (like Info(), Warn(), Error()) is invoked.
// By intercepting this call we can rate limit how often we log.
func (l *intervalLogger) Log(level logrus.Level, args ...any) {
n := l.intervalNumber()
// If Swap returns a different value that the current interval number, we haven't
// emitted a log yet this interval, so we can do so now.
if l.last.Swap(n) != n {
l.Entry.Log(level, args...)
}
// reset the Entry to the base so that any WithField/WithError calls
// don't persist across calls to Log()
}
func (l *intervalLogger) WithField(key string, value any) *intervalLogger {
cp := l.copy()
cp.Entry = cp.Entry.WithField(key, value)
return cp
}
func (l *intervalLogger) WithFields(fields logrus.Fields) *intervalLogger {
cp := l.copy()
cp.Entry = cp.Entry.WithFields(fields)
return cp
}
func (l *intervalLogger) WithError(err error) *intervalLogger {
cp := l.copy()
cp.Entry = cp.Entry.WithError(err)
return cp
}
func (l *intervalLogger) Trace(args ...any) {
l.Log(logrus.TraceLevel, args...)
}
func (l *intervalLogger) Debug(args ...any) {
l.Log(logrus.DebugLevel, args...)
}
func (l *intervalLogger) Print(args ...any) {
l.Info(args...)
}
func (l *intervalLogger) Info(args ...any) {
l.Log(logrus.InfoLevel, args...)
}
func (l *intervalLogger) Warn(args ...any) {
l.Log(logrus.WarnLevel, args...)
}
func (l *intervalLogger) Warning(args ...any) {
l.Warn(args...)
}
func (l *intervalLogger) Error(args ...any) {
l.Log(logrus.ErrorLevel, args...)
}

View File

@@ -0,0 +1,379 @@
package backfill
import (
"bytes"
"sync"
"testing"
"time"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
"github.com/stretchr/testify/require"
)
// trackingHook is a logrus hook that counts Log callCount for testing.
type trackingHook struct {
mu sync.RWMutex
entries []*logrus.Entry
}
func (h *trackingHook) Levels() []logrus.Level {
return logrus.AllLevels
}
func (h *trackingHook) Fire(entry *logrus.Entry) error {
h.mu.Lock()
defer h.mu.Unlock()
h.entries = append(h.entries, entry)
return nil
}
func (h *trackingHook) callCount() int {
h.mu.RLock()
defer h.mu.RUnlock()
return len(h.entries)
}
func (h *trackingHook) emitted(t *testing.T) []string {
h.mu.RLock()
defer h.mu.RUnlock()
e := make([]string, len(h.entries))
for i, entry := range h.entries {
entry.Buffer = new(bytes.Buffer)
serialized, err := entry.Logger.Formatter.Format(entry)
require.NoError(t, err)
e[i] = string(serialized)
}
return e
}
func entryWithHook() (*logrus.Entry, *trackingHook) {
logger := logrus.New()
logger.SetLevel(logrus.TraceLevel)
hook := &trackingHook{}
logger.AddHook(hook)
entry := logrus.NewEntry(logger)
return entry, hook
}
func intervalSecondsAndDuration(i int) (int64, time.Duration) {
return int64(i), time.Duration(i) * time.Second
}
// mockClock provides a controllable time source for testing.
// It allows tests to set the current time and advance it as needed.
type mockClock struct {
t time.Time
}
// now returns the current time.
func (c *mockClock) now() time.Time {
return c.t
}
func setupMockClock(il *intervalLogger) *mockClock {
// initialize now so that the time aligns with the start of the
// interval bucket. This ensures that adding less than an interval
// of time to the timestamp can never move into the next bucket.
interval := intervalNumber(time.Now(), il.seconds)
now := time.Unix(interval*il.seconds, 0)
clock := &mockClock{t: now}
il.now = clock.now
return clock
}
// TestNewIntervalLogger verifies logger is properly initialized
func TestNewIntervalLogger(t *testing.T) {
base := logrus.NewEntry(logrus.New())
intSec := int64(10)
il := newIntervalLogger(base, intSec)
require.NotNil(t, il)
require.Equal(t, intSec, il.seconds)
require.Equal(t, int64(0), il.last.Load())
require.Equal(t, base, il.Entry)
}
// TestLogOncePerInterval verifies that Log is called only once within an interval window
func TestLogOncePerInterval(t *testing.T) {
entry, hook := entryWithHook()
il := newIntervalLogger(entry, 10)
_ = setupMockClock(il) // use a fixed time to make sure no race is possible
// First log should call the underlying Log method
il.Log(logrus.InfoLevel, "test message 1")
require.Equal(t, 1, hook.callCount())
// Second log in same interval should not call Log
il.Log(logrus.InfoLevel, "test message 2")
require.Equal(t, 1, hook.callCount())
// Third log still in same interval should not call Log
il.Log(logrus.InfoLevel, "test message 3")
require.Equal(t, 1, hook.callCount())
// Verify last is set to current interval
require.Equal(t, il.intervalNumber(), il.last.Load())
}
// TestLogAcrossIntervalBoundary verifies logging at interval boundaries resets correctly
func TestLogAcrossIntervalBoundary(t *testing.T) {
iSec, iDur := intervalSecondsAndDuration(10)
entry, hook := entryWithHook()
il := newIntervalLogger(entry, iSec)
clock := setupMockClock(il)
il.Log(logrus.InfoLevel, "first interval")
require.Equal(t, 1, hook.callCount())
// Log in new interval should succeed
clock.t = clock.t.Add(2 * iDur)
il.Log(logrus.InfoLevel, "second interval")
require.Equal(t, 2, hook.callCount())
}
// TestWithFieldChaining verifies WithField returns logger and can be chained
func TestWithFieldChaining(t *testing.T) {
entry, hook := entryWithHook()
iSec, iDur := intervalSecondsAndDuration(10)
il := newIntervalLogger(entry, iSec)
clock := setupMockClock(il)
result := il.WithField("key1", "value1")
require.NotNil(t, result)
result.Info("test")
require.Equal(t, 1, hook.callCount())
// make sure there was no mutation of the base as a side effect
clock.t = clock.t.Add(iDur)
il.Info("another")
// Verify field is present in logged entry
emitted := hook.emitted(t)
require.Contains(t, emitted[0], "test")
require.Contains(t, emitted[0], "key1=value1")
require.Contains(t, emitted[1], "another")
require.NotContains(t, emitted[1], "key1=value1")
}
// TestWithFieldsChaining verifies WithFields properly adds multiple fields
func TestWithFieldsChaining(t *testing.T) {
entry, hook := entryWithHook()
iSec, iDur := intervalSecondsAndDuration(10)
il := newIntervalLogger(entry, iSec)
clock := setupMockClock(il)
fields := logrus.Fields{
"key1": "value1",
"key2": "value2",
}
result := il.WithFields(fields)
require.NotNil(t, result)
result.Info("test")
require.Equal(t, 1, hook.callCount())
// make sure there was no mutation of the base as a side effect
clock.t = clock.t.Add(iDur)
il.Info("another")
// Verify field is present in logged entry
emitted := hook.emitted(t)
require.Contains(t, emitted[0], "test")
require.Contains(t, emitted[0], "key1=value1")
require.Contains(t, emitted[0], "key2=value2")
require.Contains(t, emitted[1], "another")
require.NotContains(t, emitted[1], "key1=value1")
require.NotContains(t, emitted[1], "key2=value2")
}
// TestWithErrorChaining verifies WithError properly adds error field
func TestWithErrorChaining(t *testing.T) {
entry, hook := entryWithHook()
iSec, iDur := intervalSecondsAndDuration(10)
il := newIntervalLogger(entry, iSec)
clock := setupMockClock(il)
expected := errors.New("lowercase words")
result := il.WithError(expected)
require.NotNil(t, result)
result.Error("test")
require.Equal(t, 1, hook.callCount())
require.NotNil(t, result)
// make sure there was no mutation of the base as a side effect
clock.t = clock.t.Add(iDur)
il.Info("different")
// Verify field is present in logged entry
emitted := hook.emitted(t)
require.Contains(t, emitted[0], expected.Error())
require.Contains(t, emitted[0], "test")
require.Contains(t, emitted[1], "different")
require.NotContains(t, emitted[1], "test")
require.NotContains(t, emitted[1], "lowercase words")
}
// TestLogLevelMethods verifies all log level methods work and respect rate limiting
func TestLogLevelMethods(t *testing.T) {
entry, hook := entryWithHook()
il := newIntervalLogger(entry, 10)
_ = setupMockClock(il) // use a fixed time to make sure no race is possible
// First call from each level-specific method should succeed
il.Trace("trace message")
require.Equal(t, 1, hook.callCount())
// Subsequent callCount in same interval should be suppressed
il.Debug("debug message")
require.Equal(t, 1, hook.callCount())
il.Info("info message")
require.Equal(t, 1, hook.callCount())
il.Print("print message")
require.Equal(t, 1, hook.callCount())
il.Warn("warn message")
require.Equal(t, 1, hook.callCount())
il.Warning("warning message")
require.Equal(t, 1, hook.callCount())
il.Error("error message")
require.Equal(t, 1, hook.callCount())
}
// TestConcurrentLogging verifies multiple goroutines can safely call Log concurrently
func TestConcurrentLogging(t *testing.T) {
entry, hook := entryWithHook()
il := newIntervalLogger(entry, 10)
_ = setupMockClock(il) // use a fixed time to make sure no race is possible
var wg sync.WaitGroup
wait := make(chan struct{})
for range 10 {
wg.Add(1)
go func() {
<-wait
defer wg.Done()
il.Log(logrus.InfoLevel, "concurrent message")
}()
}
close(wait) // maximize raciness by unblocking goroutines together
wg.Wait()
// Only one Log call should succeed across all goroutines in the same interval
require.Equal(t, 1, hook.callCount())
}
// TestZeroInterval verifies behavior with small interval (logs every second)
func TestZeroInterval(t *testing.T) {
entry, hook := entryWithHook()
il := newIntervalLogger(entry, 1)
clock := setupMockClock(il)
il.Log(logrus.InfoLevel, "first")
require.Equal(t, 1, hook.callCount())
// Move to next second
clock.t = clock.t.Add(time.Second)
il.Log(logrus.InfoLevel, "second")
require.Equal(t, 2, hook.callCount())
}
// TestCompleteLoggingFlow tests realistic scenario with repeated logging
func TestCompleteLoggingFlow(t *testing.T) {
entry, hook := entryWithHook()
iSec, iDur := intervalSecondsAndDuration(10)
il := newIntervalLogger(entry, iSec)
clock := setupMockClock(il)
// Add field
il = il.WithField("request_id", "12345")
// Log multiple times in same interval - only first succeeds
il.Info("message 1")
require.Equal(t, 1, hook.callCount())
il.Warn("message 2")
require.Equal(t, 1, hook.callCount())
// Move to next interval
clock.t = clock.t.Add(iDur)
// Should be able to log again in new interval
il.Error("message 3")
require.Equal(t, 2, hook.callCount())
require.NotNil(t, il)
}
// TestAtomicSwapCorrectness verifies atomic swap works correctly
func TestAtomicSwapCorrectness(t *testing.T) {
il := newIntervalLogger(logrus.NewEntry(logrus.New()), 10)
_ = setupMockClock(il) // use a fixed time to make sure no race is possible
// Swap operation should return different value on first call
current := il.intervalNumber()
old := il.last.Swap(current)
require.Equal(t, int64(0), old) // initial value is 0
require.Equal(t, current, il.last.Load())
// Swap with same value should return the same value
old = il.last.Swap(current)
require.Equal(t, current, old)
}
// TestLogMethodsWithClockAdvancement verifies that log methods respect rate limiting
// within an interval but emit again after the interval passes.
func TestLogMethodsWithClockAdvancement(t *testing.T) {
entry, hook := entryWithHook()
iSec, iDur := intervalSecondsAndDuration(10)
il := newIntervalLogger(entry, iSec)
clock := setupMockClock(il)
// First Error call should log
il.Error("error 1")
require.Equal(t, 1, hook.callCount())
// Warn call in same interval should be suppressed
il.Warn("warn 1")
require.Equal(t, 1, hook.callCount())
// Info call in same interval should be suppressed
il.Info("info 1")
require.Equal(t, 1, hook.callCount())
// Debug call in same interval should be suppressed
il.Debug("debug 1")
require.Equal(t, 1, hook.callCount())
// Move forward 5 seconds - still in same 10-second interval
require.Equal(t, 5*time.Second, iDur/2)
clock.t = clock.t.Add(iDur / 2)
il.Error("error 2")
require.Equal(t, 1, hook.callCount(), "should still be suppressed within same interval")
firstInterval := il.intervalNumber()
// Move forward to next interval (10 second interval boundary)
clock.t = clock.t.Add(iDur / 2)
nextInterval := il.intervalNumber()
require.NotEqual(t, firstInterval, nextInterval, "should be in new interval now")
il.Error("error 3")
require.Equal(t, 2, hook.callCount(), "should emit in new interval")
// Another call in the new interval should be suppressed
il.Warn("warn 2")
require.Equal(t, 2, hook.callCount())
// Move forward to yet another interval
clock.t = clock.t.Add(iDur)
il.Info("info 2")
require.Equal(t, 3, hook.callCount(), "should emit in third interval")
}

View File

@@ -21,86 +21,117 @@ var (
Help: "Number of batches that are ready to be imported once they can be connected to the existing chain.",
},
)
backfillRemainingBatches = promauto.NewGauge(
batchesRemaining = promauto.NewGauge(
prometheus.GaugeOpts{
Name: "backfill_remaining_batches",
Help: "Backfill remaining batches.",
},
)
backfillBatchesImported = promauto.NewCounter(
batchesImported = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_batches_imported",
Help: "Number of backfill batches downloaded and imported.",
},
)
backfillBlocksApproximateBytes = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blocks_bytes_downloaded",
Help: "BeaconBlock bytes downloaded from peers for backfill.",
backfillBatchTimeWaiting = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_waiting_ms",
Help: "Time batch waited for a suitable peer in ms.",
Buckets: []float64{50, 100, 300, 1000, 2000},
},
)
backfillBlobsApproximateBytes = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blobs_bytes_downloaded",
Help: "BlobSidecar bytes downloaded from peers for backfill.",
backfillBatchTimeRoundtrip = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_roundtrip_ms",
Help: "Total time to import batch, from first scheduled to imported.",
Buckets: []float64{1000, 2000, 4000, 6000, 10000},
},
)
backfillBlobsDownloadCount = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blobs_download_count",
Help: "Number of BlobSidecar values downloaded from peers for backfill.",
},
)
backfillBlocksDownloadCount = promauto.NewCounter(
blockDownloadCount = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blocks_download_count",
Help: "Number of BeaconBlock values downloaded from peers for backfill.",
},
)
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},
blockDownloadBytesApprox = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blocks_downloaded_bytes",
Help: "BeaconBlock bytes downloaded from peers for backfill.",
},
)
backfillBatchTimeWaiting = promauto.NewHistogram(
blockDownloadMs = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_time_waiting",
Help: "Time batch waited for a suitable peer.",
Buckets: []float64{50, 100, 300, 1000, 2000},
},
)
backfillBatchTimeDownloadingBlocks = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_blocks_time_download",
Help: "Time, in milliseconds, batch spent downloading blocks from peer.",
Name: "backfill_batch_blocks_download_ms",
Help: "BeaconBlock download time, in ms.",
Buckets: []float64{100, 300, 1000, 2000, 4000, 8000},
},
)
backfillBatchTimeDownloadingBlobs = promauto.NewHistogram(
blockVerifyMs = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_blobs_time_download",
Help: "Time, in milliseconds, batch spent downloading blobs from peer.",
Name: "backfill_batch_verify_ms",
Help: "BeaconBlock verification time, in ms.",
Buckets: []float64{100, 300, 1000, 2000, 4000, 8000},
},
)
backfillBatchTimeVerifying = promauto.NewHistogram(
blobSidecarDownloadCount = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blobs_download_count",
Help: "Number of BlobSidecar values downloaded from peers for backfill.",
},
)
blobSidecarDownloadBytesApprox = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_blobs_downloaded_bytes",
Help: "BlobSidecar bytes downloaded from peers for backfill.",
},
)
blobSidecarDownloadMs = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_time_verify",
Help: "Time batch spent downloading blocks from peer.",
Name: "backfill_batch_blobs_download_ms",
Help: "BlobSidecar download time, in ms.",
Buckets: []float64{100, 300, 1000, 2000, 4000, 8000},
},
)
dataColumnSidecarDownloadCount = promauto.NewCounterVec(
prometheus.CounterOpts{
Name: "backfill_data_column_sidecar_downloaded",
Help: "Number of DataColumnSidecar values downloaded from peers for backfill.",
},
[]string{"index", "validity"},
)
dataColumnSidecarDownloadBytes = promauto.NewCounter(
prometheus.CounterOpts{
Name: "backfill_data_column_sidecar_downloaded_bytes",
Help: "DataColumnSidecar bytes downloaded from peers for backfill.",
},
)
dataColumnSidecarDownloadMs = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_columns_download_ms",
Help: "DataColumnSidecars download time, in ms.",
Buckets: []float64{100, 300, 1000, 2000, 4000, 8000},
},
)
dataColumnSidecarVerifyMs = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "backfill_batch_columns_verify_ms",
Help: "DataColumnSidecars verification time, in ms.",
Buckets: []float64{3, 5, 10, 20, 100, 200},
},
)
)
func blobValidationMetrics(_ blocks.ROBlob) error {
backfillBlobsDownloadCount.Inc()
blobSidecarDownloadCount.Inc()
return nil
}
func blockValidationMetrics(interfaces.ReadOnlySignedBeaconBlock) error {
backfillBlocksDownloadCount.Inc()
blockDownloadCount.Inc()
return nil
}

View File

@@ -2,22 +2,24 @@ package backfill
import (
"context"
"maps"
"math"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/peers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
type batchWorkerPool interface {
spawn(ctx context.Context, n int, clock *startup.Clock, a PeerAssigner, v *verifier, cm sync.ContextByteVersions, blobVerifier verification.NewBlobVerifier, bfs *filesystem.BlobStorage)
spawn(ctx context.Context, n int, a PeerAssigner, cfg *workerCfg)
todo(b batch)
complete() (batch, error)
}
@@ -26,47 +28,61 @@ type worker interface {
run(context.Context)
}
type newWorker func(id workerId, in, out chan batch, c *startup.Clock, v *verifier, cm sync.ContextByteVersions, nbv verification.NewBlobVerifier, bfs *filesystem.BlobStorage) worker
type newWorker func(id workerId, in, out chan batch, cfg *workerCfg) worker
func defaultNewWorker(p p2p.P2P) newWorker {
return func(id workerId, in, out chan batch, c *startup.Clock, v *verifier, cm sync.ContextByteVersions, nbv verification.NewBlobVerifier, bfs *filesystem.BlobStorage) worker {
return newP2pWorker(id, p, in, out, c, v, cm, nbv, bfs)
return func(id workerId, in, out chan batch, cfg *workerCfg) worker {
return newP2pWorker(id, p, in, out, cfg)
}
}
// minRequestInterval is the minimum amount of time between requests.
// ie a value of 1s means we'll make ~1 req/sec per peer.
const minReqInterval = time.Second
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()
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()
earliest primitives.Slot // earliest is the earliest slot a worker is processing
peerCache *sync.DASPeerCache
p2p p2p.P2P
peerFailLogger *intervalLogger
needs func() das.CurrentNeeds
}
var _ batchWorkerPool = &p2pBatchWorkerPool{}
func newP2PBatchWorkerPool(p p2p.P2P, maxBatches int) *p2pBatchWorkerPool {
func newP2PBatchWorkerPool(p p2p.P2P, maxBatches int, needs func() das.CurrentNeeds) *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),
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),
peerCache: sync.NewDASPeerCache(p),
p2p: p,
peerFailLogger: newIntervalLogger(log, 5),
earliest: primitives.Slot(math.MaxUint64),
needs: needs,
}
}
func (p *p2pBatchWorkerPool) spawn(ctx context.Context, n int, c *startup.Clock, a PeerAssigner, v *verifier, cm sync.ContextByteVersions, nbv verification.NewBlobVerifier, bfs *filesystem.BlobStorage) {
func (p *p2pBatchWorkerPool) spawn(ctx context.Context, n int, a PeerAssigner, cfg *workerCfg) {
p.ctx, p.cancel = context.WithCancel(ctx)
go p.batchRouter(a)
for i := range n {
go p.newWorker(workerId(i), p.toWorkers, p.fromWorkers, c, v, cm, nbv, bfs).run(p.ctx)
go p.newWorker(workerId(i), p.toWorkers, p.fromWorkers, cfg).run(p.ctx)
}
}
@@ -103,7 +119,6 @@ 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:
@@ -115,51 +130,129 @@ func (p *p2pBatchWorkerPool) batchRouter(pa PeerAssigner) {
// This ticker exists to periodically break out of the channel select
// to retry failed assignments.
case b := <-p.fromWorkers:
pid := b.busy
busy[pid] = false
if b.state == batchBlobSync {
todo = append(todo, b)
sortBatchDesc(todo)
} else {
p.fromRouter <- b
if b.state == batchErrFatal {
p.shutdown(b.err)
}
pid := b.assignedPeer
delete(busy, pid)
if b.workComplete() {
p.fromRouter <- b
break
}
todo = append(todo, b)
sortBatchDesc(todo)
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))
var err error
todo, err = p.processTodo(todo, pa, busy)
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 {
if err := todo[0].waitUntilReady(p.ctx); err != nil {
log.WithError(p.ctx.Err()).Info("p2pBatchWorkerPool context canceled, shutting down")
p.shutdown(p.ctx.Err())
return
}
busy[pid] = true
todo[0].busy = 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) processTodo(todo []batch, pa PeerAssigner, busy map[peer.ID]bool) ([]batch, error) {
if len(todo) == 0 {
return todo, nil
}
notBusy, err := pa.Assign(peers.NotBusy(busy))
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.
return todo, nil
}
return nil, err
}
if len(notBusy) == 0 {
log.Debug("No suitable peers available for batch assignment")
return todo, nil
}
custodied := peerdas.NewColumnIndices()
if highestEpoch(todo) >= params.BeaconConfig().FuluForkEpoch {
custodied, err = currentCustodiedColumns(p.ctx, p.p2p)
if err != nil {
return nil, errors.Wrap(err, "current custodied columns")
}
}
picker, err := p.peerCache.NewPicker(notBusy, custodied, minReqInterval)
if err != nil {
log.WithError(err).Error("Failed to compute column-weighted peer scores")
return todo, nil
}
for i, b := range todo {
needs := p.needs()
if b.expired(needs) {
p.endSeq = append(p.endSeq, b.withState(batchEndSequence))
continue
}
excludePeers := busy
if b.state == batchErrFatal {
// Fatal error detected in batch, shut down the pool.
return nil, b.err
}
if b.state == batchErrRetryable {
// Columns can fail in a partial fashion, so we nee to reset
// components that track peer interactions for multiple columns
// to enable partial retries.
b = resetToRetryColumns(b, needs)
if b.state == batchSequenced {
// Transitioning to batchSequenced means we need to download a new block batch because there was
// a problem making or verifying the last block request, so we should try to pick a different peer this time.
excludePeers = busyCopy(busy)
excludePeers[b.blockPeer] = true
b.blockPeer = "" // reset block peer so we can fail back to it next time if there is an issue with assignment.
}
}
pid, cols, err := b.selectPeer(picker, excludePeers)
if err != nil {
p.peerFailLogger.WithField("notBusy", len(notBusy)).WithError(err).WithFields(b.logFields()).Debug("Failed to select peer for batch")
// Return the remaining todo items and allow the outer loop to control when we try again.
return todo[i:], nil
}
busy[pid] = true
b.assignedPeer = pid
b.nextReqCols = cols
backfillBatchTimeWaiting.Observe(float64(time.Since(b.scheduled).Milliseconds()))
p.toWorkers <- b
p.updateEarliest(b.begin)
}
return []batch{}, nil
}
func busyCopy(busy map[peer.ID]bool) map[peer.ID]bool {
busyCp := make(map[peer.ID]bool, len(busy))
maps.Copy(busyCp, busy)
return busyCp
}
func highestEpoch(batches []batch) primitives.Epoch {
highest := primitives.Epoch(0)
for _, b := range batches {
epoch := slots.ToEpoch(b.end - 1)
if epoch > highest {
highest = epoch
}
}
return highest
}
func (p *p2pBatchWorkerPool) updateEarliest(current primitives.Slot) {
if current >= p.earliest {
return
}
p.earliest = current
oldestBatch.Set(float64(p.earliest))
}
func (p *p2pBatchWorkerPool) shutdown(err error) {
p.cancel()
p.shutdownErr <- err

View File

@@ -5,12 +5,15 @@ import (
"testing"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/peers"
p2ptest "github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
@@ -24,7 +27,7 @@ type mockAssigner struct {
// 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) {
func (m mockAssigner) Assign(filter peers.AssignmentFilter) ([]peer.ID, error) {
if m.err != nil {
return nil, m.err
}
@@ -42,7 +45,8 @@ func TestPoolDetectAllEnded(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
ctx := t.Context()
ma := &mockAssigner{}
pool := newP2PBatchWorkerPool(p2p, nw)
needs := func() das.CurrentNeeds { return das.CurrentNeeds{Block: das.NeedSpan{Begin: 10, End: 10}} }
pool := newP2PBatchWorkerPool(p2p, nw, needs)
st, err := util.NewBeaconState()
require.NoError(t, err)
keys, err := st.PublicKeys()
@@ -53,8 +57,9 @@ func TestPoolDetectAllEnded(t *testing.T) {
ctxMap, err := sync.ContextByteVersionsForValRoot(bytesutil.ToBytes32(st.GenesisValidatorsRoot()))
require.NoError(t, err)
bfs := filesystem.NewEphemeralBlobStorage(t)
pool.spawn(ctx, nw, startup.NewClock(time.Now(), [32]byte{}), ma, v, ctxMap, mockNewBlobVerifier, bfs)
br := batcher{min: 10, size: 10}
wcfg := &workerCfg{clock: startup.NewClock(time.Now(), [32]byte{}), newVB: mockNewBlobVerifier, verifier: v, ctxMap: ctxMap, blobStore: bfs}
pool.spawn(ctx, nw, ma, wcfg)
br := batcher{size: 10, currentNeeds: needs}
endSeq := br.before(0)
require.Equal(t, batchEndSequence, endSeq.state)
for range nw {
@@ -72,7 +77,7 @@ type mockPool struct {
todoChan chan batch
}
func (m *mockPool) spawn(_ context.Context, _ int, _ *startup.Clock, _ PeerAssigner, _ *verifier, _ sync.ContextByteVersions, _ verification.NewBlobVerifier, _ *filesystem.BlobStorage) {
func (m *mockPool) spawn(_ context.Context, _ int, _ PeerAssigner, _ *workerCfg) {
}
func (m *mockPool) todo(b batch) {
@@ -89,3 +94,443 @@ func (m *mockPool) complete() (batch, error) {
}
var _ batchWorkerPool = &mockPool{}
// TestProcessTodoExpiresOlderBatches tests that processTodo correctly identifies and converts expired batches
func TestProcessTodoExpiresOlderBatches(t *testing.T) {
testCases := []struct {
name string
seqLen int
min primitives.Slot
max primitives.Slot
size primitives.Slot
updateMin primitives.Slot // what we'll set minChecker to
expectedEndSeq int // how many batches should be converted to endSeq
expectedProcessed int // how many batches should be processed (assigned to peers)
}{
{
name: "NoBatchesExpired",
seqLen: 3,
min: 100,
max: 1000,
size: 50,
updateMin: 120, // doesn't expire any batches
expectedEndSeq: 0,
expectedProcessed: 3,
},
{
name: "SomeBatchesExpired",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
updateMin: 175, // expires batches with end <= 175
expectedEndSeq: 1, // [100-150] will be expired
expectedProcessed: 3,
},
{
name: "AllBatchesExpired",
seqLen: 3,
min: 100,
max: 300,
size: 50,
updateMin: 300, // expires all batches
expectedEndSeq: 3,
expectedProcessed: 0,
},
{
name: "MultipleBatchesExpired",
seqLen: 8,
min: 100,
max: 500,
size: 50,
updateMin: 320, // expires multiple batches
expectedEndSeq: 4, // [300-350] (end=350 > 320 not expired), [250-300], [200-250], [150-200], [100-150] = 4 batches
expectedProcessed: 4,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Create pool with minChecker
pool := &p2pBatchWorkerPool{
endSeq: make([]batch, 0),
}
needs := das.CurrentNeeds{Block: das.NeedSpan{Begin: tc.updateMin, End: tc.max + 1}}
// Create batches with valid slot ranges (descending order)
todo := make([]batch, tc.seqLen)
for i := 0; i < tc.seqLen; i++ {
end := tc.min + primitives.Slot((tc.seqLen-i)*int(tc.size))
begin := end - tc.size
todo[i] = batch{
begin: begin,
end: end,
state: batchInit,
}
}
// Process todo using processTodo logic (simulate without actual peer assignment)
endSeqCount := 0
processedCount := 0
for _, b := range todo {
if b.expired(needs) {
pool.endSeq = append(pool.endSeq, b.withState(batchEndSequence))
endSeqCount++
} else {
processedCount++
}
}
// Verify counts
if endSeqCount != tc.expectedEndSeq {
t.Fatalf("expected %d batches to expire, got %d", tc.expectedEndSeq, endSeqCount)
}
if processedCount != tc.expectedProcessed {
t.Fatalf("expected %d batches to be processed, got %d", tc.expectedProcessed, processedCount)
}
// Verify all expired batches are in batchEndSequence state
for _, b := range pool.endSeq {
if b.state != batchEndSequence {
t.Fatalf("expired batch should be batchEndSequence, got %s", b.state.String())
}
if b.end > tc.updateMin {
t.Fatalf("batch with end=%d should not be in endSeq when min=%d", b.end, tc.updateMin)
}
}
})
}
}
// TestExpirationAfterMoveMinimum tests that batches expire correctly after minimum is increased
func TestExpirationAfterMoveMinimum(t *testing.T) {
testCases := []struct {
name string
seqLen int
min primitives.Slot
max primitives.Slot
size primitives.Slot
firstMin primitives.Slot
secondMin primitives.Slot
expectedAfter1 int // expected expired after first processTodo
expectedAfter2 int // expected expired after second processTodo
}{
{
name: "IncrementalMinimumIncrease",
seqLen: 4,
min: 100,
max: 1000,
size: 50,
firstMin: 150, // batches with end <= 150 expire
secondMin: 200, // additional batches with end <= 200 expire
expectedAfter1: 1, // [100-150] expires
expectedAfter2: 1, // [150-200] also expires on second check (end=200 <= 200)
},
{
name: "LargeMinimumJump",
seqLen: 3,
min: 100,
max: 300,
size: 50,
firstMin: 120, // no expiration
secondMin: 300, // all expire
expectedAfter1: 0,
expectedAfter2: 3,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
pool := &p2pBatchWorkerPool{
endSeq: make([]batch, 0),
}
// Create batches
todo := make([]batch, tc.seqLen)
for i := 0; i < tc.seqLen; i++ {
end := tc.min + primitives.Slot((tc.seqLen-i)*int(tc.size))
begin := end - tc.size
todo[i] = batch{
begin: begin,
end: end,
state: batchInit,
}
}
needs := das.CurrentNeeds{Block: das.NeedSpan{Begin: tc.firstMin, End: tc.max + 1}}
// First processTodo with firstMin
endSeq1 := 0
remaining1 := make([]batch, 0)
for _, b := range todo {
if b.expired(needs) {
pool.endSeq = append(pool.endSeq, b.withState(batchEndSequence))
endSeq1++
} else {
remaining1 = append(remaining1, b)
}
}
if endSeq1 != tc.expectedAfter1 {
t.Fatalf("after first update: expected %d expired, got %d", tc.expectedAfter1, endSeq1)
}
// Second processTodo with secondMin on remaining batches
needs.Block.Begin = tc.secondMin
endSeq2 := 0
for _, b := range remaining1 {
if b.expired(needs) {
pool.endSeq = append(pool.endSeq, b.withState(batchEndSequence))
endSeq2++
}
}
if endSeq2 != tc.expectedAfter2 {
t.Fatalf("after second update: expected %d expired, got %d", tc.expectedAfter2, endSeq2)
}
// Verify total endSeq count
totalExpected := tc.expectedAfter1 + tc.expectedAfter2
if len(pool.endSeq) != totalExpected {
t.Fatalf("expected total %d expired batches, got %d", totalExpected, len(pool.endSeq))
}
})
}
}
// TestTodoInterceptsBatchEndSequence tests that todo() correctly intercepts batchEndSequence batches
func TestTodoInterceptsBatchEndSequence(t *testing.T) {
testCases := []struct {
name string
batches []batch
expectedEndSeq int
expectedToRouter int
}{
{
name: "AllRegularBatches",
batches: []batch{
{state: batchInit},
{state: batchInit},
{state: batchErrRetryable},
},
expectedEndSeq: 0,
expectedToRouter: 3,
},
{
name: "MixedBatches",
batches: []batch{
{state: batchInit},
{state: batchEndSequence},
{state: batchInit},
{state: batchEndSequence},
},
expectedEndSeq: 2,
expectedToRouter: 2,
},
{
name: "AllEndSequence",
batches: []batch{
{state: batchEndSequence},
{state: batchEndSequence},
{state: batchEndSequence},
},
expectedEndSeq: 3,
expectedToRouter: 0,
},
{
name: "EmptyBatches",
batches: []batch{},
expectedEndSeq: 0,
expectedToRouter: 0,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
pool := &p2pBatchWorkerPool{
endSeq: make([]batch, 0),
}
endSeqCount := 0
routerCount := 0
for _, b := range tc.batches {
if b.state == batchEndSequence {
pool.endSeq = append(pool.endSeq, b)
endSeqCount++
} else {
routerCount++
}
}
if endSeqCount != tc.expectedEndSeq {
t.Fatalf("expected %d batchEndSequence, got %d", tc.expectedEndSeq, endSeqCount)
}
if routerCount != tc.expectedToRouter {
t.Fatalf("expected %d batches to router, got %d", tc.expectedToRouter, routerCount)
}
if len(pool.endSeq) != tc.expectedEndSeq {
t.Fatalf("endSeq slice should have %d batches, got %d", tc.expectedEndSeq, len(pool.endSeq))
}
})
}
}
// TestCompleteShutdownCondition tests the complete() method shutdown behavior
func TestCompleteShutdownCondition(t *testing.T) {
testCases := []struct {
name string
maxBatches int
endSeqCount int
shouldShutdown bool
expectedMin primitives.Slot
}{
{
name: "AllEndSeq_Shutdown",
maxBatches: 3,
endSeqCount: 3,
shouldShutdown: true,
expectedMin: 200,
},
{
name: "PartialEndSeq_NoShutdown",
maxBatches: 3,
endSeqCount: 2,
shouldShutdown: false,
expectedMin: 200,
},
{
name: "NoEndSeq_NoShutdown",
maxBatches: 5,
endSeqCount: 0,
shouldShutdown: false,
expectedMin: 150,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
pool := &p2pBatchWorkerPool{
maxBatches: tc.maxBatches,
endSeq: make([]batch, 0),
needs: func() das.CurrentNeeds {
return das.CurrentNeeds{Block: das.NeedSpan{Begin: tc.expectedMin}}
},
}
// Add endSeq batches
for i := 0; i < tc.endSeqCount; i++ {
pool.endSeq = append(pool.endSeq, batch{state: batchEndSequence})
}
// Check shutdown condition (this is what complete() checks)
shouldShutdown := len(pool.endSeq) == pool.maxBatches
if shouldShutdown != tc.shouldShutdown {
t.Fatalf("expected shouldShutdown=%v, got %v", tc.shouldShutdown, shouldShutdown)
}
pool.needs = func() das.CurrentNeeds {
return das.CurrentNeeds{Block: das.NeedSpan{Begin: tc.expectedMin}}
}
if pool.needs().Block.Begin != tc.expectedMin {
t.Fatalf("expected minimum %d, got %d", tc.expectedMin, pool.needs().Block.Begin)
}
})
}
}
// TestExpirationFlowEndToEnd tests the complete flow of batches from batcher through pool
func TestExpirationFlowEndToEnd(t *testing.T) {
testCases := []struct {
name string
seqLen int
min primitives.Slot
max primitives.Slot
size primitives.Slot
moveMinTo primitives.Slot
expired int
description string
}{
{
name: "SingleBatchExpires",
seqLen: 2,
min: 100,
max: 300,
size: 50,
moveMinTo: 150,
expired: 1,
description: "Initial [150-200] and [100-150]; moveMinimum(150) expires [100-150]",
},
/*
{
name: "ProgressiveExpiration",
seqLen: 4,
min: 100,
max: 500,
size: 50,
moveMinTo: 250,
description: "4 batches; moveMinimum(250) expires 2 of them",
},
*/
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Simulate the flow: batcher creates batches → sequence() → pool.todo() → pool.processTodo()
// Step 1: Create sequencer (simulating batcher)
seq := newBatchSequencer(tc.seqLen, tc.max, tc.size, mockCurrentNeedsFunc(tc.min, tc.max+1))
initializeBatchWithSlots(seq.seq, tc.min, tc.size)
for i := range seq.seq {
seq.seq[i].state = batchInit
}
// Step 2: Create pool
pool := &p2pBatchWorkerPool{
endSeq: make([]batch, 0),
}
// Step 3: Initial sequence() call - all batches should be returned (none expired yet)
batches1, err := seq.sequence()
if err != nil {
t.Fatalf("initial sequence() failed: %v", err)
}
if len(batches1) != tc.seqLen {
t.Fatalf("expected %d batches from initial sequence(), got %d", tc.seqLen, len(batches1))
}
// Step 4: Move minimum (simulating epoch advancement)
seq.currentNeeds = mockCurrentNeedsFunc(tc.moveMinTo, tc.max+1)
seq.batcher.currentNeeds = seq.currentNeeds
pool.needs = seq.currentNeeds
for i := range batches1 {
seq.update(batches1[i])
}
// Step 5: Process batches through pool (second sequence call would happen here in real code)
batches2, err := seq.sequence()
if err != nil && err != errMaxBatches {
t.Fatalf("second sequence() failed: %v", err)
}
require.Equal(t, tc.seqLen-tc.expired, len(batches2))
// Step 6: Simulate pool.processTodo() checking for expiration
processedCount := 0
for _, b := range batches2 {
if b.expired(pool.needs()) {
pool.endSeq = append(pool.endSeq, b.withState(batchEndSequence))
} else {
processedCount++
}
}
// Verify: All returned non-endSeq batches should have end > moveMinTo
for _, b := range batches2 {
if b.state != batchEndSequence && b.end <= tc.moveMinTo {
t.Fatalf("batch [%d-%d] should not be returned when min=%d", b.begin, b.end, tc.moveMinTo)
}
}
})
}
}

View File

@@ -3,10 +3,11 @@ package backfill
import (
"context"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/peers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
@@ -25,47 +26,39 @@ type Service struct {
enabled bool // service is disabled by default
clock *startup.Clock
store *Store
syncNeeds das.SyncNeeds
syncNeedsWaiter func() (das.SyncNeeds, error)
ms minimumSlotter
cw startup.ClockWaiter
verifierWaiter InitializerWaiter
newBlobVerifier verification.NewBlobVerifier
nWorkers int
batchSeq *batchSequencer
batchSize uint64
pool batchWorkerPool
verifier *verifier
ctxMap sync.ContextByteVersions
p2p p2p.P2P
pa PeerAssigner
batchImporter batchImporter
blobStore *filesystem.BlobStorage
dcStore *filesystem.DataColumnStorage
initSyncWaiter func() error
complete chan struct{}
workerCfg *workerCfg
fuluStart primitives.Slot
denebStart primitives.Slot
}
var _ runtime.Service = (*Service)(nil)
// PeerAssigner describes a type that provides an Assign method, which can assign the best peer
// to service an RPC blockRequest. The Assign method takes a map of peers that should be excluded,
// to service an RPC blockRequest. The Assign method takes a callback used to filter out peers,
// 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)
Assign(filter peers.AssignmentFilter) ([]peer.ID, error)
}
type minimumSlotter func(primitives.Slot) primitives.Slot
type batchImporter func(ctx context.Context, current primitives.Slot, b batch, su *Store) (*dbval.BackfillStatus, error)
func defaultBatchImporter(ctx context.Context, current primitives.Slot, 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.
return su.fillBack(ctx, current, b.results, b.availabilityStore())
}
// ServiceOption represents a functional option for the backfill service constructor.
type ServiceOption func(*Service) error
@@ -120,66 +113,41 @@ func WithVerifierWaiter(viw InitializerWaiter) ServiceOption {
}
}
// WithMinimumSlot allows the user to specify a different backfill minimum slot than the spec default of current - MIN_EPOCHS_FOR_BLOCK_REQUESTS.
// If this value is greater than current - MIN_EPOCHS_FOR_BLOCK_REQUESTS, it will be ignored with a warning log.
func WithMinimumSlot(s primitives.Slot) ServiceOption {
ms := func(current primitives.Slot) primitives.Slot {
specMin := minimumBackfillSlot(current)
if s < specMin {
return s
}
log.WithField("userSlot", s).WithField("specMinSlot", specMin).
Warn("Ignoring user-specified slot > MIN_EPOCHS_FOR_BLOCK_REQUESTS.")
return specMin
}
func WithSyncNeedsWaiter(f func() (das.SyncNeeds, error)) ServiceOption {
return func(s *Service) error {
s.ms = ms
if f != nil {
s.syncNeedsWaiter = f
}
return nil
}
}
// 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, bStore *filesystem.BlobStorage, cw startup.ClockWaiter, p p2p.P2P, pa PeerAssigner, opts ...ServiceOption) (*Service, error) {
func NewService(ctx context.Context, su *Store, bStore *filesystem.BlobStorage, dcStore *filesystem.DataColumnStorage, cw startup.ClockWaiter, p p2p.P2P, pa PeerAssigner, opts ...ServiceOption) (*Service, error) {
s := &Service{
ctx: ctx,
store: su,
blobStore: bStore,
cw: cw,
ms: minimumBackfillSlot,
p2p: p,
pa: pa,
batchImporter: defaultBatchImporter,
complete: make(chan struct{}),
ctx: ctx,
store: su,
blobStore: bStore,
dcStore: dcStore,
cw: cw,
p2p: p,
pa: pa,
complete: make(chan struct{}),
fuluStart: slots.SafeEpochStartOrMax(params.BeaconConfig().FuluForkEpoch),
denebStart: slots.SafeEpochStartOrMax(params.BeaconConfig().DenebForkEpoch),
}
s.batchImporter = s.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, sync.ContextByteVersions, error) {
cps, err := s.store.originState(ctx)
if err != nil {
return nil, nil, err
}
keys, err := cps.PublicKeys()
if err != nil {
return nil, nil, errors.Wrap(err, "unable to retrieve public keys for all validators in the origin state")
}
vr := cps.GenesisValidatorsRoot()
ctxMap, err := sync.ContextByteVersionsForValRoot(bytesutil.ToBytes32(vr))
if err != nil {
return nil, nil, errors.Wrapf(err, "unable to initialize context version map using genesis validator root %#x", vr)
}
v, err := newBackfillVerifier(vr, keys)
return v, ctxMap, err
}
func (s *Service) updateComplete() bool {
b, err := s.pool.complete()
if err != nil {
@@ -187,7 +155,7 @@ func (s *Service) updateComplete() bool {
log.WithField("backfillSlot", b.begin).Info("Backfill is complete")
return true
}
log.WithError(err).Error("Backfill service received unhandled error from worker pool")
log.WithError(err).Error("Service received unhandled error from worker pool")
return true
}
s.batchSeq.update(b)
@@ -195,39 +163,47 @@ func (s *Service) updateComplete() bool {
}
func (s *Service) importBatches(ctx context.Context) {
importable := s.batchSeq.importable()
imported := 0
defer func() {
if imported == 0 {
return
}
backfillBatchesImported.Add(float64(imported))
}()
current := s.clock.CurrentSlot()
for i := range importable {
ib := importable[i]
if len(ib.results) == 0 {
imported := 0
importable := s.batchSeq.importable()
for _, ib := range importable {
if len(ib.blocks) == 0 {
log.WithFields(ib.logFields()).Error("Batch with no results, skipping importer")
s.batchSeq.update(ib.withError(errors.New("batch has no blocks")))
// This batch needs to be retried before we can continue importing subsequent batches.
break
}
_, err := s.batchImporter(ctx, current, ib, s.store)
if err != nil {
log.WithError(err).WithFields(ib.logFields()).Debug("Backfill batch failed to import")
s.downscorePeer(ib.blockPid, "backfillBatchImportError")
s.batchSeq.update(ib.withState(batchErrRetryable))
s.batchSeq.update(ib.withError(err))
// If a batch fails, the subsequent batches are no longer considered importable.
break
}
// Calling update with state=batchImportComplete will advance the batch list.
s.batchSeq.update(ib.withState(batchImportComplete))
imported += 1
// Calling update with state=batchImportComplete will advance the batch list.
log.WithFields(ib.logFields()).WithField("batchesRemaining", s.batchSeq.numTodo()).Debug("Imported batch")
}
nt := s.batchSeq.numTodo()
log.WithField("imported", imported).WithField("importable", len(importable)).
WithField("batchesRemaining", nt).
Info("Backfill batches processed")
batchesRemaining.Set(float64(nt))
if imported > 0 {
batchesImported.Add(float64(imported))
}
}
backfillRemainingBatches.Set(float64(nt))
func (s *Service) defaultBatchImporter(ctx context.Context, current primitives.Slot, 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.
checker := newCheckMultiplexer(s.syncNeeds.Currently(), b)
return su.fillBack(ctx, current, b.blocks, checker)
}
func (s *Service) scheduleTodos() {
@@ -240,7 +216,7 @@ func (s *Service) scheduleTodos() {
// 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")
log.Debug("Waiting for descendent batch to complete")
return
}
}
@@ -249,80 +225,83 @@ func (s *Service) scheduleTodos() {
}
}
// fuluOrigin checks whether the origin block (ie the checkpoint sync block from which backfill
// syncs backwards) is in an unsupported fork, enabling the backfill service to shut down rather than
// run with buggy behavior.
// This will be removed once DataColumnSidecar support is released.
func fuluOrigin(cfg *params.BeaconChainConfig, status *dbval.BackfillStatus) bool {
originEpoch := slots.ToEpoch(primitives.Slot(status.OriginSlot))
if originEpoch < cfg.FuluForkEpoch {
return false
}
return true
}
// Start begins the runloop of backfill.Service in the current goroutine.
func (s *Service) Start() {
if !s.enabled {
log.Info("Backfill service not enabled")
log.Info("Service not enabled")
s.markComplete()
return
}
ctx, cancel := context.WithCancel(s.ctx)
defer func() {
log.Info("Backfill service is shutting down")
log.Info("Service is shutting down")
cancel()
}()
if s.store.isGenesisSync() {
log.Info("Node synced from genesis, shutting down backfill")
s.markComplete()
return
}
clock, err := s.cw.WaitForClock(ctx)
if err != nil {
log.WithError(err).Error("Backfill service failed to start while waiting for genesis data")
log.WithError(err).Error("Service failed to start while waiting for genesis data")
return
}
s.clock = clock
v, err := s.verifierWaiter.WaitForInitializer(ctx)
s.newBlobVerifier = newBlobVerifierFromInitializer(v)
if s.syncNeedsWaiter == nil {
log.Error("Service missing sync needs waiter; cannot start")
return
}
syncNeeds, err := s.syncNeedsWaiter()
if err != nil {
log.WithError(err).Error("Could not initialize blob verifier in backfill service")
log.WithError(err).Error("Service failed to start while waiting for sync needs")
return
}
s.syncNeeds = syncNeeds
if s.store.isGenesisSync() {
log.Info("Backfill short-circuit; node synced from genesis")
s.markComplete()
return
}
status := s.store.status()
if fuluOrigin(params.BeaconConfig(), status) {
log.WithField("originSlot", s.store.status().OriginSlot).
Warn("backfill disabled; DataColumnSidecar currently unsupported, for updates follow https://github.com/OffchainLabs/prysm/issues/15982")
s.markComplete()
return
}
needs := s.syncNeeds.Currently()
// Exit early if there aren't going to be any batches to backfill.
if primitives.Slot(status.LowSlot) <= s.ms(s.clock.CurrentSlot()) {
log.WithField("minimumRequiredSlot", s.ms(s.clock.CurrentSlot())).
if !needs.Block.At(primitives.Slot(status.LowSlot)) {
log.WithField("minimumSlot", needs.Block.Begin).
WithField("backfillLowestSlot", status.LowSlot).
Info("Exiting backfill service; minimum block retention slot > lowest backfilled block")
s.markComplete()
return
}
s.verifier, s.ctxMap, err = s.initVerifier(ctx)
if err != nil {
log.WithError(err).Error("Unable to initialize backfill verifier")
return
}
if s.initSyncWaiter != nil {
log.Info("Backfill service waiting for initial-sync to reach head before starting")
log.Info("Service waiting for initial-sync to reach head before starting")
if err := s.initSyncWaiter(); err != nil {
log.WithError(err).Error("Error waiting for init-sync to complete")
return
}
}
s.pool.spawn(ctx, s.nWorkers, clock, s.pa, s.verifier, s.ctxMap, s.newBlobVerifier, s.blobStore)
s.batchSeq = newBatchSequencer(s.nWorkers, s.ms(s.clock.CurrentSlot()), primitives.Slot(status.LowSlot), primitives.Slot(s.batchSize))
if s.workerCfg == nil {
s.workerCfg = &workerCfg{
clock: s.clock,
blobStore: s.blobStore,
colStore: s.dcStore,
downscore: s.downscorePeer,
currentNeeds: s.syncNeeds.Currently,
}
if err = initWorkerCfg(ctx, s.workerCfg, s.verifierWaiter, s.store); err != nil {
log.WithError(err).Error("Could not initialize blob verifier in backfill service")
return
}
}
// Allow tests to inject a mock pool.
if s.pool == nil {
s.pool = newP2PBatchWorkerPool(s.p2p, s.nWorkers, s.syncNeeds.Currently)
}
s.pool.spawn(ctx, s.nWorkers, s.pa, s.workerCfg)
s.batchSeq = newBatchSequencer(s.nWorkers, primitives.Slot(status.LowSlot), primitives.Slot(s.batchSize), s.syncNeeds.Currently)
if err = s.initBatches(); err != nil {
log.WithError(err).Error("Non-recoverable error in backfill service")
return
@@ -338,9 +317,6 @@ func (s *Service) Start() {
}
s.importBatches(ctx)
batchesWaiting.Set(float64(s.batchSeq.countWithState(batchImportable)))
if err := s.batchSeq.moveMinimum(s.ms(s.clock.CurrentSlot())); err != nil {
log.WithError(err).Error("Non-recoverable error while adjusting backfill minimum slot")
}
s.scheduleTodos()
}
}
@@ -364,14 +340,16 @@ func (*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 := min(primitives.Epoch(params.BeaconConfig().MinEpochsForBlockRequests), slots.MaxSafeEpoch())
offset := slots.UnsafeEpochStart(oe)
// syncEpochOffset subtracts a number of epochs as slots from the current slot, with underflow checks.
// It returns slot 1 if the result would be 0 or underflow. It doesn't return slot 0 because the
// genesis block needs to be specially synced (it doesn't have a valid signature).
func syncEpochOffset(current primitives.Slot, subtract primitives.Epoch) primitives.Slot {
minEpoch := min(subtract, slots.MaxSafeEpoch())
// compute slot offset - offset is a number of slots to go back from current (not an absolute slot).
offset := slots.UnsafeEpochStart(minEpoch)
// Undeflow protection: slot 0 is the genesis block, therefore the signature in it is invalid.
// To prevent us from rejecting a batch, we restrict the minimum backfill batch till only slot 1
if offset >= current {
// Slot 0 is the genesis block, therefore the signature in it is invalid.
// To prevent us from rejecting a batch, we restrict the minimum backfill batch till only slot 1
return 1
}
return current - offset
@@ -383,9 +361,15 @@ func newBlobVerifierFromInitializer(ini *verification.Initializer) verification.
}
}
func newDataColumnVerifierFromInitializer(ini *verification.Initializer) verification.NewDataColumnsVerifier {
return func(cols []blocks.RODataColumn, reqs []verification.Requirement) verification.DataColumnsVerifier {
return ini.NewDataColumnsVerifier(cols, reqs)
}
}
func (s *Service) markComplete() {
close(s.complete)
log.Info("Backfill service marked as complete")
log.Info("Marked as complete")
}
func (s *Service) WaitForCompletion() error {
@@ -397,7 +381,11 @@ func (s *Service) WaitForCompletion() error {
}
}
func (s *Service) downscorePeer(peerID peer.ID, reason string) {
func (s *Service) downscorePeer(peerID peer.ID, reason string, err error) {
newScore := s.p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID)
log.WithFields(logrus.Fields{"peerID": peerID, "reason": reason, "newScore": newScore}).Debug("Downscore peer")
logArgs := log.WithFields(logrus.Fields{"peerID": peerID, "reason": reason, "newScore": newScore})
if err != nil {
logArgs = logArgs.WithError(err)
}
logArgs.Debug("Downscore peer")
}

View File

@@ -5,17 +5,16 @@ import (
"testing"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
p2ptest "github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/proto/dbval"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/OffchainLabs/prysm/v7/time/slots"
)
type mockMinimumSlotter struct {
@@ -40,9 +39,9 @@ func TestServiceInit(t *testing.T) {
su, err := NewUpdater(ctx, db)
require.NoError(t, err)
nWorkers := 5
var batchSize uint64 = 100
var batchSize uint64 = 4
nBatches := nWorkers * 2
var high uint64 = 11235
var high uint64 = 1 + batchSize*uint64(nBatches) // extra 1 because upper bound is exclusive
originRoot := [32]byte{}
origin, err := util.NewBeaconState()
require.NoError(t, err)
@@ -53,14 +52,24 @@ func TestServiceInit(t *testing.T) {
}
remaining := nBatches
cw := startup.NewClockSynchronizer()
require.NoError(t, cw.SetClock(startup.NewClock(time.Now(), [32]byte{})))
clock := startup.NewClock(time.Now(), [32]byte{}, startup.WithSlotAsNow(primitives.Slot(high)+1))
require.NoError(t, cw.SetClock(clock))
pool := &mockPool{todoChan: make(chan batch, nWorkers), finishedChan: make(chan batch, nWorkers)}
p2pt := p2ptest.NewTestP2P(t)
bfs := filesystem.NewEphemeralBlobStorage(t)
srv, err := NewService(ctx, su, bfs, cw, p2pt, &mockAssigner{},
WithBatchSize(batchSize), WithWorkerCount(nWorkers), WithEnableBackfill(true), WithVerifierWaiter(&mockInitalizerWaiter{}))
dcs := filesystem.NewEphemeralDataColumnStorage(t)
snw := func() (das.SyncNeeds, error) {
return das.NewSyncNeeds(
clock.CurrentSlot,
nil,
primitives.Epoch(0),
)
}
srv, err := NewService(ctx, su, bfs, dcs, cw, p2pt, &mockAssigner{},
WithBatchSize(batchSize), WithWorkerCount(nWorkers), WithEnableBackfill(true), WithVerifierWaiter(&mockInitalizerWaiter{}),
WithSyncNeedsWaiter(snw))
require.NoError(t, err)
srv.ms = mockMinimumSlotter{min: primitives.Slot(high - batchSize*uint64(nBatches))}.minimumSlot
srv.pool = pool
srv.batchImporter = func(context.Context, primitives.Slot, batch, *Store) (*dbval.BackfillStatus, error) {
return &dbval.BackfillStatus{}, nil
@@ -74,6 +83,11 @@ func TestServiceInit(t *testing.T) {
if b.state == batchSequenced {
b.state = batchImportable
}
for i := b.begin; i < b.end; i++ {
blk, _ := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, primitives.Slot(i), 0)
b.blocks = append(b.blocks, blk)
}
require.Equal(t, int(batchSize), len(b.blocks))
pool.finishedChan <- b
todo = testReadN(ctx, t, pool.todoChan, 1, todo)
}
@@ -83,18 +97,6 @@ func TestServiceInit(t *testing.T) {
}
}
func TestMinimumBackfillSlot(t *testing.T) {
oe := primitives.Epoch(params.BeaconConfig().MinEpochsForBlockRequests)
currSlot := (oe + 100).Mul(uint64(params.BeaconConfig().SlotsPerEpoch))
minSlot := minimumBackfillSlot(primitives.Slot(currSlot))
require.Equal(t, 100*params.BeaconConfig().SlotsPerEpoch, minSlot)
currSlot = oe.Mul(uint64(params.BeaconConfig().SlotsPerEpoch))
minSlot = minimumBackfillSlot(primitives.Slot(currSlot))
require.Equal(t, primitives.Slot(1), minSlot)
}
func testReadN(ctx context.Context, t *testing.T, c chan batch, n int, into []batch) []batch {
for range n {
select {
@@ -107,66 +109,3 @@ func testReadN(ctx context.Context, t *testing.T, c chan batch, n int, into []ba
}
return into
}
func TestBackfillMinSlotDefault(t *testing.T) {
oe := primitives.Epoch(params.BeaconConfig().MinEpochsForBlockRequests)
current := primitives.Slot((oe + 100).Mul(uint64(params.BeaconConfig().SlotsPerEpoch)))
s := &Service{}
specMin := minimumBackfillSlot(current)
t.Run("equal to specMin", func(t *testing.T) {
opt := WithMinimumSlot(specMin)
require.NoError(t, opt(s))
require.Equal(t, specMin, s.ms(current))
})
t.Run("older than specMin", func(t *testing.T) {
opt := WithMinimumSlot(specMin - 1)
require.NoError(t, opt(s))
// if WithMinimumSlot is older than the spec minimum, we should use it.
require.Equal(t, specMin-1, s.ms(current))
})
t.Run("newer than specMin", func(t *testing.T) {
opt := WithMinimumSlot(specMin + 1)
require.NoError(t, opt(s))
// if WithMinimumSlot is newer than the spec minimum, we should use the spec minimum
require.Equal(t, specMin, s.ms(current))
})
}
func TestFuluOrigin(t *testing.T) {
cfg := params.BeaconConfig()
fuluEpoch := cfg.FuluForkEpoch
fuluSlot, err := slots.EpochStart(fuluEpoch)
require.NoError(t, err)
cases := []struct {
name string
origin primitives.Slot
isFulu bool
}{
{
name: "before fulu",
origin: fuluSlot - 1,
isFulu: false,
},
{
name: "at fulu",
origin: fuluSlot,
isFulu: true,
},
{
name: "after fulu",
origin: fuluSlot + 1,
isFulu: true,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
status := &dbval.BackfillStatus{
OriginSlot: uint64(tc.origin),
}
result := fuluOrigin(cfg, status)
require.Equal(t, tc.isFulu, result)
})
}
}

View File

@@ -45,9 +45,10 @@ type Store struct {
bs *dbval.BackfillStatus
}
// 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.
// AvailableBlock determines if the given slot has been covered by backfill.
// If the node was synced from genesis, all slots are considered available.
// The genesis block at slot 0 is always available.
// Otherwise any slot between 0 and LowSlot are considered unavailable.
func (s *Store) AvailableBlock(sl primitives.Slot) bool {
s.RLock()
defer s.RUnlock()
@@ -71,10 +72,10 @@ func (s *Store) status() *dbval.BackfillStatus {
}
}
// 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, current primitives.Slot, blocks []blocks.ROBlock, store das.AvailabilityStore) (*dbval.BackfillStatus, error) {
// fillBack saves the slice of blocks and updates the BackfillStatus tracker to match 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.
// It also performs the blob and/or data column availability check, which will persist blobs/DCs to disk once verified.
func (s *Store) fillBack(ctx context.Context, current primitives.Slot, blocks []blocks.ROBlock, store das.AvailabilityChecker) (*dbval.BackfillStatus, error) {
status := s.status()
if len(blocks) == 0 {
return status, nil
@@ -88,10 +89,8 @@ func (s *Store) fillBack(ctx context.Context, current primitives.Slot, blocks []
status.LowParentRoot, highest.Root(), status.LowSlot, highest.Block().Slot())
}
for i := range blocks {
if err := store.IsDataAvailable(ctx, current, blocks[i]); err != nil {
return nil, err
}
if err := store.IsDataAvailable(ctx, current, blocks...); err != nil {
return nil, errors.Wrap(err, "IsDataAvailable")
}
if err := s.store.SaveROBlocks(ctx, blocks, false); err != nil {

View File

@@ -2,36 +2,41 @@ package backfill
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
)
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")
var (
errInvalidBlocks = errors.New("block validation failure")
errInvalidBatchChain = errors.Wrap(errInvalidBlocks, "parent_root of block does not match the previous block's root")
errProposerIndexTooHigh = errors.Wrap(errInvalidBlocks, "proposer index not present in origin state")
errUnknownDomain = errors.Wrap(errInvalidBlocks, "runtime error looking up signing domain for fork")
errBatchSignatureFailed = errors.Wrap(errInvalidBlocks, "failed to verify block signature in batch")
errInvalidSignatureData = errors.Wrap(errInvalidBlocks, "could not verify signatures in block batch due to invalid signature data")
errEmptyVerificationSet = errors.New("no blocks to verify in batch")
)
// verifiedROBlocks represents a slice of blocks that have passed signature verification.
type verifiedROBlocks []blocks.ROBlock
func (v verifiedROBlocks) blobIdents(retentionStart primitives.Slot) ([]blobSummary, error) {
// early return if the newest block is outside the retention window
if len(v) > 0 && v[len(v)-1].Block().Slot() < retentionStart {
func (v verifiedROBlocks) blobIdents(needed func() das.CurrentNeeds) ([]blobSummary, error) {
if len(v) == 0 {
return nil, nil
}
needs := needed()
bs := make([]blobSummary, 0)
for i := range v {
if v[i].Block().Slot() < retentionStart {
continue
}
if v[i].Block().Version() < version.Deneb {
slot := v[i].Block().Slot()
if !needs.Blob.At(slot) {
continue
}
c, err := v[i].Block().Body().BlobKzgCommitments()
@@ -56,37 +61,37 @@ type verifier struct {
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))
func (vr verifier) verify(blks []blocks.ROBlock) (verifiedROBlocks, error) {
if len(blks) == 0 {
// Returning an error here simplifies handling in the caller.
// errEmptyVerificationSet should not cause the peer to be downscored.
return nil, errEmptyVerificationSet
}
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]
if i > 0 && blks[i-1].Root() != blks[i].Block().ParentRoot() {
p, b := blks[i-1], blks[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])
set, err := vr.blockSignatureBatch(blks[i])
if err != nil {
return nil, err
return nil, errors.Wrap(err, "block signature batch")
}
sigSet.Join(set)
}
v, err := sigSet.Verify()
if err != nil {
return nil, errors.Wrap(err, "block signature verification error")
// The blst wrapper does not give us checkable errors, so we "reverse wrap"
// the error string to make it checkable for shouldDownscore.
return nil, errors.Wrap(errInvalidSignatureData, err.Error())
}
if !v {
return nil, errors.New("batch block signature verification failed")
return nil, errBatchSignatureFailed
}
return result, nil
return blks, nil
}
func (vr verifier) blockSignatureBatch(b blocks.ROBlock) (*bls.SignatureBatch, error) {

View File

@@ -0,0 +1,175 @@
package backfill
import (
"io"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
type columnBisector struct {
rootKeys map[[32]byte]rootKey
columnSource map[rootKey]map[uint64]peer.ID
bisected map[peer.ID][]blocks.RODataColumn
pidIter []peer.ID
current int
next int
downscore peerDownscorer
errs []error
failures map[rootKey]peerdas.ColumnIndices
}
type rootKey *[32]byte
var errColumnVerification = errors.New("column verification failed")
var errBisectInconsistent = errors.New("state of bisector inconsistent with columns to bisect")
func (c *columnBisector) addPeerColumns(pid peer.ID, columns ...blocks.RODataColumn) {
for _, col := range columns {
c.setColumnSource(c.rootKey(col.BlockRoot()), col.Index, pid)
}
}
// failuresFor returns the set of column indices that failed verification
// for the given block root.
func (c *columnBisector) failuresFor(root [32]byte) peerdas.ColumnIndices {
return c.failures[c.rootKey(root)]
}
func (c *columnBisector) failingRoots() [][32]byte {
roots := make([][32]byte, 0, len(c.failures))
for rk := range c.failures {
roots = append(roots, *rk)
}
return roots
}
func (c *columnBisector) setColumnSource(rk rootKey, idx uint64, pid peer.ID) {
if c.columnSource == nil {
c.columnSource = make(map[rootKey]map[uint64]peer.ID)
}
if c.columnSource[rk] == nil {
c.columnSource[rk] = make(map[uint64]peer.ID)
}
c.columnSource[rk][idx] = pid
}
func (c *columnBisector) clearColumnSource(rk rootKey, idx uint64) {
if c.columnSource == nil {
return
}
if c.columnSource[rk] == nil {
return
}
delete(c.columnSource[rk], idx)
if len(c.columnSource[rk]) == 0 {
delete(c.columnSource, rk)
}
}
func (c *columnBisector) rootKey(root [32]byte) rootKey {
ptr, ok := c.rootKeys[root]
if ok {
return ptr
}
c.rootKeys[root] = &root
return c.rootKeys[root]
}
func (c *columnBisector) peerFor(col blocks.RODataColumn) (peer.ID, error) {
r := c.columnSource[c.rootKey(col.BlockRoot())]
if len(r) == 0 {
return "", errors.Wrap(errBisectInconsistent, "root not tracked")
}
if pid, ok := r[col.Index]; ok {
return pid, nil
}
return "", errors.Wrap(errBisectInconsistent, "index not tracked for root")
}
// reset prepares the columnBisector to be used to retry failed columns.
// it resets the peer sources of the failed columns and clears the failure records.
func (c *columnBisector) reset() {
// reset all column sources for failed columns
for rk, indices := range c.failures {
for _, idx := range indices.ToSlice() {
c.clearColumnSource(rk, idx)
}
}
c.failures = make(map[rootKey]peerdas.ColumnIndices)
c.errs = nil
}
// Bisect initializes columnBisector with the set of columns to bisect.
func (c *columnBisector) Bisect(columns []blocks.RODataColumn) (das.BisectionIterator, error) {
for _, col := range columns {
pid, err := c.peerFor(col)
if err != nil {
return nil, errors.Wrap(err, "could not lookup peer for column")
}
c.bisected[pid] = append(c.bisected[pid], col)
}
c.pidIter = make([]peer.ID, 0, len(c.bisected))
for pid := range c.bisected {
c.pidIter = append(c.pidIter, pid)
}
// The implementation of Next() assumes these are equal in
// the base case.
c.current, c.next = 0, 0
return c, nil
}
// Next implements an iterator for the columnBisector.
// Each batch is from a single peer.
func (c *columnBisector) Next() ([]blocks.RODataColumn, error) {
if c.next >= len(c.pidIter) {
return nil, io.EOF
}
c.current = c.next
pid := c.pidIter[c.current]
cols := c.bisected[pid]
c.next += 1
return cols, nil
}
// Error implements das.Bisector.
func (c *columnBisector) Error() error {
if len(c.errs) > 0 {
return errColumnVerification
}
return nil
}
// OnError implements das.Bisector.
func (c *columnBisector) OnError(err error) {
c.errs = append(c.errs, err)
pid := c.pidIter[c.current]
c.downscore(pid, "column verification error", err)
// Track which roots failed by examining columns from the current peer
columns := c.bisected[pid]
for _, col := range columns {
root := col.BlockRoot()
rk := c.rootKey(root)
if c.failures[rk] == nil {
c.failures[rk] = make(peerdas.ColumnIndices)
}
c.failures[rk][col.Index] = struct{}{}
}
}
var _ das.Bisector = &columnBisector{}
var _ das.BisectionIterator = &columnBisector{}
func newColumnBisector(downscorer peerDownscorer) *columnBisector {
return &columnBisector{
rootKeys: make(map[[32]byte]rootKey),
columnSource: make(map[rootKey]map[uint64]peer.ID),
bisected: make(map[peer.ID][]blocks.RODataColumn),
failures: make(map[rootKey]peerdas.ColumnIndices),
downscore: downscorer,
}
}

View File

@@ -0,0 +1,587 @@
package backfill
import (
"io"
"slices"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
// mockDownscorer is a simple downscorer that tracks calls
type mockDownscorer struct {
calls []struct {
pid peer.ID
msg string
err error
}
}
func (m *mockDownscorer) downscoreCall(pid peer.ID, msg string, err error) {
m.calls = append(m.calls, struct {
pid peer.ID
msg string
err error
}{pid, msg, err})
}
// createTestDataColumn creates a test data column with the given parameters.
// nBlobs determines the number of cells, commitments, and proofs.
func createTestDataColumn(t *testing.T, root [32]byte, index uint64, nBlobs int) util.DataColumnParam {
commitments := make([][]byte, nBlobs)
cells := make([][]byte, nBlobs)
proofs := make([][]byte, nBlobs)
for i := range nBlobs {
commitments[i] = make([]byte, 48)
cells[i] = make([]byte, 0)
proofs[i] = make([]byte, 48)
}
return util.DataColumnParam{
Index: index,
Column: cells,
KzgCommitments: commitments,
KzgProofs: proofs,
Slot: primitives.Slot(1),
BodyRoot: root[:],
StateRoot: make([]byte, 32),
ParentRoot: make([]byte, 32),
}
}
// createTestPeerID creates a test peer ID from a string seed.
func createTestPeerID(t *testing.T, seed string) peer.ID {
pid, err := peer.Decode(seed)
require.NoError(t, err)
return pid
}
// TestNewColumnBisector verifies basic initialization
func TestNewColumnBisector(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
require.NotNil(t, cb)
require.NotNil(t, cb.rootKeys)
require.NotNil(t, cb.columnSource)
require.NotNil(t, cb.bisected)
require.Equal(t, 0, cb.current)
require.Equal(t, 0, cb.next)
}
// TestAddAndIterateColumns demonstrates creating test columns and iterating
func TestAddAndIterateColumns(t *testing.T) {
root := [32]byte{1, 0, 0}
params := []util.DataColumnParam{
createTestDataColumn(t, root, 0, 2),
createTestDataColumn(t, root, 1, 2),
}
roColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, params)
require.Equal(t, 2, len(roColumns))
// Create downscorer and bisector
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
// Create test peer ID
pid1 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
// Add columns from peer
cb.addPeerColumns(pid1, roColumns...)
// Bisect and verify iteration
iter, err := cb.Bisect(roColumns)
require.NoError(t, err)
require.NotNil(t, iter)
// Get first (and only) batch from the peer
batch, err := iter.Next()
require.NoError(t, err)
require.Equal(t, 2, len(batch))
// Next should return EOF
_, err = iter.Next()
require.Equal(t, io.EOF, err)
}
// TestRootKeyDeduplication verifies that rootKey returns the same pointer for identical roots
func TestRootKeyDeduplication(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 2, 3}
key1 := cb.rootKey(root)
key2 := cb.rootKey(root)
// Should be the same pointer
require.Equal(t, key1, key2)
}
// TestMultipleRootsAndPeers verifies handling of multiple distinct roots and peer IDs
func TestMultipleRootsAndPeers(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root1 := [32]byte{1, 0, 0}
root2 := [32]byte{2, 0, 0}
root3 := [32]byte{3, 0, 0}
pid1 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
pid2 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMr")
// Register multiple columns with different roots and peers
params1 := createTestDataColumn(t, root1, 0, 2)
params2 := createTestDataColumn(t, root2, 1, 2)
params3 := createTestDataColumn(t, root3, 2, 2)
cols1, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params1})
cols2, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params2})
cols3, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params3})
cb.addPeerColumns(pid1, cols1...)
cb.addPeerColumns(pid2, cols2...)
cb.addPeerColumns(pid1, cols3...)
// Verify roots and peers are tracked
require.Equal(t, 3, len(cb.rootKeys))
}
// TestSetColumnSource verifies that columns from different peers are properly tracked
func TestSetColumnSource(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
// Create multiple peers with columns
root1 := [32]byte{1, 0, 0}
root2 := [32]byte{2, 0, 0}
root3 := [32]byte{3, 0, 0}
pid1 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
pid2 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMr")
// Create columns for peer1: 2 columns
params1 := []util.DataColumnParam{
createTestDataColumn(t, root1, 0, 1),
createTestDataColumn(t, root2, 1, 1),
}
// Create columns for peer2: 2 columns
params2 := []util.DataColumnParam{
createTestDataColumn(t, root3, 0, 1),
createTestDataColumn(t, root1, 2, 1),
}
cols1, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, params1)
cols2, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, params2)
// Register columns from both peers
cb.addPeerColumns(pid1, cols1...)
cb.addPeerColumns(pid2, cols2...)
// Use Bisect to verify columns are grouped by peer
allCols := append(cols1, cols2...)
iter, err := cb.Bisect(allCols)
require.NoError(t, err)
// Sort the pidIter so batch order is deterministic
slices.Sort(cb.pidIter)
// Collect all batches (order is non-deterministic due to map iteration)
var batches [][]blocks.RODataColumn
for {
batch, err := iter.Next()
if err == io.EOF {
break
}
require.NoError(t, err)
batches = append(batches, batch)
}
// Verify we got exactly 2 batches
require.Equal(t, 2, len(batches))
// Find which batch is from which peer by examining the columns
pid1Batch := map[peer.ID][]blocks.RODataColumn{pid1: nil, pid2: nil}
for _, batch := range batches {
if len(batch) == 0 {
continue
}
// All columns in a batch are from the same peer
col := batch[0]
colPeer, err := cb.peerFor(col)
require.NoError(t, err)
// Compare dereferenced peer.ID values rather than pointers
if colPeer == pid1 {
pid1Batch[pid1] = batch
} else if colPeer == pid2 {
pid1Batch[pid2] = batch
}
}
// Verify peer1's batch has 2 columns
require.NotNil(t, pid1Batch[pid1])
require.Equal(t, 2, len(pid1Batch[pid1]))
for _, col := range pid1Batch[pid1] {
colPeer, err := cb.peerFor(col)
require.NoError(t, err)
require.Equal(t, pid1, colPeer)
}
// Verify peer2's batch has 2 columns
require.NotNil(t, pid1Batch[pid2])
require.Equal(t, 2, len(pid1Batch[pid2]))
for _, col := range pid1Batch[pid2] {
colPeer, err := cb.peerFor(col)
require.NoError(t, err)
require.Equal(t, pid2, colPeer)
}
}
// TestClearColumnSource verifies column removal and cleanup of empty maps
func TestClearColumnSource(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
rk := cb.rootKey(root)
pid := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
cb.setColumnSource(rk, 0, pid)
cb.setColumnSource(rk, 1, pid)
require.Equal(t, 2, len(cb.columnSource[rk]))
// Clear one column
cb.clearColumnSource(rk, 0)
require.Equal(t, 1, len(cb.columnSource[rk]))
// Clear the last column - should remove the root entry
cb.clearColumnSource(rk, 1)
_, exists := cb.columnSource[rk]
require.Equal(t, false, exists)
}
// TestClearNonexistentColumn ensures clearing non-existent columns doesn't crash
func TestClearNonexistentColumn(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
rk := cb.rootKey(root)
// Should not panic
cb.clearColumnSource(rk, 99)
}
// TestFailuresFor verifies failuresFor returns correct failures for a block root
func TestFailuresFor(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
rk := cb.rootKey(root)
// Initially no failures
failures := cb.failuresFor(root)
require.Equal(t, 0, len(failures.ToSlice()))
// Set some failures
cb.failures[rk] = peerdas.ColumnIndices{0: struct{}{}, 1: struct{}{}, 2: struct{}{}}
failures = cb.failuresFor(root)
require.Equal(t, 3, len(failures.ToSlice()))
}
// TestFailingRoots ensures failingRoots returns all roots with failures
func TestFailingRoots(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root1 := [32]byte{1, 0, 0}
root2 := [32]byte{2, 0, 0}
rk1 := cb.rootKey(root1)
rk2 := cb.rootKey(root2)
cb.failures[rk1] = peerdas.ColumnIndices{0: struct{}{}}
cb.failures[rk2] = peerdas.ColumnIndices{1: struct{}{}}
failingRoots := cb.failingRoots()
require.Equal(t, 2, len(failingRoots))
}
// TestPeerFor verifies peerFor correctly returns the peer for a column
func TestPeerFor(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
pid := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
params := createTestDataColumn(t, root, 0, 2)
cols, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params})
// Use addPeerColumns to properly register the column
cb.addPeerColumns(pid, cols[0])
peerKey, err := cb.peerFor(cols[0])
require.NoError(t, err)
require.NotNil(t, peerKey)
}
// TestPeerForNotTracked ensures error when root not tracked
func TestPeerForNotTracked(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
params := createTestDataColumn(t, root, 0, 2)
cols, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params})
// Don't add any columns - root is not tracked
_, err := cb.peerFor(cols[0])
require.ErrorIs(t, err, errBisectInconsistent)
}
// TestBisectGroupsByMultiplePeers ensures columns grouped by their peer source
func TestBisectGroupsByMultiplePeers(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
pid1 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
pid2 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMr")
params1 := createTestDataColumn(t, root, 0, 2)
params2 := createTestDataColumn(t, root, 1, 2)
cols1, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params1})
cols2, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params2})
cb.addPeerColumns(pid1, cols1...)
cb.addPeerColumns(pid2, cols2...)
// Bisect both columns
iter, err := cb.Bisect(append(cols1, cols2...))
require.NoError(t, err)
// Sort the pidIter so that batch order is deterministic
slices.Sort(cb.pidIter)
// Should get two separate batches, one from each peer
batch1, err := iter.Next()
require.NoError(t, err)
require.Equal(t, 1, len(batch1))
batch2, err := iter.Next()
require.NoError(t, err)
require.Equal(t, 1, len(batch2))
_, err = iter.Next()
require.Equal(t, io.EOF, err)
}
// TestOnError verifies OnError records errors and calls downscorer
func TestOnError(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
pid := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
cb.pidIter = append(cb.pidIter, pid)
cb.current = 0
testErr := errors.New("test error")
cb.OnError(testErr)
require.Equal(t, 1, len(cb.errs))
require.Equal(t, 1, len(downscorer.calls))
require.Equal(t, pid, downscorer.calls[0].pid)
}
// TestErrorReturnAfterOnError ensures Error() returns non-nil after OnError called
func TestErrorReturnAfterOnError(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
pid := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
cb.pidIter = append(cb.pidIter, pid)
cb.current = 0
require.NoError(t, cb.Error())
cb.OnError(errors.New("test error"))
require.NotNil(t, cb.Error())
}
// TestResetClearsFailures verifies reset clears all failures and errors
func TestResetClearsFailures(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
rk := cb.rootKey(root)
cb.failures[rk] = peerdas.ColumnIndices{0: struct{}{}, 1: struct{}{}}
cb.errs = []error{errors.New("test")}
cb.reset()
require.Equal(t, 0, len(cb.failures))
require.Equal(t, 0, len(cb.errs))
}
// TestResetClearsColumnSources ensures reset clears column sources for failed columns
func TestResetClearsColumnSources(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root := [32]byte{1, 0, 0}
rk := cb.rootKey(root)
pid := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
cb.setColumnSource(rk, 0, pid)
cb.setColumnSource(rk, 1, pid)
cb.failures[rk] = peerdas.ColumnIndices{0: struct{}{}, 1: struct{}{}}
cb.reset()
// Column sources for the failed root should be cleared
_, exists := cb.columnSource[rk]
require.Equal(t, false, exists)
}
// TestBisectResetBisectAgain tests end-to-end multiple bisect cycles with reset
func TestBisectResetBisectAgain(t *testing.T) {
downscorer := &mockDownscorer{}
root := [32]byte{1, 0, 0}
pid := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
params := createTestDataColumn(t, root, 0, 2)
cols, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{params})
// First bisect with fresh bisector
cb1 := newColumnBisector(downscorer.downscoreCall)
cb1.addPeerColumns(pid, cols...)
iter, err := cb1.Bisect(cols)
require.NoError(t, err)
batch, err := iter.Next()
require.NoError(t, err)
require.Equal(t, 1, len(batch))
_, err = iter.Next()
require.Equal(t, io.EOF, err)
// Second bisect with a new bisector (simulating retry with reset)
cb2 := newColumnBisector(downscorer.downscoreCall)
cb2.addPeerColumns(pid, cols...)
iter, err = cb2.Bisect(cols)
require.NoError(t, err)
batch, err = iter.Next()
require.NoError(t, err)
require.Equal(t, 1, len(batch))
}
// TestBisectEmptyColumns tests Bisect with empty column list
func TestBisectEmptyColumns(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
var emptyColumns []util.DataColumnParam
roColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, emptyColumns)
iter, err := cb.Bisect(roColumns)
// This should not error with empty columns
if err == nil {
_, err := iter.Next()
require.Equal(t, io.EOF, err)
}
}
// TestCompleteFailureFlow tests marking a peer as failed and tracking failure roots
func TestCompleteFailureFlow(t *testing.T) {
downscorer := &mockDownscorer{}
cb := newColumnBisector(downscorer.downscoreCall)
root1 := [32]byte{1, 0, 0}
root2 := [32]byte{2, 0, 0}
root3 := [32]byte{3, 0, 0}
pid1 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMq")
pid2 := createTestPeerID(t, "QmYyQSo1c1Ym7orWxLYvCrM2EmxFTSc34pP8r3hidQPQMr")
// Create columns: pid1 provides columns for root1 and root2, pid2 provides for root3
params1 := []util.DataColumnParam{
createTestDataColumn(t, root1, 0, 2),
createTestDataColumn(t, root2, 1, 2),
}
params2 := []util.DataColumnParam{
createTestDataColumn(t, root3, 2, 2),
}
cols1, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, params1)
cols2, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, params2)
cb.addPeerColumns(pid1, cols1...)
cb.addPeerColumns(pid2, cols2...)
allCols := append(cols1, cols2...)
iter, err := cb.Bisect(allCols)
require.NoError(t, err)
// sort the pidIter so that the test is deterministic
slices.Sort(cb.pidIter)
batch1, err := iter.Next()
require.NoError(t, err)
require.Equal(t, 2, len(batch1))
// Determine which peer the first batch came from
firstBatchPeer := batch1[0]
colPeer, err := cb.peerFor(firstBatchPeer)
require.NoError(t, err)
expectedPeer := colPeer
// Extract the roots from batch1 to ensure we can track them
rootsInBatch1 := make(map[[32]byte]bool)
for _, col := range batch1 {
rootsInBatch1[col.BlockRoot()] = true
}
// Mark the first batch's peer as failed
cb.OnError(errors.New("peer verification failed"))
// Verify downscorer was called for the peer that had the first batch
require.Equal(t, 1, len(downscorer.calls))
require.Equal(t, expectedPeer, downscorer.calls[0].pid)
// Verify that failures contains the roots from batch1
require.Equal(t, len(rootsInBatch1), len(cb.failingRoots()))
// Get remaining batches until EOF
batch2, err := iter.Next()
require.NoError(t, err)
require.Equal(t, 1, len(batch2))
_, err = iter.Next()
require.Equal(t, io.EOF, err)
// Verify failingRoots matches the roots from the failed batch
failingRoots := cb.failingRoots()
require.Equal(t, len(rootsInBatch1), len(failingRoots))
// Verify the failing roots are exactly the ones from batch1
failingRootsMap := make(map[[32]byte]bool)
for _, root := range failingRoots {
failingRootsMap[root] = true
}
for root := range rootsInBatch1 {
require.Equal(t, true, failingRootsMap[root])
}
}

View File

@@ -5,19 +5,63 @@ import (
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/runtime/interop"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/ethereum/go-ethereum/common/hexutil"
)
func mockCurrentNeeds(begin, end primitives.Slot) das.CurrentNeeds {
return das.CurrentNeeds{
Block: das.NeedSpan{
Begin: begin,
End: end,
},
Blob: das.NeedSpan{
Begin: begin,
End: end,
},
Col: das.NeedSpan{
Begin: begin,
End: end,
},
}
}
func mockCurrentSpecNeeds() das.CurrentNeeds {
cfg := params.BeaconConfig()
fuluSlot := slots.UnsafeEpochStart(cfg.FuluForkEpoch)
denebSlot := slots.UnsafeEpochStart(cfg.DenebForkEpoch)
return das.CurrentNeeds{
Block: das.NeedSpan{
Begin: 0,
End: primitives.Slot(math.MaxUint64),
},
Blob: das.NeedSpan{
Begin: denebSlot,
End: fuluSlot,
},
Col: das.NeedSpan{
Begin: fuluSlot,
End: primitives.Slot(math.MaxUint64),
},
}
}
func mockCurrentNeedsFunc(begin, end primitives.Slot) func() das.CurrentNeeds {
return func() das.CurrentNeeds {
return mockCurrentNeeds(begin, end)
}
}
func TestDomainCache(t *testing.T) {
cfg := params.MainnetConfig()
// This hack is needed not to have both Electra and Fulu fork epoch both set to the future max epoch.
@@ -70,12 +114,7 @@ func TestVerify(t *testing.T) {
}
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)
vbs, err := v.verify(blks)
require.NoError(t, err)
require.Equal(t, len(blks), len(vbs))
}

View File

@@ -4,14 +4,63 @@ import (
"context"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync"
"github.com/OffchainLabs/prysm/v7/beacon-chain/verification"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
var errInvalidBatchState = errors.New("invalid batch state")
type peerDownscorer func(peer.ID, string, error)
type workerCfg struct {
clock *startup.Clock
verifier *verifier
ctxMap sync.ContextByteVersions
newVB verification.NewBlobVerifier
newVC verification.NewDataColumnsVerifier
blobStore *filesystem.BlobStorage
colStore *filesystem.DataColumnStorage
downscore peerDownscorer
currentNeeds func() das.CurrentNeeds
}
func initWorkerCfg(ctx context.Context, cfg *workerCfg, vw InitializerWaiter, store *Store) error {
vi, err := vw.WaitForInitializer(ctx)
if err != nil {
return errors.Wrap(err, "WaitForInitializer")
}
cps, err := store.originState(ctx)
if err != nil {
return errors.Wrap(err, "originState")
}
keys, err := cps.PublicKeys()
if err != nil {
return errors.Wrap(err, "unable to retrieve public keys for all validators in the origin state")
}
vr := cps.GenesisValidatorsRoot()
cm, err := sync.ContextByteVersionsForValRoot(bytesutil.ToBytes32(vr))
if err != nil {
return errors.Wrapf(err, "unable to initialize context version map using genesis validator root %#x", vr)
}
v, err := newBackfillVerifier(vr, keys)
if err != nil {
return errors.Wrapf(err, "newBackfillVerifier failed")
}
cfg.verifier = v
cfg.ctxMap = cm
cfg.newVB = newBlobVerifierFromInitializer(vi)
cfg.newVC = newDataColumnVerifierFromInitializer(vi)
return nil
}
type workerId int
type p2pWorker struct {
@@ -19,98 +68,176 @@ type p2pWorker struct {
todo chan batch
done chan batch
p2p p2p.P2P
v *verifier
c *startup.Clock
cm sync.ContextByteVersions
nbv verification.NewBlobVerifier
bfs *filesystem.BlobStorage
cfg *workerCfg
}
func newP2pWorker(id workerId, p p2p.P2P, todo, done chan batch, cfg *workerCfg) *p2pWorker {
return &p2pWorker{
id: id,
todo: todo,
done: done,
p2p: p,
cfg: cfg,
}
}
func (w *p2pWorker) run(ctx context.Context) {
for {
select {
case b := <-w.todo:
log.WithFields(b.logFields()).WithField("backfillWorker", w.id).Debug("Backfill worker received batch")
if b.state == batchBlobSync {
w.done <- w.handleBlobs(ctx, b)
} else {
w.done <- w.handleBlocks(ctx, b)
if err := b.waitUntilReady(ctx); err != nil {
log.WithField("batchId", b.id()).WithError(ctx.Err()).Info("Worker context canceled while waiting to retry")
continue
}
log.WithFields(b.logFields()).WithField("backfillWorker", w.id).Trace("Worker received batch")
switch b.state {
case batchSequenced:
b = w.handleBlocks(ctx, b)
case batchSyncBlobs:
b = w.handleBlobs(ctx, b)
case batchSyncColumns:
b = w.handleColumns(ctx, b)
case batchImportable:
// This state indicates the batch got all the way to be imported and failed,
// so we need clear out the blocks to go all the way back to the start of the process.
b.blocks = nil
b = w.handleBlocks(ctx, b)
default:
// A batch in an unknown state represents an implementation error,
// so we treat it as a fatal error meaning the worker pool should shut down.
b = b.withFatalError(errors.Wrap(errInvalidBatchState, b.state.String()))
}
w.done <- b
case <-ctx.Done():
log.WithField("backfillWorker", w.id).Info("Backfill worker exiting after context canceled")
log.WithField("backfillWorker", w.id).Info("Worker exiting after context canceled")
return
}
}
}
func (w *p2pWorker) handleBlocks(ctx context.Context, b batch) batch {
cs := w.c.CurrentSlot()
blobRetentionStart, err := sync.BlobRPCMinValidSlot(cs)
if err != nil {
return b.withRetryableError(errors.Wrap(err, "configuration issue, could not compute minimum blob retention slot"))
}
b.blockPid = b.busy
current := w.cfg.clock.CurrentSlot()
b.blockPeer = b.assignedPeer
start := time.Now()
results, err := sync.SendBeaconBlocksByRangeRequest(ctx, w.c, w.p2p, b.blockPid, b.blockRequest(), blockValidationMetrics)
results, err := sync.SendBeaconBlocksByRangeRequest(ctx, w.cfg.clock, w.p2p, b.blockPeer, b.blockRequest(), blockValidationMetrics)
if err != nil {
log.WithError(err).WithFields(b.logFields()).Debug("Failed to request SignedBeaconBlocks by range")
return b.withRetryableError(err)
}
dlt := time.Now()
backfillBatchTimeDownloadingBlocks.Observe(float64(dlt.Sub(start).Milliseconds()))
blockDownloadMs.Observe(float64(dlt.Sub(start).Milliseconds()))
toVerify, err := blocks.NewROBlockSlice(results)
if err != nil {
log.WithError(err).WithFields(b.logFields()).Debug("Batch requesting failed")
log.WithError(err).WithFields(b.logFields()).Debug("Failed to convert blocks to ROBlock slice")
return b.withRetryableError(err)
}
vb, err := w.v.verify(results)
backfillBatchTimeVerifying.Observe(float64(time.Since(dlt).Milliseconds()))
verified, err := w.cfg.verifier.verify(toVerify)
blockVerifyMs.Observe(float64(time.Since(dlt).Milliseconds()))
if err != nil {
log.WithError(err).WithFields(b.logFields()).Debug("Batch validation failed")
if shouldDownscore(err) {
w.cfg.downscore(b.blockPeer, "invalid SignedBeaconBlock batch rpc response", err)
}
log.WithError(err).WithFields(b.logFields()).Debug("Validation failed")
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()
for i := range verified {
bdl += verified[i].SizeSSZ()
}
backfillBlocksApproximateBytes.Add(float64(bdl))
log.WithFields(b.logFields()).WithField("dlbytes", bdl).Debug("Backfill batch block bytes downloaded")
bs, err := newBlobSync(cs, vb, &blobSyncConfig{retentionStart: blobRetentionStart, nbv: w.nbv, store: w.bfs})
blockDownloadBytesApprox.Add(float64(bdl))
log.WithFields(b.logFields()).WithField("bytesDownloaded", bdl).Trace("Blocks downloaded")
b.blocks = verified
bscfg := &blobSyncConfig{currentNeeds: w.cfg.currentNeeds, nbv: w.cfg.newVB, store: w.cfg.blobStore}
bs, err := newBlobSync(current, verified, bscfg)
if err != nil {
return b.withRetryableError(err)
}
return b.withResults(vb, bs)
cs, err := newColumnSync(ctx, b, verified, current, w.p2p, w.cfg)
if err != nil {
return b.withRetryableError(err)
}
b.blobs = bs
b.columns = cs
return b.transitionToNext()
}
func (w *p2pWorker) handleBlobs(ctx context.Context, b batch) batch {
b.blobPid = b.busy
b.blobs.peer = b.assignedPeer
start := time.Now()
// we don't need to use the response for anything other than metrics, because blobResponseValidation
// adds each of them to a batch AvailabilityStore once it is checked.
blobs, err := sync.SendBlobsByRangeRequest(ctx, w.c, w.p2p, b.blobPid, w.cm, b.blobRequest(), b.blobResponseValidator(), blobValidationMetrics)
blobs, err := sync.SendBlobsByRangeRequest(ctx, w.cfg.clock, w.p2p, b.blobs.peer, w.cfg.ctxMap, b.blobRequest(), b.blobs.validateNext, blobValidationMetrics)
if err != nil {
b.bs = nil
b.blobs = nil
return b.withRetryableError(err)
}
dlt := time.Now()
backfillBatchTimeDownloadingBlobs.Observe(float64(dlt.Sub(start).Milliseconds()))
blobSidecarDownloadMs.Observe(float64(dlt.Sub(start).Milliseconds()))
if len(blobs) > 0 {
// All blobs are the same size, so we can compute 1 and use it for all in the batch.
sz := blobs[0].SizeSSZ() * len(blobs)
backfillBlobsApproximateBytes.Add(float64(sz))
log.WithFields(b.logFields()).WithField("dlbytes", sz).Debug("Backfill batch blob bytes downloaded")
blobSidecarDownloadBytesApprox.Add(float64(sz))
log.WithFields(b.logFields()).WithField("bytesDownloaded", sz).Debug("Blobs downloaded")
}
return b.postBlobSync()
if b.blobs.needed() > 0 {
// If we are missing blobs after processing the blob step, this is an error and we need to scrap the batch and start over.
b.blobs = nil
// Wipe retries periodically to avoid getting stuck on a bad block batch
if b.retries%3 == 0 {
b.blocks = []blocks.ROBlock{}
}
return b.withRetryableError(errors.New("missing blobs after blob download"))
}
return b.transitionToNext()
}
func newP2pWorker(id workerId, p p2p.P2P, todo, done chan batch, c *startup.Clock, v *verifier, cm sync.ContextByteVersions, nbv verification.NewBlobVerifier, bfs *filesystem.BlobStorage) *p2pWorker {
return &p2pWorker{
id: id,
todo: todo,
done: done,
p2p: p,
v: v,
c: c,
cm: cm,
nbv: nbv,
bfs: bfs,
func (w *p2pWorker) handleColumns(ctx context.Context, b batch) batch {
start := time.Now()
b.columns.peer = b.assignedPeer
// Bisector is used to keep track of the peer that provided each column, for scoring purposes.
// When verification of a batch of columns fails, bisector is used to retry verification with batches
// grouped by peer, to figure out if the failure is due to a specific peer.
vr, err := b.validatingColumnRequest(b.columns.bisector)
if err != nil {
return b.withRetryableError(errors.Wrap(err, "creating validating column request"))
}
p := sync.DataColumnSidecarsParams{
Ctx: ctx,
Tor: w.cfg.clock,
P2P: w.p2p,
CtxMap: w.cfg.ctxMap,
// DownscorePeerOnRPCFault is very aggressive and is only used for fetching origin blobs during startup.
DownscorePeerOnRPCFault: false,
// SendDataColumnSidecarsByRangeRequest uses the DataColumnSidecarsParams param struct to cover
// multiple different use cases. Some of them have different required fields. The following fields are
// not used in the methods that backfill invokes. SendDataColumnSidecarsByRangeRequest should be refactored
// to only require the minimum set of parameters.
//RateLimiter *leakybucket.Collector
//Storage: w.cfg.cfs,
//NewVerifier: vr.validate,
}
// The return is dropped because the validation code adds the columns
// to the columnSync AvailabilityStore under the hood.
_, err = sync.SendDataColumnSidecarsByRangeRequest(p, b.columns.peer, vr.req, vr.validate)
if err != nil {
if shouldDownscore(err) {
w.cfg.downscore(b.columns.peer, "invalid DataColumnSidecar rpc response", err)
}
return b.withRetryableError(errors.Wrap(err, "failed to request data column sidecars"))
}
dataColumnSidecarDownloadMs.Observe(float64(time.Since(start).Milliseconds()))
return b.transitionToNext()
}
func shouldDownscore(err error) bool {
return errors.Is(err, errInvalidDataColumnResponse) ||
errors.Is(err, sync.ErrInvalidFetchedData) ||
errors.Is(err, errInvalidBlocks)
}

View File

@@ -0,0 +1,247 @@
package sync
import (
"cmp"
"math"
"slices"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
)
var (
// ErrNoPeersCoverNeeded is returned when no peers are able to cover the needed columns.
ErrNoPeersCoverNeeded = errors.New("no peers able to cover needed columns")
// ErrNoPeersAvailable is returned when no peers are available for block requests.
ErrNoPeersAvailable = errors.New("no peers available")
)
// DASPeerCache caches information about a set of peers DAS peering decisions.
type DASPeerCache struct {
p2pSvc p2p.P2P
peers map[peer.ID]*dasPeer
}
// dasPeer represents a peer's custody of columns and their coverage score.
type dasPeer struct {
pid peer.ID
enid enode.ID
custodied peerdas.ColumnIndices
lastAssigned time.Time
}
// dasPeerScore is used to build a slice of peer+score pairs for ranking purproses.
type dasPeerScore struct {
peer *dasPeer
score float64
}
// PeerPicker is a structure that maps out the intersection of peer custody and column indices
// to weight each peer based on the scarcity of the columns they custody. This allows us to prioritize
// requests for more scarce columns to peers that custody them, so that we don't waste our bandwidth allocation
// making requests for more common columns from peers that can provide the more scarce columns.
type PeerPicker struct {
scores []*dasPeerScore // scores is a set of generic scores, based on the full custody group set
ranker *rarityRanker
custodians map[uint64][]*dasPeer
toCustody peerdas.ColumnIndices // full set of columns this node will try to custody
reqInterval time.Duration
}
// NewDASPeerCache initializes a DASPeerCache. This type is not currently thread safe.
func NewDASPeerCache(p2pSvc p2p.P2P) *DASPeerCache {
return &DASPeerCache{
peers: make(map[peer.ID]*dasPeer),
p2pSvc: p2pSvc,
}
}
// NewColumnScarcityRanking computes the ColumnScarcityRanking based on the current view of columns custodied
// by the given set of peers. New PeerPickers should be created somewhat frequently, as the status of peers can
// change, including the set of columns each peer custodies.
// reqInterval sets the frequency that a peer can be picked in terms of time. A peer can be picked once per reqInterval,
// eg a value of time.Second would allow 1 request per second to the peer, or a value of 500 * time.Millisecond would allow
// 2 req/sec.
func (c *DASPeerCache) NewPicker(pids []peer.ID, toCustody peerdas.ColumnIndices, reqInterval time.Duration) (*PeerPicker, error) {
// For each of the given peers, refresh the cache's view of their currently custodied columns.
// Also populate 'custodians', which stores the set of peers that custody each column index.
custodians := make(map[uint64][]*dasPeer, len(toCustody))
scores := make([]*dasPeerScore, 0, len(pids))
for _, pid := range pids {
peer, err := c.refresh(pid, toCustody)
if err != nil {
log.WithField("peerID", pid).WithError(err).Debug("Failed to convert peer ID to node ID.")
continue
}
for col := range peer.custodied {
if toCustody.Has(col) {
custodians[col] = append(custodians[col], peer)
}
}
// set score to math.MaxFloat64 so we can tell that it hasn't been initialized
scores = append(scores, &dasPeerScore{peer: peer, score: math.MaxFloat64})
}
return &PeerPicker{
toCustody: toCustody,
ranker: newRarityRanker(toCustody, custodians),
custodians: custodians,
scores: scores,
reqInterval: reqInterval,
}, nil
}
// refresh supports NewPicker in getting the latest dasPeer view for the given peer.ID. It caches the result
// of the enode.ID computation but refreshes the custody group count each time it is called, leveraging the
// cache behind peerdas.Info.
func (c *DASPeerCache) refresh(pid peer.ID, toCustody peerdas.ColumnIndices) (*dasPeer, error) {
// Computing the enode.ID seems to involve multiple parseing and validation steps followed by a
// hash computation, so it seems worth trying to cache the result.
p, ok := c.peers[pid]
if !ok {
nodeID, err := p2p.ConvertPeerIDToNodeID(pid)
if err != nil {
// If we can't convert the peer ID to a node ID, remove peer from the cache.
delete(c.peers, pid)
return nil, errors.Wrap(err, "ConvertPeerIDToNodeID")
}
p = &dasPeer{enid: nodeID, pid: pid}
}
if len(toCustody) > 0 {
dasInfo, _, err := peerdas.Info(p.enid, c.p2pSvc.CustodyGroupCountFromPeer(pid))
if err != nil {
// If we can't get the peerDAS info, remove peer from the cache.
delete(c.peers, pid)
return nil, errors.Wrapf(err, "CustodyGroupCountFromPeer, peerID=%s, nodeID=%s", pid, p.enid)
}
p.custodied = peerdas.NewColumnIndicesFromMap(dasInfo.CustodyColumns)
} else {
p.custodied = peerdas.NewColumnIndices()
}
c.peers[pid] = p
return p, nil
}
// ForColumns returns the best peer to request columns from, based on the scarcity of the columns needed.
func (m *PeerPicker) ForColumns(needed peerdas.ColumnIndices, busy map[peer.ID]bool) (peer.ID, []uint64, error) {
// - find the custodied column with the lowest frequency
// - collect all the peers that have custody of that column
// - score the peers by the rarity of the needed columns they offer
var best *dasPeer
bestScore, bestCoverage := 0.0, []uint64{}
for _, col := range m.ranker.ascendingRarity(needed) {
for _, p := range m.custodians[col] {
// enforce a minimum interval between requests to the same peer
if p.lastAssigned.Add(m.reqInterval).After(time.Now()) {
continue
}
if busy[p.pid] {
continue
}
covered := p.custodied.Intersection(needed)
if len(covered) == 0 {
continue
}
// update best if any of the following:
// - current score better than previous best
// - scores are tied, and current coverage is better than best
// - scores are tied, coverage equal, pick the least-recently used peer
score := m.ranker.score(covered)
if score < bestScore {
continue
}
if score == bestScore && best != nil {
if len(covered) < len(bestCoverage) {
continue
}
if len(covered) == len(bestCoverage) && best.lastAssigned.Before(p.lastAssigned) {
continue
}
}
best, bestScore, bestCoverage = p, score, covered.ToSlice()
}
if best != nil {
best.lastAssigned = time.Now()
slices.Sort(bestCoverage)
return best.pid, bestCoverage, nil
}
}
return "", nil, ErrNoPeersCoverNeeded
}
// ForBlocks returns the lowest scoring peer in the set. This can be used to pick a peer
// for block requests, preserving the peers that have the highest coverage scores
// for column requests.
func (m *PeerPicker) ForBlocks(busy map[peer.ID]bool) (peer.ID, error) {
slices.SortFunc(m.scores, func(a, b *dasPeerScore) int {
// MaxFloat64 is used as a sentinel value for an uninitialized score;
// check and set scores while sorting for uber-lazy initialization.
if a.score == math.MaxFloat64 {
a.score = m.ranker.score(a.peer.custodied.Intersection(m.toCustody))
}
if b.score == math.MaxFloat64 {
b.score = m.ranker.score(b.peer.custodied.Intersection(m.toCustody))
}
return cmp.Compare(a.score, b.score)
})
for _, ds := range m.scores {
if !busy[ds.peer.pid] {
return ds.peer.pid, nil
}
}
return "", ErrNoPeersAvailable
}
// rarityRanker is initialized with the set of columns this node needs to custody, and the set of
// all peer custody columns. With that information it is able to compute a numeric representation of
// column rarity, and use that number to give each peer a score that represents how fungible their
// bandwidth likely is relative to other peers given a more specific set of needed columns.
type rarityRanker struct {
// rarity maps column indices to their rarity scores.
// The rarity score is defined as the inverse of the number of custodians: 1/custodians
// So the rarity of the columns a peer custodies can be simply added together for a score
// representing how unique their custody groups are; rarer columns contribute larger values to scores.
rarity map[uint64]float64
asc []uint64 // columns indices ordered by ascending rarity
}
// newRarityRanker precomputes data used for scoring and ranking. It should be reinitialized every time
// we refresh the set of peers or the view of the peers column custody.
func newRarityRanker(toCustody peerdas.ColumnIndices, custodians map[uint64][]*dasPeer) *rarityRanker {
rarity := make(map[uint64]float64, len(toCustody))
asc := make([]uint64, 0, len(toCustody))
for col := range toCustody.ToMap() {
rarity[col] = 1 / max(1, float64(len(custodians[col])))
asc = append(asc, col)
}
slices.SortFunc(asc, func(a, b uint64) int {
return cmp.Compare(rarity[a], rarity[b])
})
return &rarityRanker{rarity: rarity, asc: asc}
}
// rank returns the requested columns sorted by ascending rarity.
func (rr *rarityRanker) ascendingRarity(cols peerdas.ColumnIndices) []uint64 {
ranked := make([]uint64, 0, len(cols))
for _, col := range rr.asc {
if cols.Has(col) {
ranked = append(ranked, col)
}
}
return ranked
}
// score gives a score representing the sum of the rarity scores of the given columns. It can be used to
// score peers based on the set intersection of their custodied indices and the indices we need to request.
func (rr *rarityRanker) score(coverage peerdas.ColumnIndices) float64 {
score := 0.0
for col := range coverage.ToMap() {
score += rr.rarity[col]
}
return score
}

View File

@@ -0,0 +1,295 @@
package sync
import (
"testing"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/libp2p/go-libp2p"
"github.com/libp2p/go-libp2p/core/crypto"
"github.com/libp2p/go-libp2p/core/peer"
)
// mockP2PForDAS wraps TestP2P to provide a known custody group count for any peer.
type mockP2PForDAS struct {
*p2ptest.TestP2P
custodyGroupCount uint64
}
func (m *mockP2PForDAS) CustodyGroupCountFromPeer(_ peer.ID) uint64 {
return m.custodyGroupCount
}
// testDASSetup provides test fixtures for DAS peer assignment tests.
type testDASSetup struct {
t *testing.T
cache *DASPeerCache
p2pService *mockP2PForDAS
peers []*p2ptest.TestP2P
peerIDs []peer.ID
}
// createSecp256k1Key generates a secp256k1 private key from a seed offset.
// These keys are compatible with ConvertPeerIDToNodeID.
func createSecp256k1Key(offset int) crypto.PrivKey {
privateKeyBytes := make([]byte, 32)
for i := range 32 {
privateKeyBytes[i] = byte(offset + i)
}
privKey, err := crypto.UnmarshalSecp256k1PrivateKey(privateKeyBytes)
if err != nil {
panic(err)
}
return privKey
}
// setupDASTest creates a test setup with the specified number of connected peers.
// custodyGroupCount is the custody count returned for all peers.
func setupDASTest(t *testing.T, peerCount int, custodyGroupCount uint64) *testDASSetup {
params.SetupTestConfigCleanup(t)
// Create main p2p service with secp256k1 key
testP2P := p2ptest.NewTestP2P(t, libp2p.Identity(createSecp256k1Key(0)))
mockP2P := &mockP2PForDAS{
TestP2P: testP2P,
custodyGroupCount: custodyGroupCount,
}
cache := NewDASPeerCache(mockP2P)
peers := make([]*p2ptest.TestP2P, peerCount)
peerIDs := make([]peer.ID, peerCount)
for i := range peerCount {
// Use offset starting at 100 to avoid collision with main p2p service
peers[i] = p2ptest.NewTestP2P(t, libp2p.Identity(createSecp256k1Key(100+i*50)))
peers[i].Connect(testP2P)
peerIDs[i] = peers[i].PeerID()
}
return &testDASSetup{
t: t,
cache: cache,
p2pService: mockP2P,
peers: peers,
peerIDs: peerIDs,
}
}
// getActualCustodyColumns returns the columns actually custodied by the test peers.
// This queries the same peerdas.Info logic used by the production code.
func (s *testDASSetup) getActualCustodyColumns() peerdas.ColumnIndices {
result := peerdas.NewColumnIndices()
custodyCount := s.p2pService.custodyGroupCount
for _, pid := range s.peerIDs {
nodeID, err := p2p.ConvertPeerIDToNodeID(pid)
if err != nil {
continue
}
info, _, err := peerdas.Info(nodeID, custodyCount)
if err != nil {
continue
}
for col := range info.CustodyColumns {
result.Set(col)
}
}
return result
}
func TestNewPicker(t *testing.T) {
custodyReq := params.BeaconConfig().CustodyRequirement
t.Run("valid peers with columns", func(t *testing.T) {
setup := setupDASTest(t, 3, custodyReq)
toCustody := setup.getActualCustodyColumns()
require.NotEqual(t, 0, toCustody.Count(), "test peers should custody some columns")
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
require.NotNil(t, picker)
require.Equal(t, 3, len(picker.scores))
})
t.Run("empty peer list", func(t *testing.T) {
setup := setupDASTest(t, 0, custodyReq)
toCustody := peerdas.NewColumnIndicesFromSlice([]uint64{0, 1})
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
require.NotNil(t, picker)
require.Equal(t, 0, len(picker.scores))
})
t.Run("empty custody columns", func(t *testing.T) {
setup := setupDASTest(t, 2, custodyReq)
toCustody := peerdas.NewColumnIndices()
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
require.NotNil(t, picker)
// With empty toCustody, peers are still added to scores but have no custodied columns
require.Equal(t, 2, len(picker.scores))
})
}
func TestForColumns(t *testing.T) {
custodyReq := params.BeaconConfig().CustodyRequirement
t.Run("basic selection returns covering peer", func(t *testing.T) {
setup := setupDASTest(t, 3, custodyReq)
toCustody := setup.getActualCustodyColumns()
require.NotEqual(t, 0, toCustody.Count(), "test peers must custody some columns")
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
// Request columns that we know peers custody
needed := toCustody
pid, cols, err := picker.ForColumns(needed, nil)
require.NoError(t, err)
require.NotEmpty(t, pid)
require.NotEmpty(t, cols)
// Verify the returned columns are a subset of what was needed
for _, col := range cols {
require.Equal(t, true, needed.Has(col), "returned column should be in needed set")
}
})
t.Run("skip busy peers", func(t *testing.T) {
setup := setupDASTest(t, 2, custodyReq)
toCustody := setup.getActualCustodyColumns()
require.NotEqual(t, 0, toCustody.Count(), "test peers must custody some columns")
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
// Mark first peer as busy
busy := map[peer.ID]bool{setup.peerIDs[0]: true}
pid, _, err := picker.ForColumns(toCustody, busy)
require.NoError(t, err)
// Should not return the busy peer
require.NotEqual(t, setup.peerIDs[0], pid)
})
t.Run("rate limiting respects reqInterval", func(t *testing.T) {
setup := setupDASTest(t, 1, custodyReq)
toCustody := setup.getActualCustodyColumns()
require.NotEqual(t, 0, toCustody.Count(), "test peers must custody some columns")
// Use a long interval so the peer can't be picked twice
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Hour)
require.NoError(t, err)
// First call should succeed
pid, _, err := picker.ForColumns(toCustody, nil)
require.NoError(t, err)
require.NotEmpty(t, pid)
// Second call should fail due to rate limiting
_, _, err = picker.ForColumns(toCustody, nil)
require.ErrorIs(t, err, ErrNoPeersCoverNeeded)
})
t.Run("no peers available returns error", func(t *testing.T) {
setup := setupDASTest(t, 2, custodyReq)
toCustody := setup.getActualCustodyColumns()
require.NotEqual(t, 0, toCustody.Count(), "test peers must custody some columns")
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
// Mark all peers as busy
busy := map[peer.ID]bool{
setup.peerIDs[0]: true,
setup.peerIDs[1]: true,
}
_, _, err = picker.ForColumns(toCustody, busy)
require.ErrorIs(t, err, ErrNoPeersCoverNeeded)
})
t.Run("empty needed columns returns error", func(t *testing.T) {
setup := setupDASTest(t, 2, custodyReq)
toCustody := setup.getActualCustodyColumns()
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
// Request empty set of columns
needed := peerdas.NewColumnIndices()
_, _, err = picker.ForColumns(needed, nil)
require.ErrorIs(t, err, ErrNoPeersCoverNeeded)
})
}
func TestForBlocks(t *testing.T) {
custodyReq := params.BeaconConfig().CustodyRequirement
t.Run("returns available peer", func(t *testing.T) {
setup := setupDASTest(t, 3, custodyReq)
toCustody := setup.getActualCustodyColumns()
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
pid, err := picker.ForBlocks(nil)
require.NoError(t, err)
require.NotEmpty(t, pid)
})
t.Run("skips busy peers", func(t *testing.T) {
setup := setupDASTest(t, 3, custodyReq)
toCustody := setup.getActualCustodyColumns()
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
// Mark first two peers as busy
busy := map[peer.ID]bool{
setup.peerIDs[0]: true,
setup.peerIDs[1]: true,
}
pid, err := picker.ForBlocks(busy)
require.NoError(t, err)
require.NotEmpty(t, pid)
// Verify returned peer is not busy
require.Equal(t, false, busy[pid], "returned peer should not be busy")
})
t.Run("all peers busy returns error", func(t *testing.T) {
setup := setupDASTest(t, 2, custodyReq)
toCustody := setup.getActualCustodyColumns()
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
// Mark all peers as busy
busy := map[peer.ID]bool{
setup.peerIDs[0]: true,
setup.peerIDs[1]: true,
}
_, err = picker.ForBlocks(busy)
require.ErrorIs(t, err, ErrNoPeersAvailable)
})
t.Run("no peers returns error", func(t *testing.T) {
setup := setupDASTest(t, 0, custodyReq)
toCustody := peerdas.NewColumnIndicesFromSlice([]uint64{0, 1, 2, 3})
picker, err := setup.cache.NewPicker(setup.peerIDs, toCustody, time.Millisecond)
require.NoError(t, err)
_, err = picker.ForBlocks(nil)
require.ErrorIs(t, err, ErrNoPeersAvailable)
})
}

View File

@@ -69,9 +69,10 @@ func (f *blocksFetcher) waitForMinimumPeers(ctx context.Context) ([]peer.ID, err
}
var peers []peer.ID
if f.mode == modeStopOnFinalizedEpoch {
cp := f.chain.FinalizedCheckpt()
headEpoch := cp.Epoch
_, peers = f.p2p.Peers().BestFinalized(params.BeaconConfig().MaxPeersToSync, headEpoch)
_, peers = f.p2p.Peers().BestFinalized(f.chain.FinalizedCheckpt().Epoch)
if len(peers) > params.BeaconConfig().MaxPeersToSync {
peers = peers[:params.BeaconConfig().MaxPeersToSync]
}
} else {
headEpoch := slots.ToEpoch(f.chain.HeadSlot())
_, peers = f.p2p.Peers().BestNonFinalized(flags.Get().MinimumSyncPeers, headEpoch)

View File

@@ -516,7 +516,10 @@ func TestBlocksFetcher_requestBeaconBlocksByRange(t *testing.T) {
p2p: p2p,
})
_, peerIDs := p2p.Peers().BestFinalized(params.BeaconConfig().MaxPeersToSync, slots.ToEpoch(mc.HeadSlot()))
_, peerIDs := p2p.Peers().BestFinalized(slots.ToEpoch(mc.HeadSlot()))
if len(peerIDs) > params.BeaconConfig().MaxPeersToSync {
peerIDs = peerIDs[:params.BeaconConfig().MaxPeersToSync]
}
req := &ethpb.BeaconBlocksByRangeRequest{
StartSlot: 1,
Step: 1,

View File

@@ -331,9 +331,7 @@ func (f *blocksFetcher) findAncestor(ctx context.Context, pid peer.ID, b interfa
// bestFinalizedSlot returns the highest finalized slot of the majority of connected peers.
func (f *blocksFetcher) bestFinalizedSlot() primitives.Slot {
cp := f.chain.FinalizedCheckpt()
finalizedEpoch, _ := f.p2p.Peers().BestFinalized(
params.BeaconConfig().MaxPeersToSync, cp.Epoch)
finalizedEpoch, _ := f.p2p.Peers().BestFinalized(f.chain.FinalizedCheckpt().Epoch)
return params.BeaconConfig().SlotsPerEpoch.Mul(uint64(finalizedEpoch))
}
@@ -350,7 +348,10 @@ func (f *blocksFetcher) calculateHeadAndTargetEpochs() (headEpoch, targetEpoch p
if f.mode == modeStopOnFinalizedEpoch {
cp := f.chain.FinalizedCheckpt()
headEpoch = cp.Epoch
targetEpoch, peers = f.p2p.Peers().BestFinalized(params.BeaconConfig().MaxPeersToSync, headEpoch)
targetEpoch, peers = f.p2p.Peers().BestFinalized(headEpoch)
if len(peers) > params.BeaconConfig().MaxPeersToSync {
peers = peers[:params.BeaconConfig().MaxPeersToSync]
}
return headEpoch, targetEpoch, peers
}

View File

@@ -28,10 +28,10 @@ const (
)
// blockReceiverFn defines block receiving function.
type blockReceiverFn func(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityStore) error
type blockReceiverFn func(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityChecker) error
// batchBlockReceiverFn defines batch receiving function.
type batchBlockReceiverFn func(ctx context.Context, blks []blocks.ROBlock, avs das.AvailabilityStore) error
type batchBlockReceiverFn func(ctx context.Context, blks []blocks.ROBlock, avs das.AvailabilityChecker) error
// Round Robin sync looks at the latest peer statuses and syncs up to the highest known epoch.
//
@@ -175,7 +175,7 @@ func (s *Service) processFetchedDataRegSync(ctx context.Context, data *blocksQue
blocksWithDataColumns := bwb[fistDataColumnIndex:]
blobBatchVerifier := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncBlobSidecarRequirements)
lazilyPersistentStoreBlobs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, blobBatchVerifier)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, blobBatchVerifier, s.blobRetentionChecker)
log := log.WithField("firstSlot", data.bwb[0].Block.Block().Slot())
logBlobs, logDataColumns := log, log
@@ -185,12 +185,12 @@ func (s *Service) processFetchedDataRegSync(ctx context.Context, data *blocksQue
}
for i, b := range blocksWithBlobs {
if err := lazilyPersistentStoreBlobs.Persist(s.clock.CurrentSlot(), b.Blobs...); err != nil {
if err := avs.Persist(s.clock.CurrentSlot(), b.Blobs...); err != nil {
logBlobs.WithError(err).WithFields(syncFields(b.Block)).Warning("Batch failure due to BlobSidecar issues")
return uint64(i), err
}
if err := s.processBlock(ctx, s.genesisTime, b, s.cfg.Chain.ReceiveBlock, lazilyPersistentStoreBlobs); err != nil {
if err := s.processBlock(ctx, s.genesisTime, b, s.cfg.Chain.ReceiveBlock, avs); err != nil {
if errors.Is(err, errParentDoesNotExist) {
logBlobs.WithField("missingParent", fmt.Sprintf("%#x", b.Block.Block().ParentRoot())).
WithFields(syncFields(b.Block)).Debug("Could not process batch blocks due to missing parent")
@@ -313,7 +313,7 @@ func (s *Service) processBlock(
genesis time.Time,
bwb blocks.BlockWithROSidecars,
blockReceiver blockReceiverFn,
avs das.AvailabilityStore,
avs das.AvailabilityChecker,
) error {
blk := bwb.Block
blkRoot := blk.Root()
@@ -404,7 +404,7 @@ func (s *Service) processBlocksWithBlobs(ctx context.Context, bwbs []blocks.Bloc
}
batchVerifier := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncBlobSidecarRequirements)
persistentStore := das.NewLazilyPersistentStore(s.cfg.BlobStorage, batchVerifier)
persistentStore := das.NewLazilyPersistentStore(s.cfg.BlobStorage, batchVerifier, s.blobRetentionChecker)
s.logBatchSyncStatus(firstBlock, bwbCount)
for _, bwb := range bwbs {

View File

@@ -317,6 +317,9 @@ func TestService_roundRobinSync(t *testing.T) {
clock: clock,
}
s.genesisTime = makeGenesisTime(tt.currentSlot)
s.blobRetentionChecker = func(primitives.Slot) bool {
return true
}
assert.NoError(t, s.roundRobinSync())
if s.cfg.Chain.HeadSlot() < tt.currentSlot {
t.Errorf("Head slot (%d) is less than expected currentSlot (%d)", s.cfg.Chain.HeadSlot(), tt.currentSlot)
@@ -376,7 +379,7 @@ func TestService_processBlock(t *testing.T) {
rowsb, err := blocks.NewROBlock(wsb)
require.NoError(t, err)
err = s.processBlock(ctx, genesis, blocks.BlockWithROSidecars{Block: rowsb}, func(
ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, _ das.AvailabilityStore) error {
ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, _ das.AvailabilityChecker) error {
assert.NoError(t, s.cfg.Chain.ReceiveBlock(ctx, block, blockRoot, nil))
return nil
}, nil)
@@ -388,7 +391,7 @@ func TestService_processBlock(t *testing.T) {
rowsb, err = blocks.NewROBlock(wsb)
require.NoError(t, err)
err = s.processBlock(ctx, genesis, blocks.BlockWithROSidecars{Block: rowsb}, func(
ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, _ das.AvailabilityStore) error {
ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, _ das.AvailabilityChecker) error {
return nil
}, nil)
assert.ErrorContains(t, errBlockAlreadyProcessed.Error(), err)
@@ -399,7 +402,7 @@ func TestService_processBlock(t *testing.T) {
rowsb, err = blocks.NewROBlock(wsb)
require.NoError(t, err)
err = s.processBlock(ctx, genesis, blocks.BlockWithROSidecars{Block: rowsb}, func(
ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, _ das.AvailabilityStore) error {
ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, _ das.AvailabilityChecker) error {
assert.NoError(t, s.cfg.Chain.ReceiveBlock(ctx, block, blockRoot, nil))
return nil
}, nil)
@@ -469,7 +472,7 @@ func TestService_processBlockBatch(t *testing.T) {
currBlockRoot = blk1Root
}
cbnormal := func(ctx context.Context, blks []blocks.ROBlock, avs das.AvailabilityStore) error {
cbnormal := func(ctx context.Context, blks []blocks.ROBlock, avs das.AvailabilityChecker) error {
assert.NoError(t, s.cfg.Chain.ReceiveBlockBatch(ctx, blks, avs))
return nil
}
@@ -478,7 +481,7 @@ func TestService_processBlockBatch(t *testing.T) {
assert.NoError(t, err)
require.Equal(t, uint64(len(batch)), count)
cbnil := func(ctx context.Context, blocks []blocks.ROBlock, _ das.AvailabilityStore) error {
cbnil := func(ctx context.Context, blocks []blocks.ROBlock, _ das.AvailabilityChecker) error {
return nil
}
@@ -851,7 +854,7 @@ func TestService_processBlocksWithDataColumns(t *testing.T) {
counter: ratecounter.NewRateCounter(counterSeconds * time.Second),
}
receiverFunc := func(ctx context.Context, blks []blocks.ROBlock, avs das.AvailabilityStore) error {
receiverFunc := func(ctx context.Context, blks []blocks.ROBlock, avs das.AvailabilityChecker) error {
require.Equal(t, 1, len(blks))
return nil
}

View File

@@ -53,6 +53,7 @@ type Config struct {
StateNotifier statefeed.Notifier
BlockNotifier blockfeed.Notifier
ClockWaiter startup.ClockWaiter
SyncNeedsWaiter func() (das.SyncNeeds, error)
InitialSyncComplete chan struct{}
BlobStorage *filesystem.BlobStorage
DataColumnStorage *filesystem.DataColumnStorage
@@ -73,6 +74,7 @@ type Service struct {
newDataColumnsVerifier verification.NewDataColumnsVerifier
ctxMap sync.ContextByteVersions
genesisTime time.Time
blobRetentionChecker das.RetentionChecker
}
// Option is a functional option for the initial-sync Service.
@@ -138,6 +140,20 @@ func (s *Service) Start() {
return
}
s.clock = clock
if s.blobRetentionChecker == nil {
if s.cfg.SyncNeedsWaiter == nil {
log.Error("Initial-sync service missing sync needs waiter; cannot start")
return
}
syncNeeds, err := s.cfg.SyncNeedsWaiter()
if err != nil {
log.WithError(err).Error("Initial-sync failed to receive sync needs")
return
}
s.blobRetentionChecker = syncNeeds.BlobRetentionChecker()
}
log.Info("Received state initialized event")
ctxMap, err := sync.ContextByteVersionsForValRoot(clock.GenesisValidatorsRoot())
if err != nil {
@@ -382,7 +398,7 @@ func (s *Service) fetchOriginBlobSidecars(pids []peer.ID, rob blocks.ROBlock) er
continue
}
bv := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncBlobSidecarRequirements)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, bv)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, bv, s.blobRetentionChecker)
current := s.clock.CurrentSlot()
if err := avs.Persist(current, blobSidecars...); err != nil {
return err

View File

@@ -11,6 +11,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/blockchain/kzg"
mock "github.com/OffchainLabs/prysm/v7/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/kv"
dbtest "github.com/OffchainLabs/prysm/v7/beacon-chain/db/testing"
@@ -152,10 +153,7 @@ func TestService_InitStartStop(t *testing.T) {
p := p2ptest.NewTestP2P(t)
connectPeers(t, p, []*peerData{}, p.Peers())
for i, tt := range tests {
if i == 0 {
continue
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
defer hook.Reset()
ctx, cancel := context.WithCancel(t.Context())
@@ -175,6 +173,8 @@ func TestService_InitStartStop(t *testing.T) {
InitialSyncComplete: make(chan struct{}),
})
s.verifierWaiter = verification.NewInitializerWaiter(gs, nil, nil, nil)
s.blobRetentionChecker = func(primitives.Slot) bool { return true }
time.Sleep(500 * time.Millisecond)
assert.NotNil(t, s)
if tt.setGenesis != nil {
@@ -207,15 +207,22 @@ func TestService_waitForStateInitialization(t *testing.T) {
cs := startup.NewClockSynchronizer()
ctx, cancel := context.WithCancel(ctx)
s := &Service{
cfg: &Config{Chain: mc, StateNotifier: mc.StateNotifier(), ClockWaiter: cs, InitialSyncComplete: make(chan struct{})},
ctx: ctx,
cancel: cancel,
synced: abool.New(),
chainStarted: abool.New(),
counter: ratecounter.NewRateCounter(counterSeconds * time.Second),
genesisChan: make(chan time.Time),
cfg: &Config{Chain: mc, StateNotifier: mc.StateNotifier(), ClockWaiter: cs, InitialSyncComplete: make(chan struct{})},
ctx: ctx,
cancel: cancel,
synced: abool.New(),
chainStarted: abool.New(),
counter: ratecounter.NewRateCounter(counterSeconds * time.Second),
genesisChan: make(chan time.Time),
blobRetentionChecker: func(primitives.Slot) bool { return true },
}
s.verifierWaiter = verification.NewInitializerWaiter(cs, nil, nil, nil)
syWait := func() (das.SyncNeeds, error) {
clock, err := cs.WaitForClock(ctx)
require.NoError(t, err)
return das.NewSyncNeeds(clock.CurrentSlot, nil, primitives.Epoch(0))
}
s.cfg.SyncNeedsWaiter = syWait
return s, cs
}
@@ -225,6 +232,7 @@ func TestService_waitForStateInitialization(t *testing.T) {
defer cancel()
s, _ := newService(ctx, &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}})
s.blobRetentionChecker = func(primitives.Slot) bool { return true }
wg := &sync.WaitGroup{}
wg.Go(func() {
s.Start()
@@ -252,6 +260,7 @@ func TestService_waitForStateInitialization(t *testing.T) {
require.NoError(t, err)
gt := st.GenesisTime()
s, gs := newService(ctx, &mock.ChainService{State: st, Genesis: gt, ValidatorsRoot: [32]byte{}})
s.blobRetentionChecker = func(primitives.Slot) bool { return true }
expectedGenesisTime := gt
wg := &sync.WaitGroup{}

View File

@@ -297,7 +297,10 @@ func (s *Service) handleBlockProcessingError(ctx context.Context, err error, b i
// getBestPeers returns the list of best peers based on finalized checkpoint epoch.
func (s *Service) getBestPeers() []core.PeerID {
_, bestPeers := s.cfg.p2p.Peers().BestFinalized(maxPeerRequest, s.cfg.chain.FinalizedCheckpt().Epoch)
_, bestPeers := s.cfg.p2p.Peers().BestFinalized(s.cfg.chain.FinalizedCheckpt().Epoch)
if len(bestPeers) > maxPeerRequest {
bestPeers = bestPeers[:maxPeerRequest]
}
return bestPeers
}

View File

@@ -46,6 +46,12 @@ var (
errDataColumnChunkedReadFailure = errors.New("failed to read stream of chunk-encoded data columns")
errMaxRequestDataColumnSidecarsExceeded = errors.New("count of requested data column sidecars exceeds MAX_REQUEST_DATA_COLUMN_SIDECARS")
errMaxResponseDataColumnSidecarsExceeded = errors.New("peer returned more data column sidecars than requested")
errSidecarRPCValidation = errors.Wrap(ErrInvalidFetchedData, "DataColumnSidecar")
errSidecarSlotsUnordered = errors.Wrap(errSidecarRPCValidation, "slots not in ascending order")
errSidecarIndicesUnordered = errors.Wrap(errSidecarRPCValidation, "sidecar indices not in ascending order")
errSidecarSlotNotRequested = errors.Wrap(errSidecarRPCValidation, "sidecar slot not in range")
errSidecarIndexNotRequested = errors.Wrap(errSidecarRPCValidation, "sidecar index not requested")
)
// ------
@@ -459,6 +465,7 @@ func SendDataColumnSidecarsByRangeRequest(
p DataColumnSidecarsParams,
pid peer.ID,
request *ethpb.DataColumnSidecarsByRangeRequest,
vfs ...DataColumnResponseValidation,
) ([]blocks.RODataColumn, error) {
// Return early if nothing to request.
if request == nil || request.Count == 0 || len(request.Columns) == 0 {
@@ -510,6 +517,16 @@ func SendDataColumnSidecarsByRangeRequest(
}
defer closeStream(stream, log)
requestedSlot, err := isSidecarSlotRequested(request)
if err != nil {
return nil, errors.Wrap(err, "is sidecar slot within bounds")
}
vfs = append([]DataColumnResponseValidation{
areSidecarsOrdered(),
isSidecarIndexRequested(request),
requestedSlot,
}, vfs...)
// Read the data column sidecars from the stream.
roDataColumns := make([]blocks.RODataColumn, 0, totalCount)
for range totalCount {
@@ -518,20 +535,7 @@ func SendDataColumnSidecarsByRangeRequest(
return nil, err
}
validatorSlotWithinBounds, err := isSidecarSlotWithinBounds(request)
if err != nil {
if p.DownscorePeerOnRPCFault {
downscorePeer(p.P2P, pid, "servedSidecarSlotOutOfBounds")
}
return nil, errors.Wrap(err, "is sidecar slot within bounds")
}
roDataColumn, err := readChunkedDataColumnSidecar(
stream, p.P2P, p.CtxMap,
validatorSlotWithinBounds,
isSidecarIndexRequested(request),
)
roDataColumn, err := readChunkedDataColumnSidecar(stream, p.P2P, p.CtxMap, vfs...)
if errors.Is(err, io.EOF) {
if p.DownscorePeerOnRPCFault && len(roDataColumns) == 0 {
downscorePeer(p.P2P, pid, "noReturnedSidecar")
@@ -566,8 +570,8 @@ func SendDataColumnSidecarsByRangeRequest(
return roDataColumns, nil
}
// isSidecarSlotWithinBounds verifies that the slot of the data column sidecar is within the bounds of the request.
func isSidecarSlotWithinBounds(request *ethpb.DataColumnSidecarsByRangeRequest) (DataColumnResponseValidation, error) {
// isSidecarSlotRequested verifies that the slot of the data column sidecar is within the bounds of the request.
func isSidecarSlotRequested(request *ethpb.DataColumnSidecarsByRangeRequest) (DataColumnResponseValidation, error) {
// endSlot is exclusive (while request.StartSlot is inclusive).
endSlot, err := request.StartSlot.SafeAdd(request.Count)
if err != nil {
@@ -578,7 +582,7 @@ func isSidecarSlotWithinBounds(request *ethpb.DataColumnSidecarsByRangeRequest)
slot := sidecar.Slot()
if !(request.StartSlot <= slot && slot < endSlot) {
return errors.Errorf("data column sidecar slot %d out of range [%d, %d[", slot, request.StartSlot, endSlot)
return errors.Wrapf(errSidecarSlotNotRequested, "got=%d, want=[%d, %d)", slot, request.StartSlot, endSlot)
}
return nil
@@ -587,6 +591,29 @@ func isSidecarSlotWithinBounds(request *ethpb.DataColumnSidecarsByRangeRequest)
return validator, nil
}
// areSidecarsOrdered enforces the p2p spec rule:
// "The following data column sidecars, where they exist, MUST be sent in (slot, column_index) order."
// via https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#datacolumnsidecarsbyrange-v1
func areSidecarsOrdered() DataColumnResponseValidation {
var prevSlot primitives.Slot
var prevIdx uint64
return func(sidecar blocks.RODataColumn) error {
if sidecar.Slot() < prevSlot {
return errors.Wrapf(errSidecarSlotsUnordered, "got=%d, want>=%d", sidecar.Slot(), prevSlot)
}
if sidecar.Slot() > prevSlot {
prevIdx = 0 // reset index tracking for new slot
prevSlot = sidecar.Slot() // move slot tracking to new slot
}
if sidecar.Index < prevIdx {
return errors.Wrapf(errSidecarIndicesUnordered, "got=%d, want>=%d", sidecar.Index, prevIdx)
}
prevIdx = sidecar.Index
return nil
}
}
// isSidecarIndexRequested verifies that the index of the data column sidecar is found in the requested indices.
func isSidecarIndexRequested(request *ethpb.DataColumnSidecarsByRangeRequest) DataColumnResponseValidation {
requestedIndices := make(map[uint64]bool)
@@ -598,7 +625,7 @@ func isSidecarIndexRequested(request *ethpb.DataColumnSidecarsByRangeRequest) Da
columnIndex := sidecar.Index
if !requestedIndices[columnIndex] {
requested := helpers.SortedPrettySliceFromMap(requestedIndices)
return errors.Errorf("data column sidecar index %d returned by the peer but not found in requested indices %v", columnIndex, requested)
return errors.Wrapf(errSidecarIndexNotRequested, "%d not in %v", columnIndex, requested)
}
return nil
@@ -785,3 +812,14 @@ func downscorePeer(p2p p2p.P2P, peerID peer.ID, reason string, fields ...logrus.
newScore := p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID)
log.WithFields(logrus.Fields{"peerID": peerID, "reason": reason, "newScore": newScore}).Debug("Downscore peer")
}
func DataColumnSidecarsByRangeRequest(columns []uint64, start, end primitives.Slot) (*ethpb.DataColumnSidecarsByRangeRequest, error) {
if end < start {
return nil, errors.Errorf("end slot %d is before start slot %d", end, start)
}
return &ethpb.DataColumnSidecarsByRangeRequest{
StartSlot: start,
Count: uint64(end-start) + 1,
Columns: columns,
}, nil
}

View File

@@ -1072,7 +1072,7 @@ func TestIsSidecarSlotWithinBounds(t *testing.T) {
Count: 10,
}
validator, err := isSidecarSlotWithinBounds(request)
validator, err := isSidecarSlotRequested(request)
require.NoError(t, err)
testCases := []struct {

View File

@@ -2,7 +2,6 @@ package sync
import (
"bytes"
"errors"
"reflect"
"testing"
"time"
@@ -22,6 +21,7 @@ import (
"github.com/OffchainLabs/prysm/v7/testing/util"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
)
@@ -193,7 +193,7 @@ func TestValidateDataColumn(t *testing.T) {
},
{
name: "nominal",
verifier: testNewDataColumnSidecarsVerifier(verification.MockDataColumnsVerifier{}),
verifier: testVerifierReturnsAll(&verification.MockDataColumnsVerifier{}),
expectedResult: pubsub.ValidationAccept,
},
}
@@ -222,3 +222,12 @@ func testNewDataColumnSidecarsVerifier(verifier verification.MockDataColumnsVeri
return &verifier
}
}
func testVerifierReturnsAll(v *verification.MockDataColumnsVerifier) verification.NewDataColumnsVerifier {
return func(cols []blocks.RODataColumn, reqs []verification.Requirement) verification.DataColumnsVerifier {
for _, col := range cols {
v.AppendRODataColumns(col)
}
return v
}
}

View File

@@ -91,12 +91,24 @@ type MockDataColumnsVerifier struct {
ErrSidecarInclusionProven error
ErrSidecarKzgProofVerified error
ErrSidecarProposerExpected error
verifiedColumns []blocks.RODataColumn
}
var _ DataColumnsVerifier = &MockDataColumnsVerifier{}
func (m *MockDataColumnsVerifier) AppendRODataColumns(columns ...blocks.RODataColumn) {
m.verifiedColumns = append(m.verifiedColumns, columns...)
}
func (m *MockDataColumnsVerifier) VerifiedRODataColumns() ([]blocks.VerifiedRODataColumn, error) {
return []blocks.VerifiedRODataColumn{{}}, nil
if len(m.verifiedColumns) > 0 {
result := make([]blocks.VerifiedRODataColumn, len(m.verifiedColumns))
for i, col := range m.verifiedColumns {
result[i] = blocks.VerifiedRODataColumn{RODataColumn: col}
}
return result, nil
}
return []blocks.VerifiedRODataColumn{}, nil
}
func (m *MockDataColumnsVerifier) SatisfyRequirement(_ Requirement) {}

View File

@@ -0,0 +1,7 @@
### Added
- Data column backfill.
- Backfill metrics for columns: backfill_data_column_sidecar_downloaded, backfill_data_column_sidecar_downloaded_bytes, backfill_batch_columns_download_ms, backfill_batch_columns_verify_ms.
### Changed
- backfill metrics that changed name and/or histogram buckets: backfill_batch_time_verify -> backfill_batch_verify_ms, backfill_batch_time_waiting -> backfill_batch_waiting_ms, backfill_batch_time_roundtrip -> backfill_batch_roundtrip_ms, backfill_blocks_bytes_downloaded -> backfill_blocks_downloaded_bytes, backfill_batch_time_verify -> backfill_batch_verify_ms, backfill_batch_blocks_time_download -> backfill_batch_blocks_download_ms, backfill_batch_blobs_time_download -> backfill_batch_blobs_download_ms, backfill_blobs_bytes_downloaded -> backfill_blocks_downloaded_bytes,

View File

@@ -16,6 +16,8 @@ go_library(
"//beacon-chain/node:go_default_library",
"//cmd:go_default_library",
"//cmd/beacon-chain/blockchain:go_default_library",
"//cmd/beacon-chain/das:go_default_library",
"//cmd/beacon-chain/das/flags:go_default_library",
"//cmd/beacon-chain/db:go_default_library",
"//cmd/beacon-chain/execution:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",

View File

@@ -0,0 +1,16 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = ["options.go"],
importpath = "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/das:go_default_library",
"//beacon-chain/node:go_default_library",
"//cmd/beacon-chain/das/flags:go_default_library",
"//consensus-types/primitives:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_urfave_cli_v2//:go_default_library",
],
)

View File

@@ -0,0 +1,12 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = ["flags.go"],
importpath = "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das/flags",
visibility = ["//visibility:public"],
deps = [
"//config/params:go_default_library",
"@com_github_urfave_cli_v2//:go_default_library",
],
)

View File

@@ -0,0 +1,25 @@
package flags
import (
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/urfave/cli/v2"
)
var (
BackfillOldestSlot = &cli.Uint64Flag{
Name: "backfill-oldest-slot",
Usage: "Specifies the oldest slot that backfill should download. " +
"If this value is greater than current_slot - MIN_EPOCHS_FOR_BLOCK_REQUESTS, it will be ignored with a warning log.",
}
BlobRetentionEpochFlag = &cli.Uint64Flag{
Name: "blob-retention-epochs",
Usage: "Override the default blob retention period (measured in epochs). The node will exit with an error at startup if the value is less than the default of 4096 epochs.",
Value: uint64(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest),
Aliases: []string{"extend-blob-retention-epoch"},
}
)
var Flags = []cli.Flag{
BackfillOldestSlot,
BlobRetentionEpochFlag,
}

View File

@@ -0,0 +1,35 @@
package options
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/das"
"github.com/OffchainLabs/prysm/v7/beacon-chain/node"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das/flags"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/pkg/errors"
"github.com/urfave/cli/v2"
)
func BeaconNodeOptions(c *cli.Context) ([]node.Option, error) {
var oldestBackfillSlot *primitives.Slot
if c.IsSet(flags.BackfillOldestSlot.Name) {
uv := c.Uint64(flags.BackfillOldestSlot.Name)
sv := primitives.Slot(uv)
oldestBackfillSlot = &sv
}
blobRetentionEpochs := primitives.Epoch(c.Uint64(flags.BlobRetentionEpochFlag.Name))
opt := func(n *node.BeaconNode) error {
n.SyncNeedsWaiter = func() (das.SyncNeeds, error) {
clock, err := n.ClockWaiter.WaitForClock(c.Context)
if err != nil {
return das.SyncNeeds{}, errors.Wrap(err, "sync needs WaitForClock")
}
return das.NewSyncNeeds(
clock.CurrentSlot,
oldestBackfillSlot,
blobRetentionEpochs,
)
}
return nil
}
return []node.Option{opt}, nil
}

View File

@@ -12,6 +12,8 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/node"
"github.com/OffchainLabs/prysm/v7/cmd"
blockchaincmd "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/blockchain"
das "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das"
dasFlags "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das/flags"
dbcommands "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/db"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/execution"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/flags"
@@ -147,12 +149,12 @@ var appFlags = []cli.Flag{
flags.JwtId,
storage.BlobStoragePathFlag,
storage.DataColumnStoragePathFlag,
storage.BlobRetentionEpochFlag,
storage.BlobStorageLayout,
bflags.EnableExperimentalBackfill,
bflags.BackfillBatchSize,
bflags.BackfillWorkerCount,
bflags.BackfillOldestSlot,
dasFlags.BackfillOldestSlot,
dasFlags.BlobRetentionEpochFlag,
flags.BatchVerifierLimit,
}
@@ -319,6 +321,7 @@ func startNode(ctx *cli.Context, cancel context.CancelFunc) error {
checkpoint.BeaconNodeOptions,
storage.BeaconNodeOptions,
backfill.BeaconNodeOptions,
das.BeaconNodeOptions,
}
for _, of := range optFuncs {
ofo, err := of(ctx)

View File

@@ -9,6 +9,7 @@ go_library(
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/node:go_default_library",
"//cmd:go_default_library",
"//cmd/beacon-chain/das/flags:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"@com_github_pkg_errors//:go_default_library",
@@ -25,6 +26,7 @@ go_test(
deps = [
"//beacon-chain/db/filesystem:go_default_library",
"//cmd:go_default_library",
"//cmd/beacon-chain/das/flags:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//testing/assert:go_default_library",

View File

@@ -11,6 +11,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/beacon-chain/node"
"github.com/OffchainLabs/prysm/v7/cmd"
das "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das/flags"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/pkg/errors"
@@ -23,12 +24,6 @@ var (
Name: "blob-path",
Usage: "Location for blob storage. Default location will be a 'blobs' directory next to the beacon db.",
}
BlobRetentionEpochFlag = &cli.Uint64Flag{
Name: "blob-retention-epochs",
Usage: "Override the default blob retention period (measured in epochs). The node will exit with an error at startup if the value is less than the default of 4096 epochs.",
Value: uint64(params.BeaconConfig().MinEpochsForBlobsSidecarsRequest),
Aliases: []string{"extend-blob-retention-epoch"},
}
BlobStorageLayout = &cli.StringFlag{
Name: "blob-storage-layout",
Usage: layoutFlagUsage(),
@@ -43,7 +38,6 @@ var (
// Flags is the list of CLI flags for configuring blob storage.
var Flags = []cli.Flag{
BlobStoragePathFlag,
BlobRetentionEpochFlag,
BlobStorageLayout,
DataColumnStoragePathFlag,
}
@@ -178,14 +172,14 @@ var errInvalidBlobRetentionEpochs = errors.New("value is smaller than spec minim
// smaller than the spec default, an error will be returned.
func blobRetentionEpoch(cliCtx *cli.Context) (primitives.Epoch, error) {
spec := params.BeaconConfig().MinEpochsForBlobsSidecarsRequest
if !cliCtx.IsSet(BlobRetentionEpochFlag.Name) {
if !cliCtx.IsSet(das.BlobRetentionEpochFlag.Name) {
return spec, nil
}
re := primitives.Epoch(cliCtx.Uint64(BlobRetentionEpochFlag.Name))
re := primitives.Epoch(cliCtx.Uint64(das.BlobRetentionEpochFlag.Name))
// Validate the epoch value against the spec default.
if re < params.BeaconConfig().MinEpochsForBlobsSidecarsRequest {
return spec, errors.Wrapf(errInvalidBlobRetentionEpochs, "%s=%d, spec=%d", BlobRetentionEpochFlag.Name, re, spec)
return spec, errors.Wrapf(errInvalidBlobRetentionEpochs, "%s=%d, spec=%d", das.BlobRetentionEpochFlag.Name, re, spec)
}
return re, nil
@@ -196,16 +190,16 @@ func blobRetentionEpoch(cliCtx *cli.Context) (primitives.Epoch, error) {
// smaller than the spec default, an error will be returned.
func dataColumnRetentionEpoch(cliCtx *cli.Context) (primitives.Epoch, error) {
defaultValue := params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest
if !cliCtx.IsSet(BlobRetentionEpochFlag.Name) {
if !cliCtx.IsSet(das.BlobRetentionEpochFlag.Name) {
return defaultValue, nil
}
// We use on purpose the same retention flag for both blobs and data columns.
customValue := primitives.Epoch(cliCtx.Uint64(BlobRetentionEpochFlag.Name))
customValue := primitives.Epoch(cliCtx.Uint64(das.BlobRetentionEpochFlag.Name))
// Validate the epoch value against the spec default.
if customValue < defaultValue {
return defaultValue, errors.Wrapf(errInvalidBlobRetentionEpochs, "%s=%d, spec=%d", BlobRetentionEpochFlag.Name, customValue, defaultValue)
return defaultValue, errors.Wrapf(errInvalidBlobRetentionEpochs, "%s=%d, spec=%d", das.BlobRetentionEpochFlag.Name, customValue, defaultValue)
}
return customValue, nil

View File

@@ -12,6 +12,7 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v7/cmd"
das "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das/flags"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/testing/assert"
@@ -52,18 +53,18 @@ func TestConfigureBlobRetentionEpoch(t *testing.T) {
require.Equal(t, specMinEpochs, epochs)
// manually define the flag in the set, so the following code can use set.Set
set.Uint64(BlobRetentionEpochFlag.Name, 0, "")
set.Uint64(das.BlobRetentionEpochFlag.Name, 0, "")
// Test case: Input epoch is greater than or equal to spec value.
expectedChange := specMinEpochs + 1
require.NoError(t, set.Set(BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expectedChange)))
require.NoError(t, set.Set(das.BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expectedChange)))
epochs, err = blobRetentionEpoch(cliCtx)
require.NoError(t, err)
require.Equal(t, primitives.Epoch(expectedChange), epochs)
// Test case: Input epoch is less than spec value.
expectedChange = specMinEpochs - 1
require.NoError(t, set.Set(BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expectedChange)))
require.NoError(t, set.Set(das.BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expectedChange)))
_, err = blobRetentionEpoch(cliCtx)
require.ErrorIs(t, err, errInvalidBlobRetentionEpochs)
}
@@ -83,12 +84,12 @@ func TestConfigureDataColumnRetentionEpoch(t *testing.T) {
require.Equal(t, expected, actual)
// Manually define the flag in the set, so the following code can use set.Set
set.Uint64(BlobRetentionEpochFlag.Name, 0, "")
set.Uint64(das.BlobRetentionEpochFlag.Name, 0, "")
// Test case: Input epoch is greater than or equal to specification value.
expected = specValue + 1
err = set.Set(BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expected))
err = set.Set(das.BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expected))
require.NoError(t, err)
actual, err = dataColumnRetentionEpoch(cliCtx)
@@ -98,7 +99,7 @@ func TestConfigureDataColumnRetentionEpoch(t *testing.T) {
// Test case: Input epoch is less than specification value.
expected = specValue - 1
err = set.Set(BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expected))
err = set.Set(das.BlobRetentionEpochFlag.Name, fmt.Sprintf("%d", expected))
require.NoError(t, err)
actual, err = dataColumnRetentionEpoch(cliCtx)

View File

@@ -9,7 +9,6 @@ go_library(
"//beacon-chain/node:go_default_library",
"//beacon-chain/sync/backfill:go_default_library",
"//cmd/beacon-chain/sync/backfill/flags:go_default_library",
"//consensus-types/primitives:go_default_library",
"@com_github_urfave_cli_v2//:go_default_library",
],
)

View File

@@ -35,9 +35,4 @@ var (
"This has a multiplicative effect with " + backfillBatchSizeName + ".",
Value: 2,
}
BackfillOldestSlot = &cli.Uint64Flag{
Name: "backfill-oldest-slot",
Usage: "Specifies the oldest slot that backfill should download. " +
"If this value is greater than current_slot - MIN_EPOCHS_FOR_BLOCK_REQUESTS, it will be ignored with a warning log.",
}
)

View File

@@ -4,7 +4,6 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/node"
"github.com/OffchainLabs/prysm/v7/beacon-chain/sync/backfill"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/sync/backfill/flags"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/urfave/cli/v2"
)
@@ -17,11 +16,6 @@ func BeaconNodeOptions(c *cli.Context) ([]node.Option, error) {
backfill.WithWorkerCount(c.Int(flags.BackfillWorkerCount.Name)),
backfill.WithEnableBackfill(c.Bool(flags.EnableExperimentalBackfill.Name)),
}
// The zero value of this uint flag would be genesis, so we use IsSet to differentiate nil from zero case.
if c.IsSet(flags.BackfillOldestSlot.Name) {
uv := c.Uint64(flags.BackfillOldestSlot.Name)
bno = append(bno, backfill.WithMinimumSlot(primitives.Slot(uv)))
}
node.BackfillOpts = bno
return nil
}

View File

@@ -6,6 +6,7 @@ import (
"sort"
"github.com/OffchainLabs/prysm/v7/cmd"
das "github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/das/flags"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/genesis"
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/storage"
@@ -115,7 +116,6 @@ var appHelpFlagGroups = []flagGroup{
Name: "db",
Flags: []cli.Flag{
backfill.BackfillBatchSize,
backfill.BackfillOldestSlot,
backfill.BackfillWorkerCount,
backfill.EnableExperimentalBackfill,
cmd.ClearDB,
@@ -123,10 +123,11 @@ var appHelpFlagGroups = []flagGroup{
cmd.ForceClearDB,
cmd.RestoreSourceFileFlag,
cmd.RestoreTargetDirFlag,
das.BackfillOldestSlot,
das.BlobRetentionEpochFlag,
flags.BeaconDBPruning,
flags.PrunerRetentionEpochs,
flags.SlotsPerArchivedPoint,
storage.BlobRetentionEpochFlag,
storage.BlobStorageLayout,
storage.BlobStoragePathFlag,
storage.DataColumnStoragePathFlag,

View File

@@ -14,6 +14,9 @@ import (
"github.com/sirupsen/logrus"
)
// errOverflow is returned when a slot calculation would overflow.
var errOverflow = errors.New("slot calculation overflows")
// MaxSlotBuffer specifies the max buffer given to slots from
// incoming objects. (24 mins with mainnet spec)
const MaxSlotBuffer = uint64(1 << 7)
@@ -108,7 +111,7 @@ func ToForkVersion(slot primitives.Slot) int {
func EpochStart(epoch primitives.Epoch) (primitives.Slot, error) {
slot, err := params.BeaconConfig().SlotsPerEpoch.SafeMul(uint64(epoch))
if err != nil {
return slot, errors.Errorf("start slot calculation overflows: %v", err)
return slot, errors.Wrap(errOverflow, "epoch start")
}
return slot, nil
}
@@ -127,7 +130,7 @@ func UnsafeEpochStart(epoch primitives.Epoch) primitives.Slot {
// current epoch.
func EpochEnd(epoch primitives.Epoch) (primitives.Slot, error) {
if epoch == math.MaxUint64 {
return 0, errors.New("start slot calculation overflows")
return 0, errors.Wrap(errOverflow, "epoch end")
}
slot, err := EpochStart(epoch + 1)
if err != nil {
@@ -284,8 +287,26 @@ func WithinVotingWindow(genesis time.Time, slot primitives.Slot) bool {
}
// MaxSafeEpoch gives the largest epoch value that can be safely converted to a slot.
// Note that just dividing max uint64 by slots per epoch is not sufficient,
// because the resulting slot could still be the start of an epoch that would overflow
// in the end slot computation. So we subtract 1 to ensure that the final epoch can always
// have 32 slots.
func MaxSafeEpoch() primitives.Epoch {
return primitives.Epoch(math.MaxUint64 / uint64(params.BeaconConfig().SlotsPerEpoch))
return primitives.Epoch(math.MaxUint64/uint64(params.BeaconConfig().SlotsPerEpoch)) - 1
}
// SafeEpochStartOrMax returns the start slot of the given epoch if it will not overflow,
// otherwise it takes the highest epoch that won't overflow,
// and to introduce a little margin for error, returns the slot beginning the prior epoch.
func SafeEpochStartOrMax(e primitives.Epoch) primitives.Slot {
// The max value converted to a slot can't be the start of a conceptual epoch,
// because the first slot of that epoch would be overflow
// so use the start slot of the epoch right before that value.
me := MaxSafeEpoch()
if e > me {
return UnsafeEpochStart(me)
}
return UnsafeEpochStart(e)
}
// SecondsUntilNextEpochStart returns how many seconds until the next Epoch start from the current time and slot

View File

@@ -117,7 +117,7 @@ func TestEpochStartSlot_OK(t *testing.T) {
require.NoError(t, err)
assert.Equal(t, tt.startSlot, ss, "EpochStart(%d)", tt.epoch)
} else {
require.ErrorContains(t, "start slot calculation overflow", err)
require.ErrorIs(t, err, errOverflow)
}
}
}
@@ -141,7 +141,7 @@ func TestEpochEndSlot_OK(t *testing.T) {
require.NoError(t, err)
assert.Equal(t, tt.startSlot, ss, "EpochStart(%d)", tt.epoch)
} else {
require.ErrorContains(t, "start slot calculation overflow", err)
require.ErrorIs(t, err, errOverflow)
}
}
}
@@ -700,3 +700,75 @@ func TestSlotTickerReplayBehaviour(t *testing.T) {
require.Equal(t, ticks, counter)
}
func TestSafeEpochStartOrMax(t *testing.T) {
farFuture := params.BeaconConfig().FarFutureEpoch
// ensure FAR_FUTURE_EPOCH is indeed larger than MaxSafeEpoch
require.Equal(t, true, farFuture > MaxSafeEpoch())
// demonstrate overflow in naive usage of FAR_FUTURE_EPOCH
require.Equal(t, true, farFuture*primitives.Epoch(params.BeaconConfig().SlotsPerEpoch) < farFuture)
// sanity check that example "ordinary" epoch does not overflow
fulu, err := EpochStart(params.BeaconConfig().FuluForkEpoch)
require.NoError(t, err)
require.Equal(t, primitives.Slot(params.BeaconConfig().FuluForkEpoch)*params.BeaconConfig().SlotsPerEpoch, fulu)
maxEpochStart := primitives.Slot(math.MaxUint64) - 63
tests := []struct {
name string
epoch primitives.Epoch
want primitives.Slot
err error
}{
{
name: "genesis",
epoch: 0,
want: 0,
},
{
name: "ordinary epoch",
epoch: params.BeaconConfig().FuluForkEpoch,
want: primitives.Slot(params.BeaconConfig().FuluForkEpoch) * params.BeaconConfig().SlotsPerEpoch,
},
{
name: "max epoch without overflow",
epoch: MaxSafeEpoch(),
want: maxEpochStart,
},
{
name: "max epoch causing overflow",
epoch: primitives.Epoch(math.MaxUint64),
want: maxEpochStart,
err: errOverflow,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := SafeEpochStartOrMax(tt.epoch)
require.Equal(t, tt.want, got)
// If we expect an error, it should be present for both start and end calculations
_, startErr := EpochStart(tt.epoch)
_, endErr := EpochEnd(tt.epoch)
if tt.err != nil {
require.ErrorIs(t, startErr, tt.err)
require.ErrorIs(t, endErr, tt.err)
} else {
require.NoError(t, startErr)
require.NoError(t, endErr)
}
})
}
}
func TestMaxEpoch(t *testing.T) {
maxEpoch := MaxSafeEpoch()
_, err := EpochStart(maxEpoch + 2)
require.ErrorIs(t, err, errOverflow)
_, err = EpochEnd(maxEpoch + 1)
require.ErrorIs(t, err, errOverflow)
require.Equal(t, primitives.Slot(math.MaxUint64)-63, primitives.Slot(maxEpoch)*params.BeaconConfig().SlotsPerEpoch)
_, err = EpochEnd(maxEpoch)
require.NoError(t, err)
_, err = EpochStart(maxEpoch)
require.NoError(t, err)
}