Support for different blob target and max values (#14678)

* Add support for different blob target and max

* Fix change log to right section
This commit is contained in:
terence
2024-12-19 06:58:24 -08:00
committed by GitHub
parent ed7b511949
commit bc69ab8a44
57 changed files with 293 additions and 201 deletions

View File

@@ -12,6 +12,7 @@ The format is based on Keep a Changelog, and this project adheres to Semantic Ve
- Added an error field to log `Finished building block`.
- Implemented a new `EmptyExecutionPayloadHeader` function.
- `Finished building block`: Display error only if not nil.
- Added support to update target and max blob count to different values per hard fork config.
### Changed

View File

@@ -15,6 +15,7 @@ go_library(
"//api/client:go_default_library",
"//api/server/structs:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",

View File

@@ -9,6 +9,7 @@ import (
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
consensusblocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
types "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -1013,7 +1014,7 @@ func (bb *BuilderBidDeneb) ToProto() (*eth.BuilderBidDeneb, error) {
if err != nil {
return nil, err
}
if len(bb.BlobKzgCommitments) > fieldparams.MaxBlobsPerBlock {
if len(bb.BlobKzgCommitments) > params.BeaconConfig().DeprecatedMaxBlobsPerBlock {
return nil, fmt.Errorf("too many blob commitments: %d", len(bb.BlobKzgCommitments))
}
kzgCommitments := make([][]byte, len(bb.BlobKzgCommitments))

View File

@@ -15,7 +15,6 @@ import (
forkchoicetypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/forkchoice/types"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
consensusblocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -496,14 +495,15 @@ func (s *Service) runLateBlockTasks() {
// It returns a map where each key represents a missing BlobSidecar index.
// An empty map means we have all indices; a non-empty map can be used to compare incoming
// BlobSidecars against the set of known missing sidecars.
func missingIndices(bs *filesystem.BlobStorage, root [32]byte, expected [][]byte) (map[uint64]struct{}, error) {
func missingIndices(bs *filesystem.BlobStorage, root [32]byte, expected [][]byte, slot primitives.Slot) (map[uint64]struct{}, error) {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
if len(expected) == 0 {
return nil, nil
}
if len(expected) > fieldparams.MaxBlobsPerBlock {
if len(expected) > maxBlobsPerBlock {
return nil, errMaxBlobsExceeded
}
indices, err := bs.Indices(root)
indices, err := bs.Indices(root, slot)
if err != nil {
return nil, err
}
@@ -552,7 +552,7 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
return nil
}
// get a map of BlobSidecar indices that are not currently available.
missing, err := missingIndices(s.blobStorage, root, kzgCommitments)
missing, err := missingIndices(s.blobStorage, root, kzgCommitments, block.Slot())
if err != nil {
return err
}
@@ -563,7 +563,7 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
// The gossip handler for blobs writes the index of each verified blob referencing the given
// root to the channel returned by blobNotifiers.forRoot.
nc := s.blobNotifiers.forRoot(root)
nc := s.blobNotifiers.forRoot(root, block.Slot())
// Log for DA checks that cross over into the next slot; helpful for debugging.
nextSlot := slots.BeginsAt(signed.Block().Slot()+1, s.genesisTime)

View File

@@ -2206,23 +2206,23 @@ func TestMissingIndices(t *testing.T) {
},
{
name: "expected exceeds max",
expected: fakeCommitments(fieldparams.MaxBlobsPerBlock + 1),
expected: fakeCommitments(params.BeaconConfig().MaxBlobsPerBlock(0) + 1),
err: errMaxBlobsExceeded,
},
{
name: "first missing",
expected: fakeCommitments(fieldparams.MaxBlobsPerBlock),
expected: fakeCommitments(params.BeaconConfig().MaxBlobsPerBlock(0)),
present: []uint64{1, 2, 3, 4, 5},
result: fakeResult([]uint64{0}),
},
{
name: "all missing",
expected: fakeCommitments(fieldparams.MaxBlobsPerBlock),
expected: fakeCommitments(params.BeaconConfig().MaxBlobsPerBlock(0)),
result: fakeResult([]uint64{0, 1, 2, 3, 4, 5}),
},
{
name: "none missing",
expected: fakeCommitments(fieldparams.MaxBlobsPerBlock),
expected: fakeCommitments(params.BeaconConfig().MaxBlobsPerBlock(0)),
present: []uint64{0, 1, 2, 3, 4, 5},
result: fakeResult([]uint64{}),
},
@@ -2256,7 +2256,7 @@ func TestMissingIndices(t *testing.T) {
bm, bs := filesystem.NewEphemeralBlobStorageWithMocker(t)
t.Run(c.name, func(t *testing.T) {
require.NoError(t, bm.CreateFakeIndices(c.root, c.present...))
missing, err := missingIndices(bs, c.root, c.expected)
missing, err := missingIndices(bs, c.root, c.expected, 0)
if c.err != nil {
require.ErrorIs(t, err, c.err)
return

View File

@@ -4,12 +4,13 @@ import (
"context"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
)
// SendNewBlobEvent sends a message to the BlobNotifier channel that the blob
// for the block root `root` is ready in the database
func (s *Service) sendNewBlobEvent(root [32]byte, index uint64) {
s.blobNotifiers.notifyIndex(root, index)
func (s *Service) sendNewBlobEvent(root [32]byte, index uint64, slot primitives.Slot) {
s.blobNotifiers.notifyIndex(root, index, slot)
}
// ReceiveBlob saves the blob to database and sends the new event
@@ -18,6 +19,6 @@ func (s *Service) ReceiveBlob(ctx context.Context, b blocks.VerifiedROBlob) erro
return err
}
s.sendNewBlobEvent(b.BlockRoot(), b.Index)
s.sendNewBlobEvent(b.BlockRoot(), b.Index, b.Slot())
return nil
}

View File

@@ -33,10 +33,10 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
@@ -104,18 +104,22 @@ var ErrMissingClockSetter = errors.New("blockchain Service initialized without a
type blobNotifierMap struct {
sync.RWMutex
notifiers map[[32]byte]chan uint64
seenIndex map[[32]byte][fieldparams.MaxBlobsPerBlock]bool
seenIndex map[[32]byte][]bool
}
// notifyIndex notifies a blob by its index for a given root.
// It uses internal maps to keep track of seen indices and notifier channels.
func (bn *blobNotifierMap) notifyIndex(root [32]byte, idx uint64) {
if idx >= fieldparams.MaxBlobsPerBlock {
func (bn *blobNotifierMap) notifyIndex(root [32]byte, idx uint64, slot primitives.Slot) {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
if idx >= uint64(maxBlobsPerBlock) {
return
}
bn.Lock()
seen := bn.seenIndex[root]
if seen == nil {
seen = make([]bool, maxBlobsPerBlock)
}
if seen[idx] {
bn.Unlock()
return
@@ -126,7 +130,7 @@ func (bn *blobNotifierMap) notifyIndex(root [32]byte, idx uint64) {
// Retrieve or create the notifier channel for the given root.
c, ok := bn.notifiers[root]
if !ok {
c = make(chan uint64, fieldparams.MaxBlobsPerBlock)
c = make(chan uint64, maxBlobsPerBlock)
bn.notifiers[root] = c
}
@@ -135,12 +139,13 @@ func (bn *blobNotifierMap) notifyIndex(root [32]byte, idx uint64) {
c <- idx
}
func (bn *blobNotifierMap) forRoot(root [32]byte) chan uint64 {
func (bn *blobNotifierMap) forRoot(root [32]byte, slot primitives.Slot) chan uint64 {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
bn.Lock()
defer bn.Unlock()
c, ok := bn.notifiers[root]
if !ok {
c = make(chan uint64, fieldparams.MaxBlobsPerBlock)
c = make(chan uint64, maxBlobsPerBlock)
bn.notifiers[root] = c
}
return c
@@ -166,7 +171,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) {
ctx, cancel := context.WithCancel(ctx)
bn := &blobNotifierMap{
notifiers: make(map[[32]byte]chan uint64),
seenIndex: make(map[[32]byte][fieldparams.MaxBlobsPerBlock]bool),
seenIndex: make(map[[32]byte][]bool),
}
srv := &Service{
ctx: ctx,

View File

@@ -587,7 +587,7 @@ func (s *MockClockSetter) SetClock(g *startup.Clock) error {
func TestNotifyIndex(t *testing.T) {
// Initialize a blobNotifierMap
bn := &blobNotifierMap{
seenIndex: make(map[[32]byte][fieldparams.MaxBlobsPerBlock]bool),
seenIndex: make(map[[32]byte][]bool),
notifiers: make(map[[32]byte]chan uint64),
}
@@ -596,7 +596,7 @@ func TestNotifyIndex(t *testing.T) {
copy(root[:], "exampleRoot")
// Test notifying a new index
bn.notifyIndex(root, 1)
bn.notifyIndex(root, 1, 1)
if !bn.seenIndex[root][1] {
t.Errorf("Index was not marked as seen")
}
@@ -607,13 +607,13 @@ func TestNotifyIndex(t *testing.T) {
}
// Test notifying an already seen index
bn.notifyIndex(root, 1)
bn.notifyIndex(root, 1, 1)
if len(bn.notifiers[root]) > 1 {
t.Errorf("Notifier channel should not receive multiple messages for the same index")
}
// Test notifying a new index again
bn.notifyIndex(root, 2)
bn.notifyIndex(root, 2, 1)
if !bn.seenIndex[root][2] {
t.Errorf("Index was not marked as seen")
}

View File

@@ -8,10 +8,11 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
consensus_types "github.com/prysmaticlabs/prysm/v5/consensus-types"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
@@ -210,7 +211,7 @@ func ProcessPayload(st state.BeaconState, body interfaces.ReadOnlyBeaconBlockBod
if err != nil {
return err
}
if err := verifyBlobCommitmentCount(body); err != nil {
if err := verifyBlobCommitmentCount(st.Slot(), body); err != nil {
return err
}
if err := ValidatePayloadWhenMergeCompletes(st, payload); err != nil {
@@ -225,7 +226,7 @@ func ProcessPayload(st state.BeaconState, body interfaces.ReadOnlyBeaconBlockBod
return nil
}
func verifyBlobCommitmentCount(body interfaces.ReadOnlyBeaconBlockBody) error {
func verifyBlobCommitmentCount(slot primitives.Slot, body interfaces.ReadOnlyBeaconBlockBody) error {
if body.Version() < version.Deneb {
return nil
}
@@ -233,7 +234,8 @@ func verifyBlobCommitmentCount(body interfaces.ReadOnlyBeaconBlockBody) error {
if err != nil {
return err
}
if len(kzgs) > field_params.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
if len(kzgs) > maxBlobsPerBlock {
return fmt.Errorf("too many kzg commitments in block: %d", len(kzgs))
}
return nil

View File

@@ -9,6 +9,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
consensusblocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
@@ -923,10 +924,10 @@ func TestVerifyBlobCommitmentCount(t *testing.T) {
b := &ethpb.BeaconBlockDeneb{Body: &ethpb.BeaconBlockBodyDeneb{}}
rb, err := consensusblocks.NewBeaconBlock(b)
require.NoError(t, err)
require.NoError(t, blocks.VerifyBlobCommitmentCount(rb.Body()))
require.NoError(t, blocks.VerifyBlobCommitmentCount(rb.Slot(), rb.Body()))
b = &ethpb.BeaconBlockDeneb{Body: &ethpb.BeaconBlockBodyDeneb{BlobKzgCommitments: make([][]byte, fieldparams.MaxBlobsPerBlock+1)}}
b = &ethpb.BeaconBlockDeneb{Body: &ethpb.BeaconBlockBodyDeneb{BlobKzgCommitments: make([][]byte, params.BeaconConfig().MaxBlobsPerBlock(rb.Slot())+1)}}
rb, err = consensusblocks.NewBeaconBlock(b)
require.NoError(t, err)
require.ErrorContains(t, fmt.Sprintf("too many kzg commitments in block: %d", fieldparams.MaxBlobsPerBlock+1), blocks.VerifyBlobCommitmentCount(rb.Body()))
require.ErrorContains(t, fmt.Sprintf("too many kzg commitments in block: %d", params.BeaconConfig().MaxBlobsPerBlock(rb.Slot())+1), blocks.VerifyBlobCommitmentCount(rb.Slot(), rb.Body()))
}

View File

@@ -13,7 +13,6 @@ go_library(
deps = [
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",
@@ -35,7 +34,6 @@ go_test(
deps = [
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",

View File

@@ -83,10 +83,10 @@ func (s *LazilyPersistentStore) Persist(current primitives.Slot, sc ...blocks.RO
func (s *LazilyPersistentStore) IsDataAvailable(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error {
blockCommitments, err := commitmentsToCheck(b, current)
if err != nil {
return errors.Wrapf(err, "could check data availability for block %#x", b.Root())
return errors.Wrapf(err, "could not check data availability for block %#x", b.Root())
}
// Return early for blocks that are pre-deneb or which do not have any commitments.
if blockCommitments.count() == 0 {
if len(blockCommitments) == 0 {
return nil
}
@@ -106,7 +106,7 @@ func (s *LazilyPersistentStore) IsDataAvailable(ctx context.Context, current pri
// Verify we have all the expected sidecars, and fail fast if any are missing or inconsistent.
// We don't try to salvage problematic batches because this indicates a misbehaving peer and we'd rather
// ignore their response and decrease their peer score.
sidecars, err := entry.filter(root, blockCommitments)
sidecars, err := entry.filter(root, blockCommitments, b.Block().Slot())
if err != nil {
return errors.Wrap(err, "incomplete BlobSidecar batch")
}
@@ -137,22 +137,28 @@ func (s *LazilyPersistentStore) IsDataAvailable(ctx context.Context, current pri
return nil
}
func commitmentsToCheck(b blocks.ROBlock, current primitives.Slot) (safeCommitmentArray, error) {
var ar safeCommitmentArray
func commitmentsToCheck(b blocks.ROBlock, current primitives.Slot) ([][]byte, error) {
if b.Version() < version.Deneb {
return ar, nil
return nil, nil
}
// We are only required to check within MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS
// We are only required to check within MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUEST
if !params.WithinDAPeriod(slots.ToEpoch(b.Block().Slot()), slots.ToEpoch(current)) {
return ar, nil
return nil, nil
}
kc, err := b.Block().Body().BlobKzgCommitments()
kzgCommitments, err := b.Block().Body().BlobKzgCommitments()
if err != nil {
return ar, err
return nil, err
}
if len(kc) > len(ar) {
return ar, errIndexOutOfBounds
maxBlobCount := params.BeaconConfig().MaxBlobsPerBlock(b.Block().Slot())
if len(kzgCommitments) > maxBlobCount {
return nil, errIndexOutOfBounds
}
copy(ar[:], kc)
return ar, nil
result := make([][]byte, len(kzgCommitments))
copy(result, kzgCommitments)
return result, nil
}

View File

@@ -8,7 +8,6 @@ import (
errors "github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -89,7 +88,7 @@ func Test_commitmentsToCheck(t *testing.T) {
require.NoError(t, err)
c, err := rb.Block().Body().BlobKzgCommitments()
require.NoError(t, err)
require.Equal(t, true, len(c) > fieldparams.MaxBlobsPerBlock)
require.Equal(t, true, len(c) > params.BeaconConfig().MaxBlobsPerBlock(sb.Block().Slot()))
return rb
},
slot: windowSlots + 1,
@@ -105,7 +104,7 @@ func Test_commitmentsToCheck(t *testing.T) {
} else {
require.NoError(t, err)
}
require.Equal(t, len(c.commits), co.count())
require.Equal(t, len(c.commits), len(co))
for i := 0; i < len(c.commits); i++ {
require.Equal(t, true, bytes.Equal(c.commits[i], co[i]))
}

View File

@@ -5,7 +5,7 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
)
@@ -60,7 +60,7 @@ func (c *cache) delete(key cacheKey) {
// cacheEntry holds a fixed-length cache of BlobSidecars.
type cacheEntry struct {
scs [fieldparams.MaxBlobsPerBlock]*blocks.ROBlob
scs []*blocks.ROBlob
diskSummary filesystem.BlobStorageSummary
}
@@ -72,9 +72,13 @@ func (e *cacheEntry) setDiskSummary(sum filesystem.BlobStorageSummary) {
// Only the first BlobSidecar of a given Index will be kept in the cache.
// stash will return an error if the given blob is already in the cache, or if the Index is out of bounds.
func (e *cacheEntry) stash(sc *blocks.ROBlob) error {
if sc.Index >= fieldparams.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(sc.Slot())
if sc.Index >= uint64(maxBlobsPerBlock) {
return errors.Wrapf(errIndexOutOfBounds, "index=%d", sc.Index)
}
if e.scs == nil {
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)
}
@@ -88,12 +92,13 @@ func (e *cacheEntry) stash(sc *blocks.ROBlob) error {
// commitments were found in the cache and the sidecar slice return value can be used
// to perform a DA check against the cached sidecars.
// filter only returns blobs that need to be checked. Blobs already available on disk will be excluded.
func (e *cacheEntry) filter(root [32]byte, kc safeCommitmentArray) ([]blocks.ROBlob, error) {
if e.diskSummary.AllAvailable(kc.count()) {
func (e *cacheEntry) filter(root [32]byte, kc [][]byte, slot primitives.Slot) ([]blocks.ROBlob, error) {
count := len(kc)
if e.diskSummary.AllAvailable(count) {
return nil, nil
}
scs := make([]blocks.ROBlob, 0, kc.count())
for i := range uint64(fieldparams.MaxBlobsPerBlock) {
scs := make([]blocks.ROBlob, 0, count)
for i := uint64(0); i < uint64(count); i++ {
// We already have this blob, we don't need to write it or validate it.
if e.diskSummary.HasIndex(i) {
continue
@@ -116,16 +121,3 @@ func (e *cacheEntry) filter(root [32]byte, kc safeCommitmentArray) ([]blocks.ROB
return scs, nil
}
// safeCommitmentArray is a fixed size array of commitment byte slices. This is helpful for avoiding
// gratuitous bounds checks.
type safeCommitmentArray [fieldparams.MaxBlobsPerBlock][]byte
func (s safeCommitmentArray) count() int {
for i := range s {
if s[i] == nil {
return i
}
}
return fieldparams.MaxBlobsPerBlock
}

View File

@@ -29,10 +29,10 @@ func TestCacheEnsureDelete(t *testing.T) {
require.Equal(t, nilEntry, c.entries[k])
}
type filterTestCaseSetupFunc func(t *testing.T) (*cacheEntry, safeCommitmentArray, []blocks.ROBlob)
type filterTestCaseSetupFunc func(t *testing.T) (*cacheEntry, [][]byte, []blocks.ROBlob)
func filterTestCaseSetup(slot primitives.Slot, nBlobs int, onDisk []int, numExpected int) filterTestCaseSetupFunc {
return func(t *testing.T) (*cacheEntry, safeCommitmentArray, []blocks.ROBlob) {
return func(t *testing.T) (*cacheEntry, [][]byte, []blocks.ROBlob) {
blk, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, slot, nBlobs)
commits, err := commitmentsToCheck(blk, blk.Block().Slot())
require.NoError(t, err)
@@ -44,7 +44,7 @@ func filterTestCaseSetup(slot primitives.Slot, nBlobs int, onDisk []int, numExpe
entry.setDiskSummary(sum)
}
expected := make([]blocks.ROBlob, 0, nBlobs)
for i := 0; i < commits.count(); i++ {
for i := 0; i < len(commits); i++ {
if entry.diskSummary.HasIndex(uint64(i)) {
continue
}
@@ -113,7 +113,7 @@ func TestFilterDiskSummary(t *testing.T) {
t.Run(c.name, func(t *testing.T) {
entry, commits, expected := c.setup(t)
// first (root) argument doesn't matter, it is just for logs
got, err := entry.filter([32]byte{}, commits)
got, err := entry.filter([32]byte{}, commits, 100)
require.NoError(t, err)
require.Equal(t, len(expected), len(got))
})
@@ -125,12 +125,12 @@ func TestFilter(t *testing.T) {
require.NoError(t, err)
cases := []struct {
name string
setup func(t *testing.T) (*cacheEntry, safeCommitmentArray, []blocks.ROBlob)
setup func(t *testing.T) (*cacheEntry, [][]byte, []blocks.ROBlob)
err error
}{
{
name: "commitments mismatch - extra sidecar",
setup: func(t *testing.T) (*cacheEntry, safeCommitmentArray, []blocks.ROBlob) {
setup: func(t *testing.T) (*cacheEntry, [][]byte, []blocks.ROBlob) {
entry, commits, expected := filterTestCaseSetup(denebSlot, 6, []int{0, 1}, 4)(t)
commits[5] = nil
return entry, commits, expected
@@ -139,7 +139,7 @@ func TestFilter(t *testing.T) {
},
{
name: "sidecar missing",
setup: func(t *testing.T) (*cacheEntry, safeCommitmentArray, []blocks.ROBlob) {
setup: func(t *testing.T) (*cacheEntry, [][]byte, []blocks.ROBlob) {
entry, commits, expected := filterTestCaseSetup(denebSlot, 6, []int{0, 1}, 4)(t)
entry.scs[5] = nil
return entry, commits, expected
@@ -148,7 +148,7 @@ func TestFilter(t *testing.T) {
},
{
name: "commitments mismatch - different bytes",
setup: func(t *testing.T) (*cacheEntry, safeCommitmentArray, []blocks.ROBlob) {
setup: func(t *testing.T) (*cacheEntry, [][]byte, []blocks.ROBlob) {
entry, commits, expected := filterTestCaseSetup(denebSlot, 6, []int{0, 1}, 4)(t)
entry.scs[5].KzgCommitment = []byte("nope")
return entry, commits, expected
@@ -160,7 +160,7 @@ func TestFilter(t *testing.T) {
t.Run(c.name, func(t *testing.T) {
entry, commits, expected := c.setup(t)
// first (root) argument doesn't matter, it is just for logs
got, err := entry.filter([32]byte{}, commits)
got, err := entry.filter([32]byte{}, commits, 100)
if c.err != nil {
require.ErrorIs(t, err, c.err)
return

View File

@@ -42,7 +42,7 @@ go_test(
embed = [":go_default_library"],
deps = [
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",

View File

@@ -13,7 +13,7 @@ import (
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
@@ -25,7 +25,7 @@ import (
)
var (
errIndexOutOfBounds = errors.New("blob index in file name >= MaxBlobsPerBlock")
errIndexOutOfBounds = errors.New("blob index in file name >= DeprecatedMaxBlobsPerBlock")
errEmptyBlobWritten = errors.New("zero bytes written to disk when saving blob sidecar")
errSidecarEmptySSZData = errors.New("sidecar marshalled to an empty ssz byte slice")
errNoBasePath = errors.New("BlobStorage base path not specified in init")
@@ -218,6 +218,7 @@ func (bs *BlobStorage) Save(sidecar blocks.VerifiedROBlob) error {
partialMoved = true
blobsWrittenCounter.Inc()
blobSaveLatency.Observe(float64(time.Since(startTime).Milliseconds()))
return nil
}
@@ -255,8 +256,10 @@ func (bs *BlobStorage) Remove(root [32]byte) error {
// Indices generates a bitmap representing which BlobSidecar.Index values are present on disk for a given root.
// This value can be compared to the commitments observed in a block to determine which indices need to be found
// on the network to confirm data availability.
func (bs *BlobStorage) Indices(root [32]byte) ([fieldparams.MaxBlobsPerBlock]bool, error) {
var mask [fieldparams.MaxBlobsPerBlock]bool
func (bs *BlobStorage) Indices(root [32]byte, s primitives.Slot) ([]bool, error) {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(s)
mask := make([]bool, maxBlobsPerBlock)
rootDir := blobNamer{root: root}.dir()
entries, err := afero.ReadDir(bs.fs, rootDir)
if err != nil {
@@ -265,6 +268,7 @@ func (bs *BlobStorage) Indices(root [32]byte) ([fieldparams.MaxBlobsPerBlock]boo
}
return mask, err
}
for i := range entries {
if entries[i].IsDir() {
continue
@@ -281,7 +285,7 @@ func (bs *BlobStorage) Indices(root [32]byte) ([fieldparams.MaxBlobsPerBlock]boo
if err != nil {
return mask, errors.Wrapf(err, "unexpected directory entry breaks listing, %s", parts[0])
}
if u >= fieldparams.MaxBlobsPerBlock {
if u >= uint64(maxBlobsPerBlock) {
return mask, errIndexOutOfBounds
}
mask[u] = true

View File

@@ -10,7 +10,7 @@ import (
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
@@ -20,7 +20,7 @@ import (
)
func TestBlobStorage_SaveBlobData(t *testing.T) {
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 1, fieldparams.MaxBlobsPerBlock)
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 1, params.BeaconConfig().MaxBlobsPerBlock(1))
testSidecars, err := verification.BlobSidecarSliceNoop(sidecars)
require.NoError(t, err)
@@ -56,10 +56,10 @@ func TestBlobStorage_SaveBlobData(t *testing.T) {
require.NoError(t, bs.Save(sc))
actualSc, err := bs.Get(sc.BlockRoot(), sc.Index)
require.NoError(t, err)
expectedIdx := [fieldparams.MaxBlobsPerBlock]bool{false, false, true}
actualIdx, err := bs.Indices(actualSc.BlockRoot())
expectedIdx := []bool{false, false, true, false, false, false}
actualIdx, err := bs.Indices(actualSc.BlockRoot(), 100)
require.NoError(t, err)
require.Equal(t, expectedIdx, actualIdx)
require.DeepEqual(t, expectedIdx, actualIdx)
})
t.Run("round trip write then read", func(t *testing.T) {
@@ -132,19 +132,19 @@ func TestBlobIndicesBounds(t *testing.T) {
fs, bs := NewEphemeralBlobStorageWithFs(t)
root := [32]byte{}
okIdx := uint64(fieldparams.MaxBlobsPerBlock - 1)
okIdx := uint64(params.BeaconConfig().MaxBlobsPerBlock(0)) - 1
writeFakeSSZ(t, fs, root, okIdx)
indices, err := bs.Indices(root)
indices, err := bs.Indices(root, 100)
require.NoError(t, err)
var expected [fieldparams.MaxBlobsPerBlock]bool
expected := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
expected[okIdx] = true
for i := range expected {
require.Equal(t, expected[i], indices[i])
}
oobIdx := uint64(fieldparams.MaxBlobsPerBlock)
oobIdx := uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
writeFakeSSZ(t, fs, root, oobIdx)
_, err = bs.Indices(root)
_, err = bs.Indices(root, 100)
require.ErrorIs(t, err, errIndexOutOfBounds)
}
@@ -163,7 +163,7 @@ func TestBlobStoragePrune(t *testing.T) {
fs, bs := NewEphemeralBlobStorageWithFs(t)
t.Run("PruneOne", func(t *testing.T) {
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 300, fieldparams.MaxBlobsPerBlock)
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 300, params.BeaconConfig().MaxBlobsPerBlock(0))
testSidecars, err := verification.BlobSidecarSliceNoop(sidecars)
require.NoError(t, err)
@@ -178,7 +178,7 @@ func TestBlobStoragePrune(t *testing.T) {
require.Equal(t, 0, len(remainingFolders))
})
t.Run("Prune dangling blob", func(t *testing.T) {
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 299, fieldparams.MaxBlobsPerBlock)
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 299, params.BeaconConfig().MaxBlobsPerBlock(0))
testSidecars, err := verification.BlobSidecarSliceNoop(sidecars)
require.NoError(t, err)
@@ -198,7 +198,7 @@ func TestBlobStoragePrune(t *testing.T) {
for j := 0; j <= blockQty; j++ {
root := bytesutil.ToBytes32(bytesutil.ToBytes(uint64(slot), 32))
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, root, slot, fieldparams.MaxBlobsPerBlock)
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, root, slot, params.BeaconConfig().MaxBlobsPerBlock(0))
testSidecars, err := verification.BlobSidecarSliceNoop(sidecars)
require.NoError(t, err)
require.NoError(t, bs.Save(testSidecars[0]))
@@ -224,7 +224,7 @@ func BenchmarkPruning(b *testing.B) {
for j := 0; j <= blockQty; j++ {
root := bytesutil.ToBytes32(bytesutil.ToBytes(uint64(slot), 32))
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, root, slot, fieldparams.MaxBlobsPerBlock)
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, root, slot, params.BeaconConfig().MaxBlobsPerBlock(0))
testSidecars, err := verification.BlobSidecarSliceNoop(sidecars)
require.NoError(t, err)
require.NoError(t, bs.Save(testSidecars[0]))

View File

@@ -9,7 +9,7 @@ import (
)
// blobIndexMask is a bitmask representing the set of blob indices that are currently set.
type blobIndexMask [fieldparams.MaxBlobsPerBlock]bool
type blobIndexMask []bool
// BlobStorageSummary represents cached information about the BlobSidecars on disk for each root the cache knows about.
type BlobStorageSummary struct {
@@ -20,7 +20,11 @@ type BlobStorageSummary struct {
// HasIndex returns true if the BlobSidecar at the given index is available in the filesystem.
func (s BlobStorageSummary) HasIndex(idx uint64) bool {
// Protect from panic, but assume callers are sophisticated enough to not need an error telling them they have an invalid idx.
if idx >= fieldparams.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(s.slot)
if idx >= uint64(maxBlobsPerBlock) {
return false
}
if idx >= uint64(len(s.mask)) {
return false
}
return s.mask[idx]
@@ -28,7 +32,11 @@ func (s BlobStorageSummary) HasIndex(idx uint64) bool {
// AllAvailable returns true if we have all blobs for all indices from 0 to count-1.
func (s BlobStorageSummary) AllAvailable(count int) bool {
if count > fieldparams.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(s.slot)
if count > maxBlobsPerBlock {
return false
}
if count > len(s.mask) {
return false
}
for i := 0; i < count; i++ {
@@ -68,13 +76,17 @@ func (s *blobStorageCache) Summary(root [32]byte) BlobStorageSummary {
}
func (s *blobStorageCache) ensure(key [32]byte, slot primitives.Slot, idx uint64) error {
if idx >= fieldparams.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
if idx >= uint64(maxBlobsPerBlock) {
return errIndexOutOfBounds
}
s.mu.Lock()
defer s.mu.Unlock()
v := s.cache[key]
v.slot = slot
if v.mask == nil {
v.mask = make(blobIndexMask, maxBlobsPerBlock)
}
if !v.mask[idx] {
s.updateMetrics(1)
}

View File

@@ -3,13 +3,17 @@ package filesystem
import (
"testing"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestSlotByRoot_Summary(t *testing.T) {
var noneSet, allSet, firstSet, lastSet, oneSet blobIndexMask
noneSet := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
allSet := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
firstSet := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
lastSet := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
oneSet := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
firstSet[0] = true
lastSet[len(lastSet)-1] = true
oneSet[1] = true
@@ -19,49 +23,49 @@ func TestSlotByRoot_Summary(t *testing.T) {
cases := []struct {
name string
root [32]byte
expected *blobIndexMask
expected blobIndexMask
}{
{
name: "not found",
},
{
name: "none set",
expected: &noneSet,
expected: noneSet,
},
{
name: "index 1 set",
expected: &oneSet,
expected: oneSet,
},
{
name: "all set",
expected: &allSet,
expected: allSet,
},
{
name: "first set",
expected: &firstSet,
expected: firstSet,
},
{
name: "last set",
expected: &lastSet,
expected: lastSet,
},
}
sc := newBlobStorageCache()
for _, c := range cases {
if c.expected != nil {
key := bytesutil.ToBytes32([]byte(c.name))
sc.cache[key] = BlobStorageSummary{slot: 0, mask: *c.expected}
sc.cache[key] = BlobStorageSummary{slot: 0, mask: c.expected}
}
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
key := bytesutil.ToBytes32([]byte(c.name))
sum := sc.Summary(key)
for i := range c.expected {
for i, has := range c.expected {
ui := uint64(i)
if c.expected == nil {
require.Equal(t, false, sum.HasIndex(ui))
} else {
require.Equal(t, c.expected[i], sum.HasIndex(ui))
require.Equal(t, has, sum.HasIndex(ui))
}
}
})
@@ -121,13 +125,13 @@ func TestAllAvailable(t *testing.T) {
},
{
name: "out of bound is safe",
count: fieldparams.MaxBlobsPerBlock + 1,
count: params.BeaconConfig().MaxBlobsPerBlock(0) + 1,
aa: false,
},
{
name: "max present",
count: fieldparams.MaxBlobsPerBlock,
idxSet: idxUpTo(fieldparams.MaxBlobsPerBlock),
count: params.BeaconConfig().MaxBlobsPerBlock(0),
idxSet: idxUpTo(params.BeaconConfig().MaxBlobsPerBlock(0)),
aa: true,
},
{
@@ -139,7 +143,7 @@ func TestAllAvailable(t *testing.T) {
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
var mask blobIndexMask
mask := make([]bool, params.BeaconConfig().MaxBlobsPerBlock(0))
for _, idx := range c.idxSet {
mask[idx] = true
}

View File

@@ -12,7 +12,7 @@ import (
"time"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/testing/require"
@@ -25,7 +25,7 @@ func TestTryPruneDir_CachedNotExpired(t *testing.T) {
pr, err := newBlobPruner(fs, 0)
require.NoError(t, err)
slot := pr.windowSize
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, slot, fieldparams.MaxBlobsPerBlock)
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, slot, params.BeaconConfig().MaxBlobsPerBlock(slot))
sc, err := verification.BlobSidecarNoop(sidecars[0])
require.NoError(t, err)
rootStr := rootString(sc.BlockRoot())

View File

@@ -177,6 +177,7 @@ func (s *Service) blobEndpoints(blocker lookup.Blocker) []endpoint {
Blocker: blocker,
OptimisticModeFetcher: s.cfg.OptimisticModeFetcher,
FinalizationFetcher: s.cfg.FinalizationFetcher,
TimeFetcher: s.cfg.GenesisTimeFetcher,
}
const namespace = "blob"

View File

@@ -14,7 +14,9 @@ go_library(
"//beacon-chain/rpc/core:go_default_library",
"//beacon-chain/rpc/lookup:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",
"//monitoring/tracing/trace:go_default_library",
"//network/httputil:go_default_library",
"//runtime/version:go_default_library",

View File

@@ -12,7 +12,9 @@ import (
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/rpc/core"
field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
"github.com/prysmaticlabs/prysm/v5/network/httputil"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
@@ -23,7 +25,7 @@ func (s *Server) Blobs(w http.ResponseWriter, r *http.Request) {
ctx, span := trace.StartSpan(r.Context(), "beacon.Blobs")
defer span.End()
indices, err := parseIndices(r.URL)
indices, err := parseIndices(r.URL, s.TimeFetcher.CurrentSlot())
if err != nil {
httputil.HandleError(w, err.Error(), http.StatusBadRequest)
return
@@ -87,9 +89,9 @@ func (s *Server) Blobs(w http.ResponseWriter, r *http.Request) {
}
// parseIndices filters out invalid and duplicate blob indices
func parseIndices(url *url.URL) ([]uint64, error) {
func parseIndices(url *url.URL, s primitives.Slot) ([]uint64, error) {
rawIndices := url.Query()["indices"]
indices := make([]uint64, 0, field_params.MaxBlobsPerBlock)
indices := make([]uint64, 0, params.BeaconConfig().MaxBlobsPerBlock(s))
invalidIndices := make([]string, 0)
loop:
for _, raw := range rawIndices {
@@ -98,7 +100,7 @@ loop:
invalidIndices = append(invalidIndices, raw)
continue
}
if ix >= field_params.MaxBlobsPerBlock {
if ix >= uint64(params.BeaconConfig().MaxBlobsPerBlock(s)) {
invalidIndices = append(invalidIndices, raw)
continue
}

View File

@@ -52,6 +52,7 @@ func TestBlobs(t *testing.T) {
s := &Server{
OptimisticModeFetcher: mockChainService,
FinalizationFetcher: mockChainService,
TimeFetcher: mockChainService,
}
t.Run("genesis", func(t *testing.T) {
@@ -400,7 +401,7 @@ func Test_parseIndices(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := parseIndices(&url.URL{RawQuery: tt.query})
got, err := parseIndices(&url.URL{RawQuery: tt.query}, 0)
if err != nil && tt.wantErr != "" {
require.StringContains(t, tt.wantErr, err.Error())
return

View File

@@ -9,4 +9,5 @@ type Server struct {
Blocker lookup.Blocker
OptimisticModeFetcher blockchain.OptimisticModeFetcher
FinalizationFetcher blockchain.FinalizationFetcher
TimeFetcher blockchain.TimeFetcher
}

View File

@@ -190,7 +190,7 @@ func TestGetSpec(t *testing.T) {
data, ok := resp.Data.(map[string]interface{})
require.Equal(t, true, ok)
assert.Equal(t, 156, len(data))
assert.Equal(t, 159, len(data))
for k, v := range data {
t.Run(k, func(t *testing.T) {
switch k {
@@ -335,7 +335,7 @@ func TestGetSpec(t *testing.T) {
case "MAX_VOLUNTARY_EXITS":
assert.Equal(t, "52", v)
case "MAX_BLOBS_PER_BLOCK":
assert.Equal(t, "4", v)
assert.Equal(t, "6", v)
case "TIMELY_HEAD_FLAG_INDEX":
assert.Equal(t, "0x35", v)
case "TIMELY_SOURCE_FLAG_INDEX":
@@ -529,6 +529,10 @@ func TestGetSpec(t *testing.T) {
assert.Equal(t, "93", v)
case "MAX_PENDING_DEPOSITS_PER_EPOCH":
assert.Equal(t, "94", v)
case "TARGET_BLOBS_PER_BLOCK_ELECTRA":
assert.Equal(t, "6", v)
case "MAX_BLOBS_PER_BLOCK_ELECTRA":
assert.Equal(t, "9", v)
default:
t.Errorf("Incorrect key: %s", k)
}

View File

@@ -235,7 +235,7 @@ func (p *BeaconDbBlocker) Blobs(ctx context.Context, id string, indices []uint64
return make([]*blocks.VerifiedROBlob, 0), nil
}
if len(indices) == 0 {
m, err := p.BlobStorage.Indices(bytesutil.ToBytes32(root))
m, err := p.BlobStorage.Indices(bytesutil.ToBytes32(root), b.Block().Slot())
if err != nil {
log.WithFields(log.Fields{
"blockRoot": hexutil.Encode(root),
@@ -244,6 +244,9 @@ func (p *BeaconDbBlocker) Blobs(ctx context.Context, id string, indices []uint64
}
for k, v := range m {
if v {
if k >= len(commitments) {
return nil, &core.RpcError{Err: fmt.Errorf("blob index %d is more than blob kzg commitments :%dd", k, len(commitments)), Reason: core.BadRequest}
}
indices = append(indices, uint64(k))
}
}

View File

@@ -270,13 +270,14 @@ func (vs *Server) getPayloadHeaderFromBuilder(
return nil, errors.Wrap(err, "could not validate builder signature")
}
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
var kzgCommitments [][]byte
if bid.Version() >= version.Deneb {
kzgCommitments, err = bid.BlobKzgCommitments()
if err != nil {
return nil, errors.Wrap(err, "could not get blob kzg commitments")
}
if len(kzgCommitments) > fieldparams.MaxBlobsPerBlock {
if len(kzgCommitments) > maxBlobsPerBlock {
return nil, fmt.Errorf("builder returned too many kzg commitments: %d", len(kzgCommitments))
}
for _, c := range kzgCommitments {

View File

@@ -9,6 +9,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
@@ -44,7 +45,7 @@ func newBlobSync(current primitives.Slot, vbs verifiedROBlocks, cfg *blobSyncCon
return &blobSync{current: current, expected: expected, bbv: bbv, store: as}, nil
}
type blobVerifierMap map[[32]byte][fieldparams.MaxBlobsPerBlock]verification.BlobVerifier
type blobVerifierMap map[[32]byte][]verification.BlobVerifier
type blobSync struct {
store das.AvailabilityStore
@@ -106,7 +107,10 @@ type blobBatchVerifier struct {
}
func (bbv *blobBatchVerifier) newVerifier(rb blocks.ROBlob) verification.BlobVerifier {
m := bbv.verifiers[rb.BlockRoot()]
m, ok := bbv.verifiers[rb.BlockRoot()]
if !ok {
m = make([]verification.BlobVerifier, params.BeaconConfig().MaxBlobsPerBlock(rb.Slot()))
}
m[rb.Index] = bbv.newBlobVerifier(rb, verification.BackfillBlobSidecarRequirements)
bbv.verifiers[rb.BlockRoot()] = m
return m[rb.Index]

View File

@@ -180,7 +180,7 @@ func (c *blobsTestCase) setup(t *testing.T) (*Service, []blocks.ROBlob, func())
cleanup := func() {
params.OverrideBeaconConfig(cfg)
}
maxBlobs := fieldparams.MaxBlobsPerBlock
maxBlobs := int(params.BeaconConfig().MaxBlobsPerBlock(0))
chain, clock := defaultMockChain(t)
if c.chain == nil {
c.chain = chain
@@ -218,8 +218,8 @@ func (c *blobsTestCase) setup(t *testing.T) (*Service, []blocks.ROBlob, func())
rateLimiter: newRateLimiter(client),
}
byRootRate := params.BeaconConfig().MaxRequestBlobSidecars * fieldparams.MaxBlobsPerBlock
byRangeRate := params.BeaconConfig().MaxRequestBlobSidecars * fieldparams.MaxBlobsPerBlock
byRootRate := params.BeaconConfig().MaxRequestBlobSidecars * uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
byRangeRate := params.BeaconConfig().MaxRequestBlobSidecars * uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
s.setRateCollector(p2p.RPCBlobSidecarsByRootTopicV1, leakybucket.NewCollector(0.000001, int64(byRootRate), time.Second, false))
s.setRateCollector(p2p.RPCBlobSidecarsByRangeTopicV1, leakybucket.NewCollector(0.000001, int64(byRangeRate), time.Second, false))
@@ -310,7 +310,7 @@ func TestTestcaseSetup_BlocksAndBlobs(t *testing.T) {
req := blobRootRequestFromSidecars(sidecars)
expect := c.filterExpectedByRoot(t, sidecars, req)
defer cleanup()
maxed := nblocks * fieldparams.MaxBlobsPerBlock
maxed := nblocks * params.BeaconConfig().MaxBlobsPerBlock(0)
require.Equal(t, maxed, len(sidecars))
require.Equal(t, maxed, len(expect))
for _, sc := range sidecars {

View File

@@ -85,7 +85,6 @@ go_test(
"//beacon-chain/verification:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",

View File

@@ -20,7 +20,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
beaconsync "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -1083,7 +1082,7 @@ func TestCommitmentCountList(t *testing.T) {
name: "nil bss, sparse slots",
cc: []commitmentCount{
{slot: 11235, count: 1},
{slot: 11240, count: fieldparams.MaxBlobsPerBlock},
{slot: 11240, count: params.BeaconConfig().MaxBlobsPerBlock(0)},
{slot: 11250, count: 3},
},
expected: &blobRange{low: 11235, high: 11250},
@@ -1100,7 +1099,7 @@ func TestCommitmentCountList(t *testing.T) {
},
cc: []commitmentCount{
{slot: 0, count: 3, root: bytesutil.ToBytes32([]byte("0"))},
{slot: 5, count: fieldparams.MaxBlobsPerBlock, root: bytesutil.ToBytes32([]byte("1"))},
{slot: 5, count: params.BeaconConfig().MaxBlobsPerBlock(0), root: bytesutil.ToBytes32([]byte("1"))},
{slot: 15, count: 3},
},
expected: &blobRange{low: 0, high: 15},
@@ -1118,7 +1117,7 @@ func TestCommitmentCountList(t *testing.T) {
cc: []commitmentCount{
{slot: 0, count: 2, root: bytesutil.ToBytes32([]byte("0"))},
{slot: 5, count: 3},
{slot: 15, count: fieldparams.MaxBlobsPerBlock, root: bytesutil.ToBytes32([]byte("2"))},
{slot: 15, count: params.BeaconConfig().MaxBlobsPerBlock(0), root: bytesutil.ToBytes32([]byte("2"))},
},
expected: &blobRange{low: 5, high: 5},
request: &ethpb.BlobSidecarsByRangeRequest{StartSlot: 5, Count: 1},
@@ -1136,7 +1135,7 @@ func TestCommitmentCountList(t *testing.T) {
{slot: 0, count: 2, root: bytesutil.ToBytes32([]byte("0"))},
{slot: 5, count: 3},
{slot: 6, count: 3},
{slot: 15, count: fieldparams.MaxBlobsPerBlock, root: bytesutil.ToBytes32([]byte("2"))},
{slot: 15, count: params.BeaconConfig().MaxBlobsPerBlock(0), root: bytesutil.ToBytes32([]byte("2"))},
},
expected: &blobRange{low: 5, high: 6},
request: &ethpb.BlobSidecarsByRangeRequest{StartSlot: 5, Count: 2},
@@ -1155,7 +1154,7 @@ func TestCommitmentCountList(t *testing.T) {
{slot: 0, count: 2, root: bytesutil.ToBytes32([]byte("0"))},
{slot: 5, count: 3, root: bytesutil.ToBytes32([]byte("1"))},
{slot: 10, count: 3},
{slot: 15, count: fieldparams.MaxBlobsPerBlock, root: bytesutil.ToBytes32([]byte("2"))},
{slot: 15, count: params.BeaconConfig().MaxBlobsPerBlock(0), root: bytesutil.ToBytes32([]byte("2"))},
},
expected: &blobRange{low: 5, high: 10},
request: &ethpb.BlobSidecarsByRangeRequest{StartSlot: 5, Count: 6},

View File

@@ -292,7 +292,7 @@ func missingBlobRequest(blk blocks.ROBlock, store *filesystem.BlobStorage) (p2pt
if len(cmts) == 0 {
return nil, nil
}
onDisk, err := store.Indices(r)
onDisk, err := store.Indices(r, blk.Block().Slot())
if err != nil {
return nil, errors.Wrapf(err, "error checking existing blobs for checkpoint sync block root %#x", r)
}
@@ -333,7 +333,7 @@ func (s *Service) fetchOriginBlobs(pids []peer.ID) error {
}
shufflePeers(pids)
for i := range pids {
sidecars, err := sync.SendBlobSidecarByRoot(s.ctx, s.clock, s.cfg.P2P, pids[i], s.ctxMap, &req)
sidecars, err := sync.SendBlobSidecarByRoot(s.ctx, s.clock, s.cfg.P2P, pids[i], s.ctxMap, &req, rob.Block().Slot())
if err != nil {
continue
}

View File

@@ -11,10 +11,10 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/verify"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
@@ -139,7 +139,7 @@ func (s *Service) sendAndSaveBlobSidecars(ctx context.Context, request types.Blo
return nil
}
sidecars, err := SendBlobSidecarByRoot(ctx, s.cfg.clock, s.cfg.p2p, peerID, s.ctxMap, &request)
sidecars, err := SendBlobSidecarByRoot(ctx, s.cfg.clock, s.cfg.p2p, peerID, s.ctxMap, &request, block.Block().Slot())
if err != nil {
return err
}
@@ -181,15 +181,15 @@ func (s *Service) pendingBlobsRequestForBlock(root [32]byte, b interfaces.ReadOn
if len(cc) == 0 {
return nil, nil
}
return s.constructPendingBlobsRequest(root, len(cc))
return s.constructPendingBlobsRequest(root, len(cc), b.Block().Slot())
}
// constructPendingBlobsRequest creates a request for BlobSidecars by root, considering blobs already in DB.
func (s *Service) constructPendingBlobsRequest(root [32]byte, commitments int) (types.BlobSidecarsByRootReq, error) {
func (s *Service) constructPendingBlobsRequest(root [32]byte, commitments int, slot primitives.Slot) (types.BlobSidecarsByRootReq, error) {
if commitments == 0 {
return nil, nil
}
stored, err := s.cfg.blobStorage.Indices(root)
stored, err := s.cfg.blobStorage.Indices(root, slot)
if err != nil {
return nil, err
}
@@ -200,7 +200,7 @@ func (s *Service) constructPendingBlobsRequest(root [32]byte, commitments int) (
// requestsForMissingIndices constructs a slice of BlobIdentifiers that are missing from
// local storage, based on a mapping that represents which indices are locally stored,
// and the highest expected index.
func requestsForMissingIndices(storedIndices [fieldparams.MaxBlobsPerBlock]bool, commitments int, root [32]byte) []*eth.BlobIdentifier {
func requestsForMissingIndices(storedIndices []bool, commitments int, root [32]byte) []*eth.BlobIdentifier {
var ids []*eth.BlobIdentifier
for i := uint64(0); i < uint64(commitments); i++ {
if !storedIndices[i] {

View File

@@ -424,7 +424,7 @@ func TestConstructPendingBlobsRequest(t *testing.T) {
// No unknown indices.
root := [32]byte{1}
count := 3
actual, err := s.constructPendingBlobsRequest(root, count)
actual, err := s.constructPendingBlobsRequest(root, count, 100)
require.NoError(t, err)
require.Equal(t, 3, len(actual))
for i, id := range actual {
@@ -454,14 +454,14 @@ func TestConstructPendingBlobsRequest(t *testing.T) {
expected := []*eth.BlobIdentifier{
{Index: 1, BlockRoot: root[:]},
}
actual, err = s.constructPendingBlobsRequest(root, count)
actual, err = s.constructPendingBlobsRequest(root, count, 100)
require.NoError(t, err)
require.Equal(t, expected[0].Index, actual[0].Index)
require.DeepEqual(t, expected[0].BlockRoot, actual[0].BlockRoot)
}
func TestFilterUnknownIndices(t *testing.T) {
haveIndices := [fieldparams.MaxBlobsPerBlock]bool{true, true, true, false, false, false}
haveIndices := []bool{true, true, true, false, false, false}
blockRoot := [32]byte{}
count := 5

View File

@@ -10,7 +10,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
@@ -28,7 +27,7 @@ func (s *Service) streamBlobBatch(ctx context.Context, batch blockBatch, wQuota
defer span.End()
for _, b := range batch.canonical() {
root := b.Root()
idxs, err := s.cfg.blobStorage.Indices(b.Root())
idxs, err := s.cfg.blobStorage.Indices(b.Root(), b.Block().Slot())
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
return wQuota, errors.Wrapf(err, "could not retrieve sidecars for block root %#x", root)
@@ -146,8 +145,9 @@ func BlobRPCMinValidSlot(current primitives.Slot) (primitives.Slot, error) {
return slots.EpochStart(minStart)
}
func blobBatchLimit() uint64 {
return uint64(flags.Get().BlockBatchLimit / fieldparams.MaxBlobsPerBlock)
func blobBatchLimit(slot primitives.Slot) uint64 {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(slot)
return uint64(flags.Get().BlockBatchLimit / maxBlobsPerBlock)
}
func validateBlobsByRange(r *pb.BlobSidecarsByRangeRequest, current primitives.Slot) (rangeParams, error) {
@@ -200,7 +200,7 @@ func validateBlobsByRange(r *pb.BlobSidecarsByRangeRequest, current primitives.S
rp.end = rp.start
}
limit := blobBatchLimit()
limit := blobBatchLimit(current)
if limit > maxRequest {
limit = maxRequest
}

View File

@@ -4,7 +4,6 @@ import (
"testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
types "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -25,7 +24,7 @@ func (c *blobsTestCase) defaultOldestSlotByRange(t *testing.T) types.Slot {
}
func blobRangeRequestFromSidecars(scs []blocks.ROBlob) interface{} {
maxBlobs := fieldparams.MaxBlobsPerBlock
maxBlobs := params.BeaconConfig().MaxBlobsPerBlock(scs[0].Slot())
count := uint64(len(scs) / maxBlobs)
return &ethpb.BlobSidecarsByRangeRequest{
StartSlot: scs[0].Slot(),
@@ -135,7 +134,7 @@ func TestBlobByRangeOK(t *testing.T) {
Count: 20,
}
},
total: func() *int { x := fieldparams.MaxBlobsPerBlock * 10; return &x }(), // 10 blocks * 4 blobs = 40
total: func() *int { x := params.BeaconConfig().MaxBlobsPerBlock(0) * 10; return &x }(), // 10 blocks * 4 blobs = 40
},
{
name: "when request count > MAX_REQUEST_BLOCKS_DENEB, MAX_REQUEST_BLOBS_SIDECARS sidecars in response",
@@ -233,7 +232,7 @@ func TestBlobsByRangeValidation(t *testing.T) {
},
start: defaultMinStart,
end: defaultMinStart + 9,
batch: blobBatchLimit(),
batch: blobBatchLimit(100),
},
{
name: "count > MAX_REQUEST_BLOB_SIDECARS",
@@ -245,7 +244,7 @@ func TestBlobsByRangeValidation(t *testing.T) {
start: defaultMinStart,
end: defaultMinStart - 10 + 999,
// a large count is ok, we just limit the amount of actual responses
batch: blobBatchLimit(),
batch: blobBatchLimit(100),
},
{
name: "start + count > current",
@@ -267,7 +266,7 @@ func TestBlobsByRangeValidation(t *testing.T) {
},
start: denebSlot,
end: denebSlot + 89,
batch: blobBatchLimit(),
batch: blobBatchLimit(100),
},
}
for _, c := range cases {

View File

@@ -8,7 +8,6 @@ import (
"github.com/libp2p/go-libp2p/core/network"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2pTypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
types "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -223,7 +222,7 @@ func TestBlobsByRootValidation(t *testing.T) {
name: "block with all indices missing between 2 full blocks",
nblocks: 3,
missing: map[int]bool{1: true},
total: func(i int) *int { return &i }(2 * fieldparams.MaxBlobsPerBlock),
total: func(i int) *int { return &i }(2 * int(params.BeaconConfig().MaxBlobsPerBlock(0))),
},
{
name: "exceeds req max",

View File

@@ -12,7 +12,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/encoder"
p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -170,9 +169,10 @@ func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle,
}
defer closeStream(stream, log)
maxBlobsPerBlock := uint64(params.BeaconConfig().MaxBlobsPerBlock(req.StartSlot))
max := params.BeaconConfig().MaxRequestBlobSidecars
if max > req.Count*fieldparams.MaxBlobsPerBlock {
max = req.Count * fieldparams.MaxBlobsPerBlock
if max > req.Count*maxBlobsPerBlock {
max = req.Count * maxBlobsPerBlock
}
vfuncs := []BlobResponseValidation{blobValidatorFromRangeReq(req), newSequentialBlobValidator()}
if len(bvs) > 0 {
@@ -183,7 +183,7 @@ func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle,
func SendBlobSidecarByRoot(
ctx context.Context, tor blockchain.TemporalOracle, p2pApi p2p.P2P, pid peer.ID,
ctxMap ContextByteVersions, req *p2ptypes.BlobSidecarsByRootReq,
ctxMap ContextByteVersions, req *p2ptypes.BlobSidecarsByRootReq, slot primitives.Slot,
) ([]blocks.ROBlob, error) {
if uint64(len(*req)) > params.BeaconConfig().MaxRequestBlobSidecars {
return nil, errors.Wrapf(p2ptypes.ErrMaxBlobReqExceeded, "length=%d", len(*req))
@@ -201,8 +201,9 @@ func SendBlobSidecarByRoot(
defer closeStream(stream, log)
max := params.BeaconConfig().MaxRequestBlobSidecars
if max > uint64(len(*req))*fieldparams.MaxBlobsPerBlock {
max = uint64(len(*req)) * fieldparams.MaxBlobsPerBlock
maxBlobCount := params.BeaconConfig().MaxBlobsPerBlock(slot)
if max > uint64(len(*req)*maxBlobCount) {
max = uint64(len(*req) * maxBlobCount)
}
return readChunkEncodedBlobs(stream, p2pApi.Encoding(), ctxMap, blobValidatorFromRootReq(req), max)
}
@@ -227,7 +228,8 @@ type seqBlobValid struct {
}
func (sbv *seqBlobValid) nextValid(blob blocks.ROBlob) error {
if blob.Index >= fieldparams.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(blob.Slot())
if blob.Index >= uint64(maxBlobsPerBlock) {
return errBlobIndexOutOfBounds
}
if sbv.prev == nil {

View File

@@ -619,7 +619,7 @@ func TestSeqBlobValid(t *testing.T) {
wrongRoot, err := blocks.NewROBlobWithRoot(oops[2].BlobSidecar, bytesutil.ToBytes32([]byte("parentderp")))
require.NoError(t, err)
oob := oops[3]
oob.Index = fieldparams.MaxBlobsPerBlock
oob.Index = uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
cases := []struct {
name string

View File

@@ -81,7 +81,7 @@ func (s *Service) reconstructAndBroadcastBlobs(ctx context.Context, block interf
if s.cfg.blobStorage == nil {
return
}
indices, err := s.cfg.blobStorage.Indices(blockRoot)
indices, err := s.cfg.blobStorage.Indices(blockRoot, block.Block().Slot())
if err != nil {
log.WithError(err).Error("Failed to retrieve indices for block")
return
@@ -93,7 +93,7 @@ func (s *Service) reconstructAndBroadcastBlobs(ctx context.Context, block interf
}
// Reconstruct blob sidecars from the EL
blobSidecars, err := s.cfg.executionReconstructor.ReconstructBlobSidecars(ctx, block, blockRoot, indices[:])
blobSidecars, err := s.cfg.executionReconstructor.ReconstructBlobSidecars(ctx, block, blockRoot, indices)
if err != nil {
log.WithError(err).Error("Failed to reconstruct blob sidecars")
return
@@ -103,7 +103,7 @@ func (s *Service) reconstructAndBroadcastBlobs(ctx context.Context, block interf
}
// Refresh indices as new blobs may have been added to the db
indices, err = s.cfg.blobStorage.Indices(blockRoot)
indices, err = s.cfg.blobStorage.Indices(blockRoot, block.Block().Slot())
if err != nil {
log.WithError(err).Error("Failed to retrieve indices for block")
return

View File

@@ -16,7 +16,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
consensusblocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -303,8 +302,10 @@ func validateDenebBeaconBlock(blk interfaces.ReadOnlyBeaconBlock) error {
}
// [REJECT] The length of KZG commitments is less than or equal to the limitation defined in Consensus Layer
// -- i.e. validate that len(body.signed_beacon_block.message.blob_kzg_commitments) <= MAX_BLOBS_PER_BLOCK
if len(commits) > fieldparams.MaxBlobsPerBlock {
return errors.Wrapf(errRejectCommitmentLen, "%d > %d", len(commits), fieldparams.MaxBlobsPerBlock)
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(blk.Slot())
if len(commits) > maxBlobsPerBlock {
return errors.Wrapf(errRejectCommitmentLen, "%d > %d", len(commits), maxBlobsPerBlock)
}
return nil
}

View File

@@ -6,7 +6,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/verify",
visibility = ["//visibility:public"],
deps = [
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//encoding/bytesutil:go_default_library",
"//runtime/version:go_default_library",

View File

@@ -2,7 +2,7 @@ package verify
import (
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
@@ -20,8 +20,9 @@ func BlobAlignsWithBlock(blob blocks.ROBlob, block blocks.ROBlock) error {
if block.Version() < version.Deneb {
return nil
}
if blob.Index >= fieldparams.MaxBlobsPerBlock {
return errors.Wrapf(ErrIncorrectBlobIndex, "index %d exceeds MAX_BLOBS_PER_BLOCK %d", blob.Index, fieldparams.MaxBlobsPerBlock)
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(blob.Slot())
if blob.Index >= uint64(maxBlobsPerBlock) {
return errors.Wrapf(ErrIncorrectBlobIndex, "index %d exceeds MAX_BLOBS_PER_BLOCK %d", blob.Index, maxBlobsPerBlock)
}
if blob.BlockRoot() != block.Root() {

View File

@@ -26,7 +26,6 @@ go_library(
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//cache/lru:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",
@@ -60,7 +59,6 @@ go_test(
"//beacon-chain/forkchoice/types:go_default_library",
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",

View File

@@ -7,7 +7,6 @@ import (
"github.com/pkg/errors"
forkchoicetypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/forkchoice/types"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
@@ -129,7 +128,8 @@ func (bv *ROBlobVerifier) recordResult(req Requirement, err *error) {
// [REJECT] The sidecar's index is consistent with MAX_BLOBS_PER_BLOCK -- i.e. blob_sidecar.index < MAX_BLOBS_PER_BLOCK.
func (bv *ROBlobVerifier) BlobIndexInBounds() (err error) {
defer bv.recordResult(RequireBlobIndexInBounds, &err)
if bv.blob.Index >= fieldparams.MaxBlobsPerBlock {
maxBlobsPerBlock := params.BeaconConfig().MaxBlobsPerBlock(bv.blob.Slot())
if bv.blob.Index >= uint64(maxBlobsPerBlock) {
log.WithFields(logging.BlobFields(bv.blob)).Debug("Sidecar index >= MAX_BLOBS_PER_BLOCK")
return blobErrBuilder(ErrBlobIndexInvalid)
}

View File

@@ -12,7 +12,6 @@ import (
forkchoicetypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/forkchoice/types"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -32,7 +31,7 @@ func TestBlobIndexInBounds(t *testing.T) {
require.Equal(t, true, v.results.executed(RequireBlobIndexInBounds))
require.NoError(t, v.results.result(RequireBlobIndexInBounds))
b.Index = fieldparams.MaxBlobsPerBlock
b.Index = uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
v = ini.NewBlobVerifier(b, GossipBlobSidecarRequirements)
require.ErrorIs(t, v.BlobIndexInBounds(), ErrBlobIndexInvalid)
require.Equal(t, true, v.results.executed(RequireBlobIndexInBounds))

View File

@@ -26,7 +26,6 @@ const (
SyncCommitteeAggregationBytesLength = 16 // SyncCommitteeAggregationBytesLength defines the length of sync committee aggregate bytes.
SyncAggregateSyncCommitteeBytesLength = 64 // SyncAggregateSyncCommitteeBytesLength defines the length of sync committee bytes in a sync aggregate.
MaxWithdrawalsPerPayload = 16 // MaxWithdrawalsPerPayloadLength defines the maximum number of withdrawals that can be included in a payload.
MaxBlobsPerBlock = 6 // MaxBlobsPerBlock defines the maximum number of blobs with respect to consensus rule can be included in a block.
MaxBlobCommitmentsPerBlock = 4096 // MaxBlobCommitmentsPerBlock defines the theoretical limit of blobs can be included in a block.
LogMaxBlobCommitments = 12 // Log_2 of MaxBlobCommitmentsPerBlock
BlobLength = 131072 // BlobLength defines the byte length of a blob.

View File

@@ -26,7 +26,6 @@ const (
SyncCommitteeAggregationBytesLength = 1 // SyncCommitteeAggregationBytesLength defines the sync committee aggregate bytes.
SyncAggregateSyncCommitteeBytesLength = 4 // SyncAggregateSyncCommitteeBytesLength defines the length of sync committee bytes in a sync aggregate.
MaxWithdrawalsPerPayload = 4 // MaxWithdrawalsPerPayloadLength defines the maximum number of withdrawals that can be included in a payload.
MaxBlobsPerBlock = 6 // MaxBlobsPerBlock defines the maximum number of blobs with respect to consensus rule can be included in a block.
MaxBlobCommitmentsPerBlock = 16 // MaxBlobCommitmentsPerBlock defines the theoretical limit of blobs can be included in a block.
LogMaxBlobCommitments = 4 // Log_2 of MaxBlobCommitmentsPerBlock
BlobLength = 131072 // BlobLength defines the byte length of a blob.

View File

@@ -280,6 +280,19 @@ type BeaconChainConfig struct {
AttestationSubnetPrefixBits uint64 `yaml:"ATTESTATION_SUBNET_PREFIX_BITS" spec:"true"` // AttestationSubnetPrefixBits is defined as (ceillog2(ATTESTATION_SUBNET_COUNT) + ATTESTATION_SUBNET_EXTRA_BITS).
SubnetsPerNode uint64 `yaml:"SUBNETS_PER_NODE" spec:"true"` // SubnetsPerNode is the number of long-lived subnets a beacon node should be subscribed to.
NodeIdBits uint64 `yaml:"NODE_ID_BITS" spec:"true"` // NodeIdBits defines the bit length of a node id.
// Blobs Values
// Deprecated_MaxBlobsPerBlock defines the max blobs that could exist in a block.
// Deprecated: This field is no longer supported. Avoid using it.
DeprecatedMaxBlobsPerBlock int `yaml:"MAX_BLOBS_PER_BLOCK" spec:"true"`
// DeprecatedMaxBlobsPerBlockElectra defines the max blobs that could exist in a block post Electra hard fork.
// Deprecated: This field is no longer supported. Avoid using it.
DeprecatedMaxBlobsPerBlockElectra int `yaml:"MAX_BLOBS_PER_BLOCK_ELECTRA" spec:"true"`
// DeprecatedTargetBlobsPerBlockElectra defines the target number of blobs per block post Electra hard fork.
// Deprecated: This field is no longer supported. Avoid using it.
DeprecatedTargetBlobsPerBlockElectra int `yaml:"TARGET_BLOBS_PER_BLOCK_ELECTRA" spec:"true"`
}
// InitializeForkSchedule initializes the schedules forks baked into the config.
@@ -357,6 +370,24 @@ func (b *BeaconChainConfig) MaximumGossipClockDisparityDuration() time.Duration
return time.Duration(b.MaximumGossipClockDisparity) * time.Millisecond
}
// TargetBlobsPerBlock returns the target number of blobs per block for the given slot,
// accounting for changes introduced by the Electra fork.
func (b *BeaconChainConfig) TargetBlobsPerBlock(slot primitives.Slot) int {
if primitives.Epoch(slot.DivSlot(32)) >= b.ElectraForkEpoch {
return b.DeprecatedTargetBlobsPerBlockElectra
}
return b.DeprecatedMaxBlobsPerBlock / 2
}
// MaxBlobsPerBlock returns the maximum number of blobs per block for the given slot,
// adjusting for the Electra fork.
func (b *BeaconChainConfig) MaxBlobsPerBlock(slot primitives.Slot) int {
if primitives.Epoch(slot.DivSlot(32)) >= b.ElectraForkEpoch {
return b.DeprecatedMaxBlobsPerBlockElectra
}
return b.DeprecatedMaxBlobsPerBlock
}
// DenebEnabled centralizes the check to determine if code paths
// that are specific to deneb should be allowed to execute. This will make it easier to find call sites that do this
// kind of check and remove them post-deneb.

View File

@@ -2,6 +2,7 @@ package params_test
import (
"bytes"
"math"
"sync"
"testing"
@@ -105,3 +106,19 @@ func TestConfigGenesisValidatorRoot(t *testing.T) {
t.Fatal("mainnet params genesis validator root does not match the mainnet genesis state value")
}
}
func Test_MaxBlobCount(t *testing.T) {
cfg := params.MainnetConfig()
cfg.ElectraForkEpoch = 10
require.Equal(t, cfg.MaxBlobsPerBlock(primitives.Slot(cfg.ElectraForkEpoch)*cfg.SlotsPerEpoch-1), 6)
require.Equal(t, cfg.MaxBlobsPerBlock(primitives.Slot(cfg.ElectraForkEpoch)*cfg.SlotsPerEpoch), 9)
cfg.ElectraForkEpoch = math.MaxUint64
}
func Test_TargetBlobCount(t *testing.T) {
cfg := params.MainnetConfig()
cfg.ElectraForkEpoch = 10
require.Equal(t, cfg.TargetBlobsPerBlock(primitives.Slot(cfg.ElectraForkEpoch)*cfg.SlotsPerEpoch-1), 3)
require.Equal(t, cfg.TargetBlobsPerBlock(primitives.Slot(cfg.ElectraForkEpoch)*cfg.SlotsPerEpoch), 6)
cfg.ElectraForkEpoch = math.MaxUint64
}

View File

@@ -35,7 +35,6 @@ var placeholderFields = []string{
"EIP7732_FORK_VERSION",
"FIELD_ELEMENTS_PER_BLOB", // Compile time constant.
"KZG_COMMITMENT_INCLUSION_PROOF_DEPTH", // Compile time constant on BlobSidecar.commitment_inclusion_proof.
"MAX_BLOBS_PER_BLOCK",
"MAX_BLOBS_PER_BLOCK_EIP7594",
"MAX_BLOB_COMMITMENTS_PER_BLOCK", // Compile time constant on BeaconBlockBodyDeneb.blob_kzg_commitments.
"MAX_BYTES_PER_TRANSACTION", // Used for ssz of EL transactions. Unused in Prysm.

View File

@@ -319,6 +319,10 @@ var mainnetBeaconConfig = &BeaconChainConfig{
AttestationSubnetPrefixBits: 6,
SubnetsPerNode: 2,
NodeIdBits: 256,
DeprecatedMaxBlobsPerBlock: 6,
DeprecatedMaxBlobsPerBlockElectra: 9,
DeprecatedTargetBlobsPerBlockElectra: 6,
}
// MainnetTestConfig provides a version of the mainnet config that has a different name

View File

@@ -7,7 +7,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v5/testing/spectest/shared/common/merkle_proof",
visibility = ["//visibility:public"],
deps = [
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//container/trie:go_default_library",
"//testing/require:go_default_library",

View File

@@ -9,7 +9,7 @@ import (
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/golang/snappy"
fssz "github.com/prysmaticlabs/fastssz"
field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
consensus_blocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/container/trie"
"github.com/prysmaticlabs/prysm/v5/testing/require"
@@ -80,7 +80,7 @@ func runSingleMerkleProofTests(t *testing.T, config, forkOrPhase string, unmarsh
if err != nil {
return
}
if index < consensus_blocks.KZGOffset || index > consensus_blocks.KZGOffset+field_params.MaxBlobsPerBlock {
if index < consensus_blocks.KZGOffset || index > uint64(consensus_blocks.KZGOffset+params.BeaconConfig().MaxBlobsPerBlock(0)) {
return
}
localProof, err := consensus_blocks.MerkleProofKZGCommitment(body, int(index-consensus_blocks.KZGOffset))

View File

@@ -3,13 +3,13 @@ package util
import (
"testing"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestInclusionProofs(t *testing.T) {
_, blobs := GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 0, fieldparams.MaxBlobsPerBlock)
_, blobs := GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 0, params.BeaconConfig().MaxBlobsPerBlock(0))
for i := range blobs {
require.NoError(t, blocks.VerifyKZGInclusionProof(blobs[i]))
}