Compare commits

...

15 Commits

Author SHA1 Message Date
Kasey Kirkham
a01a824b57 scaffolding for verification package 2023-11-16 14:59:06 -06:00
Kasey Kirkham
f1a1bffea6 mkdir for root before trying to write blob data 2023-11-16 11:22:55 -06:00
Kasey Kirkham
71c04ef6a2 fix default path construction; mkdir when blobstorage is initialized 2023-11-16 11:16:26 -06:00
Kasey Kirkham
2d5e214086 lint 2023-11-16 10:46:36 -06:00
Kasey Kirkham
41696e0e8e add unsaved file 2023-11-16 10:29:25 -06:00
Kasey Kirkham
98e5278f32 use nice Header helper 2023-11-15 17:10:05 -06:00
Kasey Kirkham
977e5bafc7 for tests only, add dummy testing.T param 2023-11-15 17:06:39 -06:00
Kasey Kirkham
d25b7d27ed add explicit continue for clarity 2023-11-15 17:00:08 -06:00
Kasey Kirkham
e3300d1c59 off-by-one 2023-11-15 16:52:56 -06:00
Kasey Kirkham
b0aa26b5f8 some spectest fixes, but skipping until we have verification 2023-11-15 16:47:53 -06:00
Kasey Kirkham
29f00e61d2 gaz 2023-11-15 15:53:51 -06:00
Kasey Kirkham
3b33a95c96 fix spectest panic by adding blobstorage to service 2023-11-15 15:52:09 -06:00
Kasey Kirkham
632010665f de-linting by hand 2023-11-15 15:52:00 -06:00
Kasey Kirkham
c324242121 deep source 2023-11-15 14:20:35 -06:00
Kasey Kirkham
d8d34fc4ff Use VerifiedROBlob in paths that interact with db 2023-11-15 14:02:50 -06:00
64 changed files with 1218 additions and 905 deletions

View File

@@ -50,6 +50,7 @@ go_library(
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/filters:go_default_library",
"//beacon-chain/db/kv:go_default_library",
"//beacon-chain/execution:go_default_library",

View File

@@ -147,15 +147,3 @@ func logPayload(block interfaces.ReadOnlyBeaconBlock) error {
log.WithFields(fields).Debug("Synced new payload")
return nil
}
func logBlobSidecar(scs []*ethpb.DeprecatedBlobSidecar, startTime time.Time) {
if len(scs) == 0 {
return
}
log.WithFields(logrus.Fields{
"count": len(scs),
"slot": scs[0].Slot,
"block": hex.EncodeToString(scs[0].BlockRoot),
"validationTime": time.Since(startTime),
}).Debug("Synced new blob sidecars")
}

View File

@@ -5,6 +5,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/cache"
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/attestations"
@@ -164,6 +165,8 @@ func WithFinalizedStateAtStartUp(st state.BeaconState) Option {
}
}
// WithClockSychronizer sets the ClockSetter/ClockWaiter values to be used by services that need to block until
// the genesis timestamp is known (ClockWaiter) or which determine the genesis timestamp (ClockSetter).
func WithClockSynchronizer(gs *startup.ClockSynchronizer) Option {
return func(s *Service) error {
s.clockSetter = gs
@@ -172,9 +175,18 @@ func WithClockSynchronizer(gs *startup.ClockSynchronizer) Option {
}
}
// WithSyncComplete sets a channel that is used to notify blockchain service that the node has synced to head.
func WithSyncComplete(c chan struct{}) Option {
return func(s *Service) error {
s.syncComplete = c
return nil
}
}
// WithBlobStorage sets the blob storage backend for the blockchain service.
func WithBlobStorage(b *filesystem.BlobStorage) Option {
return func(s *Service) error {
s.blobStorage = b
return nil
}
}

View File

@@ -6,14 +6,13 @@ import (
"time"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/kzg"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/helpers"
coreTime "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
forkchoicetypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice/types"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v4/config/features"
@@ -353,11 +352,15 @@ func (s *Service) databaseDACheck(ctx context.Context, b consensusblocks.ROBlock
if len(commitments) == 0 {
return nil
}
sidecars, err := s.cfg.BeaconDB.BlobSidecarsByRoot(ctx, b.Root())
missing, err := missingIndices(s.blobStorage, b.Root(), len(commitments))
if err != nil {
return errors.Wrap(err, "could not get blob sidecars")
return err
}
return kzg.IsDataAvailable(commitments, sidecars)
if len(missing) == 0 {
return nil
}
// TODO: don't worry that this error isn't informative, it will be superceded by a detailed sidecar cache error.
return errors.New("not all kzg commitments are available")
}
func (s *Service) updateEpochBoundaryCaches(ctx context.Context, st state.BeaconState) error {
@@ -529,11 +532,24 @@ func (s *Service) runLateBlockTasks() {
}
}
func missingIndices(bs *filesystem.BlobStorage, root [32]byte, expected int) (map[uint64]struct{}, error) {
indices, err := bs.Indices(root)
if err != nil {
return nil, err
}
missing := make(map[uint64]struct{}, expected)
for i := uint64(0); i < uint64(expected); i++ {
if !indices[i] {
missing[i] = struct{}{}
}
}
return missing, nil
}
func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed interfaces.ReadOnlySignedBeaconBlock) error {
if signed.Version() < version.Deneb {
return nil
}
t := time.Now()
block := signed.Block()
if block == nil {
@@ -556,55 +572,23 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
if expected == 0 {
return nil
}
// Read first from db in case we have the blobs
sidecars, err := s.cfg.BeaconDB.BlobSidecarsByRoot(ctx, root)
switch {
case err == nil:
if len(sidecars) >= expected {
s.blobNotifiers.delete(root)
if err := kzg.IsDataAvailable(kzgCommitments, sidecars); err != nil {
log.WithField("root", fmt.Sprintf("%#x", root)).Warn("removing blob sidecars with invalid proofs")
if err2 := s.cfg.BeaconDB.DeleteBlobSidecars(ctx, root); err2 != nil {
log.WithError(err2).Error("could not delete sidecars")
}
return err
}
logBlobSidecar(sidecars, t)
return nil
}
case errors.Is(err, db.ErrNotFound):
// If the blob sidecars haven't arrived yet, the subsequent code will wait for them.
// Note: The system will not exit with an error in this scenario.
default:
log.WithError(err).Error("could not get blob sidecars from DB")
missing, err := missingIndices(s.blobStorage, root, expected)
if err != nil {
return err
}
if len(missing) == 0 {
return nil
}
found := map[uint64]struct{}{}
for _, sc := range sidecars {
found[sc.Index] = struct{}{}
}
nc := s.blobNotifiers.forRoot(root)
for {
select {
case idx := <-nc:
found[idx] = struct{}{}
if len(found) != expected {
delete(missing, idx)
if len(missing) > 0 {
continue
}
s.blobNotifiers.delete(root)
sidecars, err := s.cfg.BeaconDB.BlobSidecarsByRoot(ctx, root)
if err != nil {
return errors.Wrap(err, "could not get blob sidecars")
}
if err := kzg.IsDataAvailable(kzgCommitments, sidecars); err != nil {
log.WithField("root", fmt.Sprintf("%#x", root)).Warn("removing blob sidecars with invalid proofs")
if err2 := s.cfg.BeaconDB.DeleteBlobSidecars(ctx, root); err2 != nil {
log.WithError(err2).Error("could not delete sidecars")
}
return err
}
logBlobSidecar(sidecars, t)
return nil
case <-ctx.Done():
return errors.Wrap(ctx.Err(), "context deadline waiting for blob sidecars")

View File

@@ -3,7 +3,7 @@ package blockchain
import (
"context"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
)
// SendNewBlobEvent sends a message to the BlobNotifier channel that the blob
@@ -13,11 +13,11 @@ func (s *Service) sendNewBlobEvent(root [32]byte, index uint64) {
}
// ReceiveBlob saves the blob to database and sends the new event
func (s *Service) ReceiveBlob(ctx context.Context, b *ethpb.DeprecatedBlobSidecar) error {
if err := s.cfg.BeaconDB.SaveBlobSidecar(ctx, []*ethpb.DeprecatedBlobSidecar{b}); err != nil {
func (s *Service) ReceiveBlob(ctx context.Context, b blocks.VerifiedROBlob) error {
if err := s.blobStorage.Save(b); err != nil {
return err
}
s.sendNewBlobEvent([32]byte(b.BlockRoot), b.Index)
s.sendNewBlobEvent(b.BlockRoot(), b.Index)
return nil
}

View File

@@ -43,7 +43,7 @@ type BlockReceiver interface {
// BlobReceiver interface defines the methods of chain service for receiving new
// blobs
type BlobReceiver interface {
ReceiveBlob(context.Context, *ethpb.DeprecatedBlobSidecar) error
ReceiveBlob(context.Context, blocks.VerifiedROBlob) error
}
// SlashingReceiver interface defines the methods of chain service for receiving validated slashing over the wire.

View File

@@ -20,6 +20,7 @@ import (
coreTime "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
f "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice"
forkchoicetypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice/types"
@@ -63,6 +64,7 @@ type Service struct {
syncComplete chan struct{}
blobNotifiers *blobNotifierMap
blockBeingSynced *currentlySyncingBlock
blobStorage *filesystem.BlobStorage
}
// config options for the service.

View File

@@ -72,7 +72,7 @@ type ChainService struct {
OptimisticRoots map[[32]byte]bool
BlockSlot primitives.Slot
SyncingRoot [32]byte
Blobs []*ethpb.DeprecatedBlobSidecar
Blobs []blocks.VerifiedROBlob
}
func (s *ChainService) Ancestor(ctx context.Context, root []byte, slot primitives.Slot) ([]byte, error) {
@@ -613,7 +613,7 @@ func (c *ChainService) BlockBeingSynced(root [32]byte) bool {
}
// ReceiveBlob implements the same method in the chain service
func (c *ChainService) ReceiveBlob(_ context.Context, b *ethpb.DeprecatedBlobSidecar) error {
func (c *ChainService) ReceiveBlob(_ context.Context, b blocks.VerifiedROBlob) error {
c.Blobs = append(c.Blobs, b)
return nil
}

View File

@@ -1,6 +1,11 @@
package db
import "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/kv"
import (
"errors"
"os"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/kv"
)
// ErrNotFound can be used to determine if an error from a method in the database package
// represents a "not found" error. These often require different handling than a low-level
@@ -19,3 +24,9 @@ var ErrNotFoundBackfillBlockRoot = kv.ErrNotFoundBackfillBlockRoot
// ErrNotFoundGenesisBlockRoot means no genesis block root was found, indicating the db was not initialized with genesis
var ErrNotFoundGenesisBlockRoot = kv.ErrNotFoundGenesisBlockRoot
// IsNotFound allows callers to treat errors from a flat-file database, where the file record is missing,
// as equivalent to db.ErrNotFound.
func IsNotFound(err error) bool {
return errors.Is(err, ErrNotFound) || os.IsNotExist(err)
}

View File

@@ -2,27 +2,34 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = ["save_blob.go"],
srcs = [
"blob.go",
"ephemeral.go",
],
importpath = "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem",
visibility = ["//visibility:private"],
visibility = ["//visibility:public"],
deps = [
"//io/file:go_default_library",
"//proto/eth/v2:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//consensus-types/blocks:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@com_github_spf13_afero//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = ["save_blob_test.go"],
srcs = ["blob_test.go"],
embed = [":go_default_library"],
deps = [
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//encoding/bytesutil:go_default_library",
"//io/file:go_default_library",
"//proto/eth/v2:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@com_github_spf13_afero//:go_default_library",
],
)

View File

@@ -0,0 +1,176 @@
package filesystem
import (
"fmt"
"os"
"path"
"strconv"
"strings"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/spf13/afero"
)
var (
errIndexOutOfBounds = errors.New("blob index in file name > MaxBlobsPerBlock")
)
const (
sszExt = "ssz"
partExt = "part"
blobLockPath = "blob.lock"
)
// NewBlobStorage creates a new instance of the BlobStorage object. Note that the implementation of BlobStorage may
// attempt to hold a file lock to guarantee exclusive control of the blob storage directory, so this should only be
// initialized once per beacon node.
func NewBlobStorage(base string) (*BlobStorage, error) {
base = path.Clean(base)
if err := os.MkdirAll(base, 0755); err != nil {
return nil, err
}
fs := afero.NewBasePathFs(afero.NewOsFs(), base)
return &BlobStorage{fs: fs}, nil
}
// BlobStorage is the concrete implementation of the filesystem backend for saving and retrieving BlobSidecars.
type BlobStorage struct {
fs afero.Fs
}
// Save saves blobs given a list of sidecars.
func (bs *BlobStorage) Save(sidecar blocks.VerifiedROBlob) error {
fname := namerForSidecar(sidecar)
sszPath := fname.ssz()
exists, err := afero.Exists(bs.fs, sszPath)
if err != nil {
return err
}
if exists {
// TODO: should it be an error to save a blob that already exists?
return nil
}
// Serialize the ethpb.BlobSidecar to binary data using SSZ.
sidecarData, err := ssz.MarshalSSZ(sidecar)
if err != nil {
return errors.Wrap(err, "failed to serialize sidecar data")
}
if err := bs.fs.Mkdir(fname.dir(), 0755); err != nil {
return err
}
partPath := fname.partial()
// Create a partial file and write the serialized data to it.
partialFile, err := bs.fs.Create(partPath)
if err != nil {
return errors.Wrap(err, "failed to create partial file")
}
_, err = partialFile.Write(sidecarData)
if err != nil {
closeErr := partialFile.Close()
if closeErr != nil {
return closeErr
}
return errors.Wrap(err, "failed to write to partial file")
}
err = partialFile.Close()
if err != nil {
return err
}
// Atomically rename the partial file to its final name.
err = bs.fs.Rename(partPath, sszPath)
if err != nil {
return errors.Wrap(err, "failed to rename partial file to final name")
}
return nil
}
// Get retrieves a single BlobSidecar by its root and index.
// Since BlobStorage only writes blobs that have undergone full verification, the return
// value is always a VerifiedROBlob.
func (bs *BlobStorage) Get(root [32]byte, idx uint64) (blocks.VerifiedROBlob, error) {
expected := blobNamer{root: root, index: idx}
encoded, err := afero.ReadFile(bs.fs, expected.ssz())
var v blocks.VerifiedROBlob
if err != nil {
return v, err
}
s := &ethpb.BlobSidecar{}
if err := s.UnmarshalSSZ(encoded); err != nil {
return v, err
}
ro, err := blocks.NewROBlobWithRoot(s, root)
if err != nil {
return blocks.VerifiedROBlob{}, err
}
return verification.BlobSidecarNoop(ro)
}
// Indices generates a bitmap representing which BlobSidecar.Index values are present on disk for a given root.
// This value can be compared to the commitments observed in a block to determine which indices need to be found
// on the network to confirm data availability.
func (bs *BlobStorage) Indices(root [32]byte) ([fieldparams.MaxBlobsPerBlock]bool, error) {
var mask [fieldparams.MaxBlobsPerBlock]bool
rootDir := blobNamer{root: root}.dir()
entries, err := afero.ReadDir(bs.fs, rootDir)
if err != nil {
if os.IsNotExist(err) {
return mask, nil
}
return mask, err
}
for i := range entries {
if entries[i].IsDir() {
continue
}
name := entries[i].Name()
if !strings.HasSuffix(name, sszExt) {
continue
}
parts := strings.Split(name, ".")
if len(parts) != 2 {
continue
}
u, err := strconv.ParseUint(parts[0], 10, 64)
if err != nil {
return mask, errors.Wrapf(err, "unexpected directory entry breaks listing, %s", parts[0])
}
if u > fieldparams.MaxBlobsPerBlock {
return mask, errIndexOutOfBounds
}
mask[u] = true
}
return mask, nil
}
type blobNamer struct {
root [32]byte
index uint64
}
func namerForSidecar(sc blocks.VerifiedROBlob) blobNamer {
return blobNamer{root: sc.BlockRoot(), index: sc.Index}
}
func (p blobNamer) dir() string {
return fmt.Sprintf("%#x", p.root)
}
func (p blobNamer) fname(ext string) string {
return path.Join(p.dir(), fmt.Sprintf("%d.%s", p.index, ext))
}
func (p blobNamer) partial() string {
return p.fname(partExt)
}
func (p blobNamer) ssz() string {
return p.fname(sszExt)
}

View File

@@ -0,0 +1,70 @@
package filesystem
import (
"bytes"
"testing"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/spf13/afero"
"github.com/prysmaticlabs/prysm/v4/testing/require"
"github.com/prysmaticlabs/prysm/v4/testing/util"
)
func TestBlobStorage_SaveBlobData(t *testing.T) {
_, sidecars := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 1, fieldparams.MaxBlobsPerBlock)
testSidecars, err := verification.BlobSidecarSliceNoop(sidecars)
require.NoError(t, err)
t.Run("no error for duplicate", func(t *testing.T) {
fs, bs := NewEphemeralBlobStorageWithFs(t)
existingSidecar := testSidecars[0]
blobPath := namerForSidecar(existingSidecar).ssz()
// Serialize the existing BlobSidecar to binary data.
existingSidecarData, err := ssz.MarshalSSZ(existingSidecar)
require.NoError(t, err)
require.NoError(t, bs.Save(existingSidecar))
// No error when attempting to write twice.
require.NoError(t, bs.Save(existingSidecar))
content, err := afero.ReadFile(fs, blobPath)
require.NoError(t, err)
require.Equal(t, true, bytes.Equal(existingSidecarData, content))
// Deserialize the BlobSidecar from the saved file data.
savedSidecar := &ethpb.BlobSidecar{}
err = savedSidecar.UnmarshalSSZ(content)
require.NoError(t, err)
// Compare the original Sidecar and the saved Sidecar.
require.DeepSSZEqual(t, existingSidecar.BlobSidecar, savedSidecar)
})
t.Run("indices", func(t *testing.T) {
bs := NewEphemeralBlobStorage(t)
sc := testSidecars[2]
require.NoError(t, bs.Save(sc))
actualSc, err := bs.Get(sc.BlockRoot(), sc.Index)
require.NoError(t, err)
expectedIdx := [fieldparams.MaxBlobsPerBlock]bool{false, false, true}
actualIdx, err := bs.Indices(actualSc.BlockRoot())
require.NoError(t, err)
require.Equal(t, expectedIdx, actualIdx)
})
t.Run("round trip write then read", func(t *testing.T) {
bs := NewEphemeralBlobStorage(t)
err := bs.Save(testSidecars[0])
require.NoError(t, err)
expected := testSidecars[0]
actual, err := bs.Get(expected.BlockRoot(), expected.Index)
require.NoError(t, err)
require.DeepSSZEqual(t, expected, actual)
})
}

View File

@@ -0,0 +1,21 @@
package filesystem
import (
"testing"
"github.com/spf13/afero"
)
// NewEphemeralBlobStorage should only be used for tests.
// The instance of BlobStorage returned is backed by an in-memory virtual filesystem,
// improving test performance and simplifying cleanup.
func NewEphemeralBlobStorage(_ testing.TB) *BlobStorage {
return &BlobStorage{fs: afero.NewMemMapFs()}
}
// NewEphemeralBlobStorageWithFs can be used by tests that want access to the virtual filesystem
// in order to interact with it outside the parameters of the BlobStorage api.
func NewEphemeralBlobStorageWithFs(_ testing.TB) (afero.Fs, *BlobStorage) {
fs := afero.NewMemMapFs()
return fs, &BlobStorage{fs: fs}
}

View File

@@ -1,98 +0,0 @@
package filesystem
import (
"crypto/sha256"
"encoding/hex"
"fmt"
"os"
"path"
"path/filepath"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v4/io/file"
"github.com/prysmaticlabs/prysm/v4/proto/eth/v2"
)
type BlobStorage struct {
baseDir string
}
// SaveBlobData saves blobs given a list of sidecars.
func (bs *BlobStorage) SaveBlobData(sidecars []*eth.BlobSidecar) error {
if len(sidecars) == 0 {
return errors.New("no blob data to save")
}
for _, sidecar := range sidecars {
blobPath := bs.sidecarFileKey(sidecar)
exists := file.Exists(blobPath)
if exists {
if err := checkDataIntegrity(sidecar, blobPath); err != nil {
// This error should never happen, if it does then the
// file has most likely been tampered with.
return errors.Wrapf(err, "failed to save blob sidecar, tried to overwrite blob (%s) with different content", blobPath)
}
continue // Blob already exists, move to the next one
}
// Serialize the ethpb.BlobSidecar to binary data using SSZ.
sidecarData, err := ssz.MarshalSSZ(sidecar)
if err != nil {
return errors.Wrap(err, "failed to serialize sidecar data")
}
// Create a partial file and write the serialized data to it.
partialFilePath := blobPath + ".partial"
partialFile, err := os.Create(filepath.Clean(partialFilePath))
if err != nil {
return errors.Wrap(err, "failed to create partial file")
}
_, err = partialFile.Write(sidecarData)
if err != nil {
closeErr := partialFile.Close()
if closeErr != nil {
return closeErr
}
return errors.Wrap(err, "failed to write to partial file")
}
err = partialFile.Close()
if err != nil {
return err
}
// Atomically rename the partial file to its final name.
err = os.Rename(partialFilePath, blobPath)
if err != nil {
return errors.Wrap(err, "failed to rename partial file to final name")
}
}
return nil
}
func (bs *BlobStorage) sidecarFileKey(sidecar *eth.BlobSidecar) string {
return path.Join(bs.baseDir, fmt.Sprintf(
"%d_%x_%d_%x.blob",
sidecar.Slot,
sidecar.BlockRoot,
sidecar.Index,
sidecar.KzgCommitment,
))
}
// checkDataIntegrity checks the data integrity by comparing the original ethpb.BlobSidecar.
func checkDataIntegrity(sidecar *eth.BlobSidecar, filePath string) error {
sidecarData, err := ssz.MarshalSSZ(sidecar)
if err != nil {
return errors.Wrap(err, "failed to serialize sidecar data")
}
originalChecksum := sha256.Sum256(sidecarData)
savedFileChecksum, err := file.HashFile(filePath)
if err != nil {
return errors.Wrap(err, "failed to calculate saved file checksum")
}
if hex.EncodeToString(originalChecksum[:]) != hex.EncodeToString(savedFileChecksum) {
return errors.New("data integrity check failed")
}
return nil
}

View File

@@ -1,191 +0,0 @@
package filesystem
import (
"bytes"
"crypto/rand"
"crypto/sha256"
"encoding/hex"
"os"
"path"
"strings"
"testing"
ssz "github.com/prysmaticlabs/fastssz"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v4/io/file"
"github.com/prysmaticlabs/prysm/v4/proto/eth/v2"
"github.com/prysmaticlabs/prysm/v4/testing/require"
)
func TestBlobStorage_SaveBlobData(t *testing.T) {
testSidecars := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock)
t.Run("NoBlobData", func(t *testing.T) {
tempDir := t.TempDir()
bs := &BlobStorage{baseDir: tempDir}
err := bs.SaveBlobData([]*eth.BlobSidecar{})
require.ErrorContains(t, "no blob data to save", err)
})
t.Run("BlobExists", func(t *testing.T) {
tempDir := t.TempDir()
bs := &BlobStorage{baseDir: tempDir}
existingSidecar := testSidecars[0]
blobPath := bs.sidecarFileKey(existingSidecar)
// Serialize the existing BlobSidecar to binary data.
existingSidecarData, err := ssz.MarshalSSZ(existingSidecar)
require.NoError(t, err)
err = os.MkdirAll(path.Dir(blobPath), os.ModePerm)
require.NoError(t, err)
// Write the serialized data to the blob file.
err = os.WriteFile(blobPath, existingSidecarData, os.ModePerm)
require.NoError(t, err)
err = bs.SaveBlobData([]*eth.BlobSidecar{existingSidecar})
require.NoError(t, err)
content, err := os.ReadFile(blobPath)
require.NoError(t, err)
// Deserialize the BlobSidecar from the saved file data.
var savedSidecar ssz.Unmarshaler
savedSidecar = &eth.BlobSidecar{}
err = savedSidecar.UnmarshalSSZ(content)
require.NoError(t, err)
// Compare the original Sidecar and the saved Sidecar.
require.DeepSSZEqual(t, existingSidecar, savedSidecar)
})
t.Run("SaveBlobDataNoErrors", func(t *testing.T) {
tempDir := t.TempDir()
bs := &BlobStorage{baseDir: tempDir}
err := bs.SaveBlobData(testSidecars)
require.NoError(t, err)
// Check the number of files in the directory.
files, err := os.ReadDir(tempDir)
require.NoError(t, err)
require.Equal(t, len(testSidecars), len(files))
for _, f := range files {
content, err := os.ReadFile(path.Join(tempDir, f.Name()))
require.NoError(t, err)
// Deserialize the BlobSidecar from the saved file data.
var savedSidecar ssz.Unmarshaler
savedSidecar = &eth.BlobSidecar{}
err = savedSidecar.UnmarshalSSZ(content)
require.NoError(t, err)
// Find the corresponding test sidecar based on the file name.
sidecar := findTestSidecarsByFileName(t, testSidecars, f.Name())
require.NotNil(t, sidecar)
// Compare the original Sidecar and the saved Sidecar.
require.DeepSSZEqual(t, sidecar, savedSidecar)
}
})
t.Run("OverwriteBlobWithDifferentContent", func(t *testing.T) {
tempDir := t.TempDir()
bs := &BlobStorage{baseDir: tempDir}
originalSidecar := []*eth.BlobSidecar{testSidecars[0]}
// Save the original sidecar
err := bs.SaveBlobData(originalSidecar)
require.NoError(t, err)
// Modify the blob data
modifiedSidecar := originalSidecar
modifiedSidecar[0].Blob = []byte("Modified Blob Data")
err = bs.SaveBlobData(modifiedSidecar)
require.ErrorContains(t, "failed to save blob sidecar, tried to overwrite blob", err)
})
}
func findTestSidecarsByFileName(t *testing.T, testSidecars []*eth.BlobSidecar, fileName string) *eth.BlobSidecar {
parts := strings.SplitN(fileName, ".", 2)
require.Equal(t, 2, len(parts))
// parts[0] contains the substring before the first period
components := strings.Split(parts[0], "_")
if len(components) == 4 {
blockRoot, err := hex.DecodeString(components[1])
require.NoError(t, err)
kzgCommitment, err := hex.DecodeString(components[3])
require.NoError(t, err)
for _, sidecar := range testSidecars {
if bytes.Equal(sidecar.BlockRoot, blockRoot) && bytes.Equal(sidecar.KzgCommitment, kzgCommitment) {
return sidecar
}
}
}
return nil
}
func TestCheckDataIntegrity(t *testing.T) {
testSidecars := generateBlobSidecars(t, fieldparams.MaxBlobsPerBlock)
originalData, err := ssz.MarshalSSZ(testSidecars[0])
require.NoError(t, err)
originalChecksum := sha256.Sum256(originalData)
tempDir := t.TempDir()
tempfile, err := os.CreateTemp(tempDir, "testfile")
require.NoError(t, err)
_, err = tempfile.Write(originalData)
require.NoError(t, err)
err = checkDataIntegrity(testSidecars[0], tempfile.Name())
require.NoError(t, err)
// Modify the data in the file to simulate data corruption
corruptedData := []byte("corrupted data")
err = os.WriteFile(tempfile.Name(), corruptedData, os.ModePerm)
require.NoError(t, err)
// Test data integrity check with corrupted data
err = checkDataIntegrity(testSidecars[0], tempfile.Name())
require.ErrorContains(t, "data integrity check failed", err)
// Modify the calculated checksum to be incorrect
wrongChecksum := hex.EncodeToString(originalChecksum[:]) + "12345"
err = os.WriteFile(tempfile.Name(), []byte(wrongChecksum), os.ModePerm)
require.NoError(t, err)
checksum, err := file.HashFile(tempfile.Name())
require.NoError(t, err)
require.NotEqual(t, wrongChecksum, hex.EncodeToString(checksum))
}
func generateBlobSidecars(t *testing.T, n uint64) []*eth.BlobSidecar {
blobSidecars := make([]*eth.BlobSidecar, n)
for i := uint64(0); i < n; i++ {
blobSidecars[i] = generateBlobSidecar(t, i)
}
return blobSidecars
}
func generateBlobSidecar(t *testing.T, index uint64) *eth.BlobSidecar {
blob := make([]byte, 131072)
_, err := rand.Read(blob)
require.NoError(t, err)
kzgCommitment := make([]byte, 48)
_, err = rand.Read(kzgCommitment)
require.NoError(t, err)
kzgProof := make([]byte, 48)
_, err = rand.Read(kzgProof)
require.NoError(t, err)
return &eth.BlobSidecar{
BlockRoot: bytesutil.PadTo([]byte{'a'}, 32),
Index: index,
Slot: 100,
BlockParentRoot: bytesutil.PadTo([]byte{'b'}, 32),
ProposerIndex: 101,
Blob: blob,
KzgCommitment: kzgCommitment,
KzgProof: kzgProof,
}
}

View File

@@ -24,6 +24,7 @@ go_library(
"//beacon-chain/cache/depositcache:go_default_library",
"//beacon-chain/cache/depositsnapshot:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/kv:go_default_library",
"//beacon-chain/db/slasherkv:go_default_library",
"//beacon-chain/deterministic-genesis:go_default_library",

View File

@@ -26,6 +26,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/cache/depositcache"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/cache/depositsnapshot"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/kv"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/slasherkv"
interopcoldstart "github.com/prysmaticlabs/prysm/v4/beacon-chain/deterministic-genesis"
@@ -112,6 +113,8 @@ type BeaconNode struct {
forkChoicer forkchoice.ForkChoicer
clockWaiter startup.ClockWaiter
initialSyncComplete chan struct{}
BlobStoragePath string
BlobStorage *filesystem.BlobStorage
}
// New creates a new node instance, sets up configuration options, and registers
@@ -200,6 +203,14 @@ func New(cliCtx *cli.Context, opts ...Option) (*BeaconNode, error) {
if err != nil {
return nil, err
}
log.Debugln("Starting Blob Storage")
blobStorage, err := filesystem.NewBlobStorage(beacon.BlobStoragePath)
if err != nil {
return nil, err
}
beacon.BlobStorage = blobStorage
log.Debugln("Starting DB")
if err := beacon.startDB(cliCtx, depositAddress); err != nil {
return nil, err
@@ -639,6 +650,7 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
blockchain.WithProposerIdsCache(b.proposerIdsCache),
blockchain.WithClockSynchronizer(gs),
blockchain.WithSyncComplete(syncComplete),
blockchain.WithBlobStorage(b.BlobStorage),
)
blockchainService, err := blockchain.NewService(b.ctx, opts...)
@@ -717,6 +729,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}) erro
regularsync.WithClockWaiter(b.clockWaiter),
regularsync.WithInitialSyncComplete(initialSyncComplete),
regularsync.WithStateNotifier(b),
regularsync.WithBlobStorage(b.BlobStorage),
)
return b.services.RegisterService(rs)
}

View File

@@ -21,7 +21,7 @@ var gossipTopicMappings = map[string]proto.Message{
SyncContributionAndProofSubnetTopicFormat: &ethpb.SignedContributionAndProof{},
SyncCommitteeSubnetTopicFormat: &ethpb.SyncCommitteeMessage{},
BlsToExecutionChangeSubnetTopicFormat: &ethpb.SignedBLSToExecutionChange{},
BlobSubnetTopicFormat: &ethpb.SignedBlobSidecar{},
BlobSubnetTopicFormat: &ethpb.BlobSidecar{},
}
// GossipTopicMappings is a function to return the assigned data type

View File

@@ -26,12 +26,12 @@ func TestConstructGenericBeaconBlock(t *testing.T) {
r1, err := b.Block().HashTreeRoot()
require.NoError(t, err)
scs := []*ethpb.DeprecatedBlobSidecar{
util.GenerateTestDenebBlobSidecar(r1, eb, 0, []byte{}),
util.GenerateTestDenebBlobSidecar(r1, eb, 1, []byte{}),
util.GenerateTestDenebBlobSidecar(r1, eb, 2, []byte{}),
util.GenerateTestDenebBlobSidecar(r1, eb, 3, []byte{}),
util.GenerateTestDenebBlobSidecar(r1, eb, 4, []byte{}),
util.GenerateTestDenebBlobSidecar(r1, eb, 5, []byte{}),
util.GenerateTestDeprecatedBlobSidecar(r1, eb, 0, []byte{}),
util.GenerateTestDeprecatedBlobSidecar(r1, eb, 1, []byte{}),
util.GenerateTestDeprecatedBlobSidecar(r1, eb, 2, []byte{}),
util.GenerateTestDeprecatedBlobSidecar(r1, eb, 3, []byte{}),
util.GenerateTestDeprecatedBlobSidecar(r1, eb, 4, []byte{}),
util.GenerateTestDeprecatedBlobSidecar(r1, eb, 5, []byte{}),
}
result, err := vs.constructGenericBeaconBlock(b, nil, scs)
require.NoError(t, err)

View File

@@ -75,6 +75,7 @@ go_library(
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/core/transition/interop:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/filters:go_default_library",
"//beacon-chain/execution:go_default_library",
"//beacon-chain/operations/attestations:go_default_library",
@@ -90,6 +91,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync/verify:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cache/lru:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
@@ -196,6 +198,7 @@ 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/filesystem:go_default_library",
"//beacon-chain/db/kv:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/execution:go_default_library",
@@ -214,6 +217,7 @@ go_test(
"//beacon-chain/state/state-native:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync/initial-sync/testing:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cache/lru:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/fieldparams:go_default_library",

View File

@@ -13,12 +13,16 @@ import (
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/protocol"
mock "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
db "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
types "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
leakybucket "github.com/prysmaticlabs/prysm/v4/container/leaky-bucket"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
@@ -48,12 +52,12 @@ type blobsTestCase struct {
}
type testHandler func(s *Service) rpcHandler
type expectedDefiner func(t *testing.T, scs []*ethpb.DeprecatedBlobSidecar, req interface{}) []*expectedBlobChunk
type requestFromSidecars func([]*ethpb.DeprecatedBlobSidecar) interface{}
type expectedDefiner func(t *testing.T, scs []blocks.ROBlob, req interface{}) []*expectedBlobChunk
type requestFromSidecars func([]blocks.ROBlob) interface{}
type oldestSlotCallback func(t *testing.T) types.Slot
type expectedRequirer func(*testing.T, *Service, []*expectedBlobChunk) func(network.Stream)
func generateTestBlockWithSidecars(t *testing.T, parent [32]byte, slot types.Slot, nblobs int) (*ethpb.SignedBeaconBlockDeneb, []*ethpb.DeprecatedBlobSidecar) {
func generateTestBlockWithSidecars(t *testing.T, parent [32]byte, slot types.Slot, nblobs int) (*ethpb.SignedBeaconBlockDeneb, []blocks.ROBlob) {
// Start service with 160 as allowed blocks capacity (and almost zero capacity recovery).
stateRoot := bytesutil.PadTo([]byte("stateRoot"), fieldparams.RootLength)
receiptsRoot := bytesutil.PadTo([]byte("receiptsRoot"), fieldparams.RootLength)
@@ -103,32 +107,45 @@ func generateTestBlockWithSidecars(t *testing.T, parent [32]byte, slot types.Slo
root, err := block.Block.HashTreeRoot()
require.NoError(t, err)
sidecars := make([]*ethpb.DeprecatedBlobSidecar, len(commitments))
sbb, err := blocks.NewSignedBeaconBlock(block)
require.NoError(t, err)
sidecars := make([]blocks.ROBlob, len(commitments))
for i, c := range block.Block.Body.BlobKzgCommitments {
sidecars[i] = generateTestSidecar(root, block, i, c)
sidecars[i] = generateTestSidecar(t, root, sbb, i, c)
}
return block, sidecars
}
func generateTestSidecar(root [32]byte, block *ethpb.SignedBeaconBlockDeneb, index int, commitment []byte) *ethpb.DeprecatedBlobSidecar {
func generateTestSidecar(t *testing.T, root [32]byte, block interfaces.ReadOnlySignedBeaconBlock, index int, commitment []byte) blocks.ROBlob {
header, err := block.Header()
require.NoError(t, err)
blob := make([]byte, fieldparams.BlobSize)
binary.LittleEndian.PutUint64(blob, uint64(index))
sc := &ethpb.DeprecatedBlobSidecar{
BlockRoot: root[:],
Index: uint64(index),
Slot: block.Block.Slot,
BlockParentRoot: block.Block.ParentRoot,
ProposerIndex: block.Block.ProposerIndex,
Blob: blob,
KzgCommitment: commitment,
KzgProof: commitment,
pb := &ethpb.BlobSidecar{
Index: uint64(index),
Blob: blob,
KzgCommitment: commitment,
KzgProof: commitment,
SignedBlockHeader: header,
}
pb.CommitmentInclusionProof = fakeEmptyProof(t, block, pb)
sc, err := blocks.NewROBlobWithRoot(pb, root)
require.NoError(t, err)
return sc
}
func fakeEmptyProof(_ *testing.T, _ interfaces.ReadOnlySignedBeaconBlock, _ *ethpb.BlobSidecar) [][]byte {
r := make([][]byte, 17)
for i := range r {
r[i] = make([]byte, 32)
}
return r
}
type expectedBlobChunk struct {
code uint8
sidecar *ethpb.DeprecatedBlobSidecar
sidecar *blocks.ROBlob
message string
}
@@ -146,17 +163,19 @@ func (r *expectedBlobChunk) requireExpected(t *testing.T, s *Service, stream net
require.NoError(t, err)
valRoot := s.cfg.chain.GenesisValidatorsRoot()
ctxBytes, err := forks.ForkDigestFromEpoch(slots.ToEpoch(r.sidecar.GetSlot()), valRoot[:])
ctxBytes, err := forks.ForkDigestFromEpoch(slots.ToEpoch(r.sidecar.Slot()), valRoot[:])
require.NoError(t, err)
require.Equal(t, ctxBytes, bytesutil.ToBytes4(c))
sc := &ethpb.DeprecatedBlobSidecar{}
sc := &ethpb.BlobSidecar{}
require.NoError(t, encoding.DecodeWithMaxLength(stream, sc))
require.Equal(t, bytesutil.ToBytes32(sc.BlockRoot), bytesutil.ToBytes32(r.sidecar.BlockRoot))
require.Equal(t, sc.Index, r.sidecar.Index)
rob, err := blocks.NewROBlob(sc)
require.NoError(t, err)
require.Equal(t, rob.BlockRoot(), r.sidecar.BlockRoot())
require.Equal(t, rob.Index, r.sidecar.Index)
}
func (c *blobsTestCase) setup(t *testing.T) (*Service, []*ethpb.DeprecatedBlobSidecar, func()) {
func (c *blobsTestCase) setup(t *testing.T) (*Service, []blocks.ROBlob, func()) {
cfg := params.BeaconConfig()
repositionFutureEpochs(cfg)
undo, err := params.SetActiveWithUndo(cfg)
@@ -174,7 +193,7 @@ func (c *blobsTestCase) setup(t *testing.T) (*Service, []*ethpb.DeprecatedBlobSi
}
d := db.SetupDB(t)
sidecars := make([]*ethpb.DeprecatedBlobSidecar, 0)
sidecars := make([]blocks.ROBlob, 0)
oldest := c.oldestSlot(t)
var parentRoot [32]byte
for i := 0; i < c.nblocks; i++ {
@@ -198,7 +217,7 @@ func (c *blobsTestCase) setup(t *testing.T) (*Service, []*ethpb.DeprecatedBlobSi
client := p2ptest.NewTestP2P(t)
s := &Service{
cfg: &config{p2p: client, chain: c.chain, clock: clock, beaconDB: d},
cfg: &config{p2p: client, chain: c.chain, clock: clock, beaconDB: d, blobStorage: filesystem.NewEphemeralBlobStorage(t)},
rateLimiter: newRateLimiter(client),
}
@@ -223,17 +242,22 @@ func (c *blobsTestCase) run(t *testing.T) {
defer cleanup()
req := c.requestFromSidecars(sidecars)
expect := c.defineExpected(t, sidecars, req)
m := map[types.Slot][]*ethpb.DeprecatedBlobSidecar{}
for _, sc := range expect {
m := map[types.Slot][]blocks.ROBlob{}
for i := range expect {
sc := expect[i]
// If define expected omits a sidecar from an expected result, we don't need to save it.
// This can happen in particular when there are no expected results, because the nth part of the
// response is an error (or none at all when the whole request is invalid).
if sc.sidecar != nil {
m[sc.sidecar.Slot] = append(m[sc.sidecar.Slot], sc.sidecar)
m[sc.sidecar.Slot()] = append(m[sc.sidecar.Slot()], *sc.sidecar)
}
}
for _, blobSidecars := range m {
require.NoError(t, s.cfg.beaconDB.SaveBlobSidecar(context.Background(), blobSidecars))
v, err := verification.BlobSidecarSliceNoop(blobSidecars)
require.NoError(t, err)
for i := range v {
require.NoError(t, s.cfg.blobStorage.Save(v[i]))
}
}
if c.total != nil {
require.Equal(t, *c.total, len(expect))
@@ -293,7 +317,7 @@ func TestTestcaseSetup_BlocksAndBlobs(t *testing.T) {
require.Equal(t, maxed, len(sidecars))
require.Equal(t, maxed, len(expect))
for _, sc := range sidecars {
blk, err := s.cfg.beaconDB.Block(ctx, bytesutil.ToBytes32(sc.BlockRoot))
blk, err := s.cfg.beaconDB.Block(ctx, sc.BlockRoot())
require.NoError(t, err)
var found *int
comms, err := blk.Block().Body().BlobKzgCommitments()

View File

@@ -38,7 +38,7 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
case strings.Contains(topic, p2p.GossipSyncCommitteeMessage) && !strings.Contains(topic, p2p.SyncContributionAndProofSubnetTopicFormat):
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.SyncCommitteeMessage{})]
case strings.Contains(topic, p2p.GossipBlobSidecarMessage):
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBlobSidecar{})]
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.BlobSidecar{})]
}
base := p2p.GossipTopicMappings(topic, 0)

View File

@@ -22,12 +22,14 @@ go_library(
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/p2p:go_default_library",
"//beacon-chain/p2p/peers/scorers:go_default_library",
"//beacon-chain/p2p/types:go_default_library",
"//beacon-chain/startup:go_default_library",
"//beacon-chain/sync:go_default_library",
"//beacon-chain/sync/verify:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/params:go_default_library",
"//consensus-types:go_default_library",
@@ -113,6 +115,7 @@ go_test(
"//async/abool:go_default_library",
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/p2p:go_default_library",
"//beacon-chain/p2p/peers:go_default_library",

View File

@@ -19,6 +19,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v4/config/params"
consensus_types "github.com/prysmaticlabs/prysm/v4/consensus-types"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
blocks2 "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
@@ -382,12 +383,12 @@ func lowestSlotNeedsBlob(retentionStart primitives.Slot, bwb []blocks2.BlockWith
return nil
}
func sortBlobs(blobs []*p2ppb.DeprecatedBlobSidecar) []*p2ppb.DeprecatedBlobSidecar {
func sortBlobs(blobs []blocks.ROBlob) []blocks.ROBlob {
sort.Slice(blobs, func(i, j int) bool {
if blobs[i].Slot == blobs[j].Slot {
if blobs[i].Slot() == blobs[j].Slot() {
return blobs[i].Index < blobs[j].Index
}
return blobs[i].Slot < blobs[j].Slot
return blobs[i].Slot() < blobs[j].Slot()
})
return blobs
@@ -396,7 +397,7 @@ func sortBlobs(blobs []*p2ppb.DeprecatedBlobSidecar) []*p2ppb.DeprecatedBlobSide
var errBlobVerification = errors.New("peer unable to serve aligned BlobSidecarsByRange and BeaconBlockSidecarsByRange responses")
var errMissingBlobsForBlockCommitments = errors.Wrap(errBlobVerification, "blobs unavailable for processing block with kzg commitments")
func verifyAndPopulateBlobs(bwb []blocks2.BlockWithVerifiedBlobs, blobs []*p2ppb.DeprecatedBlobSidecar, blobWindowStart primitives.Slot) ([]blocks2.BlockWithVerifiedBlobs, error) {
func verifyAndPopulateBlobs(bwb []blocks2.BlockWithVerifiedBlobs, blobs []blocks.ROBlob, blobWindowStart primitives.Slot) ([]blocks2.BlockWithVerifiedBlobs, error) {
// Assumes bwb has already been sorted by sortedBlockWithVerifiedBlobSlice.
blobs = sortBlobs(blobs)
blobi := 0
@@ -419,7 +420,7 @@ func verifyAndPopulateBlobs(bwb []blocks2.BlockWithVerifiedBlobs, blobs []*p2ppb
}
return nil, err
}
bb.Blobs = make([]*p2ppb.DeprecatedBlobSidecar, len(commits))
bb.Blobs = make([]blocks.ROBlob, len(commits))
for ci := range commits {
// There are more expected commitments in this block, but we've run out of blobs from the response
// (out-of-bound error guard).
@@ -502,7 +503,7 @@ func (f *blocksFetcher) requestBlocks(
return prysmsync.SendBeaconBlocksByRangeRequest(ctx, f.chain, f.p2p, pid, req, nil)
}
func (f *blocksFetcher) requestBlobs(ctx context.Context, req *p2ppb.BlobSidecarsByRangeRequest, pid peer.ID) ([]*p2ppb.DeprecatedBlobSidecar, error) {
func (f *blocksFetcher) requestBlobs(ctx context.Context, req *p2ppb.BlobSidecarsByRangeRequest, pid peer.ID) ([]blocks.ROBlob, error) {
if ctx.Err() != nil {
return nil, ctx.Err()
}

View File

@@ -962,7 +962,7 @@ func TestTimeToWait(t *testing.T) {
func TestSortBlobs(t *testing.T) {
_, blobs := util.ExtendBlocksPlusBlobs(t, []blocks.ROBlock{}, 10)
shuffled := make([]*ethpb.DeprecatedBlobSidecar, len(blobs))
shuffled := make([]blocks.ROBlob, len(blobs))
for i := range blobs {
shuffled[i] = blobs[i]
}
@@ -974,10 +974,10 @@ func TestSortBlobs(t *testing.T) {
for i := range blobs {
expect := blobs[i]
actual := sorted[i]
require.Equal(t, expect.Slot, actual.Slot)
require.Equal(t, expect.Slot(), actual.Slot())
require.Equal(t, expect.Index, actual.Index)
require.Equal(t, bytesutil.ToBytes48(expect.KzgCommitment), bytesutil.ToBytes48(actual.KzgCommitment))
require.Equal(t, bytesutil.ToBytes32(expect.BlockRoot), bytesutil.ToBytes32(actual.BlockRoot))
require.Equal(t, expect.BlockRoot(), actual.BlockRoot())
}
}
@@ -1047,7 +1047,7 @@ func TestBlobRequest(t *testing.T) {
require.Equal(t, len(allAfter), int(req.Count))
}
func testSequenceBlockWithBlob(t *testing.T, nblocks int) ([]blocks.BlockWithVerifiedBlobs, []*ethpb.DeprecatedBlobSidecar) {
func testSequenceBlockWithBlob(t *testing.T, nblocks int) ([]blocks.BlockWithVerifiedBlobs, []blocks.ROBlob) {
blks, blobs := util.ExtendBlocksPlusBlobs(t, []blocks.ROBlock{}, nblocks)
sbbs := make([]interfaces.ReadOnlySignedBeaconBlock, len(blks))
for i := range blks {
@@ -1073,19 +1073,20 @@ func TestVerifyAndPopulateBlobs(t *testing.T) {
bwb, blobs = testSequenceBlockWithBlob(t, 10)
// Misalign the slots of the blobs for the first block to simulate them being missing from the response.
offByOne := blobs[0].Slot
offByOne := blobs[0].Slot()
for i := range blobs {
if blobs[i].Slot == offByOne {
blobs[i].Slot = offByOne + 1
if blobs[i].Slot() == offByOne {
blobs[i].SignedBlockHeader.Header.Slot = offByOne + 1
}
}
_, err = verifyAndPopulateBlobs(bwb, blobs, firstBlockSlot)
require.ErrorContains(t, "BlockSlot in BlobSidecar does not match the expected slot", err)
require.ErrorIs(t, err, verify.ErrBlobBlockMisaligned)
bwb, blobs = testSequenceBlockWithBlob(t, 10)
blobs[lastBlobIdx].BlockRoot = blobs[0].BlockRoot
blobs[lastBlobIdx], err = blocks.NewROBlobWithRoot(blobs[lastBlobIdx].BlobSidecar, blobs[0].BlockRoot())
require.NoError(t, err)
_, err = verifyAndPopulateBlobs(bwb, blobs, firstBlockSlot)
require.ErrorIs(t, err, verify.ErrMismatchedBlobBlockRoot)
require.ErrorIs(t, err, verify.ErrBlobBlockMisaligned)
bwb, blobs = testSequenceBlockWithBlob(t, 10)
blobs[lastBlobIdx].Index = 100
@@ -1093,18 +1094,24 @@ func TestVerifyAndPopulateBlobs(t *testing.T) {
require.ErrorIs(t, err, verify.ErrIncorrectBlobIndex)
bwb, blobs = testSequenceBlockWithBlob(t, 10)
blobs[lastBlobIdx].ProposerIndex = 100
blobs[lastBlobIdx].SignedBlockHeader.Header.ProposerIndex = 100
blobs[lastBlobIdx], err = blocks.NewROBlob(blobs[lastBlobIdx].BlobSidecar)
require.NoError(t, err)
_, err = verifyAndPopulateBlobs(bwb, blobs, firstBlockSlot)
require.ErrorIs(t, err, verify.ErrMismatchedProposerIndex)
require.ErrorIs(t, err, verify.ErrBlobBlockMisaligned)
bwb, blobs = testSequenceBlockWithBlob(t, 10)
blobs[lastBlobIdx].BlockParentRoot = blobs[0].BlockParentRoot
blobs[lastBlobIdx].SignedBlockHeader.Header.ParentRoot = blobs[0].SignedBlockHeader.Header.ParentRoot
blobs[lastBlobIdx], err = blocks.NewROBlob(blobs[lastBlobIdx].BlobSidecar)
require.NoError(t, err)
_, err = verifyAndPopulateBlobs(bwb, blobs, firstBlockSlot)
require.ErrorIs(t, err, verify.ErrMismatchedBlobBlockRoot)
require.ErrorIs(t, err, verify.ErrBlobBlockMisaligned)
var emptyKzg [48]byte
bwb, blobs = testSequenceBlockWithBlob(t, 10)
blobs[lastBlobIdx].KzgCommitment = emptyKzg[:]
blobs[lastBlobIdx], err = blocks.NewROBlob(blobs[lastBlobIdx].BlobSidecar)
require.NoError(t, err)
_, err = verifyAndPopulateBlobs(bwb, blobs, firstBlockSlot)
require.ErrorIs(t, err, verify.ErrMismatchedBlobCommitments)

View File

@@ -11,6 +11,7 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
@@ -163,8 +164,15 @@ func (s *Service) processFetchedDataRegSync(
blksWithoutParentCount := 0
for _, b := range data.bwb {
if len(b.Blobs) > 0 {
if err := s.cfg.DB.SaveBlobSidecar(ctx, b.Blobs); err != nil {
log.WithError(err).Warn("Failed to save blob sidecar")
verified, err := verification.BlobSidecarSliceNoop(b.Blobs)
if err != nil {
log.WithField("root", b.Block.Root()).WithError(err).Error("blobs failed verification")
continue
}
for i := range verified {
if err := s.cfg.BlobStorage.Save(verified[i]); err != nil {
log.WithError(err).Warn("Failed to save blob sidecar")
}
}
}
@@ -326,8 +334,14 @@ func (s *Service) processBatchedBlocks(ctx context.Context, genesis time.Time,
if len(bb.Blobs) == 0 {
continue
}
if err := s.cfg.DB.SaveBlobSidecar(ctx, bb.Blobs); err != nil {
return errors.Wrapf(err, "failed to save blobs for block %#x", bb.Block.Root())
verified, err := verification.BlobSidecarSliceNoop(bb.Blobs)
if err != nil {
return errors.Wrapf(err, "blobs for root %#x failed verification", bb.Block.Root())
}
for i := range verified {
if err := s.cfg.BlobStorage.Save(verified[i]); err != nil {
return errors.Wrapf(err, "failed to save blobs for block %#x", bb.Block.Root())
}
}
blobCount += len(bb.Blobs)
}

View File

@@ -14,6 +14,7 @@ import (
blockfeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/block"
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
@@ -41,6 +42,7 @@ type Config struct {
BlockNotifier blockfeed.Notifier
ClockWaiter startup.ClockWaiter
InitialSyncComplete chan struct{}
BlobStorage *filesystem.BlobStorage
}
// Service service.

View File

@@ -9,6 +9,7 @@ import (
"github.com/paulbellamy/ratecounter"
"github.com/prysmaticlabs/prysm/v4/async/abool"
mock "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
dbtest "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/testing"
p2pt "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
@@ -382,6 +383,7 @@ func TestService_Resync(t *testing.T) {
P2P: p,
Chain: mc,
StateNotifier: mc.StateNotifier(),
BlobStorage: filesystem.NewEphemeralBlobStorage(t),
})
assert.NotNil(t, s)
assert.Equal(t, primitives.Slot(0), s.cfg.Chain.HeadSlot())

View File

@@ -6,6 +6,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/operation"
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/attestations"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/blstoexec"
@@ -152,3 +153,11 @@ func WithStateNotifier(n statefeed.Notifier) Option {
return nil
}
}
// WithBlobStorage gives the sync package direct access to BlobStorage.
func WithBlobStorage(b *filesystem.BlobStorage) Option {
return func(s *Service) error {
s.cfg.blobStorage = b
return nil
}
}

View File

@@ -7,10 +7,11 @@ import (
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/sync/verify"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
@@ -180,39 +181,35 @@ func (s *Service) sendAndSaveBlobSidecars(ctx context.Context, request types.Blo
log.WithFields(blobFields(sidecar)).Debug("Received blob sidecar RPC")
}
return s.cfg.beaconDB.SaveBlobSidecar(ctx, sidecars)
vscs, err := verification.BlobSidecarSliceNoop(sidecars)
if err != nil {
return err
}
for i := range vscs {
if err := s.cfg.blobStorage.Save(vscs[i]); err != nil {
return err
}
}
return nil
}
// constructPendingBlobsRequest creates a request for BlobSidecars by root, considering blobs already in DB.
func (s *Service) constructPendingBlobsRequest(ctx context.Context, blockRoot [32]byte, count int) (types.BlobSidecarsByRootReq, error) {
knownBlobs, err := s.cfg.beaconDB.BlobSidecarsByRoot(ctx, blockRoot)
if err != nil && !errors.Is(err, db.ErrNotFound) {
have, err := s.cfg.blobStorage.Indices(blockRoot)
if err != nil {
return nil, err
}
knownIndices := indexSetFromBlobs(knownBlobs)
requestedIndices := filterUnknownIndices(knownIndices, count, blockRoot)
return requestedIndices, nil
}
// Helper function to create a set of known indices.
func indexSetFromBlobs(blobs []*eth.DeprecatedBlobSidecar) map[uint64]struct{} {
indices := make(map[uint64]struct{})
for _, blob := range blobs {
indices[blob.Index] = struct{}{}
}
return indices
return requestsForMissingIndices(have, count, blockRoot), nil
}
// Helper function to filter out known indices.
func filterUnknownIndices(knownIndices map[uint64]struct{}, count int, blockRoot [32]byte) []*eth.BlobIdentifier {
func requestsForMissingIndices(haveIndices [fieldparams.MaxBlobsPerBlock]bool, count int, blockRoot [32]byte) []*eth.BlobIdentifier {
var ids []*eth.BlobIdentifier
for i := uint64(0); i < uint64(count); i++ {
if _, exists := knownIndices[i]; exists {
continue
if !haveIndices[i] {
ids = append(ids, &eth.BlobIdentifier{Index: i, BlockRoot: blockRoot[:]})
}
ids = append(ids, &eth.BlobIdentifier{Index: i, BlockRoot: blockRoot[:]})
}
return ids
}

View File

@@ -15,6 +15,7 @@ import (
gcache "github.com/patrickmn/go-cache"
mock "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
db "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/testing"
mockExecution "github.com/prysmaticlabs/prysm/v4/beacon-chain/execution/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
@@ -22,6 +23,7 @@ import (
p2ptest "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/testing"
p2pTypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
@@ -364,7 +366,7 @@ func TestRecentBeaconBlocksRPCHandler_HandleZeroBlocks(t *testing.T) {
}
func TestRequestPendingBlobs(t *testing.T) {
s := &Service{}
s := &Service{cfg: &config{blobStorage: filesystem.NewEphemeralBlobStorage(t)}}
t.Run("old block should not fail", func(t *testing.T) {
b, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlock())
require.NoError(t, err)
@@ -393,10 +395,11 @@ func TestRequestPendingBlobs(t *testing.T) {
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{FinalizedEpoch: 1})
s := &Service{
cfg: &config{
p2p: p1,
chain: chain,
clock: startup.NewClock(time.Unix(0, 0), [32]byte{}),
beaconDB: db.SetupDB(t),
p2p: p1,
chain: chain,
clock: startup.NewClock(time.Unix(0, 0), [32]byte{}),
beaconDB: db.SetupDB(t),
blobStorage: filesystem.NewEphemeralBlobStorage(t),
},
}
b := util.NewBeaconBlockDeneb()
@@ -409,7 +412,8 @@ func TestRequestPendingBlobs(t *testing.T) {
func TestConstructPendingBlobsRequest(t *testing.T) {
d := db.SetupDB(t)
s := &Service{cfg: &config{beaconDB: d}}
bs := filesystem.NewEphemeralBlobStorage(t)
s := &Service{cfg: &config{beaconDB: d, blobStorage: bs}}
ctx := context.Background()
// No unknown indices.
@@ -424,11 +428,23 @@ func TestConstructPendingBlobsRequest(t *testing.T) {
}
// Has indices.
blobSidecars := []*ethpb.DeprecatedBlobSidecar{
{Index: 0, BlockRoot: root[:]},
{Index: 2, BlockRoot: root[:]},
header := &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
ParentRoot: bytesutil.PadTo([]byte{}, 32),
StateRoot: bytesutil.PadTo([]byte{}, 32),
BodyRoot: bytesutil.PadTo([]byte{}, 32),
},
Signature: bytesutil.PadTo([]byte{}, 96),
}
blobSidecars := []blocks.ROBlob{
util.GenerateTestDenebBlobSidecar(t, root, header, 0, bytesutil.PadTo([]byte{}, 48)),
util.GenerateTestDenebBlobSidecar(t, root, header, 2, bytesutil.PadTo([]byte{}, 48)),
}
vscs, err := verification.BlobSidecarSliceNoop(blobSidecars)
require.NoError(t, err)
for i := range vscs {
require.NoError(t, bs.Save(vscs[i]))
}
require.NoError(t, d.SaveBlobSidecar(ctx, blobSidecars))
expected := []*eth.BlobIdentifier{
{Index: 1, BlockRoot: root[:]},
@@ -439,29 +455,8 @@ func TestConstructPendingBlobsRequest(t *testing.T) {
require.DeepEqual(t, expected[0].BlockRoot, actual[0].BlockRoot)
}
func TestIndexSetFromBlobs(t *testing.T) {
blobs := []*ethpb.DeprecatedBlobSidecar{
{Index: 0},
{Index: 1},
{Index: 2},
}
expected := map[uint64]struct{}{
0: {},
1: {},
2: {},
}
actual := indexSetFromBlobs(blobs)
require.DeepEqual(t, expected, actual)
}
func TestFilterUnknownIndices(t *testing.T) {
knownIndices := map[uint64]struct{}{
0: {},
1: {},
2: {},
}
haveIndices := [fieldparams.MaxBlobsPerBlock]bool{true, true, true, false, false, false}
blockRoot := [32]byte{}
count := 5
@@ -471,7 +466,7 @@ func TestFilterUnknownIndices(t *testing.T) {
{Index: 4, BlockRoot: blockRoot[:]},
}
actual := filterUnknownIndices(knownIndices, count, blockRoot)
actual := requestsForMissingIndices(haveIndices, count, blockRoot)
require.Equal(t, len(expected), len(actual))
require.Equal(t, expected[0].Index, actual[0].Index)
require.DeepEqual(t, actual[0].BlockRoot, expected[0].BlockRoot)

View File

@@ -7,7 +7,6 @@ import (
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
p2ptypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
@@ -29,15 +28,22 @@ func (s *Service) streamBlobBatch(ctx context.Context, batch blockBatch, wQuota
defer span.End()
for _, b := range batch.canonical() {
root := b.Root()
scs, err := s.cfg.beaconDB.BlobSidecarsByRoot(ctx, b.Root())
if errors.Is(err, db.ErrNotFound) {
continue
}
idxs, err := s.cfg.blobStorage.Indices(b.Root())
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
return wQuota, errors.Wrapf(err, "could not retrieve sidecars for block root %#x", root)
}
for _, sc := range scs {
for i, l := uint64(0), uint64(len(idxs)); i < l; i++ {
// index not available, skip
if !idxs[i] {
continue
}
// We won't check for file not found since the .Indices method should normally prevent that from happening.
sc, err := s.cfg.blobStorage.Get(b.Root(), i)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
return wQuota, errors.Wrapf(err, "could not retrieve sidecar: index %d, block root %#x", i, root)
}
SetStreamWriteDeadline(stream, defaultWriteDuration)
if chunkErr := WriteBlobSidecarChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), sc); chunkErr != nil {
log.WithError(chunkErr).Debug("Could not send a chunked response")

View File

@@ -6,8 +6,8 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
types "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/testing/require"
"github.com/prysmaticlabs/prysm/v4/time/slots"
@@ -24,40 +24,41 @@ func (c *blobsTestCase) defaultOldestSlotByRange(t *testing.T) types.Slot {
return oldestSlot
}
func blobRangeRequestFromSidecars(scs []*ethpb.DeprecatedBlobSidecar) interface{} {
func blobRangeRequestFromSidecars(scs []blocks.ROBlob) interface{} {
maxBlobs := fieldparams.MaxBlobsPerBlock
count := uint64(len(scs) / maxBlobs)
return &ethpb.BlobSidecarsByRangeRequest{
StartSlot: scs[0].Slot,
StartSlot: scs[0].Slot(),
Count: count,
}
}
func (c *blobsTestCase) filterExpectedByRange(t *testing.T, scs []*ethpb.DeprecatedBlobSidecar, req interface{}) []*expectedBlobChunk {
func (c *blobsTestCase) filterExpectedByRange(t *testing.T, scs []blocks.ROBlob, req interface{}) []*expectedBlobChunk {
var expect []*expectedBlobChunk
blockOffset := 0
lastRoot := bytesutil.ToBytes32(scs[0].BlockRoot)
lastRoot := scs[0].BlockRoot()
rreq, ok := req.(*ethpb.BlobSidecarsByRangeRequest)
require.Equal(t, true, ok)
var writes uint64
for _, sc := range scs {
root := bytesutil.ToBytes32(sc.BlockRoot)
for i := range scs {
sc := scs[i]
root := sc.BlockRoot()
if root != lastRoot {
blockOffset += 1
}
lastRoot = root
if sc.Slot < c.oldestSlot(t) {
if sc.Slot() < c.oldestSlot(t) {
continue
}
if sc.Slot < rreq.StartSlot || sc.Slot > rreq.StartSlot+types.Slot(rreq.Count)-1 {
if sc.Slot() < rreq.StartSlot || sc.Slot() > rreq.StartSlot+types.Slot(rreq.Count)-1 {
continue
}
if writes == params.BeaconNetworkConfig().MaxRequestBlobSidecars {
continue
}
expect = append(expect, &expectedBlobChunk{
sidecar: sc,
sidecar: &sc,
code: responseCodeSuccess,
message: "",
})
@@ -107,9 +108,9 @@ func TestBlobByRangeOK(t *testing.T) {
{
name: "10 slots before window, 10 slots after, count = 20",
nblocks: 10,
requestFromSidecars: func(scs []*ethpb.DeprecatedBlobSidecar) interface{} {
requestFromSidecars: func(scs []blocks.ROBlob) interface{} {
return &ethpb.BlobSidecarsByRangeRequest{
StartSlot: scs[0].Slot - 10,
StartSlot: scs[0].Slot() - 10,
Count: 20,
}
},
@@ -117,9 +118,9 @@ func TestBlobByRangeOK(t *testing.T) {
{
name: "request before window, empty response",
nblocks: 10,
requestFromSidecars: func(scs []*ethpb.DeprecatedBlobSidecar) interface{} {
requestFromSidecars: func(scs []blocks.ROBlob) interface{} {
return &ethpb.BlobSidecarsByRangeRequest{
StartSlot: scs[0].Slot - 10,
StartSlot: scs[0].Slot() - 10,
Count: 10,
}
},
@@ -128,9 +129,9 @@ func TestBlobByRangeOK(t *testing.T) {
{
name: "10 blocks * 4 blobs = 40",
nblocks: 10,
requestFromSidecars: func(scs []*ethpb.DeprecatedBlobSidecar) interface{} {
requestFromSidecars: func(scs []blocks.ROBlob) interface{} {
return &ethpb.BlobSidecarsByRangeRequest{
StartSlot: scs[0].Slot - 10,
StartSlot: scs[0].Slot() - 10,
Count: 20,
}
},
@@ -139,9 +140,9 @@ func TestBlobByRangeOK(t *testing.T) {
{
name: "when request count > MAX_REQUEST_BLOCKS_DENEB, MAX_REQUEST_BLOBS_SIDECARS sidecars in response",
nblocks: int(params.BeaconNetworkConfig().MaxRequestBlocksDeneb) + 10,
requestFromSidecars: func(scs []*ethpb.DeprecatedBlobSidecar) interface{} {
requestFromSidecars: func(scs []blocks.ROBlob) interface{} {
return &ethpb.BlobSidecarsByRangeRequest{
StartSlot: scs[0].Slot,
StartSlot: scs[0].Slot(),
Count: params.BeaconNetworkConfig().MaxRequestBlocksDeneb + 1,
}
},

View File

@@ -15,7 +15,6 @@ import (
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v4/monitoring/tracing"
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/time/slots"
"go.opencensus.io/trace"
)
@@ -66,10 +65,6 @@ func (s *Service) blobSidecarByRootRPCHandler(ctx context.Context, msg interface
}
minReqEpoch := blobMinReqEpoch(s.cfg.chain.FinalizedCheckpt().Epoch, slots.ToEpoch(s.cfg.clock.CurrentSlot()))
buff := struct {
root [32]byte
scs []*eth.DeprecatedBlobSidecar
}{}
for i := range blobIdents {
if err := ctx.Err(); err != nil {
closeStream(stream, log)
@@ -82,31 +77,20 @@ func (s *Service) blobSidecarByRootRPCHandler(ctx context.Context, msg interface
}
s.rateLimiter.add(stream, 1)
root, idx := bytesutil.ToBytes32(blobIdents[i].BlockRoot), blobIdents[i].Index
if root != buff.root {
scs, err := s.cfg.beaconDB.BlobSidecarsByRoot(ctx, root)
buff.root, buff.scs = root, scs
if err != nil {
if errors.Is(err, db.ErrNotFound) {
// In case db error path gave us a non-nil value, make sure that other indices for the problem root
// are not processed when we reenter the outer loop.
buff.scs = nil
log.WithError(err).Debugf("BlobSidecar not found in db, root=%x, index=%d", root, idx)
continue
}
log.WithError(err).Errorf("unexpected db error retrieving BlobSidecar, root=%x, index=%d", root, idx)
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
return err
sc, err := s.cfg.blobStorage.Get(root, idx)
if err != nil {
if db.IsNotFound(err) {
log.WithError(err).Debugf("BlobSidecar not found in db, root=%x, index=%d", root, idx)
continue
}
log.WithError(err).Errorf("unexpected db error retrieving BlobSidecar, root=%x, index=%d", root, idx)
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
return err
}
if idx >= uint64(len(buff.scs)) {
continue
}
sc := buff.scs[idx]
// If any root in the request content references a block earlier than minimum_request_epoch,
// peers MAY respond with error code 3: ResourceUnavailable or not include the blob in the response.
if slots.ToEpoch(sc.Slot) < minReqEpoch {
if slots.ToEpoch(sc.Slot()) < minReqEpoch {
s.writeErrorResponseToStream(responseCodeResourceUnavailable, types.ErrBlobLTMinRequest.Error(), stream)
log.WithError(types.ErrBlobLTMinRequest).
Debugf("requested blob for block %#x before minimum_request_epoch", blobIdents[i].BlockRoot)

View File

@@ -10,6 +10,7 @@ import (
p2pTypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/types"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
types "github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
@@ -23,15 +24,16 @@ func (c *blobsTestCase) defaultOldestSlotByRoot(t *testing.T) types.Slot {
return oldest
}
func blobRootRequestFromSidecars(scs []*ethpb.DeprecatedBlobSidecar) interface{} {
func blobRootRequestFromSidecars(scs []blocks.ROBlob) interface{} {
req := make(p2pTypes.BlobSidecarsByRootReq, 0)
for _, sc := range scs {
req = append(req, &ethpb.BlobIdentifier{BlockRoot: sc.BlockRoot, Index: sc.Index})
for i := range scs {
sc := scs[i]
req = append(req, &ethpb.BlobIdentifier{BlockRoot: sc.BlockRootSlice(), Index: sc.Index})
}
return &req
}
func (c *blobsTestCase) filterExpectedByRoot(t *testing.T, scs []*ethpb.DeprecatedBlobSidecar, r interface{}) []*expectedBlobChunk {
func (c *blobsTestCase) filterExpectedByRoot(t *testing.T, scs []blocks.ROBlob, r interface{}) []*expectedBlobChunk {
rp, ok := r.(*p2pTypes.BlobSidecarsByRootReq)
if !ok {
panic("unexpected request type in filterExpectedByRoot")
@@ -49,12 +51,13 @@ func (c *blobsTestCase) filterExpectedByRoot(t *testing.T, scs []*ethpb.Deprecat
if len(scs) == 0 {
return expect
}
lastRoot := bytesutil.ToBytes32(scs[0].BlockRoot)
lastRoot := scs[0].BlockRoot()
rootToOffset := make(map[[32]byte]int)
rootToOffset[lastRoot] = 0
scMap := make(map[[32]byte]map[uint64]*ethpb.DeprecatedBlobSidecar)
for _, sc := range scs {
root := bytesutil.ToBytes32(sc.BlockRoot)
scMap := make(map[[32]byte]map[uint64]blocks.ROBlob)
for i := range scs {
sc := scs[i]
root := sc.BlockRoot()
if root != lastRoot {
blockOffset += 1
rootToOffset[root] = blockOffset
@@ -62,11 +65,12 @@ func (c *blobsTestCase) filterExpectedByRoot(t *testing.T, scs []*ethpb.Deprecat
lastRoot = root
_, ok := scMap[root]
if !ok {
scMap[root] = make(map[uint64]*ethpb.DeprecatedBlobSidecar)
scMap[root] = make(map[uint64]blocks.ROBlob)
}
scMap[root][sc.Index] = sc
}
for _, scid := range req {
for i := range req {
scid := req[i]
rootMap, ok := scMap[bytesutil.ToBytes32(scid.BlockRoot)]
if !ok {
panic(fmt.Sprintf("test setup failure, no fixture with root %#x", scid.BlockRoot))
@@ -76,7 +80,7 @@ func (c *blobsTestCase) filterExpectedByRoot(t *testing.T, scs []*ethpb.Deprecat
panic(fmt.Sprintf("test setup failure, no fixture at index %d with root %#x", scid.Index, scid.BlockRoot))
}
// Skip sidecars that are supposed to be missing.
root := bytesutil.ToBytes32(sc.BlockRoot)
root := sc.BlockRoot()
if c.missing[rootToOffset[root]] {
continue
}
@@ -86,14 +90,14 @@ func (c *blobsTestCase) filterExpectedByRoot(t *testing.T, scs []*ethpb.Deprecat
// will set streamTerminated = true and skip everything else in the test case.
if c.expired[rootToOffset[root]] {
return append(expect, &expectedBlobChunk{
sidecar: sc,
sidecar: &sc,
code: responseCodeResourceUnavailable,
message: p2pTypes.ErrBlobLTMinRequest.Error(),
})
}
expect = append(expect, &expectedBlobChunk{
sidecar: sc,
sidecar: &sc,
code: responseCodeSuccess,
message: "",
})
@@ -148,7 +152,7 @@ func readChunkEncodedBlobsLowMax(t *testing.T, s *Service, expect []*expectedBlo
encoding := s.cfg.p2p.Encoding()
ctxMap, err := ContextByteVersionsForValRoot(s.cfg.clock.GenesisValidatorsRoot())
require.NoError(t, err)
vf := func(sidecar *ethpb.DeprecatedBlobSidecar) error {
vf := func(sidecar blocks.ROBlob) error {
return nil
}
return func(stream network.Stream) {
@@ -161,7 +165,7 @@ func readChunkEncodedBlobsAsStreamReader(t *testing.T, s *Service, expect []*exp
encoding := s.cfg.p2p.Encoding()
ctxMap, err := ContextByteVersionsForValRoot(s.cfg.clock.GenesisValidatorsRoot())
require.NoError(t, err)
vf := func(sidecar *ethpb.DeprecatedBlobSidecar) error {
vf := func(sidecar blocks.ROBlob) error {
return nil
}
return func(stream network.Stream) {
@@ -170,9 +174,9 @@ func readChunkEncodedBlobsAsStreamReader(t *testing.T, s *Service, expect []*exp
require.Equal(t, len(expect), len(scs))
for i, sc := range scs {
esc := expect[i].sidecar
require.Equal(t, esc.Slot, sc.Slot)
require.Equal(t, esc.Slot(), sc.Slot())
require.Equal(t, esc.Index, sc.Index)
require.Equal(t, bytesutil.ToBytes32(esc.BlockRoot), bytesutil.ToBytes32(sc.BlockRoot))
require.Equal(t, esc.BlockRoot(), sc.BlockRoot())
}
}
}

View File

@@ -9,10 +9,10 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/encoder"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v4/network/forks"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/runtime/version"
"github.com/prysmaticlabs/prysm/v4/time/slots"
)
@@ -159,12 +159,12 @@ func extractBlockDataType(digest []byte, tor blockchain.TemporalOracle) (interfa
// WriteBlobSidecarChunk writes blob chunk object to stream.
// response_chunk ::= <result> | <context-bytes> | <encoding-dependent-header> | <encoded-payload>
func WriteBlobSidecarChunk(stream libp2pcore.Stream, tor blockchain.TemporalOracle, encoding encoder.NetworkEncoding, sidecar *ethpb.DeprecatedBlobSidecar) error {
func WriteBlobSidecarChunk(stream libp2pcore.Stream, tor blockchain.TemporalOracle, encoding encoder.NetworkEncoding, sidecar blocks.VerifiedROBlob) error {
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return err
}
valRoot := tor.GenesisValidatorsRoot()
ctxBytes, err := forks.ForkDigestFromEpoch(slots.ToEpoch(sidecar.GetSlot()), valRoot[:])
ctxBytes, err := forks.ForkDigestFromEpoch(slots.ToEpoch(sidecar.Slot()), valRoot[:])
if err != nil {
return err
}

View File

@@ -14,9 +14,11 @@ import (
p2ptypes "github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p/types"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
pb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/runtime/version"
"github.com/prysmaticlabs/prysm/v4/time/slots"
@@ -144,7 +146,7 @@ func SendBeaconBlocksByRootRequest(
return blocks, nil
}
func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle, p2pApi p2p.SenderEncoder, pid peer.ID, ctxMap ContextByteVersions, req *pb.BlobSidecarsByRangeRequest) ([]*pb.DeprecatedBlobSidecar, error) {
func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle, p2pApi p2p.SenderEncoder, pid peer.ID, ctxMap ContextByteVersions, req *pb.BlobSidecarsByRangeRequest) ([]blocks.ROBlob, error) {
topic, err := p2p.TopicFromMessage(p2p.BlobSidecarsByRangeName, slots.ToEpoch(tor.CurrentSlot()))
if err != nil {
return nil, err
@@ -166,7 +168,7 @@ func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle,
func SendBlobSidecarByRoot(
ctx context.Context, tor blockchain.TemporalOracle, p2pApi p2p.P2P, pid peer.ID,
ctxMap ContextByteVersions, req *p2ptypes.BlobSidecarsByRootReq,
) ([]*pb.DeprecatedBlobSidecar, error) {
) ([]blocks.ROBlob, error) {
if uint64(len(*req)) > params.BeaconNetworkConfig().MaxRequestBlobSidecars {
return nil, errors.Wrapf(p2ptypes.ErrMaxBlobReqExceeded, "length=%d", len(*req))
}
@@ -189,16 +191,16 @@ func SendBlobSidecarByRoot(
return readChunkEncodedBlobs(stream, p2pApi.Encoding(), ctxMap, blobValidatorFromRootReq(req), max)
}
type blobResponseValidation func(*pb.DeprecatedBlobSidecar) error
type blobResponseValidation func(blocks.ROBlob) error
func blobValidatorFromRootReq(req *p2ptypes.BlobSidecarsByRootReq) blobResponseValidation {
roots := make(map[[32]byte]bool)
for _, sc := range *req {
roots[bytesutil.ToBytes32(sc.BlockRoot)] = true
}
return func(sc *pb.DeprecatedBlobSidecar) error {
if requested := roots[bytesutil.ToBytes32(sc.BlockRoot)]; !requested {
return errors.Wrapf(errUnrequestedRoot, "root=%#x", sc.BlockRoot)
return func(sc blocks.ROBlob) error {
if requested := roots[sc.BlockRoot()]; !requested {
return errors.Wrapf(errUnrequestedRoot, "root=%#x", sc.BlockRoot())
}
return nil
}
@@ -206,16 +208,16 @@ func blobValidatorFromRootReq(req *p2ptypes.BlobSidecarsByRootReq) blobResponseV
func blobValidatorFromRangeReq(req *pb.BlobSidecarsByRangeRequest) blobResponseValidation {
end := req.StartSlot + primitives.Slot(req.Count)
return func(sc *pb.DeprecatedBlobSidecar) error {
if sc.Slot < req.StartSlot || sc.Slot >= end {
return errors.Wrapf(errBlobResponseOutOfBounds, "req start,end:%d,%d, resp:%d", req.StartSlot, end, sc.Slot)
return func(sc blocks.ROBlob) error {
if sc.Slot() < req.StartSlot || sc.Slot() >= end {
return errors.Wrapf(errBlobResponseOutOfBounds, "req start,end:%d,%d, resp:%d", req.StartSlot, end, sc.Slot())
}
return nil
}
}
func readChunkEncodedBlobs(stream network.Stream, encoding encoder.NetworkEncoding, ctxMap ContextByteVersions, vf blobResponseValidation, max uint64) ([]*pb.DeprecatedBlobSidecar, error) {
sidecars := make([]*pb.DeprecatedBlobSidecar, 0)
func readChunkEncodedBlobs(stream network.Stream, encoding encoder.NetworkEncoding, ctxMap ContextByteVersions, vf blobResponseValidation, max uint64) ([]blocks.ROBlob, error) {
sidecars := make([]blocks.ROBlob, 0)
// Attempt an extra read beyond max to check if the peer is violating the spec by
// sending more than MAX_REQUEST_BLOB_SIDECARS, or more blobs than requested.
for i := uint64(0); i < max+1; i++ {
@@ -238,7 +240,9 @@ func readChunkEncodedBlobs(stream network.Stream, encoding encoder.NetworkEncodi
return sidecars, nil
}
func readChunkedBlobSidecar(stream network.Stream, encoding encoder.NetworkEncoding, ctxMap ContextByteVersions, vf blobResponseValidation) (*pb.DeprecatedBlobSidecar, error) {
func readChunkedBlobSidecar(stream network.Stream, encoding encoder.NetworkEncoding, ctxMap ContextByteVersions, vf blobResponseValidation) (blocks.ROBlob, error) {
var b blocks.ROBlob
pb := &ethpb.BlobSidecar{}
decode := encoding.DecodeWithMaxLength
var (
code uint8
@@ -246,31 +250,35 @@ func readChunkedBlobSidecar(stream network.Stream, encoding encoder.NetworkEncod
)
code, msg, err := ReadStatusCode(stream, encoding)
if err != nil {
return nil, err
return b, err
}
if code != 0 {
return nil, errors.Wrap(errBlobChunkedReadFailure, msg)
return b, errors.Wrap(errBlobChunkedReadFailure, msg)
}
ctxb, err := readContextFromStream(stream)
if err != nil {
return nil, errors.Wrap(err, "error reading chunk context bytes from stream")
return b, errors.Wrap(err, "error reading chunk context bytes from stream")
}
v, found := ctxMap[bytesutil.ToBytes4(ctxb)]
if !found {
return nil, errors.Wrapf(errBlobUnmarshal, fmt.Sprintf("unrecognized fork digest %#x", ctxb))
return b, errors.Wrapf(errBlobUnmarshal, fmt.Sprintf("unrecognized fork digest %#x", ctxb))
}
// Only deneb is supported at this time, because we lack a fork-spanning interface/union type for blobs.
if v != version.Deneb {
return nil, fmt.Errorf("unexpected context bytes for deneb BlobSidecar, ctx=%#x, v=%s", ctxb, version.String(v))
return b, fmt.Errorf("unexpected context bytes for deneb BlobSidecar, ctx=%#x, v=%s", ctxb, version.String(v))
}
sc := &pb.DeprecatedBlobSidecar{}
if err := decode(stream, sc); err != nil {
return nil, errors.Wrap(err, "failed to decode the protobuf-encoded BlobSidecar message from RPC chunk stream")
}
if err := vf(sc); err != nil {
return nil, errors.Wrap(err, "validation failure decoding blob RPC response")
if err := decode(stream, pb); err != nil {
return b, errors.Wrap(err, "failed to decode the protobuf-encoded BlobSidecar message from RPC chunk stream")
}
return sc, nil
rob, err := blocks.NewROBlob(pb)
if err != nil {
return b, errors.Wrap(err, "unexpected error initializing ROBlob")
}
if err := vf(rob); err != nil {
return b, errors.Wrap(err, "validation failure decoding blob RPC response")
}
return rob, nil
}

View File

@@ -481,21 +481,24 @@ func TestSendRequest_SendBeaconBlocksByRootRequest(t *testing.T) {
func TestBlobValidatorFromRootReq(t *testing.T) {
validRoot := bytesutil.PadTo([]byte("valid"), 32)
invalidRoot := bytesutil.PadTo([]byte("invalid"), 32)
header := &ethpb.SignedBeaconBlockHeader{}
validb := util.GenerateTestDenebBlobSidecar(t, bytesutil.ToBytes32(validRoot), header, 0, []byte{})
invalidb := util.GenerateTestDenebBlobSidecar(t, bytesutil.ToBytes32(invalidRoot), header, 0, []byte{})
cases := []struct {
name string
ids []*ethpb.BlobIdentifier
response []*ethpb.DeprecatedBlobSidecar
response []blocks.ROBlob
err error
}{
{
name: "valid",
ids: []*ethpb.BlobIdentifier{{BlockRoot: validRoot}},
response: []*ethpb.DeprecatedBlobSidecar{{BlockRoot: validRoot}},
response: []blocks.ROBlob{validb},
},
{
name: "invalid",
ids: []*ethpb.BlobIdentifier{{BlockRoot: validRoot}},
response: []*ethpb.DeprecatedBlobSidecar{{BlockRoot: invalidRoot}},
response: []blocks.ROBlob{invalidb},
err: errUnrequestedRoot,
},
}
@@ -517,10 +520,10 @@ func TestBlobValidatorFromRootReq(t *testing.T) {
func TestBlobValidatorFromRangeReq(t *testing.T) {
cases := []struct {
name string
req *ethpb.BlobSidecarsByRangeRequest
response []*ethpb.DeprecatedBlobSidecar
err error
name string
req *ethpb.BlobSidecarsByRangeRequest
responseSlot primitives.Slot
err error
}{
{
name: "valid - count multi",
@@ -528,7 +531,7 @@ func TestBlobValidatorFromRangeReq(t *testing.T) {
StartSlot: 10,
Count: 10,
},
response: []*ethpb.DeprecatedBlobSidecar{{Slot: 14}},
responseSlot: 14,
},
{
name: "valid - count 1",
@@ -536,7 +539,7 @@ func TestBlobValidatorFromRangeReq(t *testing.T) {
StartSlot: 10,
Count: 1,
},
response: []*ethpb.DeprecatedBlobSidecar{{Slot: 10}},
responseSlot: 10,
},
{
name: "invalid - before",
@@ -544,8 +547,8 @@ func TestBlobValidatorFromRangeReq(t *testing.T) {
StartSlot: 10,
Count: 1,
},
response: []*ethpb.DeprecatedBlobSidecar{{Slot: 9}},
err: errBlobResponseOutOfBounds,
responseSlot: 9,
err: errBlobResponseOutOfBounds,
},
{
name: "invalid - after, count 1",
@@ -553,8 +556,8 @@ func TestBlobValidatorFromRangeReq(t *testing.T) {
StartSlot: 10,
Count: 1,
},
response: []*ethpb.DeprecatedBlobSidecar{{Slot: 11}},
err: errBlobResponseOutOfBounds,
responseSlot: 11,
err: errBlobResponseOutOfBounds,
},
{
name: "invalid - after, multi",
@@ -562,8 +565,8 @@ func TestBlobValidatorFromRangeReq(t *testing.T) {
StartSlot: 10,
Count: 10,
},
response: []*ethpb.DeprecatedBlobSidecar{{Slot: 23}},
err: errBlobResponseOutOfBounds,
responseSlot: 23,
err: errBlobResponseOutOfBounds,
},
{
name: "invalid - after, at boundary, multi",
@@ -571,21 +574,23 @@ func TestBlobValidatorFromRangeReq(t *testing.T) {
StartSlot: 10,
Count: 10,
},
response: []*ethpb.DeprecatedBlobSidecar{{Slot: 20}},
err: errBlobResponseOutOfBounds,
responseSlot: 20,
err: errBlobResponseOutOfBounds,
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
vf := blobValidatorFromRangeReq(c.req)
for _, sc := range c.response {
err := vf(sc)
if c.err != nil {
require.ErrorIs(t, err, c.err)
return
}
require.NoError(t, err)
header := &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{Slot: c.responseSlot},
}
sc := util.GenerateTestDenebBlobSidecar(t, [32]byte{}, header, 0, []byte{})
err := vf(sc)
if c.err != nil {
require.ErrorIs(t, err, c.err)
return
}
require.NoError(t, err)
})
}
}

View File

@@ -23,6 +23,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/operation"
statefeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/attestations"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/blstoexec"
@@ -91,6 +92,7 @@ type config struct {
slasherBlockHeadersFeed *event.Feed
clock *startup.Clock
stateNotifier statefeed.Notifier
blobStorage *filesystem.BlobStorage
}
// This defines the interface for interacting with block chain service

View File

@@ -4,29 +4,30 @@ import (
"context"
"fmt"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed"
opfeed "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/feed/operation"
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"google.golang.org/protobuf/proto"
)
func (s *Service) blobSubscriber(ctx context.Context, msg proto.Message) error {
b, ok := msg.(*eth.SignedBlobSidecar)
b, ok := msg.(blocks.VerifiedROBlob)
if !ok {
return fmt.Errorf("message was not type *eth.SignedBlobSidecar, type=%T", msg)
return fmt.Errorf("message was not type blocks.ROBlob, type=%T", msg)
}
s.setSeenBlobIndex(b.Message.Blob, b.Message.Index)
s.setSeenBlobIndex(b.BlockRootSlice(), b.Index)
if err := s.cfg.chain.ReceiveBlob(ctx, b.Message); err != nil {
if err := s.cfg.chain.ReceiveBlob(ctx, b); err != nil {
return err
}
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
Type: opfeed.BlobSidecarReceived,
Data: &opfeed.BlobSidecarReceivedData{
Blob: b,
},
})
// TODO: convert operations feed to use ROBlob.
/*
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
Type: opfeed.BlobSidecarReceived,
Data: &opfeed.BlobSidecarReceivedData{
Blob: b,
},
})
*/
return nil
}

View File

@@ -9,15 +9,13 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/crypto/bls"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/crypto/rand"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v4/network/forks"
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
prysmTime "github.com/prysmaticlabs/prysm/v4/time"
"github.com/prysmaticlabs/prysm/v4/time/slots"
@@ -55,12 +53,15 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
return pubsub.ValidationReject, err
}
sBlob, ok := m.(*eth.SignedBlobSidecar)
bpb, ok := m.(*eth.BlobSidecar)
if !ok {
log.WithField("message", m).Error("Message is not of type *eth.SignedBlobSidecar")
log.WithField("message", m).Error("Message is not of type *eth.BlobSidecar")
return pubsub.ValidationReject, errWrongMessage
}
blob := sBlob.Message
blob, err := blocks.NewROBlob(bpb)
if err != nil {
return pubsub.ValidationReject, errors.Wrap(err, "roblob conversion failure")
}
// [REJECT] The sidecar's index is consistent with `MAX_BLOBS_PER_BLOCK` -- i.e. `sidecar.index < MAX_BLOBS_PER_BLOCK`
if blob.Index >= fieldparams.MaxBlobsPerBlock {
@@ -78,7 +79,7 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
// [IGNORE] The sidecar is not from a future slot (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) --
// i.e. validate that sidecar.slot <= current_slot (a client MAY queue future blocks for processing at the appropriate slot).
genesisTime := uint64(s.cfg.chain.GenesisTime().Unix())
if err := slots.VerifyTime(genesisTime, blob.Slot, earlyBlockProcessingTolerance); err != nil {
if err := slots.VerifyTime(genesisTime, blob.Slot(), earlyBlockProcessingTolerance); err != nil {
log.WithError(err).WithFields(blobFields(blob)).Debug("Ignored blob: too far into future")
return pubsub.ValidationIgnore, errors.Wrap(err, "blob too far into future")
}
@@ -89,14 +90,14 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
if err != nil {
return pubsub.ValidationIgnore, err
}
if startSlot >= blob.Slot {
err := fmt.Errorf("finalized slot %d greater or equal to blob slot %d", startSlot, blob.Slot)
if startSlot >= blob.Slot() {
err := fmt.Errorf("finalized slot %d greater or equal to blob slot %d", startSlot, blob.Slot())
log.WithFields(blobFields(blob)).Debug(err)
return pubsub.ValidationIgnore, err
}
// Handle the parent status (not seen or invalid cases)
parentRoot := bytesutil.ToBytes32(blob.BlockParentRoot)
parentRoot := blob.ParentRoot()
switch parentStatus := s.handleBlobParentStatus(ctx, parentRoot); parentStatus {
case pubsub.ValidationIgnore:
log.WithFields(blobFields(blob)).Debug("Parent block not found - saving blob to cache")
@@ -113,7 +114,7 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
default:
}
pubsubResult, err := s.validateBlobPostSeenParent(ctx, sBlob)
pubsubResult, err := s.validateBlobPostSeenParent(ctx, blob)
if err != nil {
return pubsubResult, err
}
@@ -121,7 +122,7 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
return pubsubResult, nil
}
startTime, err := slots.ToTime(genesisTime, blob.Slot)
startTime, err := slots.ToTime(genesisTime, blob.Slot())
if err != nil {
return pubsub.ValidationIgnore, err
}
@@ -133,22 +134,26 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
blobSidecarArrivalGossipSummary.Observe(float64(sinceSlotStartTime.Milliseconds()))
msg.ValidatorData = sBlob
vblob, err := verification.BlobSidecarNoop(blob)
if err != nil {
return pubsub.ValidationReject, errors.New("TODO: undefined verification error")
}
// use the VerifiedROBlob type from here on out
msg.ValidatorData = vblob
return pubsub.ValidationAccept, nil
}
func (s *Service) validateBlobPostSeenParent(ctx context.Context, sBlob *eth.SignedBlobSidecar) (pubsub.ValidationResult, error) {
blob := sBlob.Message
parentRoot := bytesutil.ToBytes32(blob.BlockParentRoot)
func (s *Service) validateBlobPostSeenParent(ctx context.Context, blob blocks.ROBlob) (pubsub.ValidationResult, error) {
parentRoot := blob.ParentRoot()
// [REJECT] The sidecar is from a higher slot than the sidecar's block's parent (defined by sidecar.block_parent_root).
parentSlot, err := s.cfg.chain.RecentBlockSlot(parentRoot)
if err != nil {
return pubsub.ValidationIgnore, err
}
if parentSlot >= blob.Slot {
err := fmt.Errorf("parent block slot %d greater or equal to blob slot %d", parentSlot, blob.Slot)
if parentSlot >= blob.Slot() {
err := fmt.Errorf("parent block slot %d greater or equal to blob slot %d", parentSlot, blob.Slot())
log.WithFields(blobFields(blob)).Debug(err)
return pubsub.ValidationReject, err
}
@@ -159,18 +164,21 @@ func (s *Service) validateBlobPostSeenParent(ctx context.Context, sBlob *eth.Sig
if err != nil {
return pubsub.ValidationIgnore, err
}
if err := verifyBlobSignature(parentState, sBlob); err != nil {
log.WithError(err).WithFields(blobFields(blob)).Debug("Failed to verify blob signature")
return pubsub.ValidationReject, err
}
// TODO: replace this with verification routine
/*
if err := verifyBlobSignature(parentState, blob); err != nil {
log.WithError(err).WithFields(blobFields(blob)).Debug("Failed to verify blob signature")
return pubsub.ValidationReject, err
}
*/
// [IGNORE] The sidecar is the only sidecar with valid signature received for the tuple (sidecar.block_root, sidecar.index).
if s.hasSeenBlobIndex(blob.BlockRoot, blob.Index) {
if s.hasSeenBlobIndex(blob.BlockRootSlice(), blob.Index) {
return pubsub.ValidationIgnore, nil
}
// [REJECT] The sidecar is proposed by the expected proposer_index for the block's slot in the context of the current shuffling (defined by block_parent_root/slot)
parentState, err = transition.ProcessSlotsUsingNextSlotCache(ctx, parentState, parentRoot[:], blob.Slot)
parentState, err = transition.ProcessSlotsUsingNextSlotCache(ctx, parentState, parentRoot[:], blob.Slot())
if err != nil {
return pubsub.ValidationIgnore, err
}
@@ -178,16 +186,17 @@ func (s *Service) validateBlobPostSeenParent(ctx context.Context, sBlob *eth.Sig
if err != nil {
return pubsub.ValidationIgnore, err
}
if blob.ProposerIndex != idx {
err := fmt.Errorf("expected proposer index %d, got %d", idx, blob.ProposerIndex)
if blob.ProposerIndex() != idx {
err := fmt.Errorf("expected proposer index %d, got %d", idx, blob.ProposerIndex())
log.WithFields(blobFields(blob)).Debug(err)
return pubsub.ValidationReject, err
}
return pubsub.ValidationAccept, nil
}
func verifyBlobSignature(st state.BeaconState, blob *eth.SignedBlobSidecar) error {
currentEpoch := slots.ToEpoch(blob.Message.Slot)
/*
func verifyBlobSignature(st state.BeaconState, blob blocks.ROBlob) error {
currentEpoch := slots.ToEpoch(blob.Slot())
fork, err := forks.Fork(currentEpoch)
if err != nil {
return err
@@ -196,7 +205,7 @@ func verifyBlobSignature(st state.BeaconState, blob *eth.SignedBlobSidecar) erro
if err != nil {
return err
}
proposer, err := st.ValidatorAtIndex(blob.Message.ProposerIndex)
proposer, err := st.ValidatorAtIndex(blob.ProposerIndex())
if err != nil {
return err
}
@@ -204,7 +213,7 @@ func verifyBlobSignature(st state.BeaconState, blob *eth.SignedBlobSidecar) erro
if err != nil {
return err
}
sig, err := bls.SignatureFromBytes(blob.Signature)
sig, err := bls.SignatureFromBytes(blob.Signature())
if err != nil {
return err
}
@@ -218,6 +227,7 @@ func verifyBlobSignature(st state.BeaconState, blob *eth.SignedBlobSidecar) erro
return nil
}
*/
// Returns true if the blob with the same root and index has been seen before.
func (s *Service) hasSeenBlobIndex(root []byte, index uint64) bool {
@@ -236,11 +246,11 @@ func (s *Service) setSeenBlobIndex(root []byte, index uint64) {
s.seenBlobCache.Add(string(b), true)
}
func blobFields(b *eth.DeprecatedBlobSidecar) logrus.Fields {
func blobFields(b blocks.ROBlob) logrus.Fields {
return logrus.Fields{
"slot": b.Slot,
"proposerIndex": b.ProposerIndex,
"blockRoot": fmt.Sprintf("%#x", b.BlockRoot),
"slot": b.Slot(),
"proposerIndex": b.ProposerIndex(),
"blockRoot": fmt.Sprintf("%#x", b.BlockRoot()),
"kzgCommitment": fmt.Sprintf("%#x", b.KzgCommitment),
"index": b.Index,
}

View File

@@ -10,7 +10,6 @@ import (
pubsub "github.com/libp2p/go-libp2p-pubsub"
pb "github.com/libp2p/go-libp2p-pubsub/pb"
mock "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/signing"
dbtest "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/testing"
doublylinkedtree "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice/doubly-linked-tree"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/p2p"
@@ -22,8 +21,6 @@ import (
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/config/params"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/crypto/bls"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/testing/require"
"github.com/prysmaticlabs/prysm/v4/testing/util"
@@ -88,8 +85,9 @@ func TestValidateBlob_InvalidIndex(t *testing.T) {
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
msg := util.NewBlobsidecar()
msg.Message.Index = fieldparams.MaxBlobsPerBlock
_, scs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, chainService.CurrentSlot()+1, 1)
msg := scs[0].BlobSidecar
msg.Index = fieldparams.MaxBlobsPerBlock
buf := new(bytes.Buffer)
_, err := p.Encoding().EncodeGossip(buf, msg)
require.NoError(t, err)
@@ -113,7 +111,8 @@ func TestValidateBlob_InvalidTopicIndex(t *testing.T) {
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
msg := util.NewBlobsidecar()
_, scs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, chainService.CurrentSlot()+1, 1)
msg := scs[0].BlobSidecar
buf := new(bytes.Buffer)
_, err := p.Encoding().EncodeGossip(buf, msg)
require.NoError(t, err)
@@ -141,8 +140,8 @@ func TestValidateBlob_OlderThanFinalizedEpoch(t *testing.T) {
chain: chainService,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
b := util.NewBlobsidecar()
b.Message.Slot = chainService.CurrentSlot() + 1
_, scs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, chainService.CurrentSlot()+1, 1)
b := scs[0].BlobSidecar
buf := new(bytes.Buffer)
_, err := p.Encoding().EncodeGossip(buf, b)
require.NoError(t, err)
@@ -172,18 +171,17 @@ func TestValidateBlob_HigherThanParentSlot(t *testing.T) {
chain: chainService,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
b := util.NewBlobsidecar()
b.Message.Slot = chainService.CurrentSlot() + 1
chainService.BlockSlot = chainService.CurrentSlot() + 1
bb := util.NewBeaconBlock()
bb.Block.Slot = b.Message.Slot
bb.Block.Slot = chainService.CurrentSlot() + 1
signedBb, err := blocks.NewSignedBeaconBlock(bb)
require.NoError(t, err)
require.NoError(t, db.SaveBlock(ctx, signedBb))
r, err := signedBb.Block().HashTreeRoot()
require.NoError(t, err)
b.Message.BlockParentRoot = r[:]
_, blobs := generateTestBlockWithSidecars(t, r, bb.Block.Slot, 1)
b := blobs[0].BlobSidecar
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, b)
@@ -202,52 +200,6 @@ func TestValidateBlob_HigherThanParentSlot(t *testing.T) {
require.Equal(t, result, pubsub.ValidationReject)
}
func TestValidateBlob_InvalidProposerSignature(t *testing.T) {
db := dbtest.SetupDB(t)
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
chainService := &mock.ChainService{Genesis: time.Now(), FinalizedCheckPoint: &eth.Checkpoint{}, DB: db}
stateGen := stategen.New(db, doublylinkedtree.New())
s := &Service{
cfg: &config{
p2p: p,
initialSync: &mockSync.Sync{},
chain: chainService,
stateGen: stateGen,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
b := util.NewBlobsidecar()
b.Message.Slot = chainService.CurrentSlot() + 1
sk, err := bls.SecretKeyFromBytes(bytesutil.PadTo([]byte("sk"), 32))
require.NoError(t, err)
b.Signature = sk.Sign([]byte("data")).Marshal()
bb := util.NewBeaconBlock()
signedBb, err := blocks.NewSignedBeaconBlock(bb)
require.NoError(t, err)
require.NoError(t, db.SaveBlock(ctx, signedBb))
r, err := signedBb.Block().HashTreeRoot()
require.NoError(t, err)
b.Message.BlockParentRoot = r[:]
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, b)
require.NoError(t, err)
topic := p2p.GossipTypeMapping[reflect.TypeOf(b)]
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestAndIndexToTopic(topic, digest, 0)
result, err := s.validateBlob(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
require.ErrorIs(t, signing.ErrSigFailedToVerify, err)
require.Equal(t, result, pubsub.ValidationReject)
}
func TestValidateBlob_AlreadySeenInCache(t *testing.T) {
db := dbtest.SetupDB(t)
ctx := context.Background()
@@ -263,21 +215,32 @@ func TestValidateBlob_AlreadySeenInCache(t *testing.T) {
stateGen: stateGen,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
b := util.NewBlobsidecar()
b.Message.Slot = chainService.CurrentSlot() + 1
beaconState, privKeys := util.DeterministicGenesisState(t, 100)
beaconState, _ := util.DeterministicGenesisState(t, 100)
parent := util.NewBeaconBlock()
parentBb, err := blocks.NewSignedBeaconBlock(parent)
require.NoError(t, err)
parentRoot, err := parentBb.Block().HashTreeRoot()
require.NoError(t, err)
bb := util.NewBeaconBlock()
bb.Block.Slot = 1
bb.Block.ParentRoot = parentRoot[:]
bb.Block.ProposerIndex = 19026
signedBb, err := blocks.NewSignedBeaconBlock(bb)
require.NoError(t, err)
require.NoError(t, db.SaveBlock(ctx, parentBb))
require.NoError(t, db.SaveBlock(ctx, signedBb))
r, err := signedBb.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, db.SaveState(ctx, beaconState, r))
b.Message.BlockParentRoot = r[:]
b.Signature, err = signing.ComputeDomainAndSign(beaconState, 0, b.Message, params.BeaconConfig().DomainBlobSidecar, privKeys[0])
//_, scs := util.GenerateTestDenebBlockWithSidecar(t, r, chainService.CurrentSlot()+1, 1)
header, err := signedBb.Header()
require.NoError(t, err)
sc := util.GenerateTestDenebBlobSidecar(t, r, header, 0, make([]byte, 48))
b := sc.BlobSidecar
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, b)
@@ -288,7 +251,7 @@ func TestValidateBlob_AlreadySeenInCache(t *testing.T) {
require.NoError(t, err)
topic = s.addDigestAndIndexToTopic(topic, digest, 0)
s.setSeenBlobIndex(bytesutil.PadTo([]byte{}, 32), 0)
s.setSeenBlobIndex(sc.BlockRootSlice(), 0)
result, err := s.validateBlob(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
@@ -313,9 +276,7 @@ func TestValidateBlob_IncorrectProposerIndex(t *testing.T) {
stateGen: stateGen,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
b := util.NewBlobsidecar()
b.Message.Slot = chainService.CurrentSlot() + 1
beaconState, privKeys := util.DeterministicGenesisState(t, 100)
beaconState, _ := util.DeterministicGenesisState(t, 100)
bb := util.NewBeaconBlock()
signedBb, err := blocks.NewSignedBeaconBlock(bb)
@@ -325,9 +286,8 @@ func TestValidateBlob_IncorrectProposerIndex(t *testing.T) {
require.NoError(t, err)
require.NoError(t, db.SaveState(ctx, beaconState, r))
b.Message.BlockParentRoot = r[:]
b.Signature, err = signing.ComputeDomainAndSign(beaconState, 0, b.Message, params.BeaconConfig().DomainBlobSidecar, privKeys[0])
require.NoError(t, err)
_, scs := generateTestBlockWithSidecars(t, r, chainService.CurrentSlot()+1, 1)
b := scs[0].BlobSidecar
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, b)
@@ -354,6 +314,7 @@ func TestValidateBlob_EverythingPasses(t *testing.T) {
chainService := &mock.ChainService{Genesis: time.Now(), FinalizedCheckPoint: &eth.Checkpoint{}, DB: db}
stateGen := stategen.New(db, doublylinkedtree.New())
s := &Service{
badBlockCache: lruwrpr.New(10),
seenBlobCache: lruwrpr.New(10),
cfg: &config{
p2p: p,
@@ -362,22 +323,26 @@ func TestValidateBlob_EverythingPasses(t *testing.T) {
stateGen: stateGen,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
b := util.NewBlobsidecar()
b.Message.Slot = chainService.CurrentSlot() + 1
beaconState, privKeys := util.DeterministicGenesisState(t, 100)
beaconState, _ := util.DeterministicGenesisState(t, 100)
bb := util.NewBeaconBlock()
signedBb, err := blocks.NewSignedBeaconBlock(bb)
parent := util.NewBeaconBlock()
parent.Block.Slot = 1
signedParent, err := blocks.NewSignedBeaconBlock(parent)
require.NoError(t, err)
require.NoError(t, db.SaveBlock(ctx, signedBb))
r, err := signedBb.Block().HashTreeRoot()
require.NoError(t, db.SaveBlock(ctx, signedParent))
parentRoot, err := signedParent.Block().HashTreeRoot()
require.NoError(t, err)
require.NoError(t, db.SaveState(ctx, beaconState, r))
require.NoError(t, db.SaveState(ctx, beaconState, parentRoot))
b.Message.BlockParentRoot = r[:]
b.Message.ProposerIndex = 21
b.Signature, err = signing.ComputeDomainAndSign(beaconState, 0, b.Message, params.BeaconConfig().DomainBlobSidecar, privKeys[21])
child := util.NewBeaconBlock()
child.Block.Slot = 2
child.Block.ParentRoot = parentRoot[:]
child.Block.ProposerIndex = 96
signedChild, err := blocks.NewSignedBeaconBlock(child)
require.NoError(t, err)
childRoot, err := signedChild.Block().HashTreeRoot()
require.NoError(t, err)
b := generateTestSidecar(t, childRoot, signedChild, 0, make([]byte, 48)).BlobSidecar
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, b)

View File

@@ -9,7 +9,6 @@ go_library(
"//config/fieldparams:go_default_library",
"//consensus-types/blocks:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
@@ -20,10 +19,8 @@ go_test(
srcs = ["blob_test.go"],
embed = [":go_default_library"],
deps = [
"//config/fieldparams:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//encoding/bytesutil:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
],

View File

@@ -2,70 +2,42 @@ package verify
import (
"github.com/pkg/errors"
field_params "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/runtime/version"
)
var (
errBlobVerification = errors.New("unable to verify blobs")
ErrMismatchedBlobBlockRoot = errors.Wrap(errBlobVerification, "BlockRoot in BlobSidecar does not match the expected root")
ErrMismatchedBlobBlockSlot = errors.Wrap(errBlobVerification, "BlockSlot in BlobSidecar does not match the expected slot")
ErrIncorrectBlobIndex = errors.New("incorrect blob index")
ErrBlobBlockMisaligned = errors.Wrap(errBlobVerification, "root of block header in blob sidecar does not match block root")
ErrMismatchedBlobCommitments = errors.Wrap(errBlobVerification, "commitments at given slot, root and index do not match")
ErrMismatchedProposerIndex = errors.Wrap(errBlobVerification, "proposer index does not match")
ErrIncorrectBlobIndex = errors.Wrap(errBlobVerification, "incorrect blob index")
)
// BlobAlignsWithBlock verifies if the blob aligns with the block.
func BlobAlignsWithBlock(blob *ethpb.DeprecatedBlobSidecar, block blocks.ROBlock) error {
func BlobAlignsWithBlock(blob blocks.ROBlob, block blocks.ROBlock) error {
if block.Version() < version.Deneb {
return nil
}
if blob.Index >= fieldparams.MaxBlobsPerBlock {
return errors.Wrapf(ErrIncorrectBlobIndex, "index %d exceeds MAX_BLOBS_PER_BLOCK %d", blob.Index, fieldparams.MaxBlobsPerBlock)
}
if blob.BlockRoot() != block.Root() {
return ErrBlobBlockMisaligned
}
// Verify commitment byte values match
// TODO: verify commitment inclusion proof - actually replace this with a better rpc blob verification stack altogether.
commits, err := block.Block().Body().BlobKzgCommitments()
if err != nil {
return err
}
if len(commits) == 0 {
return nil
}
if blob.Index >= field_params.MaxBlobsPerBlock {
return errors.Wrapf(ErrIncorrectBlobIndex, "blob index %d >= max blobs per block %d", blob.Index, field_params.MaxBlobsPerBlock)
}
// Verify slot
blobSlot := blob.Slot
blockSlot := block.Block().Slot()
if blobSlot != blockSlot {
return errors.Wrapf(ErrMismatchedBlobBlockSlot, "slot %d != BlobSidecar.Slot %d", blockSlot, blobSlot)
}
// Verify block and parent roots
blockRoot := bytesutil.ToBytes32(blob.BlockRoot)
if blockRoot != block.Root() {
return errors.Wrapf(ErrMismatchedBlobBlockRoot, "block root %#x != BlobSidecar.BlockRoot %#x at slot %d", block.Root(), blockRoot, blobSlot)
}
blockParentRoot := bytesutil.ToBytes32(blob.BlockParentRoot)
if blockParentRoot != block.Block().ParentRoot() {
return errors.Wrapf(ErrMismatchedBlobBlockRoot, "block parent root %#x != BlobSidecar.BlockParentRoot %#x at slot %d", block.Block().ParentRoot(), blockParentRoot, blobSlot)
}
// Verify proposer index
if blob.ProposerIndex != block.Block().ProposerIndex() {
return errors.Wrapf(ErrMismatchedProposerIndex, "proposer index %d != BlobSidecar.ProposerIndex %d at slot %d", block.Block().ProposerIndex(), blob.ProposerIndex, blobSlot)
}
// Verify commitment
blockCommitment := bytesutil.ToBytes48(commits[blob.Index])
blobCommitment := bytesutil.ToBytes48(blob.KzgCommitment)
if blobCommitment != blockCommitment {
return errors.Wrapf(ErrMismatchedBlobCommitments, "commitment %#x != block commitment %#x, at index %d for block root %#x at slot %d ", blobCommitment, blockCommitment, blob.Index, blockRoot, blobSlot)
return errors.Wrapf(ErrMismatchedBlobCommitments, "commitment %#x != block commitment %#x, at index %d for block root %#x at slot %d ", blobCommitment, blockCommitment, blob.Index, block.Root(), blob.Slot())
}
return nil
}

View File

@@ -1,142 +1,75 @@
package verify
import (
"fmt"
"testing"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v4/testing/require"
"github.com/prysmaticlabs/prysm/v4/testing/util"
)
func TestBlobAlignsWithBlock(t *testing.T) {
tests := []struct {
name string
block interfaces.ReadOnlySignedBeaconBlock
blob *ethpb.DeprecatedBlobSidecar
expectedErr string
name string
blockAndBlob func(t *testing.T) (blocks.ROBlock, []blocks.ROBlob)
err error
}{
{
name: "Block version less than Deneb",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlock()
sb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{},
name: "happy path",
blockAndBlob: func(t *testing.T) (blocks.ROBlock, []blocks.ROBlob) {
return util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 0, 1)
},
},
{
name: "No commitments in block",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
sb, err := blocks.NewSignedBeaconBlock(b)
name: "mismatched roots",
blockAndBlob: func(t *testing.T) (blocks.ROBlock, []blocks.ROBlob) {
blk, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 0, 1)
tweaked := blobs[0].BlobSidecar
tweaked.SignedBlockHeader.Header.Slot = tweaked.SignedBlockHeader.Header.Slot + 1
tampered, err := blocks.NewROBlob(tweaked)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{},
return blk, []blocks.ROBlob{tampered}
},
err: ErrBlobBlockMisaligned,
},
{
name: "Blob index exceeds max blobs per block",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Body.BlobKzgCommitments = make([][]byte, fieldparams.MaxBlobsPerBlock+1)
sb, err := blocks.NewSignedBeaconBlock(b)
name: "mismatched roots - fake",
blockAndBlob: func(t *testing.T) (blocks.ROBlock, []blocks.ROBlob) {
blk, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 0, 1)
copied := blobs[0].BlobSidecar
// exact same header, mess with the root
fake, err := blocks.NewROBlobWithRoot(copied, bytesutil.ToBytes32([]byte("derp")))
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{Index: fieldparams.MaxBlobsPerBlock},
expectedErr: "blob index 6 >= max blobs per block 6: incorrect blob index",
return blk, []blocks.ROBlob{fake}
},
err: ErrBlobBlockMisaligned,
},
{
name: "Blob slot does not match block slot",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Slot = 2
b.Block.Body.BlobKzgCommitments = make([][]byte, 1)
sb, err := blocks.NewSignedBeaconBlock(b)
name: "before deneb",
blockAndBlob: func(t *testing.T) (blocks.ROBlock, []blocks.ROBlob) {
cb := util.NewBeaconBlockCapella()
blk, err := blocks.NewSignedBeaconBlock(cb)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{Slot: 1},
expectedErr: "slot 2 != BlobSidecar.Slot 1: BlockSlot in BlobSidecar does not match the expected slot",
},
{
name: "Blob block root does not match block root",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Body.BlobKzgCommitments = make([][]byte, 1)
sb, err := blocks.NewSignedBeaconBlock(b)
rob, err := blocks.NewROBlock(blk)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{BlockRoot: []byte{1}},
expectedErr: "block root 0x0200000000000000000000000000000000000000000000000000000000000000 != " +
"BlobSidecar.BlockRoot 0x0100000000000000000000000000000000000000000000000000000000000000 at slot 0",
},
{
name: "Blob parent root does not match block parent root",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Body.BlobKzgCommitments = make([][]byte, 1)
sb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{BlockRoot: []byte{2}, BlockParentRoot: []byte{1}},
expectedErr: "block parent root 0x0000000000000000000000000000000000000000000000000000000000000000 != " +
"BlobSidecar.BlockParentRoot 0x0100000000000000000000000000000000000000000000000000000000000000 at slot 0",
},
{
name: "Blob proposer index does not match block proposer index",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Body.BlobKzgCommitments = make([][]byte, 1)
sb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{BlockRoot: []byte{2}, ProposerIndex: 1},
expectedErr: "proposer index 0 != BlobSidecar.ProposerIndex 1 at slot ",
},
{
name: "Blob commitment does not match block commitment",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Body.BlobKzgCommitments = make([][]byte, 1)
sb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{BlockRoot: []byte{2}, KzgCommitment: []byte{1}},
expectedErr: "commitment 0x010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 != " +
"block commitment 0x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",
},
{
name: "All fields are correctly aligned",
block: func() interfaces.ReadOnlySignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Body.BlobKzgCommitments = make([][]byte, 1)
sb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return sb
}(),
blob: &ethpb.DeprecatedBlobSidecar{BlockRoot: []byte{2}},
return rob, []blocks.ROBlob{blocks.ROBlob{}}
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
block, err := blocks.NewROBlockWithRoot(tt.block, [32]byte{2})
require.NoError(t, err)
err = BlobAlignsWithBlock(tt.blob, block)
if tt.expectedErr == "" {
require.NoError(t, err)
} else {
require.StringContains(t, tt.expectedErr, err.Error())
}
})
block, blobs := tt.blockAndBlob(t)
for i := range blobs {
t.Run(tt.name+fmt.Sprintf(" blob %d", i), func(t *testing.T) {
err := BlobAlignsWithBlock(blobs[i], block)
if tt.err == nil {
require.NoError(t, err)
} else {
require.ErrorIs(t, err, tt.err)
}
})
}
}
}

View File

@@ -0,0 +1,20 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = [
"blob.go",
"error.go",
"fake.go",
"initializer.go",
"result.go",
],
importpath = "github.com/prysmaticlabs/prysm/v4/beacon-chain/verification",
visibility = ["//visibility:public"],
deps = [
"//config/fieldparams:go_default_library",
"//consensus-types/blocks:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)

View File

@@ -0,0 +1,93 @@
package verification
import (
"fmt"
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/sirupsen/logrus"
log "github.com/sirupsen/logrus"
)
const (
RequireBlobIndexInBounds Requirement = iota
RequireSlotBelowMaxDisparity
RequireSlotBelowFinalized
RequireValidProposerSignature
RequireSidecarParentSeen
RequireSidecarParentValid
RequireSidecarParentSlotLower
RequireSidecarDescendsFromFinalized
RequireSidecarInclusionProven
RequireSidecarBlobCommitmentProven
RequireSidecarFirstSeen
RequireSidecarProposerExpected
)
// GossipSidecarRequirements defines the set of requirements that BlobSidecars received on gossip
// must satisfy in order to upgrade an ROBlob to a VerifiedROBlob.
var GossipSidecarRequirements = []Requirement{
RequireBlobIndexInBounds,
RequireSlotBelowMaxDisparity,
RequireSlotBelowFinalized,
RequireValidProposerSignature,
RequireSidecarParentSeen,
RequireSidecarParentValid,
RequireSidecarParentSlotLower,
RequireSidecarDescendsFromFinalized,
RequireSidecarInclusionProven,
RequireSidecarBlobCommitmentProven,
RequireSidecarFirstSeen,
RequireSidecarProposerExpected,
}
var (
ErrBlobInvalid = errors.New("blob failed verification")
ErrBlobIndexInBounds = errors.Wrap(ErrBlobInvalid, "incorrect blob sidecar index")
)
type BlobVerifier struct {
shared *sharedResources
results *results
blob blocks.ROBlob
}
func (bv *BlobVerifier) recordResult(req Requirement, err *error) {
if err == nil || *err == nil {
bv.results.record(req, nil)
return
}
bv.results.record(req, *err)
}
// BlobIndexInBounds represents the follow spec verification:
// [REJECT] The sidecar's index is consistent with MAX_BLOBS_PER_BLOCK -- i.e. blob_sidecar.index < MAX_BLOBS_PER_BLOCK.
func (bv *BlobVerifier) BlobIndexInBounds() (err error) {
defer bv.recordResult(RequireBlobIndexInBounds, &err)
if bv.blob.Index >= fieldparams.MaxBlobsPerBlock {
bv.logWithFields().Debug("Sidecar index > MAX_BLOBS_PER_BLOCK")
return ErrBlobIndexInBounds
}
return nil
}
// VerifiedROBlob "upgrades" the wrapped ROBlob to a VerifiedROBlob.
// If any of the verifications ran against the blob failed, or some required verifications
// were not run, an error will be returned.
func (bv *BlobVerifier) VerifiedROBlob() (blocks.VerifiedROBlob, error) {
if bv.results.satisfied() {
return blocks.NewVerifiedROBlob(bv.blob), nil
}
return blocks.VerifiedROBlob{}, bv.results.errors(ErrBlobInvalid)
}
func (bv *BlobVerifier) logWithFields() *logrus.Entry {
return log.WithFields(logrus.Fields{
"slot": bv.blob.Slot(),
"proposerIndex": bv.blob.ProposerIndex(),
"blockRoot": fmt.Sprintf("%#x", bv.blob.BlockRoot()),
"kzgCommitment": fmt.Sprintf("%#x", bv.blob.KzgCommitment),
"index": bv.blob.Index,
})
}

View File

@@ -0,0 +1,24 @@
package verification
import "github.com/pkg/errors"
// ErrMissingVerification indicates that the given verification function was never performed on the value.
var ErrMissingVerification = errors.New("verification was not performed for requirement")
// VerificationMultiError is a custom error that can be used to access individual verification failures.
type VerificationMultiError struct {
r *results
err error
}
func (ve VerificationMultiError) Error() string {
return ve.err.Error()
}
func (ve VerificationMultiError) Failures() map[Requirement]error {
return ve.r.failures()
}
func newVerificationMultiError(r *results, err error) VerificationMultiError {
return VerificationMultiError{r: r, err: err}
}

View File

@@ -0,0 +1,19 @@
package verification
import "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
// BlobSidecarNoop is a FAKE verification function that simply launders a ROBlob->VerifiedROBlob.
// TODO: find all code that uses this method and replace it with full verification.
func BlobSidecarNoop(b blocks.ROBlob) (blocks.VerifiedROBlob, error) {
return blocks.NewVerifiedROBlob(b), nil
}
// BlobSidecarSliceNoop is a FAKE verification function that simply launders a ROBlob->VerifiedROBlob.
// TODO: find all code that uses this method and replace it with full verification.
func BlobSidecarSliceNoop(b []blocks.ROBlob) ([]blocks.VerifiedROBlob, error) {
vbs := make([]blocks.VerifiedROBlob, len(b))
for i := range b {
vbs[i] = blocks.NewVerifiedROBlob(b[i])
}
return vbs, nil
}

View File

@@ -0,0 +1,19 @@
package verification
import "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
// sharedResources provides access to resources that are required by different verification types.
// for example, sidecar verifcation and block verification share the block signature verification cache.
type sharedResources struct{}
// Initializer is used to create different Verifiers.
// Verifiers require access to stateful data structures, like caches,
// and it is Initializer's job to provides access to those.
type Initializer struct {
shared *sharedResources
}
// NewBlobVerifier creates a BlobVerifier for a single blob, with the given set of requirements.
func (ini *Initializer) NewBlobVerifier(b blocks.ROBlob, reqs ...Requirement) *BlobVerifier {
return &BlobVerifier{shared: ini.shared, results: newResults(reqs...)}
}

View File

@@ -0,0 +1,55 @@
package verification
// represents a verification requirement.
// Requirement represents a validation check that needs to pass in order for a Verified form a consensus type to be issued.
type Requirement int
// results collects positive verification results.
// This bitmap can be used to test which verifications have been successfully completed in order to
// decide whether it is safe to issue a "Verified" type variant.
type results struct {
done map[Requirement]error
reqs []Requirement
}
func newResults(reqs ...Requirement) *results {
return &results{done: make(map[Requirement]error, len(reqs)), reqs: reqs}
}
func (r *results) record(req Requirement, err error) {
r.done[req] = err
}
// satisfied returns true if there is a nil error result for every Requirement.
func (r *results) satisfied() bool {
if len(r.done) != len(r.reqs) {
return false
}
for i := range r.reqs {
err, ok := r.done[r.reqs[i]]
if !ok || err != nil {
return false
}
}
return true
}
func (r *results) errors(err error) error {
return newVerificationMultiError(r, err)
}
func (r *results) failures() map[Requirement]error {
fail := make(map[Requirement]error, len(r.done))
for i := range r.reqs {
req := r.reqs[i]
err, ok := r.done[req]
if !ok {
fail[req] = ErrMissingVerification
continue
}
if err != nil {
fail[req] = err
}
}
return fail
}

View File

@@ -24,6 +24,7 @@ go_library(
"//cmd/beacon-chain/execution:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//cmd/beacon-chain/jwt:go_default_library",
"//cmd/beacon-chain/storage:go_default_library",
"//cmd/beacon-chain/sync/checkpoint:go_default_library",
"//cmd/beacon-chain/sync/genesis:go_default_library",
"//config/features:go_default_library",

View File

@@ -18,6 +18,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/flags"
jwtcommands "github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/jwt"
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/storage"
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/checkpoint"
"github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/sync/genesis"
"github.com/prysmaticlabs/prysm/v4/config/features"
@@ -278,6 +279,7 @@ func startNode(ctx *cli.Context) error {
optFuncs := []func(*cli.Context) (node.Option, error){
genesis.BeaconNodeOptions,
checkpoint.BeaconNodeOptions,
storage.BeaconNodeOptions,
}
for _, of := range optFuncs {
ofo, err := of(ctx)

View File

@@ -0,0 +1,13 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = ["options.go"],
importpath = "github.com/prysmaticlabs/prysm/v4/cmd/beacon-chain/storage",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/node:go_default_library",
"//cmd:go_default_library",
"@com_github_urfave_cli_v2//:go_default_library",
],
)

View File

@@ -0,0 +1,33 @@
package storage
import (
"path"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/node"
"github.com/prysmaticlabs/prysm/v4/cmd"
"github.com/urfave/cli/v2"
)
var (
// BlobStoragePath defines a flag to start the beacon chain from a give genesis state file.
BlobStoragePath = &cli.PathFlag{
Name: "blob-path",
Usage: "Location for blob storage. Default location will be a 'blobs' directory next to the beacon db.",
}
)
// BeaconNodeOptions sets configuration values on the node.BeaconNode value at node startup.
// Note: we can't get the right context from cli.Context, because the beacon node setup code uses this context to
// create a cancellable context. If we switch to using App.RunContext, we can set up this cancellation in the cmd
// package instead, and allow the functional options to tap into context cancellation.
func BeaconNodeOptions(c *cli.Context) (node.Option, error) {
blobsPath := c.Path(BlobStoragePath.Name)
if blobsPath == "" {
// append a "blobs" subdir to the end of the data dir path
blobsPath = path.Join(path.Clean(c.String(c.Path(cmd.DataDirFlag.Name))), "blobs")
}
return func(node *node.BeaconNode) (err error) {
node.BlobStoragePath = blobsPath
return nil
}, nil
}

View File

@@ -62,3 +62,31 @@ func (b *ROBlob) BodyRoot() [32]byte {
func (b *ROBlob) ProposerIndex() primitives.ValidatorIndex {
return b.SignedBlockHeader.Header.ProposerIndex
}
// BlockRootSlice returns the block root as a byte slice. This is often more conveninent/concise
// than setting a tmp var to BlockRoot(), just so that it can be sliced.
func (b *ROBlob) BlockRootSlice() []byte {
return b.root[:]
}
// ROBlobSlice is a custom type for a []ROBlob, allowing methods to be defined that act on a slice of ROBlob.
type ROBlobSlice []ROBlob
// Protos is a helper to make a more concise conversion from []ROBlob->[]*ethpb.BlobSidecar.
func (s ROBlobSlice) Protos() []*ethpb.BlobSidecar {
pb := make([]*ethpb.BlobSidecar, len(s))
for i := range s {
pb[i] = s[i].BlobSidecar
}
return pb
}
// VerifiedROBlob represents an ROBlob that has undergone full verification (eg block sig, inclusion proof, commitment check).
type VerifiedROBlob struct {
ROBlob
}
// NewVerifiedROBlob "upgrades" an ROBlob to a VerifiedROBlob. This method should only be used by the verification package.
func NewVerifiedROBlob(rob ROBlob) VerifiedROBlob {
return VerifiedROBlob{ROBlob: rob}
}

View File

@@ -5,7 +5,6 @@ import (
"sort"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
eth "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
)
// ROBlock is a value that embeds a ReadOnlySignedBeaconBlock along with its block root ([32]byte).
@@ -77,7 +76,7 @@ func (s ROBlockSlice) Len() int {
type BlockWithVerifiedBlobs struct {
Block ROBlock
Blobs []*eth.DeprecatedBlobSidecar
Blobs []ROBlob
}
type BlockWithVerifiedBlobsSlice []BlockWithVerifiedBlobs

1
go.mod
View File

@@ -67,6 +67,7 @@ require (
github.com/rs/cors v1.7.0
github.com/schollz/progressbar/v3 v3.3.4
github.com/sirupsen/logrus v1.9.0
github.com/spf13/afero v1.10.0
github.com/status-im/keycard-go v0.2.0
github.com/stretchr/testify v1.8.4
github.com/supranational/blst v0.3.11

18
go.sum
View File

@@ -7,6 +7,7 @@ cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSR
cloud.google.com/go v0.43.0/go.mod h1:BOSR3VbTLkk6FDC/TcffxP4NF/FFBGA5ku+jvKOP7pg=
cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU=
cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY=
cloud.google.com/go v0.44.3/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY=
cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc=
cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0=
cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To=
@@ -20,6 +21,7 @@ cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOY
cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY=
cloud.google.com/go v0.72.0/go.mod h1:M+5Vjvlc2wnp6tjzE102Dw08nGShTscUx2nZMufOKPI=
cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmWk=
cloud.google.com/go v0.75.0/go.mod h1:VGuuCn7PG0dwsd5XPVm2Mm3wlh3EL55/79EKB6hlPTY=
cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg=
cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8=
cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0=
@@ -42,6 +44,7 @@ cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0Zeo
cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk=
cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs=
cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0=
cloud.google.com/go/storage v1.14.0/go.mod h1:GrKmX003DSIwi9o29oFT7YDnHYwZoctc3fOKtUw0Xmo=
collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE=
contrib.go.opencensus.io/exporter/jaeger v0.2.1 h1:yGBYzYMewVL0yO9qqJv3Z5+IRhPdU7e9o/2oKpX4YvI=
contrib.go.opencensus.io/exporter/jaeger v0.2.1/go.mod h1:Y8IsLgdxqh1QxYxPC5IgXVmBaeLUeQFfBeBi9PbeZd0=
@@ -558,6 +561,7 @@ github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hf
github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM=
github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/pprof v0.0.0-20201218002935-b9804c9f04c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
@@ -577,6 +581,7 @@ github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE0
github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
github.com/googleapis/gnostic v0.4.1/go.mod h1:LRhVm6pbyptWbWbuZ38d1eyptfvIytN3ir6b65WBswg=
github.com/googleapis/google-cloud-go-testing v0.0.0-20200911160855-bcd43fbb19e8/go.mod h1:dvDLG8qkwmyD9a/MJJN3XJcT3xFxOKAvTZGvuZmac9g=
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
github.com/gordonklaus/ineffassign v0.0.0-20200309095847-7953dde2c7bf/go.mod h1:cuNKsD1zp2v6XfE/orVX2QE1LC+i254ceGcVeDT3pTU=
github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg=
@@ -764,6 +769,7 @@ github.com/koron/go-ssdp v0.0.0-20191105050749-2e1c40ed0b5d/go.mod h1:5Ky9EC2xfo
github.com/koron/go-ssdp v0.0.4 h1:1IDwrghSKYM7yLf7XCzbByg2sJ/JcNOZRXS2jczTwz0=
github.com/koron/go-ssdp v0.0.4/go.mod h1:oDXq+E5IL5q0U8uSBcoAXzTzInwy5lEgC91HoKtbmZk=
github.com/korovkin/limiter v0.0.0-20221015170604-22eb1ceceddc/go.mod h1:mM0lzivCxB6c8msz/LOP9lJgZxy92GXwGcNG1A7UZEE=
github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg=
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
@@ -1054,6 +1060,7 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
github.com/pkg/profile v1.5.0/go.mod h1:qBsxPvzyUincmltOk6iyRVxHYg4adc0OFOv72ZdLa18=
github.com/pkg/sftp v1.13.1/go.mod h1:3HaPG6Dq1ILlpPZRO0HVMrsydcdLt6HRDccSgb87qRg=
github.com/pkg/term v0.0.0-20180730021639-bffc007b7fd5/go.mod h1:eCbImbZ95eXtAUIbLAuAVnBnwf83mjf6QIVH8SHYwqQ=
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
@@ -1212,6 +1219,8 @@ github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2
github.com/spf13/afero v0.0.0-20170901052352-ee1bd8ee15a1/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ=
github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ=
github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk=
github.com/spf13/afero v1.10.0 h1:EaGW2JJh15aKOejeuJ+wpFSHnbd7GE6Wvp3TsNhb6LY=
github.com/spf13/afero v1.10.0/go.mod h1:UBogFpq8E9Hx+xc5CNTTEpTnuHVmXDwZcZcE1eb/UhQ=
github.com/spf13/cast v1.1.0/go.mod h1:r2rcYCSwa1IExKTDiTfzaxqT2FNHs8hODu4LnUfgKEg=
github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE=
github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
@@ -1424,7 +1433,9 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPh
golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I=
golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4=
golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4=
golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4=
golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4=
golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc=
golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4=
@@ -1526,6 +1537,7 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY
golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210220033124-5f55cee0dc0d/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
@@ -1536,6 +1548,7 @@ golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qx
golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.0.0-20211008194852-3b03d305991f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk=
golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
@@ -1651,6 +1664,7 @@ golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210225134936-a50acf3fe073/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -1662,6 +1676,7 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210420205809-ac73e9fd8988/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210423185535-09eb48e85fd7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
@@ -1787,6 +1802,7 @@ golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4f
golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0=
golang.org/x/tools v0.1.1-0.20210205202024-ef80cdb6ec6d/go.mod h1:9bzcO0MWcOuT0tm1iBGzDVPshzfwoVvREIui8C+MHqU=
golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
@@ -1892,8 +1908,10 @@ google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6D
google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210108203827-ffc7fda8c3d7/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210207032614-bba0dbe2a9ea/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210222152913-aa3ee6e6a81c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210226172003-ab064af71705/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=

View File

@@ -8,5 +8,6 @@ import (
)
func TestMainnet_Deneb_Forkchoice(t *testing.T) {
t.Skip("This will fail until we re-integrate proof verification")
forkchoice.Run(t, "mainnet", version.Deneb)
}

View File

@@ -19,6 +19,7 @@ go_library(
"//beacon-chain/cache/depositcache:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/execution:go_default_library",
"//beacon-chain/forkchoice/doubly-linked-tree:go_default_library",
@@ -27,6 +28,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",

View File

@@ -12,6 +12,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/state"
state_native "github.com/prysmaticlabs/prysm/v4/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v4/config/fieldparams"
"github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
@@ -295,7 +296,6 @@ func runBlobStep(t *testing.T,
block := beaconBlock.Block()
root, err := block.HashTreeRoot()
require.NoError(t, err)
parentRoot := block.ParentRoot()
kzgs, err := block.Body().BlobKzgCommitments()
require.NoError(t, err)
@@ -303,6 +303,8 @@ func runBlobStep(t *testing.T,
require.NoError(t, err)
blobsSSZ, err := snappy.Decode(nil /* dst */, blobsFile)
require.NoError(t, err)
sh, err := beaconBlock.Header()
require.NoError(t, err)
for index := uint64(0); index*fieldparams.BlobLength < uint64(len(blobsSSZ)); index++ {
var proof []byte
if index < uint64(len(proofs)) {
@@ -316,19 +318,31 @@ func runBlobStep(t *testing.T,
if uint64(len(kzgs)) < index {
kzg = kzgs[index]
}
if len(kzg) == 0 {
kzg = make([]byte, 48)
}
blob := [fieldparams.BlobLength]byte{}
copy(blob[:], blobsSSZ[index*fieldparams.BlobLength:])
sidecar := &ethpb.DeprecatedBlobSidecar{
BlockRoot: root[:],
Index: index,
Slot: block.Slot(),
BlockParentRoot: parentRoot[:],
ProposerIndex: block.ProposerIndex(),
Blob: blob[:],
KzgCommitment: kzg,
KzgProof: proof,
fakeProof := make([][]byte, 17)
for i := range fakeProof {
fakeProof[i] = make([]byte, 32)
}
require.NoError(t, builder.service.ReceiveBlob(context.Background(), sidecar))
if len(proof) == 0 {
proof = make([]byte, 48)
}
pb := &ethpb.BlobSidecar{
Index: index,
Blob: blob[:],
KzgCommitment: kzg,
KzgProof: proof,
SignedBlockHeader: sh,
CommitmentInclusionProof: fakeProof,
}
ro, err := blocks.NewROBlobWithRoot(pb, root)
require.NoError(t, err)
vsc, err := verification.BlobSidecarNoop(ro)
require.NoError(t, err)
require.NoError(t, builder.service.ReceiveBlob(context.Background(), vsc))
}
}
}

View File

@@ -14,6 +14,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/cache/depositcache"
coreTime "github.com/prysmaticlabs/prysm/v4/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/db/filesystem"
testDB "github.com/prysmaticlabs/prysm/v4/beacon-chain/db/testing"
doublylinkedtree "github.com/prysmaticlabs/prysm/v4/beacon-chain/forkchoice/doubly-linked-tree"
"github.com/prysmaticlabs/prysm/v4/beacon-chain/operations/attestations"
@@ -69,6 +70,7 @@ func startChainService(t testing.TB,
blockchain.WithDepositCache(depositCache),
blockchain.WithProposerIdsCache(cache.NewProposerPayloadIDsCache()),
blockchain.WithClockSynchronizer(startup.NewClockSynchronizer()),
blockchain.WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)),
)
service, err := blockchain.NewService(context.Background(), opts...)
require.NoError(t, err)

View File

@@ -16,7 +16,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/testing/require"
)
func GenerateTestDenebBlockWithSidecar(t *testing.T, parent [32]byte, slot primitives.Slot, nblobs int) (blocks.ROBlock, []*ethpb.DeprecatedBlobSidecar) {
func GenerateTestDenebBlockWithSidecar(t *testing.T, parent [32]byte, slot primitives.Slot, nblobs int) (blocks.ROBlock, []blocks.ROBlob) {
// Start service with 160 as allowed blocks capacity (and almost zero capacity recovery).
stateRoot := bytesutil.PadTo([]byte("stateRoot"), fieldparams.RootLength)
receiptsRoot := bytesutil.PadTo([]byte("receiptsRoot"), fieldparams.RootLength)
@@ -67,22 +67,48 @@ func GenerateTestDenebBlockWithSidecar(t *testing.T, parent [32]byte, slot primi
root, err := block.Block.HashTreeRoot()
require.NoError(t, err)
sidecars := make([]*ethpb.DeprecatedBlobSidecar, len(commitments))
for i, c := range block.Block.Body.BlobKzgCommitments {
sidecars[i] = GenerateTestDenebBlobSidecar(root, block, i, c)
}
sidecars := make([]blocks.ROBlob, len(commitments))
sbb, err := blocks.NewSignedBeaconBlock(block)
require.NoError(t, err)
sh, err := sbb.Header()
require.NoError(t, err)
for i, c := range block.Block.Body.BlobKzgCommitments {
sidecars[i] = GenerateTestDenebBlobSidecar(t, root, sh, i, c)
}
rob, err := blocks.NewROBlock(sbb)
require.NoError(t, err)
return rob, sidecars
}
func GenerateTestDenebBlobSidecar(root [32]byte, block *ethpb.SignedBeaconBlockDeneb, index int, commitment []byte) *ethpb.DeprecatedBlobSidecar {
func GenerateTestDenebBlobSidecar(t *testing.T, root [32]byte, header *ethpb.SignedBeaconBlockHeader, index int, commitment []byte) blocks.ROBlob {
blob := make([]byte, fieldparams.BlobSize)
binary.LittleEndian.PutUint64(blob, uint64(index))
sc := &ethpb.DeprecatedBlobSidecar{
pb := &ethpb.BlobSidecar{
SignedBlockHeader: header,
Index: uint64(index),
Blob: blob,
KzgCommitment: commitment,
KzgProof: commitment,
}
pb.CommitmentInclusionProof = fakeEmptyProof(t, pb)
r, err := blocks.NewROBlobWithRoot(pb, root)
require.NoError(t, err)
return r
}
func fakeEmptyProof(_ *testing.T, _ *ethpb.BlobSidecar) [][]byte {
r := make([][]byte, 17)
for i := range r {
r[i] = make([]byte, 32)
}
return r
}
func GenerateTestDeprecatedBlobSidecar(root [32]byte, block *ethpb.SignedBeaconBlockDeneb, index int, commitment []byte) *ethpb.DeprecatedBlobSidecar {
blob := make([]byte, fieldparams.BlobSize)
binary.LittleEndian.PutUint64(blob, uint64(index))
pb := &ethpb.DeprecatedBlobSidecar{
BlockRoot: root[:],
Index: uint64(index),
Slot: block.Block.Slot,
@@ -92,11 +118,11 @@ func GenerateTestDenebBlobSidecar(root [32]byte, block *ethpb.SignedBeaconBlockD
KzgCommitment: commitment,
KzgProof: commitment,
}
return sc
return pb
}
func ExtendBlocksPlusBlobs(t *testing.T, blks []blocks.ROBlock, size int) ([]blocks.ROBlock, []*ethpb.DeprecatedBlobSidecar) {
blobs := make([]*ethpb.DeprecatedBlobSidecar, 0)
func ExtendBlocksPlusBlobs(t *testing.T, blks []blocks.ROBlock, size int) ([]blocks.ROBlock, []blocks.ROBlob) {
blobs := make([]blocks.ROBlob, 0)
if len(blks) == 0 {
blk, blb := GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 0, 6)
blobs = append(blobs, blb...)