Compare commits

...

6 Commits

Author SHA1 Message Date
terence tsao
365c253cdd Make CI/CD green again 2023-03-11 12:59:50 -08:00
terence tsao
4f53fe0600 Make CI/CD green again 2023-03-11 12:36:38 -08:00
terence tsao
856ce6d82a Rm BlobsSidecar 2023-03-11 12:07:43 -08:00
terence tsao
6d2e597967 Revert pending block and blob changes 2023-03-11 10:38:36 -08:00
terence tsao
3919c214b9 Use blob by root for pending cache 2023-03-11 10:28:10 -08:00
terence tsao
38dee4077f Use block and blob cache before import 2023-03-10 09:46:10 -08:00
46 changed files with 944 additions and 2420 deletions

View File

@@ -21,7 +21,7 @@ linters:
linters-settings:
gocognit:
# TODO: We should target for < 50
min-complexity: 69
min-complexity: 100
output:
print-issued-lines: true

View File

@@ -17,7 +17,6 @@ go_library(
"//config/features:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//io/file:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)

View File

@@ -8,7 +8,6 @@ import (
"github.com/prysmaticlabs/prysm/v3/config/features"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/io/file"
eth "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
// WriteBlockToDisk as a block ssz. Writes to temp directory. Debug!
@@ -32,21 +31,3 @@ func WriteBlockToDisk(prefix string, block interfaces.ReadOnlySignedBeaconBlock,
log.WithError(err).Error("Failed to write to disk")
}
}
func WriteBadBlobsToDisk(prefix string, sideCar *eth.BlobsSidecar) {
if !features.Get().WriteSSZStateTransitions {
return
}
filename := fmt.Sprintf(prefix+"_blobs_%d.ssz", sideCar.BeaconBlockSlot)
fp := path.Join(os.TempDir(), filename)
log.Warnf("Writing blobs to disk at %s", fp)
enc, err := sideCar.MarshalSSZ()
if err != nil {
log.WithError(err).Error("Failed to ssz encode blobs")
return
}
if err := file.WriteFile(fp, enc); err != nil {
log.WithError(err).Error("Failed to write to disk")
}
}

View File

@@ -54,9 +54,6 @@ type ReadOnlyDatabase interface {
// Fee recipients operations.
FeeRecipientByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (common.Address, error)
RegistrationByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (*ethpb.ValidatorRegistrationV1, error)
// Blobs related methods.
BlobsSidecar(ctx context.Context, blockRoot [32]byte) (*ethpb.BlobsSidecar, error)
BlobsSidecarsBySlot(ctx context.Context, slot primitives.Slot) ([]*ethpb.BlobsSidecar, error)
// origin checkpoint sync support
OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error)
@@ -72,9 +69,6 @@ type NoHeadAccessDatabase interface {
SaveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock) error
SaveBlocks(ctx context.Context, blocks []interfaces.ReadOnlySignedBeaconBlock) error
SaveGenesisBlockRoot(ctx context.Context, blockRoot [32]byte) error
// Blob related methods.
SaveBlobsSidecar(ctx context.Context, blob *ethpb.BlobsSidecar) error
DeleteBlobsSidecar(ctx context.Context, blockRoot [32]byte) error
// State related methods.
SaveState(ctx context.Context, state state.ReadOnlyBeaconState, blockRoot [32]byte) error
SaveStates(ctx context.Context, states []state.ReadOnlyBeaconState, blockRoots [][32]byte) error

View File

@@ -5,7 +5,6 @@ go_library(
srcs = [
"archived_point.go",
"backup.go",
"blobs.go",
"blocks.go",
"checkpoint.go",
"deposit_contract.go",

View File

@@ -1,172 +0,0 @@
package kv
import (
"bytes"
"context"
"fmt"
"github.com/prysmaticlabs/prysm/v3/config/params"
types "github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
bolt "go.etcd.io/bbolt"
"go.opencensus.io/trace"
)
const blobSidecarKeyLength = 48 // slot_to_rotating_buffer(blob.slot) ++ blob.slot ++ blob.block_root
// SaveBlobsSidecar saves the blobs for a given epoch in the sidecar bucket. When we receive a blob:
//
// 1. Convert slot using a modulo operator to [0, maxSlots] where maxSlots = MAX_BLOB_EPOCHS*SLOTS_PER_EPOCH
//
// 2. Compute key for blob as bytes(slot_to_rotating_buffer(blob.slot)) ++ bytes(blob.slot) ++ blob.block_root
//
// 3. Begin the save algorithm: If the incoming blob has a slot bigger than the saved slot at the spot
// in the rotating keys buffer, we overwrite all elements for that slot.
//
// firstElemKey = getFirstElement(bucket)
// shouldOverwrite = blob.slot > bytes_to_slot(firstElemKey[8:16])
// if shouldOverwrite:
// for existingKey := seek prefix bytes(slot_to_rotating_buffer(blob.slot))
// bucket.delete(existingKey)
// bucket.put(key, blob)
func (s *Store) SaveBlobsSidecar(ctx context.Context, blobSidecar *ethpb.BlobsSidecar) error {
ctx, span := trace.StartSpan(ctx, "BeaconDB.SaveBlobsSidecar")
defer span.End()
return s.db.Update(func(tx *bolt.Tx) error {
encodedBlobSidecar, err := encode(ctx, blobSidecar)
if err != nil {
return err
}
bkt := tx.Bucket(blobsBucket)
c := bkt.Cursor()
key := blobSidecarKey(blobSidecar)
rotatingBufferPrefix := key[0:8]
var firstElementKey []byte
for k, _ := c.Seek(rotatingBufferPrefix); bytes.HasPrefix(k, rotatingBufferPrefix); k, _ = c.Next() {
if len(k) != 0 {
firstElementKey = k
break
}
}
// If there is no element stored at blob.slot % MAX_SLOTS_TO_PERSIST_BLOBS, then we simply
// store the blob by key and exit early.
if len(firstElementKey) == 0 {
return bkt.Put(key, encodedBlobSidecar)
} else if len(firstElementKey) != len(key) {
return fmt.Errorf(
"key length %d (%#x) != existing key length %d (%#x)",
len(key),
key,
len(firstElementKey),
firstElementKey,
)
}
slotOfFirstElement := firstElementKey[8:16]
// If we should overwrite old blobs at the spot in the rotating buffer, we clear data at that spot.
shouldOverwrite := blobSidecar.BeaconBlockSlot > bytesutil.BytesToSlotBigEndian(slotOfFirstElement)
if shouldOverwrite {
for k, _ := c.Seek(rotatingBufferPrefix); bytes.HasPrefix(k, rotatingBufferPrefix); k, _ = c.Next() {
if err := bkt.Delete(k); err != nil {
log.WithError(err).Warnf("Could not delete blob with key %#x", k)
}
}
}
return bkt.Put(key, encodedBlobSidecar)
})
}
// BlobsSidecar retrieves the blobs given a beacon block root.
func (s *Store) BlobsSidecar(ctx context.Context, beaconBlockRoot [32]byte) (*ethpb.BlobsSidecar, error) {
ctx, span := trace.StartSpan(ctx, "BeaconDB.BlobsSidecar")
defer span.End()
var enc []byte
if err := s.db.View(func(tx *bolt.Tx) error {
c := tx.Bucket(blobsBucket).Cursor()
// Bucket size is bounded and bolt cursors are fast. Moreover, a thin caching layer can be added.
for k, v := c.First(); k != nil; k, v = c.Next() {
if bytes.HasSuffix(k, beaconBlockRoot[:]) {
enc = v
break
}
}
return nil
}); err != nil {
return nil, err
}
if len(enc) == 0 {
return nil, nil
}
blob := &ethpb.BlobsSidecar{}
if err := decode(ctx, enc, blob); err != nil {
return nil, err
}
return blob, nil
}
// BlobsSidecarsBySlot retrieves sidecars from a slot.
func (s *Store) BlobsSidecarsBySlot(ctx context.Context, slot types.Slot) ([]*ethpb.BlobsSidecar, error) {
ctx, span := trace.StartSpan(ctx, "BeaconDB.BlobsSidecarsBySlot")
defer span.End()
encodedItems := make([][]byte, 0)
if err := s.db.View(func(tx *bolt.Tx) error {
c := tx.Bucket(blobsBucket).Cursor()
// Bucket size is bounded and bolt cursors are fast. Moreover, a thin caching layer can be added.
for k, v := c.First(); k != nil; k, v = c.Next() {
if len(k) != blobSidecarKeyLength {
continue
}
slotInKey := bytesutil.BytesToSlotBigEndian(k[8:16])
if slotInKey == slot {
encodedItems = append(encodedItems, v)
}
}
return nil
}); err != nil {
return nil, err
}
sidecars := make([]*ethpb.BlobsSidecar, len(encodedItems))
if len(encodedItems) == 0 {
return sidecars, nil
}
for i, enc := range encodedItems {
blob := &ethpb.BlobsSidecar{}
if err := decode(ctx, enc, blob); err != nil {
return nil, err
}
sidecars[i] = blob
}
return sidecars, nil
}
// DeleteBlobsSidecar returns true if the blobs are in the db.
func (s *Store) DeleteBlobsSidecar(ctx context.Context, beaconBlockRoot [32]byte) error {
ctx, span := trace.StartSpan(ctx, "BeaconDB.DeleteBlobsSidecar")
defer span.End()
return s.db.Update(func(tx *bolt.Tx) error {
bkt := tx.Bucket(blobsBucket)
c := bkt.Cursor()
for k, _ := c.First(); k != nil; k, _ = c.Next() {
if bytes.HasSuffix(k, beaconBlockRoot[:]) {
if err := bkt.Delete(k); err != nil {
return err
}
}
}
return nil
})
}
// We define a blob sidecar key as: bytes(slot_to_rotating_buffer(blob.slot)) ++ bytes(blob.slot) ++ blob.block_root
// where slot_to_rotating_buffer(slot) = slot % MAX_SLOTS_TO_PERSIST_BLOBS.
func blobSidecarKey(blob *ethpb.BlobsSidecar) []byte {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
maxEpochsToPersistBlobs := params.BeaconNetworkConfig().MinEpochsForBlobsSidecarsRequest
maxSlotsToPersistBlobs := types.Slot(maxEpochsToPersistBlobs.Mul(uint64(slotsPerEpoch)))
slotInRotatingBuffer := blob.BeaconBlockSlot.ModSlot(maxSlotsToPersistBlobs)
key := bytesutil.SlotToBytesBigEndian(slotInRotatingBuffer)
key = append(key, bytesutil.SlotToBytesBigEndian(blob.BeaconBlockSlot)...)
key = append(key, blob.BeaconBlockRoot...)
return key
}

View File

@@ -240,10 +240,6 @@ func (s *Store) DeleteBlock(ctx context.Context, root [32]byte) error {
return err
}
if err := s.DeleteBlobsSidecar(ctx, root); err != nil {
return err
}
return s.db.Update(func(tx *bolt.Tx) error {
bkt := tx.Bucket(finalizedBlockRootsIndexBucket)
if b := bkt.Get(root[:]); b != nil {

View File

@@ -2,44 +2,12 @@ package beacon
import (
"context"
"fmt"
"github.com/pkg/errors"
ethpbv1 "github.com/prysmaticlabs/prysm/v3/proto/eth/v1"
)
func (bs *Server) GetBlobsSidecar(ctx context.Context, req *ethpbv1.BlobsRequest) (*ethpbv1.BlobsResponse, error) {
sblk, err := bs.blockFromBlockID(ctx, req.BlockId)
err = handleGetBlockError(sblk, err)
if err != nil {
return nil, errors.Wrap(err, "GetBlobs")
}
block := sblk.Block()
root, err := block.HashTreeRoot()
if err != nil {
return nil, errors.Wrap(err, "failed to htr block")
}
sidecar, err := bs.BeaconDB.BlobsSidecar(ctx, root)
if err != nil {
return nil, fmt.Errorf("failed to get blobs sidecar for block %x", root)
}
var blobs []*ethpbv1.Blob
var aggregatedProof []byte
if sidecar != nil {
aggregatedProof = sidecar.AggregatedProof
for _, b := range sidecar.Blobs {
var data []byte
// go through each element, concat them
for range b.Data {
data = append(data, b.Data...)
}
blobs = append(blobs, &ethpbv1.Blob{Data: data})
}
}
return &ethpbv1.BlobsResponse{
BeaconBlockRoot: root[:],
BeaconBlockSlot: uint64(block.Slot()),
Blobs: blobs,
AggregatedProof: aggregatedProof,
}, nil
// TODO: implement this with new blob type request
return nil, errors.New("not implemented")
}

View File

@@ -11,7 +11,6 @@ import (
"github.com/ethereum/go-ethereum/common/hexutil"
emptypb "github.com/golang/protobuf/ptypes/empty"
"github.com/pkg/errors"
"github.com/protolambda/go-kzg/eth"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/builder"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/feed"
@@ -21,7 +20,6 @@ import (
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/transition/interop"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/db/kv"
"github.com/prysmaticlabs/prysm/v3/config/params"
blobs2 "github.com/prysmaticlabs/prysm/v3/consensus-types/blobs"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
@@ -355,22 +353,6 @@ func (vs *Server) proposeGenericBeaconBlock(ctx context.Context, req *ethpb.Gene
}
func (vs *Server) proposeBlockAndBlobs(ctx context.Context, root [32]byte, blk interfaces.ReadOnlySignedBeaconBlock, blobSidecars []*ethpb.SignedBlobSidecar) error {
blobs, err := vs.BlobsCache.Get(blk.Block().Slot())
if err != nil {
return errors.Wrap(err, "could not get blobs from cache")
}
sc := &ethpb.BlobsSidecar{
Blobs: blobs,
BeaconBlockSlot: blk.Block().Slot(),
BeaconBlockRoot: root[:],
}
aggregatedProof, err := eth.ComputeAggregateKZGProof(blobs2.BlobsSequenceImpl(blobs))
if err != nil {
interop.WriteBadBlobsToDisk("proposer", sc)
return fmt.Errorf("failed to compute aggregated kzg proof: %v", err)
}
sc.AggregatedProof = aggregatedProof[:]
for _, sidecar := range blobSidecars {
if err := vs.P2P.BroadcastBlob(ctx, sidecar.Message.Index, sidecar); err != nil {
return errors.Wrap(err, "could not broadcast blob sidecar")
@@ -379,9 +361,6 @@ func (vs *Server) proposeBlockAndBlobs(ctx context.Context, root [32]byte, blk i
if err := vs.BlockReceiver.ReceiveBlock(ctx, blk, root); err != nil {
return fmt.Errorf("could not process beacon block: %v", err)
}
if err := vs.BeaconDB.SaveBlobsSidecar(ctx, sc); err != nil {
return errors.Wrap(err, "could not save sidecar to DB")
}
return nil
}

View File

@@ -24,7 +24,6 @@ go_library(
"rpc_beacon_blocks_by_range.go",
"rpc_beacon_blocks_by_root.go",
"rpc_blob_sidecars_by_root.go",
"rpc_blobs_sidecars_by_range.go",
"rpc_chunked_response.go",
"rpc_goodbye.go",
"rpc_metadata.go",
@@ -94,7 +93,6 @@ go_library(
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blobs:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
@@ -155,7 +153,6 @@ go_test(
"rpc_beacon_blocks_by_range_test.go",
"rpc_beacon_blocks_by_root_test.go",
"rpc_blob_sidecars_by_root_test.go",
"rpc_blobs_sidecars_by_range_test.go",
"rpc_chunked_response_test.go",
"rpc_goodbye_test.go",
"rpc_metadata_test.go",
@@ -197,7 +194,6 @@ go_test(
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/iface:go_default_library",
"//beacon-chain/db/kv:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/execution:go_default_library",

View File

@@ -12,8 +12,8 @@ import (
)
type blockAndBlobs struct {
blk interfaces.SignedBeaconBlock
blobs []*eth.SignedBlobSidecar
blk interfaces.ReadOnlySignedBeaconBlock
blobs []*eth.BlobSidecar
}
type blockAndBlocksQueue struct {
@@ -27,7 +27,7 @@ func newBlockAndBlobs() *blockAndBlocksQueue {
}
}
func (q *blockAndBlocksQueue) addBlock(b interfaces.SignedBeaconBlock) error {
func (q *blockAndBlocksQueue) addBlock(b interfaces.ReadOnlySignedBeaconBlock) error {
q.lock.Lock()
defer q.lock.Unlock()
@@ -43,7 +43,7 @@ func (q *blockAndBlocksQueue) addBlock(b interfaces.SignedBeaconBlock) error {
if !ok {
q.queue[r] = blockAndBlobs{
blk: b,
blobs: make([]*eth.SignedBlobSidecar, 0, 4),
blobs: make([]*eth.BlobSidecar, 0, 4),
}
return nil
}
@@ -53,15 +53,15 @@ func (q *blockAndBlocksQueue) addBlock(b interfaces.SignedBeaconBlock) error {
return nil
}
func (q *blockAndBlocksQueue) addBlob(b *eth.SignedBlobSidecar) error {
func (q *blockAndBlocksQueue) addBlob(b *eth.BlobSidecar) error {
q.lock.Lock()
defer q.lock.Unlock()
r := bytesutil.ToBytes32(b.Message.BlockRoot)
r := bytesutil.ToBytes32(b.BlockRoot)
bnb, ok := q.queue[r]
if !ok {
q.queue[r] = blockAndBlobs{
blobs: make([]*eth.SignedBlobSidecar, 0, 4),
blobs: make([]*eth.BlobSidecar, 0, 4),
}
bnb = q.queue[r]
}
@@ -71,7 +71,7 @@ func (q *blockAndBlocksQueue) addBlob(b *eth.SignedBlobSidecar) error {
return nil
}
func (q *blockAndBlocksQueue) getBlock(r [32]byte) (interfaces.SignedBeaconBlock, error) {
func (q *blockAndBlocksQueue) getBlock(r [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
q.lock.RLock()
defer q.lock.RUnlock()
@@ -85,7 +85,7 @@ func (q *blockAndBlocksQueue) getBlock(r [32]byte) (interfaces.SignedBeaconBlock
return bnb.blk, nil
}
func (q *blockAndBlocksQueue) getBlob(r [32]byte, i uint64) (*eth.SignedBlobSidecar, error) {
func (q *blockAndBlocksQueue) getBlob(r [32]byte, i uint64) (*eth.BlobSidecar, error) {
q.lock.RLock()
defer q.lock.RUnlock()
@@ -98,7 +98,7 @@ func (q *blockAndBlocksQueue) getBlob(r [32]byte, i uint64) (*eth.SignedBlobSide
return nil, errors.New("blob does not exist")
}
for _, blob := range bnb.blobs {
if i == blob.Message.Index {
if i == blob.Index {
return blob, nil
}
}

View File

@@ -40,7 +40,7 @@ func Test_blockAndBlocksQueue(t *testing.T) {
})
t.Run("has one blob, no block", func(t *testing.T) {
q := newBlockAndBlobs()
require.NoError(t, q.addBlob(&eth.SignedBlobSidecar{Message: &eth.BlobSidecar{Index: 1}}))
require.NoError(t, q.addBlob(&eth.BlobSidecar{Index: 1}))
_, err := q.getBlock([32]byte{})
require.ErrorContains(t, "block does not exist", err)
_, err = q.getBlob([32]byte{}, 0)
@@ -62,21 +62,21 @@ func Test_blockAndBlocksQueue(t *testing.T) {
r, err := sb.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, q.addBlock(sb))
require.NoError(t, q.addBlob(&eth.SignedBlobSidecar{Message: &eth.BlobSidecar{BlockRoot: r[:], Index: 0}}))
require.NoError(t, q.addBlob(&eth.SignedBlobSidecar{Message: &eth.BlobSidecar{BlockRoot: r[:], Index: 1}}))
require.NoError(t, q.addBlob(&eth.SignedBlobSidecar{Message: &eth.BlobSidecar{BlockRoot: r[:], Index: 2}}))
require.NoError(t, q.addBlob(&eth.BlobSidecar{BlockRoot: r[:], Index: 0}))
require.NoError(t, q.addBlob(&eth.BlobSidecar{BlockRoot: r[:], Index: 1}))
require.NoError(t, q.addBlob(&eth.BlobSidecar{BlockRoot: r[:], Index: 2}))
blk, err := q.getBlock(r)
require.NoError(t, err)
require.DeepEqual(t, sb, blk)
blob, err := q.getBlob(r, 0)
require.NoError(t, err)
require.Equal(t, uint64(0), blob.Message.Index)
require.Equal(t, uint64(0), blob.Index)
blob, err = q.getBlob(r, 1)
require.NoError(t, err)
require.Equal(t, uint64(1), blob.Message.Index)
require.Equal(t, uint64(1), blob.Index)
blob, err = q.getBlob(r, 2)
require.NoError(t, err)
require.Equal(t, uint64(2), blob.Message.Index)
require.Equal(t, uint64(2), blob.Index)
has, err := q.hasEverything(r)
require.NoError(t, err)
require.Equal(t, true, has)
@@ -90,7 +90,7 @@ func Test_blockAndBlocksQueue(t *testing.T) {
r, err := sb.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, q.addBlock(sb))
require.NoError(t, q.addBlob(&eth.SignedBlobSidecar{Message: &eth.BlobSidecar{BlockRoot: r[:], Index: 2}}))
require.NoError(t, q.addBlob(&eth.BlobSidecar{BlockRoot: r[:], Index: 2}))
blk, err := q.getBlock(r)
require.NoError(t, err)
require.DeepEqual(t, sb, blk)
@@ -100,7 +100,7 @@ func Test_blockAndBlocksQueue(t *testing.T) {
require.ErrorContains(t, "blob does not exist", err)
blob, err := q.getBlob(r, 2)
require.NoError(t, err)
require.Equal(t, uint64(2), blob.Message.Index)
require.Equal(t, uint64(2), blob.Index)
has, err := q.hasEverything(r)
require.NoError(t, err)
require.Equal(t, false, has)

View File

@@ -29,7 +29,6 @@ go_library(
"//beacon-chain/sync:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blobs:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//container/leaky-bucket:go_default_library",

View File

@@ -20,7 +20,6 @@ import (
"github.com/prysmaticlabs/prysm/v3/crypto/rand"
"github.com/prysmaticlabs/prysm/v3/math"
p2ppb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/time/slots"
"github.com/sirupsen/logrus"
"go.opencensus.io/trace"
)
@@ -109,7 +108,6 @@ type fetchRequestResponse struct {
start primitives.Slot
count uint64
blocks []interfaces.ReadOnlySignedBeaconBlock
blobs []*p2ppb.BlobsSidecar
err error
}
@@ -251,7 +249,6 @@ func (f *blocksFetcher) handleRequest(ctx context.Context, start primitives.Slot
start: start,
count: count,
blocks: []interfaces.ReadOnlySignedBeaconBlock{},
blobs: []*p2ppb.BlobsSidecar{},
err: nil,
}
@@ -277,19 +274,7 @@ func (f *blocksFetcher) handleRequest(ctx context.Context, start primitives.Slot
}
response.blocks, response.pid, response.err = f.fetchBlocksFromPeer(ctx, start, count, peers)
var blobStart primitives.Slot
var blobCount uint64
for i := primitives.Slot(0); i < primitives.Slot(count); i++ {
if slots.WithinDataAvailabilityBound(uint64(f.chain.GenesisTime().Unix()), slots.ToEpoch(start+i)) {
blobStart = start + i
blobCount = count - uint64(i)
break
}
}
if slots.WithinDataAvailabilityBound(uint64(f.chain.GenesisTime().Unix()), slots.ToEpoch(blobStart)) {
response.blobs, response.pid, response.err = f.fetchBlobsFromPeer(ctx, blobStart, blobCount, peers)
}
return response
}
@@ -320,32 +305,6 @@ func (f *blocksFetcher) fetchBlocksFromPeer(
return nil, "", errNoPeersAvailable
}
// fetchBlobsFromPeer fetches blocks from a single randomly selected peer.
func (f *blocksFetcher) fetchBlobsFromPeer(
ctx context.Context,
start primitives.Slot, count uint64,
peers []peer.ID,
) ([]*p2ppb.BlobsSidecar, peer.ID, error) {
ctx, span := trace.StartSpan(ctx, "initialsync.fetchBlobsFromPeer")
defer span.End()
peers = f.filterPeers(ctx, peers, peersPercentagePerRequest)
req := &p2ppb.BlobsSidecarsByRangeRequest{
StartSlot: start,
Count: count,
}
for i := 0; i < len(peers); i++ {
blobs, err := f.requestBlobs(ctx, req, peers[i])
if err == nil {
f.p2p.Peers().Scorers().BlockProviderScorer().Touch(peers[i])
return blobs, peers[i], err
} else {
log.WithError(err).Debug("Could not request blobs by range")
}
}
return nil, "", errNoPeersAvailable
}
// requestBlocks is a wrapper for handling BeaconBlocksByRangeRequest requests/streams.
func (f *blocksFetcher) requestBlocks(
ctx context.Context,
@@ -376,30 +335,6 @@ func (f *blocksFetcher) requestBlocks(
return prysmsync.SendBeaconBlocksByRangeRequest(ctx, f.chain, f.p2p, pid, req, nil)
}
func (f *blocksFetcher) requestBlobs(ctx context.Context, req *p2ppb.BlobsSidecarsByRangeRequest, pid peer.ID) ([]*p2ppb.BlobsSidecar, error) {
if ctx.Err() != nil {
return nil, ctx.Err()
}
l := f.peerLock(pid)
l.Lock()
log.WithFields(logrus.Fields{
"peer": pid,
"start": req.StartSlot,
"count": req.Count,
"capacity": f.rateLimiter.Remaining(pid.String()),
"score": f.p2p.Peers().Scorers().BlockProviderScorer().FormatScorePretty(pid),
}).Debug("Requesting blobs")
if f.rateLimiter.Remaining(pid.String()) < int64(req.Count) {
if err := f.waitForBandwidth(pid, req.Count); err != nil {
l.Unlock()
return nil, err
}
}
f.rateLimiter.Add(pid.String(), int64(req.Count))
l.Unlock()
return prysmsync.SendBlobsSidecarsByRangeRequest(ctx, f.chain, f.p2p, pid, req, nil)
}
// requestBlocksByRoot is a wrapper for handling BeaconBlockByRootsReq requests/streams.
func (f *blocksFetcher) requestBlocksByRoot(
ctx context.Context,

View File

@@ -24,7 +24,6 @@ import (
type forkData struct {
peer peer.ID
blocks []interfaces.ReadOnlySignedBeaconBlock
blobs []*p2ppb.BlobsSidecar
}
// nonSkippedSlotAfter checks slots after the given one in an attempt to find a non-empty future slot.

View File

@@ -11,7 +11,6 @@ import (
beaconsync "github.com/prysmaticlabs/prysm/v3/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
eth "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/time/slots"
"github.com/sirupsen/logrus"
)
@@ -91,7 +90,6 @@ type blocksQueue struct {
type blocksQueueFetchedData struct {
pid peer.ID
blocks []interfaces.ReadOnlySignedBeaconBlock
blobs []*eth.BlobsSidecar
}
// newBlocksQueue creates initialized priority queue.
@@ -335,7 +333,6 @@ func (q *blocksQueue) onDataReceivedEvent(ctx context.Context) eventHandlerFn {
}
m.pid = response.pid
m.blocks = response.blocks
m.blobs = response.blobs
return stateDataParsed, nil
}
}
@@ -358,7 +355,6 @@ func (q *blocksQueue) onReadyToSendEvent(ctx context.Context) eventHandlerFn {
data := &blocksQueueFetchedData{
pid: m.pid,
blocks: m.blocks,
blobs: m.blobs,
}
select {
case <-ctx.Done():

View File

@@ -29,7 +29,6 @@ func (q *blocksQueue) resetFromFork(fork *forkData) error {
fsm := q.smm.addStateMachine(firstBlock.Slot())
fsm.pid = fork.peer
fsm.blocks = fork.blocks
fsm.blobs = fork.blobs
fsm.state = stateDataParsed
// The rest of machines are in skipped state.

View File

@@ -9,7 +9,6 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
eth "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
prysmTime "github.com/prysmaticlabs/prysm/v3/time"
"github.com/prysmaticlabs/prysm/v3/time/slots"
)
@@ -48,7 +47,6 @@ type stateMachine struct {
state stateID
pid peer.ID
blocks []interfaces.ReadOnlySignedBeaconBlock
blobs []*eth.BlobsSidecar
updated time.Time
}
@@ -81,7 +79,6 @@ func (smm *stateMachineManager) addStateMachine(startSlot primitives.Slot) *stat
start: startSlot,
state: stateNew,
blocks: []interfaces.ReadOnlySignedBeaconBlock{},
blobs: []*eth.BlobsSidecar{},
updated: prysmTime.Now(),
}
smm.recalculateMachineAttribs()
@@ -94,7 +91,6 @@ func (smm *stateMachineManager) removeStateMachine(startSlot primitives.Slot) er
return fmt.Errorf("state for machine %v is not found", startSlot)
}
smm.machines[startSlot].blocks = nil
smm.machines[startSlot].blobs = nil
delete(smm.machines, startSlot)
smm.recalculateMachineAttribs()
return nil

View File

@@ -10,10 +10,8 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
"github.com/paulbellamy/ratecounter"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/transition"
blobs2 "github.com/prysmaticlabs/prysm/v3/consensus-types/blobs"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
eth "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/time/slots"
"github.com/sirupsen/logrus"
)
@@ -83,15 +81,10 @@ func (s *Service) syncToFinalizedEpoch(ctx context.Context, genesis time.Time) e
}
for data := range queue.fetchedData {
blobsMap := make(map[primitives.Slot]*eth.BlobsSidecar)
for _, blob := range data.blobs {
blobsMap[blob.BeaconBlockSlot] = blob
}
// If blobs are available. Verify blobs and blocks are consistence.
// We can't import a block if there's no associated blob within DA bound.
// The blob has to pass aggregated proof check.
s.processFetchedData(ctx, genesis, s.cfg.Chain.HeadSlot(), data, blobsMap)
s.processFetchedData(ctx, genesis, s.cfg.Chain.HeadSlot(), data)
}
log.WithFields(logrus.Fields{
@@ -119,12 +112,7 @@ func (s *Service) syncToNonFinalizedEpoch(ctx context.Context, genesis time.Time
return err
}
for data := range queue.fetchedData {
blobsMap := make(map[primitives.Slot]*eth.BlobsSidecar)
for _, blob := range data.blobs {
blobsMap[blob.BeaconBlockSlot] = blob
}
s.processFetchedDataRegSync(ctx, genesis, s.cfg.Chain.HeadSlot(), data, blobsMap)
s.processFetchedDataRegSync(ctx, genesis, s.cfg.Chain.HeadSlot(), data)
}
log.WithFields(logrus.Fields{
"syncedSlot": s.cfg.Chain.HeadSlot(),
@@ -139,45 +127,24 @@ func (s *Service) syncToNonFinalizedEpoch(ctx context.Context, genesis time.Time
// processFetchedData processes data received from queue.
func (s *Service) processFetchedData(
ctx context.Context, genesis time.Time, startSlot primitives.Slot, data *blocksQueueFetchedData, blobs map[primitives.Slot]*eth.BlobsSidecar) {
ctx context.Context, genesis time.Time, startSlot primitives.Slot, data *blocksQueueFetchedData) {
defer s.updatePeerScorerStats(data.pid, startSlot)
// Use Batch Block Verify to process and verify batches directly.
if err := s.processBatchedBlocks(ctx, genesis, data.blocks, s.cfg.Chain.ReceiveBlockBatch, blobs); err != nil {
if err := s.processBatchedBlocks(ctx, genesis, data.blocks, s.cfg.Chain.ReceiveBlockBatch); err != nil {
log.WithError(err).Warn("Skip processing batched blocks")
}
}
// processFetchedData processes data received from queue.
func (s *Service) processFetchedDataRegSync(
ctx context.Context, genesis time.Time, startSlot primitives.Slot, data *blocksQueueFetchedData, blobs map[primitives.Slot]*eth.BlobsSidecar) {
ctx context.Context, genesis time.Time, startSlot primitives.Slot, data *blocksQueueFetchedData) {
defer s.updatePeerScorerStats(data.pid, startSlot)
blockReceiver := s.cfg.Chain.ReceiveBlock
invalidBlocks := 0
blksWithoutParentCount := 0
for _, blk := range data.blocks {
blkSlot := blk.Block().Slot()
if slots.WithinDataAvailabilityBound(uint64(s.cfg.Chain.GenesisTime().Unix()), slots.ToEpoch(blkSlot)) {
kzgs, err := blk.Block().Body().BlobKzgCommitments()
if err != nil {
log.WithError(err).Error("Failed to get kzg commitments")
continue
}
if len(kzgs) > 0 {
blob, ok := blobs[blkSlot]
if !ok {
log.Errorf("No blob found for block %d", blkSlot)
}
blkRoot, err := blk.Block().HashTreeRoot()
if err != nil {
log.WithError(err).Error("Failed to get block root")
}
if err := blobs2.ValidateBlobsSidecar(blkSlot, blkRoot, kzgs, blob); err != nil {
log.WithError(err).Error("Failed to validate blobs sidecar")
}
}
}
if err := s.processBlock(ctx, genesis, blk, blockReceiver); err != nil {
switch {
case errors.Is(err, errBlockAlreadyProcessed):
@@ -281,7 +248,7 @@ func (s *Service) processBlock(
}
func (s *Service) processBatchedBlocks(ctx context.Context, genesis time.Time,
blks []interfaces.ReadOnlySignedBeaconBlock, bFunc batchBlockReceiverFn, blobs map[primitives.Slot]*eth.BlobsSidecar) error {
blks []interfaces.ReadOnlySignedBeaconBlock, bFunc batchBlockReceiverFn) error {
if len(blks) == 0 {
return errors.New("0 blocks provided into method")
}
@@ -320,25 +287,6 @@ func (s *Service) processBatchedBlocks(ctx context.Context, genesis time.Time,
return err
}
blockRoots[i] = blkRoot
blkSlot := b.Block().Slot()
if slots.WithinDataAvailabilityBound(uint64(s.cfg.Chain.GenesisTime().Unix()), slots.ToEpoch(blkSlot)) {
kzgs, err := b.Block().Body().BlobKzgCommitments()
if err != nil {
return fmt.Errorf("failed to get blob kzg commitments: %w", err)
}
if len(kzgs) > 0 {
blob, ok := blobs[b.Block().Slot()]
if !ok {
return fmt.Errorf("missing sidecar blob for slot %d", b.Block().Slot())
}
if err := blobs2.ValidateBlobsSidecar(blkSlot, blkRoot, kzgs, blob); err != nil {
return err
}
if err := s.cfg.DB.SaveBlobsSidecar(ctx, blob); err != nil {
return err
}
}
}
}
return bFunc(ctx, blks, blockRoots)
}

View File

@@ -454,14 +454,14 @@ func TestService_processBlockBatch(t *testing.T) {
ctx context.Context, blks []interfaces.ReadOnlySignedBeaconBlock, blockRoots [][32]byte) error {
assert.NoError(t, s.cfg.Chain.ReceiveBlockBatch(ctx, blks, blockRoots))
return nil
}, map[primitives.Slot]*eth.BlobsSidecar{})
})
assert.NoError(t, err)
// Duplicate processing should trigger error.
err = s.processBatchedBlocks(ctx, genesis, batch, func(
ctx context.Context, blocks []interfaces.ReadOnlySignedBeaconBlock, blockRoots [][32]byte) error {
return nil
}, map[primitives.Slot]*eth.BlobsSidecar{})
})
assert.ErrorContains(t, "block is already processed", err)
var badBatch2 []interfaces.ReadOnlySignedBeaconBlock
@@ -477,7 +477,7 @@ func TestService_processBlockBatch(t *testing.T) {
err = s.processBatchedBlocks(ctx, genesis, badBatch2, func(
ctx context.Context, blks []interfaces.ReadOnlySignedBeaconBlock, blockRoots [][32]byte) error {
return nil
}, map[primitives.Slot]*eth.BlobsSidecar{})
})
expectedSubErr := "expected linear block list"
assert.ErrorContains(t, expectedSubErr, err)
@@ -486,7 +486,7 @@ func TestService_processBlockBatch(t *testing.T) {
ctx context.Context, blks []interfaces.ReadOnlySignedBeaconBlock, blockRoots [][32]byte) error {
assert.NoError(t, s.cfg.Chain.ReceiveBlockBatch(ctx, blks, blockRoots))
return nil
}, map[primitives.Slot]*eth.BlobsSidecar{})
})
assert.NoError(t, err)
assert.Equal(t, primitives.Slot(19), s.cfg.Chain.HeadSlot(), "Unexpected head slot")
})

View File

@@ -3,17 +3,14 @@ package sync
import (
"context"
"encoding/hex"
"fmt"
"sort"
"sync"
"time"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/async"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/blockchain"
p2ptypes "github.com/prysmaticlabs/prysm/v3/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v3/config/params"
blobs2 "github.com/prysmaticlabs/prysm/v3/consensus-types/blobs"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
@@ -21,7 +18,7 @@ import (
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v3/encoding/ssz/equality"
"github.com/prysmaticlabs/prysm/v3/monitoring/tracing"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/runtime/version"
"github.com/prysmaticlabs/prysm/v3/time/slots"
"github.com/sirupsen/logrus"
"github.com/trailofbits/go-mutexasserts"
@@ -87,18 +84,10 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
span.End()
continue
}
var blobs []*ethpb.BlobsSidecar
if slots.ToEpoch(slot) >= params.BeaconConfig().DenebForkEpoch {
blobs = s.pendingBlobsInCache(slot)
if len(bs) != len(blobs) {
log.Errorf("Processing pending blocks and blobs are not the same length: %d != %d", len(bs), len(blobs))
continue
}
}
s.pendingQueueLock.RUnlock()
// Loop through the pending queue and mark the potential parent blocks as seen.
for i, b := range bs {
for _, b := range bs {
if b == nil || b.IsNil() || b.Block().IsNil() {
span.End()
continue
@@ -170,55 +159,40 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
default:
}
if err := s.cfg.chain.ReceiveBlock(ctx, b, blkRoot); err != nil {
if blockchain.IsInvalidBlock(err) {
r := blockchain.InvalidBlockRoot(err)
if r != [32]byte{} {
s.setBadBlock(ctx, r) // Setting head block as bad.
} else {
s.setBadBlock(ctx, blkRoot)
}
if b.Version() >= version.Deneb {
if err := s.blockAndBlobs.addBlock(b); err != nil {
return err
}
log.WithError(err).WithField("slot", b.Block().Slot()).Debug("Could not process block")
// In the next iteration of the queue, this block will be removed from
// the pending queue as it has been marked as a 'bad' block.
span.End()
continue
}
if slots.ToEpoch(slot) >= params.BeaconConfig().DenebForkEpoch {
sc := blobs[i]
// TODO(TT): Add blob validation
kzgs, err := b.Block().Body().BlobKzgCommitments()
root, err := b.Block().HashTreeRoot()
if err != nil {
log.WithError(err).WithField("slot", b.Block().Slot()).Error("Could not get blob kzg commitments")
s.setBadBlock(ctx, blkRoot)
continue
return err
}
if err := blobs2.ValidateBlobsSidecar(b.Block().Slot(), blkRoot, kzgs, sc); err != nil {
log.WithError(err).WithField("slot", b.Block().Slot()).Error("Could not validate blobs sidecar from geth")
s.setBadBlock(ctx, blkRoot)
continue
hasEverything, err := s.blockAndBlobs.hasEverything(root)
if err != nil {
return err
}
if err := s.cfg.beaconDB.SaveBlobsSidecar(ctx, sc); err != nil {
log.WithError(err).WithField("slot", b.Block().Slot()).Error("Could not save blobs sidecar")
continue
if hasEverything {
if err := s.receiveBlockAndBlobs(ctx, b, root); err != nil {
return err
}
s.blockAndBlobs.delete(root)
}
} else {
if err := s.cfg.chain.ReceiveBlock(ctx, b, blkRoot); err != nil {
if s.receiveBlock(ctx, b, blkRoot) != nil {
log.WithError(err).WithField("slot", b.Block().Slot()).Debug("Could not process block")
continue
}
}
}
s.setSeenBlockIndexSlot(b.Block().Slot(), b.Block().ProposerIndex())
// Broadcasting the block again once a node is able to process it.
if slots.ToEpoch(slot) >= params.BeaconConfig().DenebForkEpoch {
if err := s.broadcastBlockAndBlobsSidecar(ctx, b, blobs[i]); err != nil {
log.WithError(err).Debug("Could not broadcast block and blobs sidecar")
}
pb, err := b.Proto()
if err != nil {
log.WithError(err).Debug("Could not get protobuf block")
} else {
pb, err := b.Proto()
if err != nil {
log.WithError(err).Debug("Could not get protobuf block")
}
if err := s.cfg.p2p.Broadcast(ctx, pb); err != nil {
log.WithError(err).Debug("Could not broadcast block")
}
@@ -243,18 +217,6 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
return s.sendBatchRootRequest(ctx, parentRoots, randGen)
}
func (s *Service) broadcastBlockAndBlobsSidecar(ctx context.Context, b interfaces.ReadOnlySignedBeaconBlock, blobs *ethpb.BlobsSidecar) error {
blkPb, err := b.PbDenebBlock()
if err != nil {
return err
}
pb := &ethpb.SignedBeaconBlockAndBlobsSidecar{
BeaconBlock: blkPb,
BlobsSidecar: blobs,
}
return s.cfg.p2p.Broadcast(ctx, pb)
}
func (s *Service) checkIfBlockIsBad(
ctx context.Context,
span *trace.Span,
@@ -305,21 +267,10 @@ func (s *Service) sendBatchRootRequest(ctx context.Context, roots [][32]byte, ra
if len(roots) > int(params.BeaconNetworkConfig().MaxRequestBlocks) {
req = roots[:params.BeaconNetworkConfig().MaxRequestBlocks]
}
if slots.ToEpoch(s.cfg.chain.CurrentSlot()) >= params.BeaconConfig().DenebForkEpoch {
if err := s.sendBlocksAndSidecarsRequest(ctx, &req, pid); err != nil {
log.WithError(err).Error("Could not send recent block and blob request, falling back to block only")
if err := s.sendRecentBeaconBlocksRequest(ctx, &req, pid); err != nil {
tracing.AnnotateError(span, err)
log.WithError(err).Error("Could not send recent block request after Deneb fork epoch")
}
}
} else {
if err := s.sendRecentBeaconBlocksRequest(ctx, &req, pid); err != nil {
tracing.AnnotateError(span, err)
log.WithError(err).Debug("Could not send recent block request")
}
if err := s.sendRecentBeaconBlocksRequest(ctx, &req, pid); err != nil {
tracing.AnnotateError(span, err)
log.WithError(err).Debug("Could not send recent block request")
}
newRoots := make([][32]byte, 0, len(roots))
s.pendingQueueLock.RLock()
for _, rt := range roots {
@@ -345,26 +296,15 @@ func (s *Service) sortedPendingSlots() []primitives.Slot {
items := s.slotToPendingBlocks.Items()
ss1 := make([]primitives.Slot, 0, len(items))
ss := make([]primitives.Slot, 0, len(items))
for k := range items {
slot := cacheKeyToSlot(k)
ss1 = append(ss1, slot)
ss = append(ss, slot)
}
sort.Slice(ss1, func(i, j int) bool {
return ss1[i] < ss1[j]
sort.Slice(ss, func(i, j int) bool {
return ss[i] < ss[j]
})
items = s.slotToPendingBlobs.Items()
ss2 := make([]primitives.Slot, 0, len(items))
for k := range items {
slot := cacheKeyToSlot(k)
ss2 = append(ss2, slot)
}
sort.Slice(ss2, func(i, j int) bool {
return ss2[i] < ss2[j]
})
return ss1
return ss
}
// validatePendingSlots validates the pending blocks
@@ -380,21 +320,10 @@ func (s *Service) validatePendingSlots() error {
if s.slotToPendingBlocks == nil {
return errors.New("slotToPendingBlocks cache can't be nil")
}
if s.slotToPendingBlobs == nil {
return errors.New("slotToPendingBlobs cache can't be nil")
}
items := s.slotToPendingBlocks.Items()
for k := range items {
slot := cacheKeyToSlot(k)
blks := s.pendingBlocksInCache(slot)
blobs := s.pendingBlobsInCache(slot)
if slots.ToEpoch(slot) >= params.BeaconConfig().DenebForkEpoch {
if len(blks) != len(blobs) {
return fmt.Errorf("blks and blobs are different length in pending queue: %d, %d", len(blks), len(blobs))
}
}
for _, b := range blks {
epoch := slots.ToEpoch(slot)
// remove all descendant blocks of old blocks
@@ -429,7 +358,6 @@ func (s *Service) clearPendingSlots() {
s.pendingQueueLock.Lock()
defer s.pendingQueueLock.Unlock()
s.slotToPendingBlocks.Flush()
s.slotToPendingBlobs.Flush()
s.seenPendingBlocks = make(map[[32]byte]bool)
}
@@ -443,22 +371,13 @@ func (s *Service) deleteBlockFromPendingQueue(slot primitives.Slot, b interfaces
return nil
}
postDeneb := slots.ToEpoch(slot) >= params.BeaconConfig().DenebForkEpoch
blobs := s.pendingBlobsInCache(slot)
if postDeneb {
if len(blobs) != len(blks) {
return fmt.Errorf("blobs and blks are different length in pending queue for deletion: %d, %d", len(blobs), len(blks))
}
}
// Defensive check to ignore nil blocks
if err := blocks.BeaconBlockIsNil(b); err != nil {
return err
}
newBlks := make([]interfaces.ReadOnlySignedBeaconBlock, 0, len(blks))
newBlobs := make([]*ethpb.BlobsSidecar, 0, len(blobs))
for i, blk := range blks {
for _, blk := range blks {
blkPb, err := blk.Proto()
if err != nil {
return err
@@ -471,15 +390,9 @@ func (s *Service) deleteBlockFromPendingQueue(slot primitives.Slot, b interfaces
continue
}
newBlks = append(newBlks, blk)
if postDeneb {
newBlobs = append(newBlobs, blobs[i])
}
}
if len(newBlks) == 0 {
s.slotToPendingBlocks.Delete(slotToCacheKey(slot))
if postDeneb {
s.slotToPendingBlobs.Delete(slotToCacheKey(slot))
}
delete(s.seenPendingBlocks, r)
return nil
}
@@ -489,20 +402,13 @@ func (s *Service) deleteBlockFromPendingQueue(slot primitives.Slot, b interfaces
if err := s.slotToPendingBlocks.Replace(slotToCacheKey(slot), newBlks, d); err != nil {
return err
}
if postDeneb {
if err := s.slotToPendingBlobs.Replace(slotToCacheKey(slot), newBlobs, d); err != nil {
return err
}
}
delete(s.seenPendingBlocks, r)
return nil
}
// Insert block to the list in the pending queue using the slot as key.
// Note: this helper is not thread safe.
func (s *Service) insertBlkAndBlobToQueue(_ primitives.Slot, b interfaces.ReadOnlySignedBeaconBlock, r [32]byte, blob *ethpb.BlobsSidecar) error {
func (s *Service) insertBlockToPendingQueue(_ primitives.Slot, b interfaces.ReadOnlySignedBeaconBlock, r [32]byte) error {
mutexasserts.AssertRWMutexLocked(&s.pendingQueueLock)
if s.seenPendingBlocks[r] {
@@ -512,9 +418,6 @@ func (s *Service) insertBlkAndBlobToQueue(_ primitives.Slot, b interfaces.ReadOn
if err := s.addPendingBlockToCache(b); err != nil {
return err
}
if err := s.addPeningBlobsToCache(blob); err != nil {
return err
}
s.seenPendingBlocks[r] = true
return nil
@@ -534,20 +437,6 @@ func (s *Service) pendingBlocksInCache(slot primitives.Slot) []interfaces.ReadOn
return blks
}
// This returns blobs given input key from slotToPendingBlobs.
func (s *Service) pendingBlobsInCache(slot primitives.Slot) []*ethpb.BlobsSidecar {
k := slotToCacheKey(slot)
value, ok := s.slotToPendingBlobs.Get(k)
if !ok {
return []*ethpb.BlobsSidecar{}
}
blobs, ok := value.([]*ethpb.BlobsSidecar)
if !ok {
return []*ethpb.BlobsSidecar{}
}
return blobs
}
// This adds input signed beacon block to slotToPendingBlocks cache.
func (s *Service) addPendingBlockToCache(b interfaces.ReadOnlySignedBeaconBlock) error {
if err := blocks.BeaconBlockIsNil(b); err != nil {
@@ -566,23 +455,6 @@ func (s *Service) addPendingBlockToCache(b interfaces.ReadOnlySignedBeaconBlock)
return nil
}
// This adds input signed beacon block to slotToPendingBlobs cache.
func (s *Service) addPeningBlobsToCache(b *ethpb.BlobsSidecar) error {
if b == nil {
return nil
}
blobs := s.pendingBlobsInCache(b.BeaconBlockSlot)
if len(blobs) >= maxBlocksPerSlot {
return nil
}
blobs = append(blobs, b)
k := slotToCacheKey(b.BeaconBlockSlot)
s.slotToPendingBlobs.Set(k, blobs, pendingBlockExpTime)
return nil
}
// Returns true if the genesis time has been set in chain service.
// Without the genesis time, the chain does not start.
func (s *Service) isGenesisTimeSet() bool {

View File

@@ -82,7 +82,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks1(t *testing.T) {
// Add b2 to the cache
wsb, err := blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b2.Block.Slot, wsb, b2Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wsb, b2Root))
require.NoError(t, r.processPendingBlocks(context.Background()))
assert.Equal(t, 1, len(r.slotToPendingBlocks.Items()), "Incorrect size for slot to pending blocks cache")
@@ -91,7 +91,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks1(t *testing.T) {
// Add b1 to the cache
wsb, err = blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b1.Block.Slot, wsb, b1Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wsb, b1Root))
util.SaveBlock(t, context.Background(), r.cfg.beaconDB, b1)
nBlock := util.NewBeaconBlock()
@@ -102,7 +102,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks1(t *testing.T) {
// Insert bad b1 in the cache to verify the good one doesn't get replaced.
wsb, err = blocks.NewSignedBeaconBlock(nBlock)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(nBlock.Block.Slot, wsb, nRoot, nil))
require.NoError(t, r.insertBlockToPendingQueue(nBlock.Block.Slot, wsb, nRoot))
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
require.NoError(t, r.processPendingBlocks(context.Background())) // Bad block removed on second run
@@ -155,7 +155,7 @@ func TestRegularSyncBeaconBlockSubscriber_OptimisticStatus(t *testing.T) {
// Add b2 to the cache
wsb, err := blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b2.Block.Slot, wsb, b2Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wsb, b2Root))
require.NoError(t, r.processPendingBlocks(context.Background()))
assert.Equal(t, 1, len(r.slotToPendingBlocks.Items()), "Incorrect size for slot to pending blocks cache")
@@ -164,7 +164,7 @@ func TestRegularSyncBeaconBlockSubscriber_OptimisticStatus(t *testing.T) {
// Add b1 to the cache
wsb, err = blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b1.Block.Slot, wsb, b1Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wsb, b1Root))
util.SaveBlock(t, context.Background(), r.cfg.beaconDB, b1)
nBlock := util.NewBeaconBlock()
@@ -175,7 +175,7 @@ func TestRegularSyncBeaconBlockSubscriber_OptimisticStatus(t *testing.T) {
// Insert bad b1 in the cache to verify the good one doesn't get replaced.
wsb, err = blocks.NewSignedBeaconBlock(nBlock)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(nBlock.Block.Slot, wsb, nRoot, nil))
require.NoError(t, r.insertBlockToPendingQueue(nBlock.Block.Slot, wsb, nRoot))
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
require.NoError(t, r.processPendingBlocks(context.Background())) // Bad block removed on second run
@@ -229,7 +229,7 @@ func TestRegularSyncBeaconBlockSubscriber_ExecutionEngineTimesOut(t *testing.T)
// Add b2 to the cache
wsb, err := blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b2.Block.Slot, wsb, b2Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wsb, b2Root))
require.NoError(t, r.processPendingBlocks(context.Background()))
assert.Equal(t, 1, len(r.slotToPendingBlocks.Items()), "Incorrect size for slot to pending blocks cache")
@@ -238,7 +238,7 @@ func TestRegularSyncBeaconBlockSubscriber_ExecutionEngineTimesOut(t *testing.T)
// Add b1 to the cache
wsb, err = blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b1.Block.Slot, wsb, b1Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wsb, b1Root))
util.SaveBlock(t, context.Background(), r.cfg.beaconDB, b1)
nBlock := util.NewBeaconBlock()
@@ -249,7 +249,7 @@ func TestRegularSyncBeaconBlockSubscriber_ExecutionEngineTimesOut(t *testing.T)
// Insert bad b1 in the cache to verify the good one doesn't get replaced.
wsb, err = blocks.NewSignedBeaconBlock(nBlock)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(nBlock.Block.Slot, wsb, nRoot, nil))
require.NoError(t, r.insertBlockToPendingQueue(nBlock.Block.Slot, wsb, nRoot))
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
require.NoError(t, r.processPendingBlocks(context.Background())) // Bad block removed on second run
@@ -292,24 +292,24 @@ func TestRegularSync_InsertDuplicateBlocks(t *testing.T) {
wsb, err := blocks.NewSignedBeaconBlock(b0)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b0.Block.Slot, wsb, b0r, nil))
require.NoError(t, r.insertBlockToPendingQueue(b0.Block.Slot, wsb, b0r))
require.Equal(t, 1, len(r.pendingBlocksInCache(b0.Block.Slot)), "Block was not added to map")
wsb, err = blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b1.Block.Slot, wsb, b1r, nil))
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wsb, b1r))
require.Equal(t, 1, len(r.pendingBlocksInCache(b1.Block.Slot)), "Block was not added to map")
// Add duplicate block which should not be saved.
wsb, err = blocks.NewSignedBeaconBlock(b0)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b0.Block.Slot, wsb, b0r, nil))
require.NoError(t, r.insertBlockToPendingQueue(b0.Block.Slot, wsb, b0r))
require.Equal(t, 1, len(r.pendingBlocksInCache(b0.Block.Slot)), "Block was added to map")
// Add duplicate block which should not be saved.
wsb, err = blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b1.Block.Slot, wsb, b1r, nil))
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wsb, b1r))
require.Equal(t, 1, len(r.pendingBlocksInCache(b1.Block.Slot)), "Block was added to map")
}
@@ -350,7 +350,7 @@ func TestRegularSyncBeaconBlockSubscriber_DoNotReprocessBlock(t *testing.T) {
// Add b3 to the cache
wsb, err := blocks.NewSignedBeaconBlock(b3)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b3.Block.Slot, wsb, b3Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b3.Block.Slot, wsb, b3Root))
require.NoError(t, r.processPendingBlocks(context.Background()))
assert.Equal(t, 0, len(r.slotToPendingBlocks.Items()), "Incorrect size for slot to pending blocks cache")
@@ -443,10 +443,10 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
wsb, err := blocks.NewSignedBeaconBlock(b4)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b4.Block.Slot, wsb, b4Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b4.Block.Slot, wsb, b4Root))
wsb, err = blocks.NewSignedBeaconBlock(b5)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b5.Block.Slot, wsb, b5Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b5.Block.Slot, wsb, b5Root))
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
require.NoError(t, r.processPendingBlocks(context.Background())) // Bad block removed on second run
@@ -457,7 +457,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
// Add b3 to the cache
wsb, err = blocks.NewSignedBeaconBlock(b3)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b3.Block.Slot, wsb, b3Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b3.Block.Slot, wsb, b3Root))
util.SaveBlock(t, context.Background(), r.cfg.beaconDB, b3)
require.NoError(t, r.processPendingBlocks(context.Background())) // Marks a block as bad
@@ -469,7 +469,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
// Add b2 to the cache
wsb, err = blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b2.Block.Slot, wsb, b2Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wsb, b2Root))
util.SaveBlock(t, context.Background(), r.cfg.beaconDB, b2)
@@ -543,16 +543,16 @@ func TestRegularSyncBeaconBlockSubscriber_PruneOldPendingBlocks(t *testing.T) {
wsb, err := blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b2.Block.Slot, wsb, b2Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wsb, b2Root))
wsb, err = blocks.NewSignedBeaconBlock(b3)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b3.Block.Slot, wsb, b3Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b3.Block.Slot, wsb, b3Root))
wsb, err = blocks.NewSignedBeaconBlock(b4)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b4.Block.Slot, wsb, b4Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b4.Block.Slot, wsb, b4Root))
wsb, err = blocks.NewSignedBeaconBlock(b5)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b5.Block.Slot, wsb, b5Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b5.Block.Slot, wsb, b5Root))
require.NoError(t, r.processPendingBlocks(context.Background()))
assert.Equal(t, 0, len(r.slotToPendingBlocks.Items()), "Incorrect size for slot to pending blocks cache")
@@ -569,16 +569,16 @@ func TestService_sortedPendingSlots(t *testing.T) {
var lastSlot primitives.Slot = math.MaxUint64
wsb, err := blocks.NewSignedBeaconBlock(util.HydrateSignedBeaconBlock(&ethpb.SignedBeaconBlock{Block: &ethpb.BeaconBlock{Slot: lastSlot}}))
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(lastSlot, wsb, [32]byte{1}, nil))
require.NoError(t, r.insertBlockToPendingQueue(lastSlot, wsb, [32]byte{1}))
wsb, err = blocks.NewSignedBeaconBlock(util.HydrateSignedBeaconBlock(&ethpb.SignedBeaconBlock{Block: &ethpb.BeaconBlock{Slot: lastSlot - 3}}))
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(lastSlot-3, wsb, [32]byte{2}, nil))
require.NoError(t, r.insertBlockToPendingQueue(lastSlot-3, wsb, [32]byte{2}))
wsb, err = blocks.NewSignedBeaconBlock(util.HydrateSignedBeaconBlock(&ethpb.SignedBeaconBlock{Block: &ethpb.BeaconBlock{Slot: lastSlot - 5}}))
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(lastSlot-5, wsb, [32]byte{3}, nil))
require.NoError(t, r.insertBlockToPendingQueue(lastSlot-5, wsb, [32]byte{3}))
wsb, err = blocks.NewSignedBeaconBlock(util.HydrateSignedBeaconBlock(&ethpb.SignedBeaconBlock{Block: &ethpb.BeaconBlock{Slot: lastSlot - 2}}))
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(lastSlot-2, wsb, [32]byte{4}, nil))
require.NoError(t, r.insertBlockToPendingQueue(lastSlot-2, wsb, [32]byte{4}))
want := []primitives.Slot{lastSlot - 5, lastSlot - 3, lastSlot - 2, lastSlot}
assert.DeepEqual(t, want, r.sortedPendingSlots(), "Unexpected pending slots list")
@@ -691,19 +691,19 @@ func TestService_AddPendingBlockToQueueOverMax(t *testing.T) {
b2.Block.StateRoot = []byte{'b'}
wsb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(0, wsb, [32]byte{}, nil))
require.NoError(t, r.insertBlockToPendingQueue(0, wsb, [32]byte{}))
wsb, err = blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(0, wsb, [32]byte{1}, nil))
require.NoError(t, r.insertBlockToPendingQueue(0, wsb, [32]byte{1}))
wsb, err = blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(0, wsb, [32]byte{2}, nil))
require.NoError(t, r.insertBlockToPendingQueue(0, wsb, [32]byte{2}))
b3 := ethpb.CopySignedBeaconBlock(b)
b3.Block.StateRoot = []byte{'c'}
wsb, err = blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(0, wsb, [32]byte{3}, nil))
require.NoError(t, r.insertBlockToPendingQueue(0, wsb, [32]byte{3}))
require.Equal(t, maxBlocksPerSlot, len(r.pendingBlocksInCache(0)))
}
@@ -772,15 +772,15 @@ func TestService_ProcessPendingBlockOnCorrectSlot(t *testing.T) {
// Add block1 for slot1
wsb, err := blocks.NewSignedBeaconBlock(b1)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b1.Block.Slot, wsb, b1Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b1.Block.Slot, wsb, b1Root))
// Add block2 for slot2
wsb, err = blocks.NewSignedBeaconBlock(b2)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b2.Block.Slot, wsb, b2Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b2.Block.Slot, wsb, b2Root))
// Add block3 for slot3
wsb, err = blocks.NewSignedBeaconBlock(b3)
require.NoError(t, err)
require.NoError(t, r.insertBlkAndBlobToQueue(b3.Block.Slot, wsb, b3Root, nil))
require.NoError(t, r.insertBlockToPendingQueue(b3.Block.Slot, wsb, b3Root))
// processPendingBlocks should process only blocks of the current slot. i.e. slot 1.
// Then check if the other two blocks are still in the pendingQueue.
@@ -843,9 +843,8 @@ func TestService_ProcessBadPendingBlocks(t *testing.T) {
assert.NoError(t, err)
// TODO: using the old deprecated type here because that's what the code expects, will get fixed with new sync queue
bs := &ethpb.BlobsSidecar{}
// Add block1 for slot 55
require.NoError(t, r.insertBlkAndBlobToQueue(b.Block.Slot, bA, b1Root, bs))
require.NoError(t, r.insertBlockToPendingQueue(b.Block.Slot, bA, b1Root))
bB, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlock())
assert.NoError(t, err)
// remove with a different block from the same slot.

View File

@@ -98,10 +98,6 @@ func (s *Service) registerRPCHandlersAltair() {
}
func (s *Service) registerRPCHandlersDeneb() {
s.registerRPC(
p2p.RPCBlobsSidecarsByRangeTopicV1,
s.blobsSidecarsByRangeRPCHandler,
)
s.registerRPC(
p2p.RPCBlobSidecarsByRootTopicV1,
s.blobSidecarByRootRPCHandler,

View File

@@ -11,6 +11,8 @@ import (
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
eth "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/runtime/version"
)
// sendRecentBeaconBlocksRequest sends a recent beacon blocks request to a peer to get
@@ -26,9 +28,32 @@ func (s *Service) sendRecentBeaconBlocksRequest(ctx context.Context, blockRoots
}
s.pendingQueueLock.Lock()
defer s.pendingQueueLock.Unlock()
if err := s.insertBlkAndBlobToQueue(blk.Block().Slot(), blk, blkRoot, nil); err != nil {
if err := s.insertBlockToPendingQueue(blk.Block().Slot(), blk, blkRoot); err != nil {
return err
}
if blk.Version() >= version.Deneb {
c, err := blk.Block().Body().BlobKzgCommitments()
if err != nil {
return err
}
blobIdentifiers := make([]*eth.BlobIdentifier, len(c))
for i := 0; i < len(c); i++ {
blobIdentifiers[i] = &eth.BlobIdentifier{
BlockRoot: blkRoot[:],
Index: uint64(i),
}
}
blobs, err := SendBlobSidecarByRoot(ctx, s.cfg.chain, s.cfg.p2p, id, blobIdentifiers)
if err != nil {
return err
}
for _, blob := range blobs {
// TODO(TT): Validate the blob is valid
if err := s.blockAndBlobs.addBlob(blob); err != nil {
return err
}
}
}
return nil
})
return err

View File

@@ -4,17 +4,14 @@ import (
"context"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v3/monitoring/tracing"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/time/slots"
"go.opencensus.io/trace"
)
@@ -81,26 +78,3 @@ func (s *Service) blobSidecarByRootRPCHandler(ctx context.Context, msg interface
}
return nil
}
func (s *Service) sendBlocksAndSidecarsRequest(ctx context.Context, blockRoots *types.BeaconBlockByRootsReq, id peer.ID) error {
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
_, err := SendBlocksAndSidecarsByRootRequest(ctx, s.cfg.chain, s.cfg.p2p, id, blockRoots, func(blkAndSidecar *ethpb.SignedBeaconBlockAndBlobsSidecar) error {
blk, err := blocks.NewSignedBeaconBlock(blkAndSidecar.BeaconBlock)
if err != nil {
return err
}
blkRoot, err := blk.Block().HashTreeRoot()
if err != nil {
return err
}
s.pendingQueueLock.Lock()
defer s.pendingQueueLock.Unlock()
if err := s.insertBlkAndBlobToQueue(blk.Block().Slot(), blk, blkRoot, blkAndSidecar.BlobsSidecar); err != nil {
return err
}
return nil
})
return err
}

View File

@@ -1,97 +0,0 @@
package sync
import (
"context"
"time"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/pkg/errors"
p2ptypes "github.com/prysmaticlabs/prysm/v3/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v3/monitoring/tracing"
pb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"go.opencensus.io/trace"
)
type BlobsSidecarProcessor func(sidecar *pb.BlobsSidecar) error
// blobsSidecarsByRangeRPCHandler looks up the request blobs from the database from a given start slot index
func (s *Service) blobsSidecarsByRangeRPCHandler(ctx context.Context, msg interface{}, stream libp2pcore.Stream) error {
ctx, span := trace.StartSpan(ctx, "sync.BlobsSidecarsByRangeHandler")
defer span.End()
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
SetRPCStreamDeadlines(stream)
r, ok := msg.(*pb.BlobsSidecarsByRangeRequest)
if !ok {
return errors.New("message is not type *pb.BlobsSidecarsByRangeRequest")
}
if err := s.rateLimiter.validateRequest(stream, 1); err != nil {
return err
}
sidecarLimiter, err := s.rateLimiter.topicCollector(string(stream.Protocol()))
if err != nil {
return err
}
startSlot := r.StartSlot
count := r.Count
endSlot := startSlot.Add(count)
var numBlobs uint64
maxRequestBlobsSidecars := params.BeaconNetworkConfig().MaxRequestBlobsSidecars
for slot := startSlot; slot < endSlot && numBlobs < maxRequestBlobsSidecars; slot = slot.Add(1) {
sidecars, err := s.cfg.beaconDB.BlobsSidecarsBySlot(ctx, slot)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)
return err
}
if len(sidecars) == 0 {
continue
}
for i, sidecar := range sidecars {
if bytesutil.ToBytes32(sidecar.BeaconBlockRoot) == params.BeaconConfig().ZeroHash {
continue
}
SetStreamWriteDeadline(stream, defaultWriteDuration)
if chunkErr := WriteBlobsSidecarChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), sidecar); chunkErr != nil {
log.WithError(chunkErr).Debug("Could not send a chunked response")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, chunkErr)
return chunkErr
}
s.rateLimiter.add(stream, 1)
numBlobs += 1
// Short-circuit immediately once we've sent the last blob
if slot.Add(1) >= endSlot && i == len(sidecars)-1 {
break
}
key := stream.Conn().RemotePeer().String()
for {
if sidecarLimiter.Remaining(key) != 0 {
break
}
// Throttling - wait until we have enough tokens to send the next blobs
timer := time.NewTimer(time.Second * 1)
select {
case <-ctx.Done():
timer.Stop()
return ctx.Err()
case <-timer.C:
timer.Stop()
}
}
}
}
closeStream(stream, log)
return nil
}

View File

@@ -1,182 +0,0 @@
package sync
import (
"context"
"sync"
"testing"
"time"
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/protocol"
chainMock "github.com/prysmaticlabs/prysm/v3/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/db/iface"
db "github.com/prysmaticlabs/prysm/v3/beacon-chain/db/testing"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/v3/beacon-chain/p2p/testing"
fieldparams "github.com/prysmaticlabs/prysm/v3/config/fieldparams"
"github.com/prysmaticlabs/prysm/v3/config/params"
types "github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
leakybucket "github.com/prysmaticlabs/prysm/v3/container/leaky-bucket"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/testing/assert"
"github.com/prysmaticlabs/prysm/v3/testing/require"
"github.com/prysmaticlabs/prysm/v3/testing/util"
"github.com/prysmaticlabs/prysm/v3/time/slots"
)
func TestRPCBBlobsSidecarsByRange_RPCHandlerRateLimit_NoOverflow(t *testing.T) {
d := db.SetupDB(t)
p1 := p2ptest.NewTestP2P(t)
p2 := p2ptest.NewTestP2P(t)
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(params.BeaconNetworkConfig().MaxRequestBlobsSidecars)
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(0.000001, capacity, time.Second, false)
req := &ethpb.BlobsSidecarsByRangeRequest{
Count: uint64(capacity),
}
saveBlobs(d, req, 1)
assert.NoError(t, sendBlobsRequest(t, p1, p2, r, req, true, true, 1))
remainingCapacity := r.rateLimiter.limiterMap[topic].Remaining(p2.PeerID().String())
expectedCapacity := int64(0) // Whole capacity is used, but no overflow.
assert.Equal(t, expectedCapacity, remainingCapacity, "Unexpected rate limiting capacity")
}
func TestRPCBBlobsSidecarsByRange_RPCHandlerRateLimit_ThrottleWithoutOverflowAndMultipleBlobsPerSlot(t *testing.T) {
d := db.SetupDB(t)
p1 := p2ptest.NewTestP2P(t)
p2 := p2ptest.NewTestP2P(t)
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(params.BeaconNetworkConfig().MaxRequestBlobsSidecars)
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(0.000001, capacity, time.Second, false)
req := &ethpb.BlobsSidecarsByRangeRequest{
Count: uint64(capacity) / 2,
}
saveBlobs(d, req, 2)
assert.NoError(t, sendBlobsRequest(t, p1, p2, r, req, true, true, 2))
remainingCapacity := r.rateLimiter.limiterMap[topic].Remaining(p2.PeerID().String())
expectedCapacity := int64(0) // Whole capacity is used, but no overflow.
assert.Equal(t, expectedCapacity, remainingCapacity, "Unexpected rate limiting capacity")
}
func TestRPCBBlobsSidecarsByRange_RPCHandlerRateLimit_ThrottleWithOverflow(t *testing.T) {
d := db.SetupDB(t)
p1 := p2ptest.NewTestP2P(t)
p2 := p2ptest.NewTestP2P(t)
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(params.BeaconNetworkConfig().MaxRequestBlobsSidecars) / 2
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
// 30 blobs per second streams blobs in ~4s
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(30, capacity, time.Second, false)
req := &ethpb.BlobsSidecarsByRangeRequest{
Count: uint64(capacity) * 2,
}
saveBlobs(d, req, 1)
assert.NoError(t, sendBlobsRequest(t, p1, p2, r, req, true, true, 1))
remainingCapacity := r.rateLimiter.limiterMap[topic].Remaining(p2.PeerID().String())
expectedCapacity := int64(0)
assert.NotEqual(t, expectedCapacity, remainingCapacity, "Unexpected rate limiting capacity")
}
func TestRPCBBlobsSidecarsByRange_RPCHandlerRateLimit_MultipleRequestsThrottle(t *testing.T) {
d := db.SetupDB(t)
p1 := p2ptest.NewTestP2P(t)
p2 := p2ptest.NewTestP2P(t)
p1.Connect(p2)
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
capacity := int64(params.BeaconNetworkConfig().MaxRequestBlobsSidecars) / 2
r := &Service{cfg: &config{p2p: p1, beaconDB: d, chain: &chainMock.ChainService{}}, rateLimiter: newRateLimiter(p1)}
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
topic := string(pcl)
// 64 blobs per second - takes ~6 seconds to stream all blobs
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(64, capacity, time.Second, false)
for i := 0; i < 2; i++ {
req := &ethpb.BlobsSidecarsByRangeRequest{
Count: uint64(capacity) * 2,
}
saveBlobs(d, req, 1)
assert.NoError(t, sendBlobsRequest(t, p1, p2, r, req, true, true, 1))
}
}
func saveBlobs(d iface.Database, req *ethpb.BlobsSidecarsByRangeRequest, blobsPerSlot uint64) {
for i := req.StartSlot; i < req.StartSlot.Add(req.Count); i += types.Slot(1) {
for j := uint64(0); j < blobsPerSlot; j++ {
blob := &ethpb.BlobsSidecar{
BeaconBlockSlot: i,
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
AggregatedProof: make([]byte, 48),
}
// always non-empty to ensure that blobs are always sent
blob.BeaconBlockRoot[0] = 0x1
blob.BeaconBlockRoot[1] = byte(j)
if err := d.SaveBlobsSidecar(context.Background(), blob); err != nil {
panic(err)
}
}
}
}
func sendBlobsRequest(t *testing.T, p1, p2 *p2ptest.TestP2P, r *Service,
req *ethpb.BlobsSidecarsByRangeRequest, validateBlocks bool, success bool, blobsPerSlot uint64) error {
var wg sync.WaitGroup
wg.Add(1)
pcl := protocol.ID(p2p.RPCBlocksByRangeTopicV1)
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
if !validateBlocks {
return
}
for i := req.StartSlot; i < req.StartSlot.Add(req.Count); i += types.Slot(1) {
if !success {
continue
}
for j := uint64(0); j < blobsPerSlot; j++ {
SetRPCStreamDeadlines(stream)
expectSuccess(t, stream)
res := new(ethpb.BlobsSidecar)
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, res))
if slots.AbsoluteValueSlotDifference(res.BeaconBlockSlot, i) != 0 {
t.Errorf("Received unexpected blob slot %d. expected %d", res.BeaconBlockSlot, i)
}
}
}
})
stream, err := p1.BHost.NewStream(context.Background(), p2.BHost.ID(), pcl)
require.NoError(t, err)
if err := r.blobsSidecarsByRangeRPCHandler(context.Background(), req, stream); err != nil {
return err
}
if util.WaitTimeout(&wg, 20*time.Second) {
t.Fatal("Did not receive stream within 10 sec")
}
return nil
}

View File

@@ -89,25 +89,6 @@ func ReadChunkedBlock(stream libp2pcore.Stream, chain blockchain.ForkFetcher, p2
return readResponseChunk(stream, chain, p2p)
}
// WriteBlobsSidecarChunk writes blobs chunk object to stream.
// response_chunk ::= <result> | <context-bytes> | <encoding-dependent-header> | <encoded-payload>
func WriteBlobsSidecarChunk(stream libp2pcore.Stream, chain blockchain.ChainInfoFetcher, encoding encoder.NetworkEncoding, blobs *ethpb.BlobsSidecar) error {
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return err
}
valRoot := chain.GenesisValidatorsRoot()
ctxBytes, err := forks.ForkDigestFromEpoch(params.BeaconConfig().DenebForkEpoch, valRoot[:])
if err != nil {
return err
}
if err := writeContextToStream(ctxBytes[:], stream, chain); err != nil {
return err
}
_, err = encoding.EncodeWithMaxLength(stream, blobs)
return err
}
// WriteBlobSidecarChunk writes blob chunk object to stream.
// response_chunk ::= <result> | <context-bytes> | <encoding-dependent-header> | <encoded-payload>
func WriteBlobSidecarChunk(stream libp2pcore.Stream, chain blockchain.ChainInfoFetcher, encoding encoder.NetworkEncoding, sidecar *ethpb.BlobSidecar) error {
@@ -127,56 +108,7 @@ func WriteBlobSidecarChunk(stream libp2pcore.Stream, chain blockchain.ChainInfoF
return err
}
func WriteBlockAndBlobsSidecarChunk(stream libp2pcore.Stream, chain blockchain.ChainInfoFetcher, encoding encoder.NetworkEncoding, b *ethpb.SignedBeaconBlockAndBlobsSidecar) error {
SetStreamWriteDeadline(stream, defaultWriteDuration)
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return err
}
valRoot := chain.GenesisValidatorsRoot()
ctxBytes, err := forks.ForkDigestFromEpoch(params.BeaconConfig().DenebForkEpoch, valRoot[:])
if err != nil {
return err
}
if err := writeContextToStream(ctxBytes[:], stream, chain); err != nil {
return err
}
_, err = encoding.EncodeWithMaxLength(stream, b)
return err
}
func ReadChunkedBlockAndBlobsSidecar(stream libp2pcore.Stream, chain blockchain.ChainInfoFetcher, p2p p2p.P2P, isFirstChunk bool) (*ethpb.SignedBeaconBlockAndBlobsSidecar, error) {
var (
code uint8
errMsg string
err error
)
if isFirstChunk {
code, errMsg, err = ReadStatusCode(stream, p2p.Encoding())
} else {
SetStreamReadDeadline(stream, respTimeout)
code, errMsg, err = readStatusCodeNoDeadline(stream, p2p.Encoding())
}
if err != nil {
return nil, err
}
if code != 0 {
return nil, errors.New(errMsg)
}
// No-op for now with the rpc context.
rpcCtx, err := readContextFromStream(stream, chain)
if err != nil {
return nil, err
}
b, err := extractBeaconBlockAndBlobsSidecarDataType(rpcCtx, chain)
if err != nil {
return nil, err
}
err = p2p.Encoding().DecodeWithMaxLength(stream, b)
return b, err
}
func ReadChunkedBlobsSidecar(stream libp2pcore.Stream, chain blockchain.ForkFetcher, p2p p2p.EncodingProvider, isFirstChunk bool) (*ethpb.BlobsSidecar, error) {
func ReadChunkedBlobsSidecar(stream libp2pcore.Stream, chain blockchain.ForkFetcher, p2p p2p.EncodingProvider, isFirstChunk bool) (*ethpb.BlobSidecar, error) {
var (
code uint8
errMsg string
@@ -277,7 +209,7 @@ func extractBlockDataType(digest []byte, chain blockchain.ForkFetcher) (interfac
return nil, errors.New("no valid digest matched")
}
func extractBeaconBlockAndBlobsSidecarDataType(digest []byte, chain blockchain.ForkFetcher) (*ethpb.SignedBeaconBlockAndBlobsSidecar, error) {
func extractBlobsSidecarDataType(digest []byte, chain blockchain.ForkFetcher) (*ethpb.BlobSidecar, error) {
if len(digest) != forkDigestLength {
return nil, errors.Errorf("invalid digest returned, wanted a length of %d but received %d", forkDigestLength, len(digest))
}
@@ -289,20 +221,5 @@ func extractBeaconBlockAndBlobsSidecarDataType(digest []byte, chain blockchain.F
if rDigest != bytesutil.ToBytes4(digest) {
return nil, errors.Errorf("invalid digest returned, wanted %x but received %x", rDigest, digest)
}
return &ethpb.SignedBeaconBlockAndBlobsSidecar{}, nil
}
func extractBlobsSidecarDataType(digest []byte, chain blockchain.ForkFetcher) (*ethpb.BlobsSidecar, error) {
if len(digest) != forkDigestLength {
return nil, errors.Errorf("invalid digest returned, wanted a length of %d but received %d", forkDigestLength, len(digest))
}
vRoot := chain.GenesisValidatorsRoot()
rDigest, err := signing.ComputeForkDigest(params.BeaconConfig().DenebForkVersion, vRoot[:])
if err != nil {
return nil, err
}
if rDigest != bytesutil.ToBytes4(digest) {
return nil, errors.Errorf("invalid digest returned, wanted %x but received %x", rDigest, digest)
}
return &ethpb.BlobsSidecar{}, nil
return &ethpb.BlobSidecar{}, nil
}

View File

@@ -23,8 +23,6 @@ var ErrInvalidFetchedData = errors.New("invalid data returned from peer")
// blocks even before all blocks are ready.
type BeaconBlockProcessor func(block interfaces.ReadOnlySignedBeaconBlock) error
type BeaconBlockAndSidecarProcessor func(blockAndSidecar *pb.SignedBeaconBlockAndBlobsSidecar) error
// SendBeaconBlocksByRangeRequest sends BeaconBlocksByRange and returns fetched blocks, if any.
func SendBeaconBlocksByRangeRequest(
ctx context.Context, chain blockchain.ForkFetcher, p2pProvider p2p.SenderEncoder, pid peer.ID,
@@ -132,10 +130,10 @@ func SendBeaconBlocksByRootRequest(
return blocks, nil
}
func SendSidecarsByRoot(
func SendBlobSidecarByRoot(
ctx context.Context, ci blockchain.ChainInfoFetcher, p2pApi p2p.P2P, pid peer.ID,
req *p2ptypes.BlobSidecarsByRootReq, processor BlobsSidecarProcessor,
) ([]*pb.BlobsSidecar, error) {
req p2ptypes.BlobSidecarsByRootReq,
) ([]*pb.BlobSidecar, error) {
topic, err := p2p.TopicFromMessage(p2p.BlobSidecarsByRootName, slots.ToEpoch(ci.CurrentSlot()))
if err != nil {
return nil, err
@@ -146,17 +144,10 @@ func SendSidecarsByRoot(
}
defer closeStream(stream, log)
sidecars := make([]*pb.BlobsSidecar, 0, len(*req))
process := func(s *pb.BlobsSidecar) error {
sidecars = append(sidecars, s)
if processor != nil {
return processor(s)
}
return nil
}
sidecars := make([]*pb.BlobSidecar, 0, len(req))
max := params.BeaconNetworkConfig().MaxRequestBlobsSidecars * params.BeaconConfig().MaxBlobsPerBlock
for i := 0; i < len(*req); i++ {
for i := 0; i < len(req); i++ {
// Exit if peer sends more than MAX_REQUEST_BLOBS_SIDECARS.
if uint64(i) >= max {
break
@@ -169,104 +160,7 @@ func SendSidecarsByRoot(
if err != nil {
return nil, err
}
if err := process(sc); err != nil {
return nil, err
}
}
return sidecars, nil
}
func SendBlocksAndSidecarsByRootRequest(
ctx context.Context, chain blockchain.ChainInfoFetcher, p2pProvider p2p.P2P, pid peer.ID,
req *p2ptypes.BeaconBlockByRootsReq, processor BeaconBlockAndSidecarProcessor,
) ([]*pb.SignedBeaconBlockAndBlobsSidecar, error) {
topic, err := p2p.TopicFromMessage(p2p.BeaconBlockAndBlobsSidecarByRootName, slots.ToEpoch(chain.CurrentSlot()))
if err != nil {
return nil, err
}
stream, err := p2pProvider.Send(ctx, req, topic, pid)
if err != nil {
return nil, err
}
defer closeStream(stream, log)
blkAndSidecars := make([]*pb.SignedBeaconBlockAndBlobsSidecar, 0, len(*req))
process := func(b *pb.SignedBeaconBlockAndBlobsSidecar) error {
blkAndSidecars = append(blkAndSidecars, b)
if processor != nil {
return processor(b)
}
return nil
}
for i := 0; i < len(*req); i++ {
// Exit if peer sends more than max request blocks.
if uint64(i) >= params.BeaconNetworkConfig().MaxRequestBlocks {
break
}
isFirstChunk := i == 0
blkAndSidecar, err := ReadChunkedBlockAndBlobsSidecar(stream, chain, p2pProvider, isFirstChunk)
if errors.Is(err, io.EOF) {
break
}
if err != nil {
return nil, err
}
if err := process(blkAndSidecar); err != nil {
return nil, err
}
}
return blkAndSidecars, nil
}
func SendBlobsSidecarsByRangeRequest(
ctx context.Context, chain blockchain.ForkFetcher, p2pProvider p2p.SenderEncoder, pid peer.ID,
req *pb.BlobsSidecarsByRangeRequest, sidecarProcessor BlobsSidecarProcessor) ([]*pb.BlobsSidecar, error) {
topic, err := p2p.TopicFromMessage(p2p.BlobsSidecarsByRangeMessageName, slots.ToEpoch(chain.CurrentSlot()))
if err != nil {
return nil, err
}
stream, err := p2pProvider.Send(ctx, req, topic, pid)
if err != nil {
return nil, err
}
defer closeStream(stream, log)
var sidecars []*pb.BlobsSidecar
process := func(sidecar *pb.BlobsSidecar) error {
sidecars = append(sidecars, sidecar)
if sidecarProcessor != nil {
return sidecarProcessor(sidecar)
}
return nil
}
var prevSlot primitives.Slot
for i := uint64(0); ; i++ {
isFirstChunk := len(sidecars) == 0
sidecar, err := ReadChunkedBlobsSidecar(stream, chain, p2pProvider, isFirstChunk)
if errors.Is(err, io.EOF) {
break
}
if err != nil {
return nil, err
}
if i >= req.Count || i >= params.BeaconNetworkConfig().MaxRequestBlobsSidecars {
return nil, ErrInvalidFetchedData
}
if sidecar.BeaconBlockSlot < req.StartSlot || sidecar.BeaconBlockSlot >= req.StartSlot.Add(req.Count) {
return nil, ErrInvalidFetchedData
}
// assert slots aren't out of order and always increasing
if prevSlot >= sidecar.BeaconBlockSlot {
return nil, ErrInvalidFetchedData
}
prevSlot = sidecar.BeaconBlockSlot
if err := process(sidecar); err != nil {
return nil, err
}
sidecars = append(sidecars, sc)
}
return sidecars, nil
}

View File

@@ -6,6 +6,8 @@ import (
"github.com/prysmaticlabs/prysm/v3/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/transition/interop"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/runtime/version"
"google.golang.org/protobuf/proto"
)
@@ -27,6 +29,48 @@ func (s *Service) beaconBlockSubscriber(ctx context.Context, msg proto.Message)
return err
}
if block.Version() >= version.Deneb {
if err := s.blockAndBlobs.addBlock(signed); err != nil {
return err
}
hasEverything, err := s.blockAndBlobs.hasEverything(root)
if err != nil {
return err
}
if hasEverything {
if err := s.receiveBlockAndBlobs(ctx, signed, root); err != nil {
return err
}
s.blockAndBlobs.delete(root)
}
}
return s.receiveBlock(ctx, signed, root)
}
func (s *Service) receiveBlockAndBlobs(ctx context.Context, signed interfaces.ReadOnlySignedBeaconBlock, root [32]byte) error {
if err := s.receiveBlock(ctx, signed, root); err != nil {
return err
}
kzgs, err := signed.Block().Body().BlobKzgCommitments()
if err != nil {
return err
}
for i := 0; i < len(kzgs); i++ {
index := uint64(i)
sb, err := s.blockAndBlobs.getBlob(root, index)
if err != nil {
return err
}
if err := s.blobs.WriteBlobSidecar(root, index, sb); err != nil {
return err
}
}
return nil
}
func (s *Service) receiveBlock(ctx context.Context, signed interfaces.ReadOnlySignedBeaconBlock, root [32]byte) error {
if err := s.cfg.chain.ReceiveBlock(ctx, signed, root); err != nil {
if blockchain.IsInvalidBlock(err) {
r := blockchain.InvalidBlockRoot(err)
@@ -43,5 +87,5 @@ func (s *Service) beaconBlockSubscriber(ctx context.Context, msg proto.Message)
}
return err
}
return err
return nil
}

View File

@@ -5,6 +5,7 @@ import (
"fmt"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
eth "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"google.golang.org/protobuf/proto"
)
@@ -15,9 +16,26 @@ func (s *Service) blobSubscriber(ctx context.Context, msg proto.Message) error {
return fmt.Errorf("message was not type *eth.Attestation, type=%T", msg)
}
if err := s.blockAndBlobs.addBlob(b); err != nil {
if err := s.blockAndBlobs.addBlob(b.Message); err != nil {
return errors.Wrap(err, "could not add blob to queue")
}
root := bytesutil.ToBytes32(b.Message.BlockRoot)
hasEverything, err := s.blockAndBlobs.hasEverything(root)
if err != nil {
return err
}
if hasEverything {
signed, err := s.blockAndBlobs.getBlock(root)
if err != nil {
return err
}
if err := s.receiveBlockAndBlobs(ctx, signed, root); err != nil {
return err
}
s.blockAndBlobs.delete(root)
}
return nil
}

View File

@@ -318,7 +318,7 @@ func TestStaticSubnets(t *testing.T) {
r.subscribeStaticWithSubnets(defaultTopic, r.noopValidator, func(_ context.Context, msg proto.Message) error {
// no-op
return nil
}, d)
}, d, 0)
topics := r.cfg.p2p.PubSub().GetTopics()
if uint64(len(topics)) != params.BeaconNetworkConfig().AttestationSubnetCount {
t.Errorf("Wanted the number of subnet topics registered to be %d but got %d", params.BeaconNetworkConfig().AttestationSubnetCount, len(topics))

View File

@@ -150,7 +150,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
// Otherwise queue it for processing in the right slot.
if isBlockQueueable(genesisTime, blk.Block().Slot(), receivedTime) {
s.pendingQueueLock.Lock()
if err := s.insertBlkAndBlobToQueue(blk.Block().Slot(), blk, blockRoot, nil); err != nil {
if err := s.insertBlockToPendingQueue(blk.Block().Slot(), blk, blockRoot); err != nil {
s.pendingQueueLock.Unlock()
log.WithError(err).WithFields(getBlockFields(blk)).Debug("Could not insert block to pending queue")
return pubsub.ValidationIgnore, err
@@ -164,7 +164,7 @@ func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, ms
// Handle block when the parent is unknown.
if !s.cfg.chain.HasBlock(ctx, blk.Block().ParentRoot()) {
s.pendingQueueLock.Lock()
if err := s.insertBlkAndBlobToQueue(blk.Block().Slot(), blk, blockRoot, nil); err != nil {
if err := s.insertBlockToPendingQueue(blk.Block().Slot(), blk, blockRoot); err != nil {
s.pendingQueueLock.Unlock()
log.WithError(err).WithFields(getBlockFields(blk)).Debug("Could not insert block to pending queue")
return pubsub.ValidationIgnore, err

View File

@@ -6,11 +6,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v3/consensus-types/blobs",
visibility = ["//visibility:public"],
deps = [
"//consensus-types/primitives:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_protolambda_go_kzg//eth:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)

View File

@@ -1,28 +1,10 @@
package blobs
import (
"fmt"
"github.com/protolambda/go-kzg/eth"
types "github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
v1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
log "github.com/sirupsen/logrus"
)
type commitmentSequenceImpl [][]byte
func (s commitmentSequenceImpl) At(i int) eth.KZGCommitment {
var out eth.KZGCommitment
copy(out[:], s[i])
return out
}
func (s commitmentSequenceImpl) Len() int {
return len(s)
}
type BlobImpl []byte
func (b BlobImpl) At(i int) [32]byte {
@@ -44,21 +26,3 @@ func (s BlobsSequenceImpl) At(i int) eth.Blob {
func (s BlobsSequenceImpl) Len() int {
return len(s)
}
// ValidateBlobsSidecar verifies the integrity of a sidecar, returning nil if the blob is valid.
func ValidateBlobsSidecar(slot types.Slot, root [32]byte, commitments [][]byte, sidecar *ethpb.BlobsSidecar) error {
kzgSidecar := eth.BlobsSidecar{
BeaconBlockRoot: eth.Root(bytesutil.ToBytes32(sidecar.BeaconBlockRoot)),
BeaconBlockSlot: eth.Slot(sidecar.BeaconBlockSlot),
Blobs: BlobsSequenceImpl(sidecar.Blobs),
KZGAggregatedProof: eth.KZGProof(bytesutil.ToBytes48(sidecar.AggregatedProof)),
}
log.WithFields(log.Fields{
"slot": slot,
"root": fmt.Sprintf("%#x", bytesutil.Trunc(root[:])),
"commitments": len(commitments),
"sidecarBlobs": len(sidecar.Blobs),
"aggregatedProof": fmt.Sprintf("%#x", bytesutil.Trunc(sidecar.AggregatedProof)),
}).Info("Validating blobs")
return eth.ValidateBlobsSidecar(eth.Slot(slot), root, commitmentSequenceImpl(commitments), kzgSidecar)
}

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: f179210312cdf4d6daddd16abbe6484fe0405db496c7b47bb3325eedb8cf9918
// Hash: 08cc68bba152b00ea6bc21e70baa7977a4d2312d2d6a290315bde7d77322b9d6
package enginev1
import (

View File

@@ -18,7 +18,7 @@ import (
"github.com/grpc-ecosystem/grpc-gateway/v2/runtime"
"github.com/grpc-ecosystem/grpc-gateway/v2/utilities"
github_com_prysmaticlabs_prysm_v3_consensus_types_primitives "github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v3/proto/eth/v1"
v1 "github.com/prysmaticlabs/prysm/v3/proto/eth/v1"
"github.com/prysmaticlabs/prysm/v3/proto/eth/v2"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 04bade3bf8672ae75769cb5f179fe0c5921577d1d4c61f3b82fb09a1ae8f085b
// Hash: 7c27b881c879b0b666ae0e078fa61fb09ed0694888c7ea09c7ed729942a81e56
package v1
import (

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 4a3207ccd0cfa725c04271e1a2a39525dfa5f6a9c52557fc40d31207aa309034
// Hash: 55d934e826515b29e1a10370082a3b771f7579a70ef2ae7c20a43d1b7be7d9cd
package eth
import (

View File

@@ -130,8 +130,6 @@ ssz_gen_marshal(
"BuilderBid",
"BlobsAndCommitments",
"PolynomialAndCommitment",
"BlobsSidecar",
"SignedBeaconBlockAndBlobsSidecar",
"BuilderBidCapella",
"BlobSidecar",
"SignedBlobSidecar",

File diff suppressed because it is too large Load Diff

View File

@@ -445,15 +445,6 @@ message BlindedBeaconBlockBodyBellatrix {
ethereum.engine.v1.ExecutionPayloadHeader execution_payload_header = 10;
}
// This contains beacon block and the blob sidecar objects under single container.
// It is to be gossip'ed over p2p to peers to validate and process.
message SignedBeaconBlockAndBlobsSidecar {
// The block of the beacon chain.
SignedBeaconBlockDeneb beacon_block = 1;
// The blob sidecar which corresponds to the block itself.
BlobsSidecar blobs_sidecar = 2;
}
message SignedBeaconBlockDenebAndBlobs {
SignedBeaconBlockDeneb block = 1;
repeated SignedBlobSidecar blobs = 2 [(ethereum.eth.ext.ssz_max) = "4"];

View File

@@ -24,77 +24,6 @@ const (
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
type BlobsSidecar struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
BeaconBlockRoot []byte `protobuf:"bytes,1,opt,name=beacon_block_root,json=beaconBlockRoot,proto3" json:"beacon_block_root,omitempty" ssz-size:"32"`
BeaconBlockSlot github_com_prysmaticlabs_prysm_v3_consensus_types_primitives.Slot `protobuf:"varint,2,opt,name=beacon_block_slot,json=beaconBlockSlot,proto3" json:"beacon_block_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v3/consensus-types/primitives.Slot"`
Blobs []*v1.Blob `protobuf:"bytes,3,rep,name=blobs,proto3" json:"blobs,omitempty" ssz-max:"4"`
AggregatedProof []byte `protobuf:"bytes,4,opt,name=aggregated_proof,json=aggregatedProof,proto3" json:"aggregated_proof,omitempty" ssz-size:"48"`
}
func (x *BlobsSidecar) Reset() {
*x = BlobsSidecar{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BlobsSidecar) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BlobsSidecar) ProtoMessage() {}
func (x *BlobsSidecar) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BlobsSidecar.ProtoReflect.Descriptor instead.
func (*BlobsSidecar) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{0}
}
func (x *BlobsSidecar) GetBeaconBlockRoot() []byte {
if x != nil {
return x.BeaconBlockRoot
}
return nil
}
func (x *BlobsSidecar) GetBeaconBlockSlot() github_com_prysmaticlabs_prysm_v3_consensus_types_primitives.Slot {
if x != nil {
return x.BeaconBlockSlot
}
return github_com_prysmaticlabs_prysm_v3_consensus_types_primitives.Slot(0)
}
func (x *BlobsSidecar) GetBlobs() []*v1.Blob {
if x != nil {
return x.Blobs
}
return nil
}
func (x *BlobsSidecar) GetAggregatedProof() []byte {
if x != nil {
return x.AggregatedProof
}
return nil
}
type BlobSidecar struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -113,7 +42,7 @@ type BlobSidecar struct {
func (x *BlobSidecar) Reset() {
*x = BlobSidecar{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1]
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -126,7 +55,7 @@ func (x *BlobSidecar) String() string {
func (*BlobSidecar) ProtoMessage() {}
func (x *BlobSidecar) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1]
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -139,7 +68,7 @@ func (x *BlobSidecar) ProtoReflect() protoreflect.Message {
// Deprecated: Use BlobSidecar.ProtoReflect.Descriptor instead.
func (*BlobSidecar) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{1}
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{0}
}
func (x *BlobSidecar) GetBlockRoot() []byte {
@@ -210,7 +139,7 @@ type SignedBlobSidecar struct {
func (x *SignedBlobSidecar) Reset() {
*x = SignedBlobSidecar{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2]
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -223,7 +152,7 @@ func (x *SignedBlobSidecar) String() string {
func (*SignedBlobSidecar) ProtoMessage() {}
func (x *SignedBlobSidecar) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2]
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -236,7 +165,7 @@ func (x *SignedBlobSidecar) ProtoReflect() protoreflect.Message {
// Deprecated: Use SignedBlobSidecar.ProtoReflect.Descriptor instead.
func (*SignedBlobSidecar) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{2}
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{1}
}
func (x *SignedBlobSidecar) GetMessage() *BlobSidecar {
@@ -265,7 +194,7 @@ type BlobIdentifier struct {
func (x *BlobIdentifier) Reset() {
*x = BlobIdentifier{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3]
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -278,7 +207,7 @@ func (x *BlobIdentifier) String() string {
func (*BlobIdentifier) ProtoMessage() {}
func (x *BlobIdentifier) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3]
mi := &file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -291,7 +220,7 @@ func (x *BlobIdentifier) ProtoReflect() protoreflect.Message {
// Deprecated: Use BlobIdentifier.ProtoReflect.Descriptor instead.
func (*BlobIdentifier) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{3}
return file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP(), []int{2}
}
func (x *BlobIdentifier) GetBlockRoot() []byte {
@@ -318,77 +247,59 @@ var file_proto_prysm_v1alpha1_blobs_proto_rawDesc = []byte{
0x2f, 0x65, 0x74, 0x68, 0x2f, 0x65, 0x78, 0x74, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x6e,
0x67, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
0x6e, 0x5f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x9f,
0x02, 0x0a, 0x0c, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12,
0x32, 0x0a, 0x11, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x0f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52,
0x6f, 0x6f, 0x74, 0x12, 0x71, 0x0a, 0x11, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45,
0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70,
0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x33, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d,
0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73,
0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x0f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x35, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18,
0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x42,
0x05, 0x92, 0xb5, 0x18, 0x01, 0x34, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x12, 0x31, 0x0a,
0x10, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f,
0x66, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52,
0x0f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66,
0x22, 0xd3, 0x03, 0x0a, 0x0b, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72,
0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x59, 0x0a,
0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18,
0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f,
0x76, 0x33, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70,
0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c,
0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x32, 0x0a, 0x11, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x5f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x76, 0x0a, 0x0e,
0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05,
0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
0x6e, 0x5f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd3,
0x03, 0x0a, 0x0b, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x25,
0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01,
0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02,
0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x59, 0x0a, 0x04, 0x73,
0x6c, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67,
0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61,
0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x33,
0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73,
0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74,
0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x32, 0x0a, 0x11, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x76, 0x0a, 0x0e, 0x70, 0x72,
0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01,
0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73,
0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x33, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e,
0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74,
0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64,
0x65, 0x78, 0x12, 0x2c, 0x0a, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x18, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69,
0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x04, 0x62, 0x6c, 0x6f, 0x62,
0x12, 0x2d, 0x0a, 0x0e, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65,
0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38,
0x52, 0x0d, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x12,
0x23, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x08, 0x20, 0x01,
0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x08, 0x6b, 0x7a, 0x67, 0x50,
0x72, 0x6f, 0x6f, 0x66, 0x22, 0x77, 0x0a, 0x11, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x6c,
0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x3c, 0x0a, 0x07, 0x6d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x52, 0x07,
0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61,
0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x4d, 0x0a,
0x0e, 0x42, 0x6c, 0x6f, 0x62, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12,
0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18,
0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x95, 0x01, 0x0a,
0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x0a, 0x42, 0x6c, 0x6f, 0x62,
0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61,
0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x33, 0x2f, 0x63, 0x6f, 0x6e, 0x73,
0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d,
0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x49,
0x6e, 0x64, 0x65, 0x78, 0x12, 0x2c, 0x0a, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x18, 0x06, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x18, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e,
0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x52, 0x04, 0x62, 0x6c,
0x6f, 0x62, 0x12, 0x2d, 0x0a, 0x0e, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74,
0x6d, 0x65, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x34, 0x38, 0x52, 0x0d, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e,
0x74, 0x12, 0x23, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x08,
0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x08, 0x6b, 0x7a,
0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, 0x77, 0x0a, 0x11, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64,
0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x3c, 0x0a, 0x07, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c,
0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72,
0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67,
0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22,
0x4d, 0x0a, 0x0e, 0x42, 0x6c, 0x6f, 0x62, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65,
0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65,
0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x95,
0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x0a, 0x42, 0x6c,
0x6f, 0x62, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63,
0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x33, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02,
0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x33, 0x2f, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c,
0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -403,23 +314,21 @@ func file_proto_prysm_v1alpha1_blobs_proto_rawDescGZIP() []byte {
return file_proto_prysm_v1alpha1_blobs_proto_rawDescData
}
var file_proto_prysm_v1alpha1_blobs_proto_msgTypes = make([]protoimpl.MessageInfo, 4)
var file_proto_prysm_v1alpha1_blobs_proto_msgTypes = make([]protoimpl.MessageInfo, 3)
var file_proto_prysm_v1alpha1_blobs_proto_goTypes = []interface{}{
(*BlobsSidecar)(nil), // 0: ethereum.eth.v1alpha1.BlobsSidecar
(*BlobSidecar)(nil), // 1: ethereum.eth.v1alpha1.BlobSidecar
(*SignedBlobSidecar)(nil), // 2: ethereum.eth.v1alpha1.SignedBlobSidecar
(*BlobIdentifier)(nil), // 3: ethereum.eth.v1alpha1.BlobIdentifier
(*v1.Blob)(nil), // 4: ethereum.engine.v1.Blob
(*BlobSidecar)(nil), // 0: ethereum.eth.v1alpha1.BlobSidecar
(*SignedBlobSidecar)(nil), // 1: ethereum.eth.v1alpha1.SignedBlobSidecar
(*BlobIdentifier)(nil), // 2: ethereum.eth.v1alpha1.BlobIdentifier
(*v1.Blob)(nil), // 3: ethereum.engine.v1.Blob
}
var file_proto_prysm_v1alpha1_blobs_proto_depIdxs = []int32{
4, // 0: ethereum.eth.v1alpha1.BlobsSidecar.blobs:type_name -> ethereum.engine.v1.Blob
4, // 1: ethereum.eth.v1alpha1.BlobSidecar.blob:type_name -> ethereum.engine.v1.Blob
1, // 2: ethereum.eth.v1alpha1.SignedBlobSidecar.message:type_name -> ethereum.eth.v1alpha1.BlobSidecar
3, // [3:3] is the sub-list for method output_type
3, // [3:3] is the sub-list for method input_type
3, // [3:3] is the sub-list for extension type_name
3, // [3:3] is the sub-list for extension extendee
0, // [0:3] is the sub-list for field type_name
3, // 0: ethereum.eth.v1alpha1.BlobSidecar.blob:type_name -> ethereum.engine.v1.Blob
0, // 1: ethereum.eth.v1alpha1.SignedBlobSidecar.message:type_name -> ethereum.eth.v1alpha1.BlobSidecar
2, // [2:2] is the sub-list for method output_type
2, // [2:2] is the sub-list for method input_type
2, // [2:2] is the sub-list for extension type_name
2, // [2:2] is the sub-list for extension extendee
0, // [0:2] is the sub-list for field type_name
}
func init() { file_proto_prysm_v1alpha1_blobs_proto_init() }
@@ -429,18 +338,6 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() {
}
if !protoimpl.UnsafeEnabled {
file_proto_prysm_v1alpha1_blobs_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlobsSidecar); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlobSidecar); i {
case 0:
return &v.state
@@ -452,7 +349,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() {
return nil
}
}
file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
file_proto_prysm_v1alpha1_blobs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SignedBlobSidecar); i {
case 0:
return &v.state
@@ -464,7 +361,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() {
return nil
}
}
file_proto_prysm_v1alpha1_blobs_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
file_proto_prysm_v1alpha1_blobs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlobIdentifier); i {
case 0:
return &v.state
@@ -483,7 +380,7 @@ func file_proto_prysm_v1alpha1_blobs_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_prysm_v1alpha1_blobs_proto_rawDesc,
NumEnums: 0,
NumMessages: 4,
NumMessages: 3,
NumExtensions: 0,
NumServices: 0,
},

View File

@@ -25,19 +25,6 @@ option java_outer_classname = "BlobsProto";
option java_package = "org.ethereum.eth.v1alpha1";
option php_namespace = "Ethereum\\Eth\\v1alpha1";
// The blobs sidecar which contains the beacon block root, slot, blobs and an aggregated proof
// for fast validation.
message BlobsSidecar { // TODO: To be removed soon once spec is finalized.
// The beacon block root which corresponds to the beacon block.
bytes beacon_block_root = 1 [(ethereum.eth.ext.ssz_size) = "32"];
// The beacon block slot which corresponds ot the beacon block.
uint64 beacon_block_slot = 2 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v3/consensus-types/primitives.Slot"];
// The blobs itself which contains data to be committed.
repeated ethereum.engine.v1.Blob blobs = 3 [(ethereum.eth.ext.ssz_max) = "4"];
// The aggregated proof for blobs and commitments for validation.
bytes aggregated_proof = 4 [(ethereum.eth.ext.ssz_size) = "48"];
}
message BlobSidecar {
bytes block_root = 1 [(ethereum.eth.ext.ssz_size) = "32"];
uint64 index = 2;

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 1284dc1619a47949c34f8502554268149d98b7a9e609cf703f2dc5203f86293c
// Hash: 0727379338827ff4bb1b0f0141993507db61eb862f1e7c503dfa55644cd5f8a3
package eth
import (
@@ -4613,138 +4613,6 @@ func (b *BlindedBeaconBlockBodyBellatrix) HashTreeRootWith(hh *ssz.Hasher) (err
return
}
// MarshalSSZ ssz marshals the SignedBeaconBlockAndBlobsSidecar object
func (s *SignedBeaconBlockAndBlobsSidecar) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(s)
}
// MarshalSSZTo ssz marshals the SignedBeaconBlockAndBlobsSidecar object to a target array
func (s *SignedBeaconBlockAndBlobsSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
offset := int(8)
// Offset (0) 'BeaconBlock'
dst = ssz.WriteOffset(dst, offset)
if s.BeaconBlock == nil {
s.BeaconBlock = new(SignedBeaconBlockDeneb)
}
offset += s.BeaconBlock.SizeSSZ()
// Offset (1) 'BlobsSidecar'
dst = ssz.WriteOffset(dst, offset)
if s.BlobsSidecar == nil {
s.BlobsSidecar = new(BlobsSidecar)
}
offset += s.BlobsSidecar.SizeSSZ()
// Field (0) 'BeaconBlock'
if dst, err = s.BeaconBlock.MarshalSSZTo(dst); err != nil {
return
}
// Field (1) 'BlobsSidecar'
if dst, err = s.BlobsSidecar.MarshalSSZTo(dst); err != nil {
return
}
return
}
// UnmarshalSSZ ssz unmarshals the SignedBeaconBlockAndBlobsSidecar object
func (s *SignedBeaconBlockAndBlobsSidecar) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size < 8 {
return ssz.ErrSize
}
tail := buf
var o0, o1 uint64
// Offset (0) 'BeaconBlock'
if o0 = ssz.ReadOffset(buf[0:4]); o0 > size {
return ssz.ErrOffset
}
if o0 < 8 {
return ssz.ErrInvalidVariableOffset
}
// Offset (1) 'BlobsSidecar'
if o1 = ssz.ReadOffset(buf[4:8]); o1 > size || o0 > o1 {
return ssz.ErrOffset
}
// Field (0) 'BeaconBlock'
{
buf = tail[o0:o1]
if s.BeaconBlock == nil {
s.BeaconBlock = new(SignedBeaconBlockDeneb)
}
if err = s.BeaconBlock.UnmarshalSSZ(buf); err != nil {
return err
}
}
// Field (1) 'BlobsSidecar'
{
buf = tail[o1:]
if s.BlobsSidecar == nil {
s.BlobsSidecar = new(BlobsSidecar)
}
if err = s.BlobsSidecar.UnmarshalSSZ(buf); err != nil {
return err
}
}
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the SignedBeaconBlockAndBlobsSidecar object
func (s *SignedBeaconBlockAndBlobsSidecar) SizeSSZ() (size int) {
size = 8
// Field (0) 'BeaconBlock'
if s.BeaconBlock == nil {
s.BeaconBlock = new(SignedBeaconBlockDeneb)
}
size += s.BeaconBlock.SizeSSZ()
// Field (1) 'BlobsSidecar'
if s.BlobsSidecar == nil {
s.BlobsSidecar = new(BlobsSidecar)
}
size += s.BlobsSidecar.SizeSSZ()
return
}
// HashTreeRoot ssz hashes the SignedBeaconBlockAndBlobsSidecar object
func (s *SignedBeaconBlockAndBlobsSidecar) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(s)
}
// HashTreeRootWith ssz hashes the SignedBeaconBlockAndBlobsSidecar object with a hasher
func (s *SignedBeaconBlockAndBlobsSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'BeaconBlock'
if err = s.BeaconBlock.HashTreeRootWith(hh); err != nil {
return
}
// Field (1) 'BlobsSidecar'
if err = s.BlobsSidecar.HashTreeRootWith(hh); err != nil {
return
}
if ssz.EnableVectorizedHTR {
hh.MerkleizeVectorizedHTR(indx)
} else {
hh.Merkleize(indx)
}
return
}
// MarshalSSZ ssz marshals the SignedBeaconBlockDeneb object
func (s *SignedBeaconBlockDeneb) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(s)
@@ -14534,170 +14402,6 @@ func (h *HistoricalSummary) HashTreeRootWith(hh *ssz.Hasher) (err error) {
return
}
// MarshalSSZ ssz marshals the BlobsSidecar object
func (b *BlobsSidecar) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(b)
}
// MarshalSSZTo ssz marshals the BlobsSidecar object to a target array
func (b *BlobsSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
offset := int(92)
// Field (0) 'BeaconBlockRoot'
if size := len(b.BeaconBlockRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.BeaconBlockRoot", size, 32)
return
}
dst = append(dst, b.BeaconBlockRoot...)
// Field (1) 'BeaconBlockSlot'
dst = ssz.MarshalUint64(dst, uint64(b.BeaconBlockSlot))
// Offset (2) 'Blobs'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.Blobs) * 131072
// Field (3) 'AggregatedProof'
if size := len(b.AggregatedProof); size != 48 {
err = ssz.ErrBytesLengthFn("--.AggregatedProof", size, 48)
return
}
dst = append(dst, b.AggregatedProof...)
// Field (2) 'Blobs'
if size := len(b.Blobs); size > 4 {
err = ssz.ErrListTooBigFn("--.Blobs", size, 4)
return
}
for ii := 0; ii < len(b.Blobs); ii++ {
if dst, err = b.Blobs[ii].MarshalSSZTo(dst); err != nil {
return
}
}
return
}
// UnmarshalSSZ ssz unmarshals the BlobsSidecar object
func (b *BlobsSidecar) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size < 92 {
return ssz.ErrSize
}
tail := buf
var o2 uint64
// Field (0) 'BeaconBlockRoot'
if cap(b.BeaconBlockRoot) == 0 {
b.BeaconBlockRoot = make([]byte, 0, len(buf[0:32]))
}
b.BeaconBlockRoot = append(b.BeaconBlockRoot, buf[0:32]...)
// Field (1) 'BeaconBlockSlot'
b.BeaconBlockSlot = github_com_prysmaticlabs_prysm_v3_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[32:40]))
// Offset (2) 'Blobs'
if o2 = ssz.ReadOffset(buf[40:44]); o2 > size {
return ssz.ErrOffset
}
if o2 < 92 {
return ssz.ErrInvalidVariableOffset
}
// Field (3) 'AggregatedProof'
if cap(b.AggregatedProof) == 0 {
b.AggregatedProof = make([]byte, 0, len(buf[44:92]))
}
b.AggregatedProof = append(b.AggregatedProof, buf[44:92]...)
// Field (2) 'Blobs'
{
buf = tail[o2:]
num, err := ssz.DivideInt2(len(buf), 131072, 4)
if err != nil {
return err
}
b.Blobs = make([]*v1.Blob, num)
for ii := 0; ii < num; ii++ {
if b.Blobs[ii] == nil {
b.Blobs[ii] = new(v1.Blob)
}
if err = b.Blobs[ii].UnmarshalSSZ(buf[ii*131072 : (ii+1)*131072]); err != nil {
return err
}
}
}
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the BlobsSidecar object
func (b *BlobsSidecar) SizeSSZ() (size int) {
size = 92
// Field (2) 'Blobs'
size += len(b.Blobs) * 131072
return
}
// HashTreeRoot ssz hashes the BlobsSidecar object
func (b *BlobsSidecar) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(b)
}
// HashTreeRootWith ssz hashes the BlobsSidecar object with a hasher
func (b *BlobsSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'BeaconBlockRoot'
if size := len(b.BeaconBlockRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.BeaconBlockRoot", size, 32)
return
}
hh.PutBytes(b.BeaconBlockRoot)
// Field (1) 'BeaconBlockSlot'
hh.PutUint64(uint64(b.BeaconBlockSlot))
// Field (2) 'Blobs'
{
subIndx := hh.Index()
num := uint64(len(b.Blobs))
if num > 4 {
err = ssz.ErrIncorrectListSize
return
}
for _, elem := range b.Blobs {
if err = elem.HashTreeRootWith(hh); err != nil {
return
}
}
if ssz.EnableVectorizedHTR {
hh.MerkleizeWithMixinVectorizedHTR(subIndx, num, 4)
} else {
hh.MerkleizeWithMixin(subIndx, num, 4)
}
}
// Field (3) 'AggregatedProof'
if size := len(b.AggregatedProof); size != 48 {
err = ssz.ErrBytesLengthFn("--.AggregatedProof", size, 48)
return
}
hh.PutBytes(b.AggregatedProof)
if ssz.EnableVectorizedHTR {
hh.MerkleizeVectorizedHTR(indx)
} else {
hh.Merkleize(indx)
}
return
}
// MarshalSSZ ssz marshals the BlobSidecar object
func (b *BlobSidecar) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(b)

View File

@@ -134,8 +134,6 @@ func unmarshalledSSZ(t *testing.T, serializedBytes []byte, folderName string) (i
obj = &ethpb.BlobSidecar{}
case "SignedBlobSidecar":
obj = &ethpb.SignedBlobSidecar{}
case "SignedBeaconBlockAndBlobsSidecar":
obj = &ethpb.SignedBeaconBlockAndBlobsSidecar{}
case "PowBlock":
obj = &ethpb.PowBlock{}
case "Withdrawal":