Compare commits

...

35 Commits

Author SHA1 Message Date
Manu NALEPA
7fcd5a5460 PeerDAS: Generate private key to match subnets. 2024-06-11 10:46:44 +02:00
Manu NALEPA
d0f2789e25 CustodyColumnSubnets: Returns a slice instead of a map 2024-06-10 22:02:28 +02:00
Manu NALEPA
fe7cb7e5e2 PeerDAS: Remove unused ComputeExtendedMatrix and RecoverMatrix functions. 2024-06-10 22:02:24 +02:00
Manu NALEPA
0f74569012 peerDAS: Move functions in file. Add documentation. 2024-06-10 14:25:44 +02:00
Manu NALEPA
55e436eceb privKey: Improve logs. 2024-06-10 14:25:31 +02:00
Manu NALEPA
a6496b9408 PeerDAS: Gossip the reconstructed columns (#14079)
* PeerDAS: Broadcast not seen via gossip but reconstructed data columns.

* Address Nishant's comment.
2024-06-06 12:28:21 +02:00
Manu NALEPA
3b62c7bb4a PeerDAS: Only saved custodied columns even after reconstruction. (#14083) 2024-06-06 10:35:12 +02:00
Manu NALEPA
c109f28bdc recoverBlobs: Cover the 0 < blobsCount < fieldparams.MaxBlobsPerBlock case. (#14066)
* `recoverBlobs`: Cover the `0 < blobsCount < fieldparams.MaxBlobsPerBlock` case.

* Fix Nishant's comment.
2024-06-04 17:08:02 +08:00
Manu NALEPA
3f6e9c8420 PeerDAS: Withhold data on purpose. (#14076)
* Introduce hidden flag `data-columns-withhold-count`.

* Address Nishant's comment.
2024-06-04 10:38:59 +02:00
Manu NALEPA
54f2d91ef0 PeerDAS: Implement / use data column feed from database. (#14062)
* Remove some `_` identifiers.

* Blob storage: Implement a notifier system for data columns.

* `dataColumnSidecarByRootRPCHandler`: Remove ugly `time.Sleep(100 * time.Millisecond)`.

* Address Nishant's comment.
2024-06-04 09:52:25 +02:00
Manu NALEPA
b692050079 PeerDAS: Implement reconstruction. (#14036)
* Wrap errors, add logs.

* `missingColumnRequest`: Fix blobs <-> data columns mix.

* `ColumnIndices`: Return `map[uint64]bool` instead of `[fieldparams.NumberOfColumns]bool`.

* `DataColumnSidecars`: `interfaces.SignedBeaconBlock` ==> `interfaces.ReadOnlySignedBeaconBlock`.

We don't need any of the non read-only methods.

* Fix comments.

* `handleUnblidedBlock` ==> `handleUnblindedBlock`.

* `SaveDataColumn`: Move log from debug to trace.

If we attempt to save an already existing data column sidecar,
a debug log was printed.

This case could be quite common now with the data column reconstruction enabled.

* `sampling_data_columns.go` --> `data_columns_sampling.go`.

* Reconstruct data columns.
2024-05-30 10:00:09 +02:00
Nishant Das
ac7e5b6618 Fix Custody Columns (#14021) 2024-05-30 10:00:09 +02:00
Nishant Das
e82a2488ee Disable Evaluators For E2E (#14019)
* Hack E2E

* Fix it For Real

* Gofmt

* Remove
2024-05-30 10:00:09 +02:00
Nishant Das
ce1e6be98d Request Data Columns When Fetching Pending Blocks (#14007)
* Support Data Columns For By Root Requests

* Revert Config Changes

* Fix Panic

* Fix Process Block

* Fix Flags

* Lint

* Support Checkpoint Sync

* Manu's Review

* Add Support For Columns in Remaining Methods

* Unmarshal Uncorrectly
2024-05-30 10:00:09 +02:00
Manu NALEPA
463d45970b Fix CustodyColumns to comply with alpha-2 spectests. (#14008)
* Adding error wrapping

* Fix `CustodyColumnSubnets` tests.
2024-05-30 10:00:09 +02:00
Manu NALEPA
fbb07226f6 Fix beacon chain config. (#14017) 2024-05-30 10:00:09 +02:00
Nishant Das
f10805fe4b Adding Back Generated Objects (#14009) 2024-05-30 10:00:09 +02:00
Nishant Das
812dbc9eb9 Set Custody Count Correctly (#14004)
* Set Custody Count Correctly

* Fix Discovery Count
2024-05-30 10:00:09 +02:00
Manu NALEPA
de7e5deca5 Sample from peers some data columns. (#13980)
* PeerDAS: Implement sampling.

* `TestNewRateLimiter`: Fix with the new number of expected registered topics.
2024-05-30 10:00:09 +02:00
Nishant Das
8b40a6a591 Implement Data Columns By Range Request And Response Methods (#13972)
* Add Data Structure for New Request Type

* Add Data Column By Range Handler

* Add Data Column Request Methods

* Add new validation for columns by range requests

* Fix Build

* Allow Prysm Node To Fetch Data Columns

* Allow Prysm Node To Fetch Data Columns And Sync

* Bug Fixes For Interop

* GoFmt

* Use different var

* Manu's Review
2024-05-30 10:00:09 +02:00
Nishant Das
c49863b8c3 Enable E2E For PeerDAS (#13945)
* Enable E2E And Add Fixes

* Register Same Topic For Data Columns

* Initialize Capacity Of Slice

* Fix Initialization of Data Column Receiver

* Remove Mix In From Merkle Proof

* E2E: Subscribe to all subnets.

* Remove Index Check

* Remaining Bug Fixes to Get It Working

* Change Evaluator to Allow Test to Finish

* Fix Build

* Add Data Column Verification

* Fix LoopVar Bug

* Do Not Allocate Memory

* Update beacon-chain/blockchain/process_block.go

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>

* Update beacon-chain/core/peerdas/helpers.go

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>

* Update beacon-chain/core/peerdas/helpers.go

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>

* Gofmt

* Fix It Again

* Fix Test Setup

* Fix Build

* Fix Trusted Setup panic

* Fix Trusted Setup panic

* Use New Test

---------

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>
2024-05-30 10:00:09 +02:00
Justin Traglia
23b8bb0802 [PeerDAS] Upgrade c-kzg-4844 package (#13967)
* Upgrade c-kzg-4844 package

* Upgrade bazel deps
2024-05-30 10:00:09 +02:00
Manu NALEPA
1aca805036 SendDataColumnSidecarByRoot: Return RODataColumn instead of ROBlob. (#13957)
* `SendDataColumnSidecarByRoot`: Return `RODataColumn` instead of `ROBlob`.

* Make deepsource happier.
2024-05-30 10:00:09 +02:00
Manu NALEPA
559d7eae0f Spectests (#13940)
* Update `consensus_spec_version` to `v1.5.0-alpha.1`.

* `CustodyColumns`: Fix and implement spec tests.

* Make deepsource happy.

* `^uint64(0)` => `math.MaxUint64`.

* Fix `TestLoadConfigFile` test.
2024-05-30 10:00:09 +02:00
Nishant Das
ba74366801 Add DA Check For Data Columns (#13938)
* Add new DA check

* Exit early in the event no commitments exist.

* Gazelle

* Fix Mock Broadcaster

* Fix Test Setup

* Update beacon-chain/blockchain/process_block.go

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>

* Manu's Review

* Fix Build

---------

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>
2024-05-30 10:00:09 +02:00
Manu NALEPA
63bd276c48 Implement peer DAS proposer RPC (#13922)
* Remove capital letter from error messages.

* `[4]byte` => `[fieldparams.VersionLength]byte`.

* Prometheus: Remove extra `committee`.

They are probably due to a bad copy/paste.

Note: The name of the probe itself is remaining,
to ensure backward compatibility.

* Implement Proposer RPC for data columns.

* Fix TestProposer_ProposeBlock_OK test.

* Remove default peerDAS activation.

* `validateDataColumn`: Workaround to return a `VerifiedRODataColumn`
2024-05-30 10:00:09 +02:00
Nishant Das
0cd2786153 Update .bazelrc (#13931) 2024-05-30 10:00:09 +02:00
Manu NALEPA
32b2991dd1 Implement custody_subnet_count ENR field. (#13915)
https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/p2p-interface.md#the-discovery-domain-discv5
2024-05-30 10:00:09 +02:00
Manu NALEPA
5cfa5f3950 Peer das core (#13877)
* Bump `c-kzg-4844` lib to the `das` branch.

* Implement `MerkleProofKZGCommitments`.

* Implement `das-core.md`.

* Use `peerdas.CustodyColumnSubnets` and `peerdas.CustodyColumns`.

* `CustodyColumnSubnets`: Include `i` in the for loop.

* Remove `computeSubscribedColumnSubnet`.

* Remove `peerdas.CustodyColumns` out of the for loop.
2024-05-30 10:00:09 +02:00
Nishant Das
e7672c7801 Add Request And Response RPC Methods For Data Columns (#13909)
* Add RPC Handler

* Add Column Requests

* Update beacon-chain/db/filesystem/blob.go

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>

* Update beacon-chain/p2p/rpc_topic_mappings.go

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>

* Manu's Review

* Manu's Review

* Interface Fixes

* mock manager

---------

Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>
2024-05-30 10:00:09 +02:00
Nishant Das
67c648bcad Add Data Column Gossip Handlers (#13894)
* Add Data Column Subscriber

* Add Data Column Vaidator

* Wire all Handlers In

* Fix Build

* Fix Test

* Fix IP in Test

* Fix IP in Test
2024-05-30 10:00:09 +02:00
Nishant Das
76cf2c3e0e Add Support For Discovery Of Column Subnets (#13883)
* Add Support For Discovery Of Column Subnets

* Lint for SubnetsPerNode

* Manu's Review

* Change to a better name
2024-05-30 10:00:09 +02:00
Nishant Das
b753b11356 add in networking params (#13866) 2024-05-30 10:00:09 +02:00
Nishant Das
acd93daf00 add it (#13865) 2024-05-30 10:00:09 +02:00
Nishant Das
57c9d59996 Add in column sidecars protos (#13862) 2024-05-30 10:00:09 +02:00
120 changed files with 5612 additions and 322 deletions

View File

@@ -22,6 +22,7 @@ coverage --define=coverage_enabled=1
build --workspace_status_command=./hack/workspace_status.sh
build --define blst_disabled=false
build --compilation_mode=opt
run --define blst_disabled=false
build:blst_disabled --define blst_disabled=true

View File

@@ -26,6 +26,7 @@ go_library(
"receive_attestation.go",
"receive_blob.go",
"receive_block.go",
"receive_data_column.go",
"service.go",
"tracked_proposer.go",
"weak_subjectivity_checks.go",
@@ -48,6 +49,7 @@ go_library(
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",
@@ -67,6 +69,7 @@ go_library(
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
@@ -157,6 +160,7 @@ go_test(
"//beacon-chain/operations/slashings:go_default_library",
"//beacon-chain/operations/voluntaryexits:go_default_library",
"//beacon-chain/p2p:go_default_library",
"//beacon-chain/p2p/testing:go_default_library",
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",

View File

@@ -33,6 +33,7 @@ var (
)
var errMaxBlobsExceeded = errors.New("Expected commitments in block exceeds MAX_BLOBS_PER_BLOCK")
var errMaxDataColumnsExceeded = errors.New("Expected data columns for node exceeds NUMBER_OF_COLUMNS")
// An invalid block is the block that fails state transition based on the core protocol rules.
// The beacon node shall not be accepting nor building blocks that branch off from an invalid block.

View File

@@ -12,6 +12,8 @@ go_library(
deps = [
"//consensus-types/blocks:go_default_library",
"@com_github_crate_crypto_go_kzg_4844//:go_default_library",
"@com_github_ethereum_c_kzg_4844//bindings/go:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
)

View File

@@ -5,6 +5,8 @@ import (
"encoding/json"
GoKZG "github.com/crate-crypto/go-kzg-4844"
CKZG "github.com/ethereum/c-kzg-4844/bindings/go"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/pkg/errors"
)
@@ -12,17 +14,38 @@ var (
//go:embed trusted_setup.json
embeddedTrustedSetup []byte // 1.2Mb
kzgContext *GoKZG.Context
kzgLoaded bool
)
func Start() error {
parsedSetup := GoKZG.JSONTrustedSetup{}
err := json.Unmarshal(embeddedTrustedSetup, &parsedSetup)
parsedSetup := &GoKZG.JSONTrustedSetup{}
err := json.Unmarshal(embeddedTrustedSetup, parsedSetup)
if err != nil {
return errors.Wrap(err, "could not parse trusted setup JSON")
}
kzgContext, err = GoKZG.NewContext4096(&parsedSetup)
kzgContext, err = GoKZG.NewContext4096(parsedSetup)
if err != nil {
return errors.Wrap(err, "could not initialize go-kzg context")
}
g1Lagrange := &parsedSetup.SetupG1Lagrange
// Length of a G1 point, converted from hex to binary.
g1s := make([]byte, len(g1Lagrange)*(len(g1Lagrange[0])-2)/2)
for i, g1 := range g1Lagrange {
copy(g1s[i*(len(g1)-2)/2:], hexutil.MustDecode(g1))
}
// Length of a G2 point, converted from hex to binary.
g2s := make([]byte, len(parsedSetup.SetupG2)*(len(parsedSetup.SetupG2[0])-2)/2)
for i, g2 := range parsedSetup.SetupG2 {
copy(g2s[i*(len(g2)-2)/2:], hexutil.MustDecode(g2))
}
if !kzgLoaded {
// Free the current trusted setup before running this method. CKZG
// panics if the same setup is run multiple times.
if err = CKZG.LoadTrustedSetup(g1s, g2s); err != nil {
panic(err)
}
}
kzgLoaded = true
return nil
}

View File

@@ -118,9 +118,9 @@ func WithBLSToExecPool(p blstoexec.PoolManager) Option {
}
// WithP2PBroadcaster to broadcast messages after appropriate processing.
func WithP2PBroadcaster(p p2p.Broadcaster) Option {
func WithP2PBroadcaster(p p2p.Acceser) Option {
return func(s *Service) error {
s.cfg.P2p = p
s.cfg.P2P = p
return nil
}
}

View File

@@ -6,6 +6,8 @@ import (
"time"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/sirupsen/logrus"
"go.opencensus.io/trace"
@@ -500,7 +502,7 @@ func missingIndices(bs *filesystem.BlobStorage, root [32]byte, expected [][]byte
}
indices, err := bs.Indices(root)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "indices")
}
missing := make(map[uint64]struct{}, len(expected))
for i := range expected {
@@ -514,12 +516,35 @@ func missingIndices(bs *filesystem.BlobStorage, root [32]byte, expected [][]byte
return missing, nil
}
func missingDataColumns(bs *filesystem.BlobStorage, root [32]byte, expected map[uint64]bool) (map[uint64]bool, error) {
if len(expected) == 0 {
return nil, nil
}
if len(expected) > int(params.BeaconConfig().NumberOfColumns) {
return nil, errMaxDataColumnsExceeded
}
indices, err := bs.ColumnIndices(root)
if err != nil {
return nil, err
}
missing := make(map[uint64]bool, len(expected))
for col := range expected {
if !indices[col] {
missing[col] = true
}
}
return missing, nil
}
// isDataAvailable blocks until all BlobSidecars committed to in the block are available,
// or an error or context cancellation occurs. A nil result means that the data availability check is successful.
// The function will first check the database to see if all sidecars have been persisted. If any
// sidecars are missing, it will then read from the blobNotifier channel for the given root until the channel is
// closed, the context hits cancellation/timeout, or notifications have been received for all the missing sidecars.
func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed interfaces.ReadOnlySignedBeaconBlock) error {
if features.Get().EnablePeerDAS {
return s.isDataAvailableDataColumns(ctx, root, signed)
}
if signed.Version() < version.Deneb {
return nil
}
@@ -549,7 +574,7 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
// get a map of BlobSidecar indices that are not currently available.
missing, err := missingIndices(s.blobStorage, root, kzgCommitments)
if err != nil {
return err
return errors.Wrap(err, "missing indices")
}
// If there are no missing indices, all BlobSidecars are available.
if len(missing) == 0 {
@@ -568,8 +593,13 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
if len(missing) == 0 {
return
}
log.WithFields(daCheckLogFields(root, signed.Block().Slot(), expected, len(missing))).
Error("Still waiting for DA check at slot end.")
log.WithFields(logrus.Fields{
"slot": signed.Block().Slot(),
"root": fmt.Sprintf("%#x", root),
"blobsExpected": expected,
"blobsWaiting": len(missing),
}).Error("Still waiting for blobs DA check at slot end.")
})
defer nst.Stop()
}
@@ -591,12 +621,104 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
}
}
func daCheckLogFields(root [32]byte, slot primitives.Slot, expected, missing int) logrus.Fields {
return logrus.Fields{
"slot": slot,
"root": fmt.Sprintf("%#x", root),
"blobsExpected": expected,
"blobsWaiting": missing,
func (s *Service) isDataAvailableDataColumns(ctx context.Context, root [32]byte, signed interfaces.ReadOnlySignedBeaconBlock) error {
if signed.Version() < version.Deneb {
return nil
}
block := signed.Block()
if block == nil {
return errors.New("invalid nil beacon block")
}
// We are only required to check within MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS
if !params.WithinDAPeriod(slots.ToEpoch(block.Slot()), slots.ToEpoch(s.CurrentSlot())) {
return nil
}
body := block.Body()
if body == nil {
return errors.New("invalid nil beacon block body")
}
kzgCommitments, err := body.BlobKzgCommitments()
if err != nil {
return errors.Wrap(err, "could not get KZG commitments")
}
// If block has not commitments there is nothing to wait for.
if len(kzgCommitments) == 0 {
return nil
}
custodiedSubnetCount := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodiedSubnetCount = params.BeaconConfig().DataColumnSidecarSubnetCount
}
colMap, err := peerdas.CustodyColumns(s.cfg.P2P.NodeID(), custodiedSubnetCount)
if err != nil {
return err
}
// Expected is the number of custodied data columnns a node is expected to have.
expected := len(colMap)
if expected == 0 {
return nil
}
// Subscribe to newsly data columns stored in the database.
rootIndexChan := make(chan filesystem.RootIndexPair)
subscription := s.blobStorage.DataColumnFeed.Subscribe(rootIndexChan)
defer subscription.Unsubscribe()
// Get a map of data column indices that are not currently available.
missing, err := missingDataColumns(s.blobStorage, root, colMap)
if err != nil {
return err
}
// If there are no missing indices, all data column sidecars are available.
// This is the happy path.
if len(missing) == 0 {
return nil
}
// Log for DA checks that cross over into the next slot; helpful for debugging.
nextSlot := slots.BeginsAt(signed.Block().Slot()+1, s.genesisTime)
// Avoid logging if DA check is called after next slot start.
if nextSlot.After(time.Now()) {
nst := time.AfterFunc(time.Until(nextSlot), func() {
if len(missing) == 0 {
return
}
log.WithFields(logrus.Fields{
"slot": signed.Block().Slot(),
"root": fmt.Sprintf("%#x", root),
"columnsExpected": expected,
"columnsWaiting": len(missing),
}).Error("Still waiting for data columns DA check at slot end.")
})
defer nst.Stop()
}
for {
select {
case rootIndex := <-rootIndexChan:
if rootIndex.Root != root {
// This is not the root we are looking for.
continue
}
// Remove the index from the missing map.
delete(missing, rootIndex.Index)
// Exit if there is no more missing data columns.
if len(missing) == 0 {
return nil
}
case <-ctx.Done():
missingIndexes := make([]uint64, 0, len(missing))
for val := range missing {
copiedVal := val
missingIndexes = append(missingIndexes, copiedVal)
}
return errors.Wrapf(ctx.Err(), "context deadline waiting for data column sidecars slot: %d, BlockRoot: %#x, missing %v", block.Slot(), root, missingIndexes)
}
}
}

View File

@@ -50,6 +50,12 @@ type BlobReceiver interface {
ReceiveBlob(context.Context, blocks.VerifiedROBlob) error
}
// DataColumnReceiver interface defines the methods of chain service for receiving new
// data columns
type DataColumnReceiver interface {
ReceiveDataColumn(context.Context, blocks.VerifiedRODataColumn) error
}
// SlashingReceiver interface defines the methods of chain service for receiving validated slashing over the wire.
type SlashingReceiver interface {
ReceiveAttesterSlashing(ctx context.Context, slashing ethpb.AttSlashing)

View File

@@ -0,0 +1,16 @@
package blockchain
import (
"context"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
)
func (s *Service) ReceiveDataColumn(ctx context.Context, ds blocks.VerifiedRODataColumn) error {
if err := s.blobStorage.SaveDataColumn(ds); err != nil {
return errors.Wrap(err, "save data column")
}
return nil
}

View File

@@ -82,7 +82,7 @@ type config struct {
ExitPool voluntaryexits.PoolManager
SlashingPool slashings.PoolManager
BLSToExecPool blstoexec.PoolManager
P2p p2p.Broadcaster
P2P p2p.Acceser
MaxRoutines int
StateNotifier statefeed.Notifier
ForkChoiceStore f.ForkChoicer
@@ -107,15 +107,17 @@ var ErrMissingClockSetter = errors.New("blockchain Service initialized without a
type blobNotifierMap struct {
sync.RWMutex
notifiers map[[32]byte]chan uint64
seenIndex map[[32]byte][fieldparams.MaxBlobsPerBlock]bool
seenIndex map[[32]byte][fieldparams.NumberOfColumns]bool
}
// notifyIndex notifies a blob by its index for a given root.
// It uses internal maps to keep track of seen indices and notifier channels.
func (bn *blobNotifierMap) notifyIndex(root [32]byte, idx uint64) {
if idx >= fieldparams.MaxBlobsPerBlock {
return
}
// TODO: Separate Data Columns from blobs
/*
if idx >= fieldparams.MaxBlobsPerBlock {
return
}*/
bn.Lock()
seen := bn.seenIndex[root]
@@ -129,7 +131,7 @@ func (bn *blobNotifierMap) notifyIndex(root [32]byte, idx uint64) {
// Retrieve or create the notifier channel for the given root.
c, ok := bn.notifiers[root]
if !ok {
c = make(chan uint64, fieldparams.MaxBlobsPerBlock)
c = make(chan uint64, fieldparams.NumberOfColumns)
bn.notifiers[root] = c
}
@@ -143,7 +145,7 @@ func (bn *blobNotifierMap) forRoot(root [32]byte) chan uint64 {
defer bn.Unlock()
c, ok := bn.notifiers[root]
if !ok {
c = make(chan uint64, fieldparams.MaxBlobsPerBlock)
c = make(chan uint64, fieldparams.NumberOfColumns)
bn.notifiers[root] = c
}
return c
@@ -169,7 +171,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) {
ctx, cancel := context.WithCancel(ctx)
bn := &blobNotifierMap{
notifiers: make(map[[32]byte]chan uint64),
seenIndex: make(map[[32]byte][fieldparams.MaxBlobsPerBlock]bool),
seenIndex: make(map[[32]byte][fieldparams.NumberOfColumns]bool),
}
srv := &Service{
ctx: ctx,

View File

@@ -95,7 +95,7 @@ func setupBeaconChain(t *testing.T, beaconDB db.Database) *Service {
WithAttestationPool(attestations.NewPool()),
WithSlashingPool(slashings.NewPool()),
WithExitPool(voluntaryexits.NewPool()),
WithP2PBroadcaster(&mockBroadcaster{}),
WithP2PBroadcaster(&mockAccesser{}),
WithStateNotifier(&mockBeaconNode{}),
WithForkChoiceStore(fc),
WithAttestationService(attService),
@@ -518,7 +518,7 @@ func (s *MockClockSetter) SetClock(g *startup.Clock) error {
func TestNotifyIndex(t *testing.T) {
// Initialize a blobNotifierMap
bn := &blobNotifierMap{
seenIndex: make(map[[32]byte][fieldparams.MaxBlobsPerBlock]bool),
seenIndex: make(map[[32]byte][fieldparams.NumberOfColumns]bool),
notifiers: make(map[[32]byte]chan uint64),
}

View File

@@ -19,6 +19,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/operations/attestations"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/operations/blstoexec"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2pTesting "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
@@ -45,6 +46,11 @@ type mockBroadcaster struct {
broadcastCalled bool
}
type mockAccesser struct {
mockBroadcaster
p2pTesting.MockPeerManager
}
func (mb *mockBroadcaster) Broadcast(_ context.Context, _ proto.Message) error {
mb.broadcastCalled = true
return nil
@@ -65,6 +71,11 @@ func (mb *mockBroadcaster) BroadcastBlob(_ context.Context, _ uint64, _ *ethpb.B
return nil
}
func (mb *mockBroadcaster) BroadcastDataColumn(_ context.Context, _ uint64, _ *ethpb.DataColumnSidecar) error {
mb.broadcastCalled = true
return nil
}
func (mb *mockBroadcaster) BroadcastBLSChanges(_ context.Context, _ []*ethpb.SignedBLSToExecutionChange) {
}

View File

@@ -628,6 +628,11 @@ func (c *ChainService) ReceiveBlob(_ context.Context, b blocks.VerifiedROBlob) e
return nil
}
// ReceiveDataColumn implements the same method in chain service
func (c *ChainService) ReceiveDataColumn(_ context.Context, _ blocks.VerifiedRODataColumn) error {
return nil
}
// TargetRootForEpoch mocks the same method in the chain service
func (c *ChainService) TargetRootForEpoch(_ [32]byte, _ primitives.Epoch) ([32]byte, error) {
return c.TargetRoot, nil

View File

@@ -8,6 +8,7 @@ go_library(
"attestation_data.go",
"balance_cache_key.go",
"checkpoint_state.go",
"column_subnet_ids.go",
"committee.go",
"committee_disabled.go", # keep
"committees.go",

65
beacon-chain/cache/column_subnet_ids.go vendored Normal file
View File

@@ -0,0 +1,65 @@
package cache
import (
"sync"
"time"
"github.com/patrickmn/go-cache"
"github.com/prysmaticlabs/prysm/v5/config/params"
)
type columnSubnetIDs struct {
colSubCache *cache.Cache
colSubLock sync.RWMutex
}
// ColumnSubnetIDs for column subnet participants
var ColumnSubnetIDs = newColumnSubnetIDs()
const columnKey = "columns"
func newColumnSubnetIDs() *columnSubnetIDs {
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
// Set the default duration of a column subnet subscription as the column expiry period.
subLength := epochDuration * time.Duration(params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest)
persistentCache := cache.New(subLength*time.Second, epochDuration*time.Second)
return &columnSubnetIDs{colSubCache: persistentCache}
}
// GetColumnSubnets retrieves the data column subnets.
func (s *columnSubnetIDs) GetColumnSubnets() ([]uint64, bool, time.Time) {
s.colSubLock.RLock()
defer s.colSubLock.RUnlock()
id, duration, ok := s.colSubCache.GetWithExpiration(columnKey)
if !ok {
return nil, false, time.Time{}
}
// Retrieve indices from the cache.
idxs, ok := id.([]uint64)
if !ok {
return nil, false, time.Time{}
}
return idxs, ok, duration
}
// AddColumnSubnets adds the relevant data column subnets.
func (s *columnSubnetIDs) AddColumnSubnets(colIdx []uint64) {
s.colSubLock.Lock()
defer s.colSubLock.Unlock()
s.colSubCache.Set(columnKey, colIdx, 0)
}
// EmptyAllCaches empties out all the related caches and flushes any stored
// entries on them. This should only ever be used for testing, in normal
// production, handling of the relevant subnets for each role is done
// separately.
func (s *columnSubnetIDs) EmptyAllCaches() {
// Clear the cache.
s.colSubLock.Lock()
defer s.colSubLock.Unlock()
s.colSubCache.Flush()
}

View File

@@ -96,6 +96,24 @@ func VerifyBlockHeaderSignature(beaconState state.BeaconState, header *ethpb.Sig
return signing.VerifyBlockHeaderSigningRoot(header.Header, proposerPubKey, header.Signature, domain)
}
func VerifyBlockHeaderSignatureUsingCurrentFork(beaconState state.BeaconState, header *ethpb.SignedBeaconBlockHeader) error {
currentEpoch := slots.ToEpoch(header.Header.Slot)
fork, err := forks.Fork(currentEpoch)
if err != nil {
return err
}
domain, err := signing.Domain(fork, currentEpoch, params.BeaconConfig().DomainBeaconProposer, beaconState.GenesisValidatorsRoot())
if err != nil {
return err
}
proposer, err := beaconState.ValidatorAtIndex(header.Header.ProposerIndex)
if err != nil {
return err
}
proposerPubKey := proposer.PublicKey
return signing.VerifyBlockHeaderSigningRoot(header.Header, proposerPubKey, header.Signature, domain)
}
// VerifyBlockSignatureUsingCurrentFork verifies the proposer signature of a beacon block. This differs
// from the above method by not using fork data from the state and instead retrieving it
// via the respective epoch.

View File

@@ -32,6 +32,9 @@ const (
// AttesterSlashingReceived is sent after an attester slashing is received from gossip or rpc
AttesterSlashingReceived = 8
// DataColumnSidecarReceived is sent after a data column sidecar is received from gossip or rpc.
DataColumnSidecarReceived = 9
)
// UnAggregatedAttReceivedData is the data sent with UnaggregatedAttReceived events.
@@ -77,3 +80,7 @@ type ProposerSlashingReceivedData struct {
type AttesterSlashingReceivedData struct {
AttesterSlashing ethpb.AttSlashing
}
type DataColumnSidecarReceivedData struct {
DataColumn *blocks.VerifiedRODataColumn
}

View File

@@ -0,0 +1,36 @@
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = ["helpers.go"],
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas",
visibility = ["//visibility:public"],
deps = [
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//crypto/hash:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_ethereum_c_kzg_4844//bindings/go:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enode:go_default_library",
"@com_github_holiman_uint256//:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = ["helpers_test.go"],
deps = [
":go_default_library",
"//beacon-chain/blockchain/kzg:go_default_library",
"//consensus-types/blocks:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"@com_github_consensys_gnark_crypto//ecc/bls12-381/fr:go_default_library",
"@com_github_crate_crypto_go_kzg_4844//:go_default_library",
"@com_github_ethereum_c_kzg_4844//bindings/go:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)

View File

@@ -0,0 +1,304 @@
package peerdas
import (
"encoding/binary"
"math"
cKzg4844 "github.com/ethereum/c-kzg-4844/bindings/go"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/holiman/uint256"
errors "github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/crypto/hash"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
// Bytes per cell
const bytesPerCell = cKzg4844.FieldElementsPerCell * cKzg4844.BytesPerFieldElement
var (
// Custom errors
errCustodySubnetCountTooLarge = errors.New("custody subnet count larger than data column sidecar subnet count")
errIndexTooLarge = errors.New("column index is larger than the specified number of columns")
errMismatchLength = errors.New("mismatch in the length of the commitments and proofs")
// maxUint256 is the maximum value of a uint256.
maxUint256 = &uint256.Int{math.MaxUint64, math.MaxUint64, math.MaxUint64, math.MaxUint64}
)
// CustodyColumnSubnets computes the subnets the node should participate in for custody.
func CustodyColumnSubnets(nodeId enode.ID, custodySubnetCount uint64) ([]uint64, error) {
dataColumnSidecarSubnetCount := params.BeaconConfig().DataColumnSidecarSubnetCount
// Check if the custody subnet count is larger than the data column sidecar subnet count.
if custodySubnetCount > dataColumnSidecarSubnetCount {
return nil, errCustodySubnetCountTooLarge
}
one := uint256.NewInt(1)
subnetIds, subnetIdsMap := make([]uint64, 0, custodySubnetCount), make(map[uint64]bool, custodySubnetCount)
for currentId := new(uint256.Int).SetBytes(nodeId.Bytes()); uint64(len(subnetIds)) < custodySubnetCount; currentId.Add(currentId, one) {
// Convert to big endian bytes.
currentIdBytesBigEndian := currentId.Bytes32()
// Convert to little endian.
currentIdBytesLittleEndian := bytesutil.ReverseByteOrder(currentIdBytesBigEndian[:])
// Hash the result.
hashedCurrentId := hash.Hash(currentIdBytesLittleEndian)
// Get the subnet ID.
subnetId := binary.LittleEndian.Uint64(hashedCurrentId[:8]) % dataColumnSidecarSubnetCount
// Add the subnet to the slice.
exists := subnetIdsMap[subnetId]
if !exists {
subnetIds = append(subnetIds, subnetId)
subnetIdsMap[subnetId] = true
}
// Overflow prevention.
if currentId.Cmp(maxUint256) == 0 {
currentId = uint256.NewInt(0)
}
}
return subnetIds, nil
}
// CustodyColumns computes the columns the node should custody.
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#helper-functions
func CustodyColumns(nodeId enode.ID, custodySubnetCount uint64) (map[uint64]bool, error) {
dataColumnSidecarSubnetCount := params.BeaconConfig().DataColumnSidecarSubnetCount
// Compute the custodied subnets.
subnetIds, err := CustodyColumnSubnets(nodeId, custodySubnetCount)
if err != nil {
return nil, errors.Wrap(err, "custody subnets")
}
columnsPerSubnet := cKzg4844.CellsPerExtBlob / dataColumnSidecarSubnetCount
// Knowing the subnet ID and the number of columns per subnet, select all the columns the node should custody.
// Columns belonging to the same subnet are contiguous.
columnIndices := make(map[uint64]bool, custodySubnetCount*columnsPerSubnet)
for i := uint64(0); i < columnsPerSubnet; i++ {
for _, subnetId := range subnetIds {
columnIndex := dataColumnSidecarSubnetCount*i + subnetId
columnIndices[columnIndex] = true
}
}
return columnIndices, nil
}
// DataColumnSidecars computes the data column sidecars from the signed block and blobs.
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#recover_matrix
func DataColumnSidecars(signedBlock interfaces.ReadOnlySignedBeaconBlock, blobs []cKzg4844.Blob) ([]*ethpb.DataColumnSidecar, error) {
blobsCount := len(blobs)
if blobsCount == 0 {
return nil, nil
}
// Get the signed block header.
signedBlockHeader, err := signedBlock.Header()
if err != nil {
return nil, errors.Wrap(err, "signed block header")
}
// Get the block body.
block := signedBlock.Block()
blockBody := block.Body()
// Get the blob KZG commitments.
blobKzgCommitments, err := blockBody.BlobKzgCommitments()
if err != nil {
return nil, errors.Wrap(err, "blob KZG commitments")
}
// Compute the KZG commitments inclusion proof.
kzgCommitmentsInclusionProof, err := blocks.MerkleProofKZGCommitments(blockBody)
if err != nil {
return nil, errors.Wrap(err, "merkle proof ZKG commitments")
}
// Compute cells and proofs.
cells := make([][cKzg4844.CellsPerExtBlob]cKzg4844.Cell, 0, blobsCount)
proofs := make([][cKzg4844.CellsPerExtBlob]cKzg4844.KZGProof, 0, blobsCount)
for i := range blobs {
blob := &blobs[i]
blobCells, blobProofs, err := cKzg4844.ComputeCellsAndKZGProofs(blob)
if err != nil {
return nil, errors.Wrap(err, "compute cells and KZG proofs")
}
cells = append(cells, blobCells)
proofs = append(proofs, blobProofs)
}
// Get the column sidecars.
sidecars := make([]*ethpb.DataColumnSidecar, 0, cKzg4844.CellsPerExtBlob)
for columnIndex := uint64(0); columnIndex < cKzg4844.CellsPerExtBlob; columnIndex++ {
column := make([]cKzg4844.Cell, 0, blobsCount)
kzgProofOfColumn := make([]cKzg4844.KZGProof, 0, blobsCount)
for rowIndex := 0; rowIndex < blobsCount; rowIndex++ {
cell := cells[rowIndex][columnIndex]
column = append(column, cell)
kzgProof := proofs[rowIndex][columnIndex]
kzgProofOfColumn = append(kzgProofOfColumn, kzgProof)
}
columnBytes := make([][]byte, 0, blobsCount)
for i := range column {
cell := column[i]
cellBytes := make([]byte, 0, bytesPerCell)
for _, fieldElement := range cell {
copiedElem := fieldElement
cellBytes = append(cellBytes, copiedElem[:]...)
}
columnBytes = append(columnBytes, cellBytes)
}
kzgProofOfColumnBytes := make([][]byte, 0, blobsCount)
for _, kzgProof := range kzgProofOfColumn {
copiedProof := kzgProof
kzgProofOfColumnBytes = append(kzgProofOfColumnBytes, copiedProof[:])
}
sidecar := &ethpb.DataColumnSidecar{
ColumnIndex: columnIndex,
DataColumn: columnBytes,
KzgCommitments: blobKzgCommitments,
KzgProof: kzgProofOfColumnBytes,
SignedBlockHeader: signedBlockHeader,
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
sidecars = append(sidecars, sidecar)
}
return sidecars, nil
}
// DataColumnSidecarsForReconstruct is a TEMPORARY function until there is an official specification for it.
// It is scheduled for deletion.
func DataColumnSidecarsForReconstruct(
blobKzgCommitments [][]byte,
signedBlockHeader *ethpb.SignedBeaconBlockHeader,
kzgCommitmentsInclusionProof [][]byte,
blobs []cKzg4844.Blob,
) ([]*ethpb.DataColumnSidecar, error) {
blobsCount := len(blobs)
if blobsCount == 0 {
return nil, nil
}
// Compute cells and proofs.
cells := make([][cKzg4844.CellsPerExtBlob]cKzg4844.Cell, 0, blobsCount)
proofs := make([][cKzg4844.CellsPerExtBlob]cKzg4844.KZGProof, 0, blobsCount)
for i := range blobs {
blob := &blobs[i]
blobCells, blobProofs, err := cKzg4844.ComputeCellsAndKZGProofs(blob)
if err != nil {
return nil, errors.Wrap(err, "compute cells and KZG proofs")
}
cells = append(cells, blobCells)
proofs = append(proofs, blobProofs)
}
// Get the column sidecars.
sidecars := make([]*ethpb.DataColumnSidecar, 0, cKzg4844.CellsPerExtBlob)
for columnIndex := uint64(0); columnIndex < cKzg4844.CellsPerExtBlob; columnIndex++ {
column := make([]cKzg4844.Cell, 0, blobsCount)
kzgProofOfColumn := make([]cKzg4844.KZGProof, 0, blobsCount)
for rowIndex := 0; rowIndex < blobsCount; rowIndex++ {
cell := cells[rowIndex][columnIndex]
column = append(column, cell)
kzgProof := proofs[rowIndex][columnIndex]
kzgProofOfColumn = append(kzgProofOfColumn, kzgProof)
}
columnBytes := make([][]byte, 0, blobsCount)
for i := range column {
cell := column[i]
cellBytes := make([]byte, 0, bytesPerCell)
for _, fieldElement := range cell {
copiedElem := fieldElement
cellBytes = append(cellBytes, copiedElem[:]...)
}
columnBytes = append(columnBytes, cellBytes)
}
kzgProofOfColumnBytes := make([][]byte, 0, blobsCount)
for _, kzgProof := range kzgProofOfColumn {
copiedProof := kzgProof
kzgProofOfColumnBytes = append(kzgProofOfColumnBytes, copiedProof[:])
}
sidecar := &ethpb.DataColumnSidecar{
ColumnIndex: columnIndex,
DataColumn: columnBytes,
KzgCommitments: blobKzgCommitments,
KzgProof: kzgProofOfColumnBytes,
SignedBlockHeader: signedBlockHeader,
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
sidecars = append(sidecars, sidecar)
}
return sidecars, nil
}
// VerifyDataColumnSidecarKZGProofs verifies the provided KZG Proofs for the particular
// data column.
func VerifyDataColumnSidecarKZGProofs(sc *ethpb.DataColumnSidecar) (bool, error) {
if sc.ColumnIndex >= params.BeaconConfig().NumberOfColumns {
return false, errIndexTooLarge
}
if len(sc.DataColumn) != len(sc.KzgCommitments) || len(sc.KzgCommitments) != len(sc.KzgProof) {
return false, errMismatchLength
}
blobsCount := len(sc.DataColumn)
rowIdx := make([]uint64, 0, blobsCount)
colIdx := make([]uint64, 0, blobsCount)
for i := 0; i < len(sc.DataColumn); i++ {
copiedI := uint64(i)
rowIdx = append(rowIdx, copiedI)
colI := sc.ColumnIndex
colIdx = append(colIdx, colI)
}
ckzgComms := make([]cKzg4844.Bytes48, 0, len(sc.KzgCommitments))
for _, com := range sc.KzgCommitments {
ckzgComms = append(ckzgComms, cKzg4844.Bytes48(com))
}
var cells []cKzg4844.Cell
for _, ce := range sc.DataColumn {
var newCell []cKzg4844.Bytes32
for i := 0; i < len(ce); i += 32 {
newCell = append(newCell, cKzg4844.Bytes32(ce[i:i+32]))
}
cells = append(cells, cKzg4844.Cell(newCell))
}
var proofs []cKzg4844.Bytes48
for _, p := range sc.KzgProof {
proofs = append(proofs, cKzg4844.Bytes48(p))
}
return cKzg4844.VerifyCellKZGProofBatch(ckzgComms, rowIdx, colIdx, cells, proofs)
}

View File

@@ -0,0 +1,91 @@
package peerdas_test
import (
"bytes"
"crypto/sha256"
"encoding/binary"
"fmt"
"testing"
"github.com/consensys/gnark-crypto/ecc/bls12-381/fr"
GoKZG "github.com/crate-crypto/go-kzg-4844"
ckzg4844 "github.com/ethereum/c-kzg-4844/bindings/go"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/kzg"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
"github.com/sirupsen/logrus"
)
func deterministicRandomness(seed int64) [32]byte {
// Converts an int64 to a byte slice
buf := new(bytes.Buffer)
err := binary.Write(buf, binary.BigEndian, seed)
if err != nil {
logrus.WithError(err).Error("Failed to write int64 to bytes buffer")
return [32]byte{}
}
bytes := buf.Bytes()
return sha256.Sum256(bytes)
}
// Returns a serialized random field element in big-endian
func GetRandFieldElement(seed int64) [32]byte {
bytes := deterministicRandomness(seed)
var r fr.Element
r.SetBytes(bytes[:])
return GoKZG.SerializeScalar(r)
}
// Returns a random blob using the passed seed as entropy
func GetRandBlob(seed int64) ckzg4844.Blob {
var blob ckzg4844.Blob
bytesPerBlob := GoKZG.ScalarsPerBlob * GoKZG.SerializedScalarSize
for i := 0; i < bytesPerBlob; i += GoKZG.SerializedScalarSize {
fieldElementBytes := GetRandFieldElement(seed + int64(i))
copy(blob[i:i+GoKZG.SerializedScalarSize], fieldElementBytes[:])
}
return blob
}
func GenerateCommitmentAndProof(blob ckzg4844.Blob) (ckzg4844.KZGCommitment, ckzg4844.KZGProof, error) {
commitment, err := ckzg4844.BlobToKZGCommitment(&blob)
if err != nil {
return ckzg4844.KZGCommitment{}, ckzg4844.KZGProof{}, err
}
proof, err := ckzg4844.ComputeBlobKZGProof(&blob, ckzg4844.Bytes48(commitment))
if err != nil {
return ckzg4844.KZGCommitment{}, ckzg4844.KZGProof{}, err
}
return commitment, proof, err
}
func TestVerifyDataColumnSidecarKZGProofs(t *testing.T) {
dbBlock := util.NewBeaconBlockDeneb()
require.NoError(t, kzg.Start())
comms := [][]byte{}
blobs := []ckzg4844.Blob{}
for i := int64(0); i < 6; i++ {
blob := GetRandBlob(i)
commitment, _, err := GenerateCommitmentAndProof(blob)
require.NoError(t, err)
comms = append(comms, commitment[:])
blobs = append(blobs, blob)
}
dbBlock.Block.Body.BlobKzgCommitments = comms
sBlock, err := blocks.NewSignedBeaconBlock(dbBlock)
require.NoError(t, err)
sCars, err := peerdas.DataColumnSidecars(sBlock, blobs)
require.NoError(t, err)
for i, sidecar := range sCars {
verified, err := peerdas.VerifyDataColumnSidecarKZGProofs(sidecar)
require.NoError(t, err)
require.Equal(t, true, verified, fmt.Sprintf("sidecar %d failed", i))
}
}

View File

@@ -4,6 +4,7 @@ go_library(
name = "go_default_library",
srcs = [
"availability.go",
"availability_columns.go",
"cache.go",
"iface.go",
"mock.go",
@@ -20,6 +21,7 @@ go_library(
"//runtime/logging:go_default_library",
"//runtime/version:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enode:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],

View File

@@ -0,0 +1,151 @@
package das
import (
"context"
"fmt"
"github.com/ethereum/go-ethereum/p2p/enode"
errors "github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
log "github.com/sirupsen/logrus"
)
// LazilyPersistentStoreColumn is an implementation of AvailabilityStore to be used when batch syncing data columns.
// This implementation will hold any blobs passed to Persist until the IsDataAvailable is called for their
// block, at which time they will undergo full verification and be saved to the disk.
type LazilyPersistentStoreColumn struct {
store *filesystem.BlobStorage
cache *cache
verifier ColumnBatchVerifier
nodeID enode.ID
}
type ColumnBatchVerifier interface {
VerifiedRODataColumns(ctx context.Context, blk blocks.ROBlock, sc []blocks.RODataColumn) ([]blocks.VerifiedRODataColumn, error)
}
func NewLazilyPersistentStoreColumn(store *filesystem.BlobStorage, verifier ColumnBatchVerifier, id enode.ID) *LazilyPersistentStoreColumn {
return &LazilyPersistentStoreColumn{
store: store,
cache: newCache(),
verifier: verifier,
nodeID: id,
}
}
// TODO: Very Ugly, change interface to allow for columns and blobs
func (s *LazilyPersistentStoreColumn) Persist(current primitives.Slot, sc ...blocks.ROBlob) error {
return nil
}
// PersistColumns adds columns to the working column cache. columns stored in this cache will be persisted
// for at least as long as the node is running. Once IsDataAvailable succeeds, all blobs referenced
// by the given block are guaranteed to be persisted for the remainder of the retention period.
func (s *LazilyPersistentStoreColumn) PersistColumns(current primitives.Slot, sc ...blocks.RODataColumn) error {
if len(sc) == 0 {
return nil
}
if len(sc) > 1 {
first := sc[0].BlockRoot()
for i := 1; i < len(sc); i++ {
if first != sc[i].BlockRoot() {
return errMixedRoots
}
}
}
if !params.WithinDAPeriod(slots.ToEpoch(sc[0].Slot()), slots.ToEpoch(current)) {
return nil
}
key := keyFromColumn(sc[0])
entry := s.cache.ensure(key)
for i := range sc {
if err := entry.stashColumns(&sc[i]); err != nil {
return err
}
}
return nil
}
// IsDataAvailable returns nil if all the commitments in the given block are persisted to the db and have been verified.
// BlobSidecars already in the db are assumed to have been previously verified against the block.
func (s *LazilyPersistentStoreColumn) IsDataAvailable(ctx context.Context, current primitives.Slot, b blocks.ROBlock) error {
blockCommitments, err := fullCommitmentsToCheck(b, current)
if err != nil {
return errors.Wrapf(err, "could check data availability for block %#x", b.Root())
}
// Return early for blocks that are pre-deneb or which do not have any commitments.
if blockCommitments.count() == 0 {
return nil
}
key := keyFromBlock(b)
entry := s.cache.ensure(key)
defer s.cache.delete(key)
root := b.Root()
sumz, err := s.store.WaitForSummarizer(ctx)
if err != nil {
log.WithField("root", fmt.Sprintf("%#x", b.Root())).
WithError(err).
Debug("Failed to receive BlobStorageSummarizer within IsDataAvailable")
} else {
entry.setDiskSummary(sumz.Summary(root))
}
// Verify we have all the expected sidecars, and fail fast if any are missing or inconsistent.
// We don't try to salvage problematic batches because this indicates a misbehaving peer and we'd rather
// ignore their response and decrease their peer score.
sidecars, err := entry.filterColumns(root, blockCommitments)
if err != nil {
return errors.Wrap(err, "incomplete BlobSidecar batch")
}
// Do thorough verifications of each BlobSidecar for the block.
// Same as above, we don't save BlobSidecars if there are any problems with the batch.
vscs, err := s.verifier.VerifiedRODataColumns(ctx, b, sidecars)
if err != nil {
var me verification.VerificationMultiError
ok := errors.As(err, &me)
if ok {
fails := me.Failures()
lf := make(log.Fields, len(fails))
for i := range fails {
lf[fmt.Sprintf("fail_%d", i)] = fails[i].Error()
}
log.WithFields(lf).
Debug("invalid ColumnSidecars received")
}
return errors.Wrapf(err, "invalid ColumnSidecars received for block %#x", root)
}
// Ensure that each column sidecar is written to disk.
for i := range vscs {
if err := s.store.SaveDataColumn(vscs[i]); err != nil {
return errors.Wrapf(err, "failed to save ColumnSidecar index %d for block %#x", vscs[i].ColumnIndex, root)
}
}
// All ColumnSidecars are persisted - da check succeeds.
return nil
}
func fullCommitmentsToCheck(b blocks.ROBlock, current primitives.Slot) (safeCommitmentsArray, error) {
var ar safeCommitmentsArray
if b.Version() < version.Deneb {
return ar, nil
}
// We are only required to check within MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS
if !params.WithinDAPeriod(slots.ToEpoch(b.Block().Slot()), slots.ToEpoch(current)) {
return ar, nil
}
kc, err := b.Block().Body().BlobKzgCommitments()
if err != nil {
return ar, err
}
for i := range ar {
copy(ar[i], kc)
}
return ar, nil
}

View File

@@ -2,6 +2,7 @@ package das
import (
"bytes"
"reflect"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
@@ -38,6 +39,10 @@ func keyFromSidecar(sc blocks.ROBlob) cacheKey {
return cacheKey{slot: sc.Slot(), root: sc.BlockRoot()}
}
func keyFromColumn(sc blocks.RODataColumn) cacheKey {
return cacheKey{slot: sc.Slot(), root: sc.BlockRoot()}
}
// keyFromBlock is a convenience method for constructing a cacheKey from a ROBlock value.
func keyFromBlock(b blocks.ROBlock) cacheKey {
return cacheKey{slot: b.Block().Slot(), root: b.Root()}
@@ -61,6 +66,7 @@ func (c *cache) delete(key cacheKey) {
// cacheEntry holds a fixed-length cache of BlobSidecars.
type cacheEntry struct {
scs [fieldparams.MaxBlobsPerBlock]*blocks.ROBlob
colScs [fieldparams.NumberOfColumns]*blocks.RODataColumn
diskSummary filesystem.BlobStorageSummary
}
@@ -82,6 +88,17 @@ func (e *cacheEntry) stash(sc *blocks.ROBlob) error {
return nil
}
func (e *cacheEntry) stashColumns(sc *blocks.RODataColumn) error {
if sc.ColumnIndex >= fieldparams.NumberOfColumns {
return errors.Wrapf(errIndexOutOfBounds, "index=%d", sc.ColumnIndex)
}
if e.colScs[sc.ColumnIndex] != nil {
return errors.Wrapf(ErrDuplicateSidecar, "root=%#x, index=%d, commitment=%#x", sc.BlockRoot(), sc.ColumnIndex, sc.KzgCommitments)
}
e.colScs[sc.ColumnIndex] = sc
return nil
}
// filter evicts sidecars that are not committed to by the block and returns custom
// errors if the cache is missing any of the commitments, or if the commitments in
// the cache do not match those found in the block. If err is nil, then all expected
@@ -117,6 +134,35 @@ func (e *cacheEntry) filter(root [32]byte, kc safeCommitmentArray) ([]blocks.ROB
return scs, nil
}
func (e *cacheEntry) filterColumns(root [32]byte, kc safeCommitmentsArray) ([]blocks.RODataColumn, error) {
if e.diskSummary.AllAvailable(kc.count()) {
return nil, nil
}
scs := make([]blocks.RODataColumn, 0, kc.count())
for i := uint64(0); i < fieldparams.NumberOfColumns; i++ {
// We already have this blob, we don't need to write it or validate it.
if e.diskSummary.HasIndex(i) {
continue
}
if kc[i] == nil {
if e.colScs[i] != nil {
return nil, errors.Wrapf(errCommitmentMismatch, "root=%#x, index=%#x, commitment=%#x, no block commitment", root, i, e.scs[i].KzgCommitment)
}
continue
}
if e.colScs[i] == nil {
return nil, errors.Wrapf(errMissingSidecar, "root=%#x, index=%#x", root, i)
}
if !reflect.DeepEqual(kc[i], e.colScs[i].KzgCommitments) {
return nil, errors.Wrapf(errCommitmentMismatch, "root=%#x, index=%#x, commitment=%#x, block commitment=%#x", root, i, e.colScs[i].KzgCommitments, kc[i])
}
scs = append(scs, *e.colScs[i])
}
return scs, nil
}
// safeCommitmentArray is a fixed size array of commitment byte slices. This is helpful for avoiding
// gratuitous bounds checks.
type safeCommitmentArray [fieldparams.MaxBlobsPerBlock][]byte
@@ -129,3 +175,14 @@ func (s safeCommitmentArray) count() int {
}
return fieldparams.MaxBlobsPerBlock
}
type safeCommitmentsArray [fieldparams.NumberOfColumns][][]byte
func (s safeCommitmentsArray) count() int {
for i := range s {
if s[i] == nil {
return i
}
}
return fieldparams.NumberOfColumns
}

View File

@@ -13,6 +13,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem",
visibility = ["//visibility:public"],
deps = [
"//async/event:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",

View File

@@ -12,6 +12,7 @@ import (
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/async/event"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
@@ -39,8 +40,15 @@ const (
directoryPermissions = 0700
)
// BlobStorageOption is a functional option for configuring a BlobStorage.
type BlobStorageOption func(*BlobStorage) error
type (
// BlobStorageOption is a functional option for configuring a BlobStorage.
BlobStorageOption func(*BlobStorage) error
RootIndexPair struct {
Root [fieldparams.RootLength]byte
Index uint64
}
)
// WithBasePath is a required option that sets the base path of blob storage.
func WithBasePath(base string) BlobStorageOption {
@@ -70,7 +78,10 @@ func WithSaveFsync(fsync bool) BlobStorageOption {
// 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(opts ...BlobStorageOption) (*BlobStorage, error) {
b := &BlobStorage{}
b := &BlobStorage{
DataColumnFeed: new(event.Feed),
}
for _, o := range opts {
if err := o(b); err != nil {
return nil, errors.Wrap(err, "failed to create blob storage")
@@ -99,6 +110,7 @@ type BlobStorage struct {
fsync bool
fs afero.Fs
pruner *blobPruner
DataColumnFeed *event.Feed
}
// WarmCache runs the prune routine with an expiration of slot of 0, so nothing will be pruned, but the pruner's cache
@@ -221,6 +233,110 @@ func (bs *BlobStorage) Save(sidecar blocks.VerifiedROBlob) error {
return nil
}
// SaveDataColumn saves a data column to our local filesystem.
func (bs *BlobStorage) SaveDataColumn(column blocks.VerifiedRODataColumn) error {
startTime := time.Now()
fname := namerForDataColumn(column)
sszPath := fname.path()
exists, err := afero.Exists(bs.fs, sszPath)
if err != nil {
return err
}
if exists {
log.Trace("Ignoring a duplicate data column sidecar save attempt")
return nil
}
if bs.pruner != nil {
hRoot, err := column.SignedBlockHeader.Header.HashTreeRoot()
if err != nil {
return err
}
if err := bs.pruner.notify(hRoot, column.SignedBlockHeader.Header.Slot, column.ColumnIndex); err != nil {
return errors.Wrapf(err, "problem maintaining pruning cache/metrics for sidecar with root=%#x", hRoot)
}
}
// Serialize the ethpb.DataColumnSidecar to binary data using SSZ.
sidecarData, err := column.MarshalSSZ()
if err != nil {
return errors.Wrap(err, "failed to serialize sidecar data")
} else if len(sidecarData) == 0 {
return errSidecarEmptySSZData
}
if err := bs.fs.MkdirAll(fname.dir(), directoryPermissions); err != nil {
return err
}
partPath := fname.partPath(fmt.Sprintf("%p", sidecarData))
partialMoved := false
// Ensure the partial file is deleted.
defer func() {
if partialMoved {
return
}
// It's expected to error if the save is successful.
err = bs.fs.Remove(partPath)
if err == nil {
log.WithFields(logrus.Fields{
"partPath": partPath,
}).Debugf("Removed partial file")
}
}()
// 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")
}
n, 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")
}
if bs.fsync {
if err := partialFile.Sync(); err != nil {
return err
}
}
if err := partialFile.Close(); err != nil {
return err
}
if n != len(sidecarData) {
return fmt.Errorf("failed to write the full bytes of sidecarData, wrote only %d of %d bytes", n, len(sidecarData))
}
if n == 0 {
return errEmptyBlobWritten
}
// 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")
}
partialMoved = true
// Notify the data column notifier that a new data column has been saved.
bs.DataColumnFeed.Send(RootIndexPair{
Root: column.BlockRoot(),
Index: column.ColumnIndex,
})
// TODO: Use new metrics for data columns
blobsWrittenCounter.Inc()
blobSaveLatency.Observe(float64(time.Since(startTime).Milliseconds()))
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.
@@ -246,6 +362,20 @@ func (bs *BlobStorage) Get(root [32]byte, idx uint64) (blocks.VerifiedROBlob, er
return verification.BlobSidecarNoop(ro)
}
// GetColumn retrieves a single DataColumnSidecar by its root and index.
func (bs *BlobStorage) GetColumn(root [32]byte, idx uint64) (*ethpb.DataColumnSidecar, error) {
expected := blobNamer{root: root, index: idx}
encoded, err := afero.ReadFile(bs.fs, expected.path())
if err != nil {
return nil, err
}
s := &ethpb.DataColumnSidecar{}
if err := s.UnmarshalSSZ(encoded); err != nil {
return nil, err
}
return s, nil
}
// Remove removes all blobs for a given root.
func (bs *BlobStorage) Remove(root [32]byte) error {
rootDir := blobNamer{root: root}.dir()
@@ -289,6 +419,61 @@ func (bs *BlobStorage) Indices(root [32]byte) ([fieldparams.MaxBlobsPerBlock]boo
return mask, nil
}
// ColumnIndices retrieve the stored column indexes from our filesystem.
func (bs *BlobStorage) ColumnIndices(root [32]byte) (map[uint64]bool, error) {
custody := make(map[uint64]bool, fieldparams.NumberOfColumns)
// Get all the files in the directory.
rootDir := blobNamer{root: root}.dir()
entries, err := afero.ReadDir(bs.fs, rootDir)
if err != nil {
// If the directory does not exist, we do not custody any columns.
if os.IsNotExist(err) {
return nil, nil
}
return nil, errors.Wrap(err, "read directory")
}
// Iterate over all the entries in the directory.
for _, entry := range entries {
// If the entry is a directory, skip it.
if entry.IsDir() {
continue
}
// If the entry does not have the correct extension, skip it.
name := entry.Name()
if !strings.HasSuffix(name, sszExt) {
continue
}
// The file should be in the `<index>.<extension>` format.
// Skip the file if it does not match the format.
parts := strings.Split(name, ".")
if len(parts) != 2 {
continue
}
// Get the column index from the file name.
columnIndexStr := parts[0]
columnIndex, err := strconv.ParseUint(columnIndexStr, 10, 64)
if err != nil {
return nil, errors.Wrapf(err, "unexpected directory entry breaks listing, %s", parts[0])
}
// If the column index is out of bounds, return an error.
if columnIndex >= fieldparams.NumberOfColumns {
return nil, errors.Wrapf(errIndexOutOfBounds, "invalid index %d", columnIndex)
}
// Mark the column index as in custody.
custody[columnIndex] = true
}
return custody, nil
}
// Clear deletes all files on the filesystem.
func (bs *BlobStorage) Clear() error {
dirs, err := listDir(bs.fs, ".")
@@ -321,6 +506,10 @@ func namerForSidecar(sc blocks.VerifiedROBlob) blobNamer {
return blobNamer{root: sc.BlockRoot(), index: sc.Index}
}
func namerForDataColumn(col blocks.VerifiedRODataColumn) blobNamer {
return blobNamer{root: col.BlockRoot(), index: col.ColumnIndex}
}
func (p blobNamer) dir() string {
return rootString(p.root)
}

View File

@@ -9,7 +9,7 @@ import (
)
// blobIndexMask is a bitmask representing the set of blob indices that are currently set.
type blobIndexMask [fieldparams.MaxBlobsPerBlock]bool
type blobIndexMask [fieldparams.NumberOfColumns]bool
// BlobStorageSummary represents cached information about the BlobSidecars on disk for each root the cache knows about.
type BlobStorageSummary struct {
@@ -68,9 +68,12 @@ func (s *blobStorageCache) Summary(root [32]byte) BlobStorageSummary {
}
func (s *blobStorageCache) ensure(key [32]byte, slot primitives.Slot, idx uint64) error {
if idx >= fieldparams.MaxBlobsPerBlock {
return errIndexOutOfBounds
}
// TODO: Separate blob index checks from data column index checks
/*
if idx >= fieldparams.MaxBlobsPerBlock {
return errIndexOutOfBounds
}
*/
s.mu.Lock()
defer s.mu.Unlock()
v := s.cache[key]

View File

@@ -9,6 +9,7 @@ import (
)
func TestSlotByRoot_Summary(t *testing.T) {
t.Skip("Use new test for data columns")
var noneSet, allSet, firstSet, lastSet, oneSet blobIndexMask
firstSet[0] = true
lastSet[len(lastSet)-1] = true

View File

@@ -23,10 +23,10 @@ import (
"go.opencensus.io/trace"
)
// used to represent errors for inconsistent slot ranges.
// Used to represent errors for inconsistent slot ranges.
var errInvalidSlotRange = errors.New("invalid end slot and start slot provided")
// Block retrieval by root.
// Block retrieval by root. Return nil if block is not found.
func (s *Store) Block(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
ctx, span := trace.StartSpan(ctx, "BeaconDB.Block")
defer span.End()

View File

@@ -992,6 +992,7 @@ func (b *BeaconNode) registerRPCService(router *mux.Router) error {
FinalizationFetcher: chainService,
BlockReceiver: chainService,
BlobReceiver: chainService,
DataColumnReceiver: chainService,
AttestationReceiver: chainService,
GenesisTimeFetcher: chainService,
GenesisFetcher: chainService,

View File

@@ -7,6 +7,7 @@ go_library(
"broadcaster.go",
"config.go",
"connection_gater.go",
"custody.go",
"dial_relay_node.go",
"discovery.go",
"doc.go",
@@ -46,6 +47,7 @@ go_library(
"//beacon-chain/core/altair:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/p2p/encoder:go_default_library",
@@ -56,6 +58,7 @@ go_library(
"//beacon-chain/startup:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",
@@ -74,9 +77,12 @@ go_library(
"//runtime/version:go_default_library",
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_btcsuite_btcd_btcec_v2//:go_default_library",
"@com_github_ethereum_go_ethereum//crypto:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/discover:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enode:go_default_library",
"@com_github_ethereum_go_ethereum//p2p/enr:go_default_library",
"@com_github_ferranbt_fastssz//:go_default_library",
"@com_github_holiman_uint256//:go_default_library",
"@com_github_kr_pretty//:go_default_library",
"@com_github_libp2p_go_libp2p//:go_default_library",
@@ -139,6 +145,7 @@ go_test(
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/cache:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/p2p/encoder:go_default_library",
@@ -156,6 +163,7 @@ go_test(
"//container/leaky-bucket:go_default_library",
"//crypto/ecdsa:go_default_library",
"//crypto/hash:go_default_library",
"//crypto/rand:go_default_library",
"//encoding/bytesutil:go_default_library",
"//network:go_default_library",
"//network/forks:go_default_library",

View File

@@ -11,6 +11,7 @@ import (
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/altair"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/crypto/hash"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
@@ -96,7 +97,12 @@ func (s *Service) BroadcastSyncCommitteeMessage(ctx context.Context, subnet uint
return nil
}
func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint64, att ethpb.Att, forkDigest [4]byte) {
func (s *Service) internalBroadcastAttestation(
ctx context.Context,
subnet uint64,
att ethpb.Att,
forkDigest [fieldparams.VersionLength]byte,
) {
_, span := trace.StartSpan(ctx, "p2p.internalBroadcastAttestation")
defer span.End()
ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline.
@@ -152,7 +158,7 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6
}
}
func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMsg *ethpb.SyncCommitteeMessage, forkDigest [4]byte) {
func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMsg *ethpb.SyncCommitteeMessage, forkDigest [fieldparams.VersionLength]byte) {
_, span := trace.StartSpan(ctx, "p2p.broadcastSyncCommittee")
defer span.End()
ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline.
@@ -228,7 +234,12 @@ func (s *Service) BroadcastBlob(ctx context.Context, subnet uint64, blob *ethpb.
return nil
}
func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blobSidecar *ethpb.BlobSidecar, forkDigest [4]byte) {
func (s *Service) internalBroadcastBlob(
ctx context.Context,
subnet uint64,
blobSidecar *ethpb.BlobSidecar,
forkDigest [fieldparams.VersionLength]byte,
) {
_, span := trace.StartSpan(ctx, "p2p.internalBroadcastBlob")
defer span.End()
ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline.
@@ -243,7 +254,7 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob
s.subnetLocker(wrappedSubIdx).RUnlock()
if !hasPeer {
blobSidecarCommitteeBroadcastAttempts.Inc()
blobSidecarBroadcastAttempts.Inc()
if err := func() error {
s.subnetLocker(wrappedSubIdx).Lock()
defer s.subnetLocker(wrappedSubIdx).Unlock()
@@ -252,7 +263,7 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob
return err
}
if ok {
blobSidecarCommitteeBroadcasts.Inc()
blobSidecarBroadcasts.Inc()
return nil
}
return errors.New("failed to find peers for subnet")
@@ -268,6 +279,99 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob
}
}
// BroadcastDataColumn broadcasts a data column to the p2p network, the message is assumed to be
// broadcasted to the current fork and to the input column subnet.
// TODO: Add tests
func (s *Service) BroadcastDataColumn(ctx context.Context, columnSubnet uint64, dataColumnSidecar *ethpb.DataColumnSidecar) error {
// Add tracing to the function.
ctx, span := trace.StartSpan(ctx, "p2p.BroadcastBlob")
defer span.End()
// Ensure the data column sidecar is not nil.
if dataColumnSidecar == nil {
return errors.Errorf("attempted to broadcast nil data column sidecar at subnet %d", columnSubnet)
}
// Retrieve the current fork digest.
forkDigest, err := s.currentForkDigest()
if err != nil {
err := errors.Wrap(err, "current fork digest")
tracing.AnnotateError(span, err)
return err
}
// Non-blocking broadcast, with attempts to discover a column subnet peer if none available.
go s.internalBroadcastDataColumn(ctx, columnSubnet, dataColumnSidecar, forkDigest)
return nil
}
func (s *Service) internalBroadcastDataColumn(
ctx context.Context,
columnSubnet uint64,
dataColumnSidecar *ethpb.DataColumnSidecar,
forkDigest [fieldparams.VersionLength]byte,
) {
// Add tracing to the function.
_, span := trace.StartSpan(ctx, "p2p.internalBroadcastDataColumn")
defer span.End()
// Increase the number of broadcast attempts.
dataColumnSidecarBroadcastAttempts.Inc()
// Clear parent context / deadline.
ctx = trace.NewContext(context.Background(), span)
// Define a one-slot length context timeout.
oneSlot := time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
ctx, cancel := context.WithTimeout(ctx, oneSlot)
defer cancel()
// Build the topic corresponding to this column subnet and this fork digest.
topic := dataColumnSubnetToTopic(columnSubnet, forkDigest)
// Compute the wrapped subnet index.
wrappedSubIdx := columnSubnet + dataColumnSubnetVal
// Check if we have peers with this subnet.
hasPeer := func() bool {
s.subnetLocker(wrappedSubIdx).RLock()
defer s.subnetLocker(wrappedSubIdx).RUnlock()
return s.hasPeerWithSubnet(topic)
}()
// If no peers are found, attempt to find peers with this subnet.
if !hasPeer {
if err := func() error {
s.subnetLocker(wrappedSubIdx).Lock()
defer s.subnetLocker(wrappedSubIdx).Unlock()
ok, err := s.FindPeersWithSubnet(ctx, topic, columnSubnet, 1 /*threshold*/)
if err != nil {
return errors.Wrap(err, "find peers for subnet")
}
if ok {
return nil
}
return errors.New("failed to find peers for subnet")
}(); err != nil {
log.WithError(err).Error("Failed to find peers")
tracing.AnnotateError(span, err)
}
}
// Broadcast the data column sidecar to the network.
if err := s.broadcastObject(ctx, dataColumnSidecar, topic); err != nil {
log.WithError(err).Error("Failed to broadcast data column sidecar")
tracing.AnnotateError(span, err)
}
// Increase the number of successful broadcasts.
blobSidecarBroadcasts.Inc()
}
// method to broadcast messages to other peers in our gossip mesh.
func (s *Service) broadcastObject(ctx context.Context, obj ssz.Marshaler, topic string) error {
ctx, span := trace.StartSpan(ctx, "p2p.broadcastObject")
@@ -297,14 +401,18 @@ func (s *Service) broadcastObject(ctx context.Context, obj ssz.Marshaler, topic
return nil
}
func attestationToTopic(subnet uint64, forkDigest [4]byte) string {
func attestationToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string {
return fmt.Sprintf(AttestationSubnetTopicFormat, forkDigest, subnet)
}
func syncCommitteeToTopic(subnet uint64, forkDigest [4]byte) string {
func syncCommitteeToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string {
return fmt.Sprintf(SyncCommitteeSubnetTopicFormat, forkDigest, subnet)
}
func blobSubnetToTopic(subnet uint64, forkDigest [4]byte) string {
func blobSubnetToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string {
return fmt.Sprintf(BlobSubnetTopicFormat, forkDigest, subnet)
}
func dataColumnSubnetToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string {
return fmt.Sprintf(DataColumnSubnetTopicFormat, forkDigest, subnet)
}

View File

@@ -0,0 +1,83 @@
package p2p
import (
ssz "github.com/ferranbt/fastssz"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/sirupsen/logrus"
)
func (s *Service) GetValidCustodyPeers(peers []peer.ID) ([]peer.ID, error) {
custodiedSubnetCount := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodiedSubnetCount = params.BeaconConfig().DataColumnSidecarSubnetCount
}
custodiedColumns, err := peerdas.CustodyColumns(s.NodeID(), custodiedSubnetCount)
if err != nil {
return nil, err
}
var validPeers []peer.ID
for _, pid := range peers {
remoteCount := s.CustodyCountFromRemotePeer(pid)
nodeId, err := ConvertPeerIDToNodeID(pid)
if err != nil {
return nil, errors.Wrap(err, "convert peer ID to node ID")
}
remoteCustodiedColumns, err := peerdas.CustodyColumns(nodeId, remoteCount)
if err != nil {
return nil, errors.Wrap(err, "custody columns")
}
invalidPeer := false
for c := range custodiedColumns {
if !remoteCustodiedColumns[c] {
invalidPeer = true
break
}
}
if invalidPeer {
continue
}
copiedId := pid
// Add valid peer to list
validPeers = append(validPeers, copiedId)
}
return validPeers, nil
}
func (s *Service) CustodyCountFromRemotePeer(pid peer.ID) uint64 {
// By default, we assume the peer custodies the minimum number of subnets.
peerCustodyCountCount := params.BeaconConfig().CustodyRequirement
// Retrieve the ENR of the peer.
peerRecord, err := s.peers.ENR(pid)
if err != nil {
log.WithError(err).WithField("peerID", pid).Error("Failed to retrieve ENR for peer")
return peerCustodyCountCount
}
if peerRecord == nil {
// This is the case for inbound peers. So we don't log an error for this.
log.WithField("peerID", pid).Debug("No ENR found for peer")
return peerCustodyCountCount
}
// Load the `custody_subnet_count`
custodyObj := CustodySubnetCount(make([]byte, 8))
if err := peerRecord.Load(&custodyObj); err != nil {
log.WithField("peerID", pid).Error("Cannot load the custody_subnet_count from peer")
return peerCustodyCountCount
}
// Unmarshal the custody count from the peer's ENR.
peerCustodyCountFromRecord := ssz.UnmarshallUint64(custodyObj)
log.WithFields(logrus.Fields{
"peerID": pid,
"custodyCount": peerCustodyCountFromRecord,
}).Debug("Custody count read from peer's ENR")
return peerCustodyCountFromRecord
}

View File

@@ -14,6 +14,7 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
ma "github.com/multiformats/go-multiaddr"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
@@ -42,15 +43,21 @@ const (
udp6
)
type quicProtocol uint16
type (
quicProtocol uint16
CustodySubnetCount []byte
)
// quicProtocol is the "quic" key, which holds the QUIC port of the node.
func (quicProtocol) ENRKey() string { return "quic" }
// RefreshENR uses an epoch to refresh the enr entry for our node
// with the tracked committee ids for the epoch, allowing our node
// to be dynamically discoverable by others given our tracked committee ids.
func (s *Service) RefreshENR() {
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/p2p-interface.md#the-discovery-domain-discv5
func (CustodySubnetCount) ENRKey() string { return "custody_subnet_count" }
// RefreshPersistentSubnets checks that we are tracking our local persistent subnets for a variety of gossip topics.
// This routine checks for our attestation, sync committee and data column subnets and updates them if they have
// been rotated.
func (s *Service) RefreshPersistentSubnets() {
// return early if discv5 isnt running
if s.dv5Listener == nil || !s.isInitialized() {
return
@@ -60,6 +67,10 @@ func (s *Service) RefreshENR() {
log.WithError(err).Error("Could not initialize persistent subnets")
return
}
if err := initializePersistentColumnSubnets(s.dv5Listener.LocalNode().ID()); err != nil {
log.WithError(err).Error("Could not initialize persistent column subnets")
return
}
bitV := bitfield.NewBitvector64()
committees := cache.SubnetIDs.GetAllSubnets()
@@ -258,6 +269,20 @@ func (s *Service) createLocalNode(
localNode.Set(quicEntry)
}
if features.Get().EnablePeerDAS {
var custodyBytes []byte
custodyBytes = ssz.MarshalUint64(custodyBytes, params.BeaconConfig().CustodyRequirement)
custodySubnetEntry := CustodySubnetCount(custodyBytes)
if flags.Get().SubscribeToAllSubnets {
var allCustodyBytes []byte
allCustodyBytes = ssz.MarshalUint64(allCustodyBytes, params.BeaconConfig().DataColumnSidecarSubnetCount)
custodySubnetEntry = CustodySubnetCount(allCustodyBytes)
}
localNode.Set(custodySubnetEntry)
}
localNode.SetFallbackIP(ipAddr)
localNode.SetFallbackUDP(udpPort)
@@ -346,6 +371,8 @@ func (s *Service) filterPeer(node *enode.Node) bool {
// Ignore nodes that are already active.
if s.peers.IsActive(peerData.ID) {
// Constantly update enr for known peers
s.peers.UpdateENR(node.Record(), peerData.ID)
return false
}

View File

@@ -601,7 +601,7 @@ func TestRefreshENR_ForkBoundaries(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
s := tt.svcBuilder(t)
s.RefreshENR()
s.RefreshPersistentSubnets()
tt.postValidation(t, s)
s.dv5Listener.Close()
cache.SubnetIDs.EmptyAllCaches()

View File

@@ -121,7 +121,7 @@ func (s *Service) topicScoreParams(topic string) (*pubsub.TopicScoreParams, erro
return defaultAttesterSlashingTopicParams(), nil
case strings.Contains(topic, GossipBlsToExecutionChangeMessage):
return defaultBlsToExecutionChangeTopicParams(), nil
case strings.Contains(topic, GossipBlobSidecarMessage):
case strings.Contains(topic, GossipBlobSidecarMessage), strings.Contains(topic, GossipDataColumnSidecarMessage):
// TODO(Deneb): Using the default block scoring. But this should be updated.
return defaultBlockTopicParams(), nil
default:

View File

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

View File

@@ -3,6 +3,7 @@ package p2p
import (
"context"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/connmgr"
@@ -28,6 +29,12 @@ type P2P interface {
ConnectionHandler
PeersProvider
MetadataProvider
CustodyHandler
}
type Acceser interface {
Broadcaster
PeerManager
}
// Broadcaster broadcasts messages to peers over the p2p pubsub protocol.
@@ -36,6 +43,7 @@ type Broadcaster interface {
BroadcastAttestation(ctx context.Context, subnet uint64, att ethpb.Att) error
BroadcastSyncCommitteeMessage(ctx context.Context, subnet uint64, sMsg *ethpb.SyncCommitteeMessage) error
BroadcastBlob(ctx context.Context, subnet uint64, blob *ethpb.BlobSidecar) error
BroadcastDataColumn(ctx context.Context, columnSubnet uint64, dataColumnSidecar *ethpb.DataColumnSidecar) error
}
// SetStreamHandler configures p2p to handle streams of a certain topic ID.
@@ -81,8 +89,9 @@ type PeerManager interface {
PeerID() peer.ID
Host() host.Host
ENR() *enr.Record
NodeID() enode.ID
DiscoveryAddresses() ([]multiaddr.Multiaddr, error)
RefreshENR()
RefreshPersistentSubnets()
FindPeersWithSubnet(ctx context.Context, topic string, subIndex uint64, threshold int) (bool, error)
AddPingMethod(reqFunc func(ctx context.Context, id peer.ID) error)
}
@@ -102,3 +111,8 @@ type MetadataProvider interface {
Metadata() metadata.Metadata
MetadataSeq() uint64
}
type CustodyHandler interface {
CustodyCountFromRemotePeer(peer.ID) uint64
GetValidCustodyPeers([]peer.ID) ([]peer.ID, error)
}

View File

@@ -60,17 +60,21 @@ var (
"the subnet. The beacon node increments this counter when the broadcast is blocked " +
"until a subnet peer can be found.",
})
blobSidecarCommitteeBroadcasts = promauto.NewCounter(prometheus.CounterOpts{
blobSidecarBroadcasts = promauto.NewCounter(prometheus.CounterOpts{
Name: "p2p_blob_sidecar_committee_broadcasts",
Help: "The number of blob sidecar committee messages that were broadcast with no peer on.",
Help: "The number of blob sidecar messages that were broadcast with no peer on.",
})
syncCommitteeBroadcastAttempts = promauto.NewCounter(prometheus.CounterOpts{
Name: "p2p_sync_committee_subnet_attempted_broadcasts",
Help: "The number of sync committee that were attempted to be broadcast.",
})
blobSidecarCommitteeBroadcastAttempts = promauto.NewCounter(prometheus.CounterOpts{
blobSidecarBroadcastAttempts = promauto.NewCounter(prometheus.CounterOpts{
Name: "p2p_blob_sidecar_committee_attempted_broadcasts",
Help: "The number of blob sidecar committee messages that were attempted to be broadcast.",
Help: "The number of blob sidecar messages that were attempted to be broadcast.",
})
dataColumnSidecarBroadcastAttempts = promauto.NewCounter(prometheus.CounterOpts{
Name: "p2p_data_column_sidecar_attempted_broadcasts",
Help: "The number of data column sidecar messages that were attempted to be broadcast.",
})
// Gossip Tracer Metrics

View File

@@ -159,6 +159,14 @@ func (p *Status) Add(record *enr.Record, pid peer.ID, address ma.Multiaddr, dire
p.addIpToTracker(pid)
}
func (p *Status) UpdateENR(record *enr.Record, pid peer.ID) {
p.store.Lock()
defer p.store.Unlock()
if peerData, ok := p.store.PeerData(pid); ok {
peerData.Enr = record
}
}
// Address returns the multiaddress of the given remote peer.
// This will error if the peer does not exist.
func (p *Status) Address(pid peer.ID) (ma.Multiaddr, error) {

View File

@@ -165,14 +165,14 @@ func (s *Service) pubsubOptions() []pubsub.Option {
func parsePeersEnr(peers []string) ([]peer.AddrInfo, error) {
addrs, err := PeersFromStringAddrs(peers)
if err != nil {
return nil, fmt.Errorf("Cannot convert peers raw ENRs into multiaddresses: %v", err)
return nil, fmt.Errorf("cannot convert peers raw ENRs into multiaddresses: %v", err)
}
if len(addrs) == 0 {
return nil, fmt.Errorf("Converting peers raw ENRs into multiaddresses resulted in an empty list")
return nil, fmt.Errorf("converting peers raw ENRs into multiaddresses resulted in an empty list")
}
directAddrInfos, err := peer.AddrInfosFromP2pAddrs(addrs...)
if err != nil {
return nil, fmt.Errorf("Cannot convert peers multiaddresses into AddrInfos: %v", err)
return nil, fmt.Errorf("cannot convert peers multiaddresses into AddrInfos: %v", err)
}
return directAddrInfos, nil
}

View File

@@ -90,7 +90,7 @@ func TestService_CanSubscribe(t *testing.T) {
formatting := []interface{}{digest}
// Special case for attestation subnets which have a second formatting placeholder.
if topic == AttestationSubnetTopicFormat || topic == SyncCommitteeSubnetTopicFormat || topic == BlobSubnetTopicFormat {
if topic == AttestationSubnetTopicFormat || topic == SyncCommitteeSubnetTopicFormat || topic == BlobSubnetTopicFormat || topic == DataColumnSubnetTopicFormat {
formatting = append(formatting, 0 /* some subnet ID */)
}

View File

@@ -43,6 +43,12 @@ const BlobSidecarsByRangeName = "/blob_sidecars_by_range"
// BlobSidecarsByRootName is the name for the BlobSidecarsByRoot v1 message topic.
const BlobSidecarsByRootName = "/blob_sidecars_by_root"
// DataColumnSidecarsByRootName is the name for the DataColumnSidecarsByRoot v1 message topic.
const DataColumnSidecarsByRootName = "/data_column_sidecars_by_root"
// DataColumnSidecarsByRangeName is the name for the DataColumnSidecarsByRange v1 message topic.
const DataColumnSidecarsByRangeName = "/data_column_sidecars_by_range"
const (
// V1 RPC Topics
// RPCStatusTopicV1 defines the v1 topic for the status rpc method.
@@ -65,6 +71,12 @@ const (
// RPCBlobSidecarsByRootTopicV1 is a topic for requesting blob sidecars by their block root. New in deneb.
// /eth2/beacon_chain/req/blob_sidecars_by_root/1/
RPCBlobSidecarsByRootTopicV1 = protocolPrefix + BlobSidecarsByRootName + SchemaVersionV1
// RPCDataColumnSidecarsByRootTopicV1 is a topic for requesting data column sidecars by their block root. New in PeerDAS.
// /eth2/beacon_chain/req/data_column_sidecars_by_root/1
RPCDataColumnSidecarsByRootTopicV1 = protocolPrefix + DataColumnSidecarsByRootName + SchemaVersionV1
// RPCDataColumnSidecarsByRangeTopicV1 is a topic for requesting data column sidecars by their slot. New in PeerDAS.
// /eth2/beacon_chain/req/data_column_sidecars_by_range/1
RPCDataColumnSidecarsByRangeTopicV1 = protocolPrefix + DataColumnSidecarsByRangeName + SchemaVersionV1
// V2 RPC Topics
// RPCBlocksByRangeTopicV2 defines v2 the topic for the blocks by range rpc method.
@@ -101,6 +113,10 @@ var RPCTopicMappings = map[string]interface{}{
RPCBlobSidecarsByRangeTopicV1: new(pb.BlobSidecarsByRangeRequest),
// BlobSidecarsByRoot v1 Message
RPCBlobSidecarsByRootTopicV1: new(p2ptypes.BlobSidecarsByRootReq),
// DataColumnSidecarsByRange v1 Message
RPCDataColumnSidecarsByRangeTopicV1: new(pb.DataColumnSidecarsByRangeRequest),
// DataColumnSidecarsByRoot v1 Message
RPCDataColumnSidecarsByRootTopicV1: new(p2ptypes.BlobSidecarsByRootReq),
}
// Maps all registered protocol prefixes.
@@ -119,6 +135,8 @@ var messageMapping = map[string]bool{
MetadataMessageName: true,
BlobSidecarsByRangeName: true,
BlobSidecarsByRootName: true,
DataColumnSidecarsByRootName: true,
DataColumnSidecarsByRangeName: true,
}
// Maps all the RPC messages which are to updated in altair.

View File

@@ -226,7 +226,7 @@ func (s *Service) Start() {
}
// Initialize metadata according to the
// current epoch.
s.RefreshENR()
s.RefreshPersistentSubnets()
// Periodic functions.
async.RunEvery(s.ctx, params.BeaconConfig().TtfbTimeoutDuration(), func() {
@@ -234,7 +234,7 @@ func (s *Service) Start() {
})
async.RunEvery(s.ctx, 30*time.Minute, s.Peers().Prune)
async.RunEvery(s.ctx, time.Duration(params.BeaconConfig().RespTimeout)*time.Second, s.updateMetrics)
async.RunEvery(s.ctx, refreshRate, s.RefreshENR)
async.RunEvery(s.ctx, refreshRate, s.RefreshPersistentSubnets)
async.RunEvery(s.ctx, 1*time.Minute, func() {
inboundQUICCount := len(s.peers.InboundConnectedWithProtocol(peers.QUIC))
inboundTCPCount := len(s.peers.InboundConnectedWithProtocol(peers.TCP))
@@ -358,6 +358,15 @@ func (s *Service) ENR() *enr.Record {
return s.dv5Listener.Self().Record()
}
// NodeID returns the local node's node ID
// for discovery.
func (s *Service) NodeID() enode.ID {
if s.dv5Listener == nil {
return [32]byte{}
}
return s.dv5Listener.Self().ID()
}
// DiscoveryAddresses represents our enr addresses as multiaddresses.
func (s *Service) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
if s.dv5Listener == nil {

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/consensus-types/wrapper"
@@ -34,8 +35,8 @@ var syncCommsSubnetEnrKey = params.BeaconNetworkConfig().SyncCommsSubnetKey
// The value used with the subnet, inorder
// to create an appropriate key to retrieve
// the relevant lock. This is used to differentiate
// sync subnets from attestation subnets. This is deliberately
// chosen as more than 64(attestation subnet count).
// sync subnets from others. This is deliberately
// chosen as more than 64 (attestation subnet count).
const syncLockerVal = 100
// The value used with the blob sidecar subnet, in order
@@ -45,6 +46,13 @@ const syncLockerVal = 100
// chosen more than sync and attestation subnet combined.
const blobSubnetLockerVal = 110
// The value used with the data column sidecar subnet, in order
// to create an appropriate key to retrieve
// the relevant lock. This is used to differentiate
// data column subnets from others. This is deliberately
// chosen more than sync, attestation and blob subnet (6) combined.
const dataColumnSubnetVal = 150
// FindPeersWithSubnet performs a network search for peers
// subscribed to a particular subnet. Then it tries to connect
// with those peers. This method will block until either:
@@ -207,6 +215,20 @@ func initializePersistentSubnets(id enode.ID, epoch primitives.Epoch) error {
return nil
}
func initializePersistentColumnSubnets(id enode.ID) error {
_, ok, expTime := cache.ColumnSubnetIDs.GetColumnSubnets()
if ok && expTime.After(time.Now()) {
return nil
}
subnetsId, err := peerdas.CustodyColumnSubnets(id, params.BeaconConfig().CustodyRequirement)
if err != nil {
return err
}
cache.ColumnSubnetIDs.AddColumnSubnets(subnetsId)
return nil
}
// Spec pseudocode definition:
//
// def compute_subscribed_subnets(node_id: NodeID, epoch: Epoch) -> Sequence[SubnetID]:
@@ -356,10 +378,11 @@ func syncBitvector(record *enr.Record) (bitfield.Bitvector4, error) {
// The subnet locker is a map which keeps track of all
// mutexes stored per subnet. This locker is re-used
// between both the attestation and sync subnets. In
// order to differentiate between attestation and sync
// subnets. Sync subnets are stored by (subnet+syncLockerVal). This
// is to prevent conflicts while allowing both subnets
// between both the attestation, sync and blob subnets.
// Sync subnets are stored by (subnet+syncLockerVal).
// Blob subnets are stored by (subnet+blobSubnetLockerVal).
// Data column subnets are stored by (subnet+dataColumnSubnetVal).
// This is to prevent conflicts while allowing subnets
// to use a single locker.
func (s *Service) subnetLocker(i uint64) *sync.RWMutex {
s.subnetsLockLock.Lock()

View File

@@ -3,6 +3,7 @@ package testing
import (
"context"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/control"
@@ -55,6 +56,11 @@ func (_ *FakeP2P) ENR() *enr.Record {
return new(enr.Record)
}
// NodeID returns the node id of the local peer.
func (_ *FakeP2P) NodeID() enode.ID {
return [32]byte{}
}
// DiscoveryAddresses -- fake
func (_ *FakeP2P) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
return nil, nil
@@ -66,7 +72,7 @@ func (_ *FakeP2P) FindPeersWithSubnet(_ context.Context, _ string, _ uint64, _ i
}
// RefreshENR mocks the p2p func.
func (_ *FakeP2P) RefreshENR() {}
func (_ *FakeP2P) RefreshPersistentSubnets() {}
// LeaveTopic -- fake.
func (_ *FakeP2P) LeaveTopic(_ string) error {
@@ -148,6 +154,11 @@ func (_ *FakeP2P) BroadcastBlob(_ context.Context, _ uint64, _ *ethpb.BlobSideca
return nil
}
// BroadcastDataColumn -- fake.
func (_ *FakeP2P) BroadcastDataColumn(_ context.Context, _ uint64, _ *ethpb.DataColumnSidecar) error {
return nil
}
// InterceptPeerDial -- fake.
func (_ *FakeP2P) InterceptPeerDial(peer.ID) (allow bool) {
return true
@@ -172,3 +183,11 @@ func (_ *FakeP2P) InterceptSecured(network.Direction, peer.ID, network.ConnMulti
func (_ *FakeP2P) InterceptUpgraded(network.Conn) (allow bool, reason control.DisconnectReason) {
return true, 0
}
func (_ *FakeP2P) CustodyCountFromRemotePeer(peer.ID) uint64 {
return 0
}
func (_ *FakeP2P) GetValidCustodyPeers(peers []peer.ID) ([]peer.ID, error) {
return peers, nil
}

View File

@@ -48,6 +48,12 @@ func (m *MockBroadcaster) BroadcastBlob(context.Context, uint64, *ethpb.BlobSide
return nil
}
// BroadcastDataColumn broadcasts a data column for mock.
func (m *MockBroadcaster) BroadcastDataColumn(context.Context, uint64, *ethpb.DataColumnSidecar) error {
m.BroadcastCalled.Store(true)
return nil
}
// NumMessages returns the number of messages broadcasted.
func (m *MockBroadcaster) NumMessages() int {
m.msgLock.Lock()

View File

@@ -4,6 +4,7 @@ import (
"context"
"errors"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
"github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p/core/peer"
@@ -39,6 +40,11 @@ func (m MockPeerManager) ENR() *enr.Record {
return m.Enr
}
// NodeID .
func (m MockPeerManager) NodeID() enode.ID {
return [32]byte{}
}
// DiscoveryAddresses .
func (m MockPeerManager) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
if m.FailDiscoveryAddr {
@@ -48,7 +54,7 @@ func (m MockPeerManager) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
}
// RefreshENR .
func (_ MockPeerManager) RefreshENR() {}
func (_ MockPeerManager) RefreshPersistentSubnets() {}
// FindPeersWithSubnet .
func (_ MockPeerManager) FindPeersWithSubnet(_ context.Context, _ string, _ uint64, _ int) (bool, error) {

View File

@@ -10,6 +10,7 @@ import (
"testing"
"time"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
pubsub "github.com/libp2p/go-libp2p-pubsub"
core "github.com/libp2p/go-libp2p/core"
@@ -183,6 +184,12 @@ func (p *TestP2P) BroadcastBlob(context.Context, uint64, *ethpb.BlobSidecar) err
return nil
}
// BroadcastDataColumn broadcasts a data column for mock.
func (p *TestP2P) BroadcastDataColumn(context.Context, uint64, *ethpb.DataColumnSidecar) error {
p.BroadcastCalled.Store(true)
return nil
}
// SetStreamHandler for RPC.
func (p *TestP2P) SetStreamHandler(topic string, handler network.StreamHandler) {
p.BHost.SetStreamHandler(protocol.ID(topic), handler)
@@ -263,6 +270,11 @@ func (_ *TestP2P) ENR() *enr.Record {
return new(enr.Record)
}
// NodeID returns the node id of the local peer.
func (_ *TestP2P) NodeID() enode.ID {
return [32]byte{}
}
// DiscoveryAddresses --
func (_ *TestP2P) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
return nil, nil
@@ -361,7 +373,7 @@ func (_ *TestP2P) FindPeersWithSubnet(_ context.Context, _ string, _ uint64, _ i
}
// RefreshENR mocks the p2p func.
func (_ *TestP2P) RefreshENR() {}
func (_ *TestP2P) RefreshPersistentSubnets() {}
// ForkDigest mocks the p2p func.
func (p *TestP2P) ForkDigest() ([4]byte, error) {
@@ -407,3 +419,11 @@ func (_ *TestP2P) InterceptSecured(network.Direction, peer.ID, network.ConnMulti
func (_ *TestP2P) InterceptUpgraded(network.Conn) (allow bool, reason control.DisconnectReason) {
return true, 0
}
func (_ *TestP2P) CustodyCountFromRemotePeer(peer.ID) uint64 {
return 0
}
func (_ *TestP2P) GetValidCustodyPeers(peers []peer.ID) ([]peer.ID, error) {
return peers, nil
}

View File

@@ -30,6 +30,9 @@ const (
GossipBlsToExecutionChangeMessage = "bls_to_execution_change"
// GossipBlobSidecarMessage is the name for the blob sidecar message type.
GossipBlobSidecarMessage = "blob_sidecar"
// GossipDataColumnSidecarMessage is the name for the data column sidecar message type.
GossipDataColumnSidecarMessage = "data_column_sidecar"
// Topic Formats
//
// AttestationSubnetTopicFormat is the topic format for the attestation subnet.
@@ -52,4 +55,6 @@ const (
BlsToExecutionChangeSubnetTopicFormat = GossipProtocolAndDigest + GossipBlsToExecutionChangeMessage
// BlobSubnetTopicFormat is the topic format for the blob subnet.
BlobSubnetTopicFormat = GossipProtocolAndDigest + GossipBlobSidecarMessage + "_%d"
// DataColumnSubnetTopicFormat is the topic format for the data column subnet.
DataColumnSubnetTopicFormat = GossipProtocolAndDigest + GossipDataColumnSidecarMessage + "_%d"
)

View File

@@ -9,10 +9,15 @@ var (
ErrInvalidSequenceNum = errors.New("invalid sequence number provided")
ErrGeneric = errors.New("internal service error")
ErrRateLimited = errors.New("rate limited")
ErrIODeadline = errors.New("i/o deadline exceeded")
ErrInvalidRequest = errors.New("invalid range, step or count")
ErrBlobLTMinRequest = errors.New("blob slot < minimum_request_epoch")
ErrMaxBlobReqExceeded = errors.New("requested more than MAX_REQUEST_BLOB_SIDECARS")
ErrRateLimited = errors.New("rate limited")
ErrIODeadline = errors.New("i/o deadline exceeded")
ErrInvalidRequest = errors.New("invalid range, step or count")
ErrBlobLTMinRequest = errors.New("blob epoch < minimum_request_epoch")
ErrDataColumnLTMinRequest = errors.New("data column epoch < minimum_request_epoch")
ErrMaxBlobReqExceeded = errors.New("requested more than MAX_REQUEST_BLOB_SIDECARS")
ErrMaxDataColumnReqExceeded = errors.New("requested more than MAX_REQUEST_DATA_COLUMN_SIDECARS")
ErrResourceUnavailable = errors.New("resource requested unavailable")
ErrInvalidColumnIndex = errors.New("invalid column index requested")
)

View File

@@ -6,16 +6,25 @@ import (
"crypto/rand"
"encoding/base64"
"encoding/hex"
"encoding/json"
"fmt"
"net"
"os"
"path"
"time"
"github.com/btcsuite/btcd/btcec/v2"
gCrypto "github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/ethereum/go-ethereum/p2p/enr"
"github.com/libp2p/go-libp2p/core/crypto"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/wrapper"
ecdsaprysm "github.com/prysmaticlabs/prysm/v5/crypto/ecdsa"
"github.com/prysmaticlabs/prysm/v5/io/file"
@@ -25,10 +34,13 @@ import (
"google.golang.org/protobuf/proto"
)
const keyPath = "network-keys"
const metaDataPath = "metaData"
const (
keyPath = "network-keys"
custodyColumnSubnetsPath = "custodyColumnsSubnets.json"
metaDataPath = "metaData"
const dialTimeout = 1 * time.Second
dialTimeout = 1 * time.Second
)
// SerializeENR takes the enr record in its key-value form and serializes it.
func SerializeENR(record *enr.Record) (string, error) {
@@ -43,32 +55,235 @@ func SerializeENR(record *enr.Record) (string, error) {
return enrString, nil
}
// Determines a private key for p2p networking from the p2p service's
// randomPrivKeyWithSubnets generates a random private key which, when derived into a node ID, matches expectedSubnets.
// This is done by brute forcing the generation of a private key until it matches the desired subnets.
// TODO: Run multiple goroutines to speed up the process.
func randomPrivKeyWithSubnets(expectedSubnets map[uint64]bool) (crypto.PrivKey, uint64, time.Duration, error) {
// Get the current time.
start := time.Now()
mainLoop:
for i := uint64(1); ; /* No exit condition */ i++ {
// Get the subnets count.
expectedSubnetsCount := len(expectedSubnets)
// Generate a random keys pair
privKey, _, err := crypto.GenerateSecp256k1Key(rand.Reader)
if err != nil {
return nil, 0, time.Duration(0), errors.Wrap(err, "generate SECP256K1 key")
}
ecdsaPrivKey, err := ecdsaprysm.ConvertFromInterfacePrivKey(privKey)
if err != nil {
return nil, 0, time.Duration(0), errors.Wrap(err, "convert from interface private key")
}
// Compute the node ID from the public key.
nodeID := enode.PubkeyToIDV4(&ecdsaPrivKey.PublicKey)
// Retrieve the custody column subnets of the node.
actualSubnets, err := peerdas.CustodyColumnSubnets(nodeID, uint64(expectedSubnetsCount))
if err != nil {
return nil, 0, time.Duration(0), errors.Wrap(err, "custody column subnets")
}
// Safe check, just in case.
actualSubnetsCount := len(actualSubnets)
if actualSubnetsCount != expectedSubnetsCount {
return nil, 0, time.Duration(0), errors.Errorf("mismatch counts of custody subnets. Actual %d - Required %d", actualSubnetsCount, expectedSubnetsCount)
}
// Check if the expected subnets are the same as the actual subnets.
for _, subnet := range actualSubnets {
if !expectedSubnets[subnet] {
// At least one subnet does not match, so we need to generate a new key.
continue mainLoop
}
}
// It's a match, return the private key.
return privKey, i, time.Since(start), nil
}
}
// privateKeyWithConstraint reads the subnets from a file and generates a private key that matches the subnets.
func privateKeyWithConstraint(subnetsPath string) (crypto.PrivKey, error) {
// Read the subnets from the file.
data, err := file.ReadFileAsBytes(subnetsPath)
if err != nil {
return nil, errors.Wrapf(err, "read file %s", subnetsPath)
}
var storedSubnets []uint64
if err := json.Unmarshal(data, &storedSubnets); err != nil {
return nil, errors.Wrapf(err, "unmarshal subnets %s", subnetsPath)
}
storedSubnetsCount := uint64(len(storedSubnets))
// Retrieve the subnets to custody.
custodySubnetsCount := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodySubnetsCount = params.BeaconConfig().DataColumnSidecarSubnetCount
}
// Check our subnets count is not greater than the subnet count in the file.
// Such a case is possible if the number of subnets increased after the file was created.
// This is possible only within a new release. If this happens, we should implement a modification
// of the file. At the moment, we raise an error.
if custodySubnetsCount > storedSubnetsCount {
return nil, errors.Errorf(
"subnets count in the file %s (%d) is less than the current subnets count (%d)",
subnetsPath,
storedSubnetsCount,
custodySubnetsCount,
)
}
subnetsMap := make(map[uint64]bool, custodySubnetsCount)
custodySubnetsMap := make(map[uint64]bool, len(storedSubnets))
for i, subnet := range storedSubnets {
subnetsMap[subnet] = true
if uint64(i) < custodySubnetsCount {
custodySubnetsMap[subnet] = true
}
}
if len(subnetsMap) != len(storedSubnets) {
return nil, errors.Errorf("duplicated subnets found in the file %s", subnetsPath)
}
// Generate a private key that matches the subnets.
privKey, iterations, duration, err := randomPrivKeyWithSubnets(custodySubnetsMap)
log.WithFields(logrus.Fields{
"iterations": iterations,
"duration": duration,
}).Info("Generated P2P private key")
return privKey, err
}
// privateKeyWithoutConstraint generates a private key, computes the subnets and stores them in a file.
func privateKeyWithoutConstraint(subnetsPath string) (crypto.PrivKey, error) {
// Get the total number of subnets.
subnetCount := params.BeaconConfig().DataColumnSidecarSubnetCount
// Generate the private key.
privKey, _, err := crypto.GenerateSecp256k1Key(rand.Reader)
if err != nil {
return nil, errors.Wrap(err, "generate SECP256K1 key")
}
convertedKey, err := ecdsaprysm.ConvertFromInterfacePrivKey(privKey)
if err != nil {
return nil, errors.Wrap(err, "convert from interface private key")
}
// Compute the node ID from the public key.
nodeID := enode.PubkeyToIDV4(&convertedKey.PublicKey)
// Retrieve the custody column subnets of the node.
subnets, err := peerdas.CustodyColumnSubnets(nodeID, subnetCount)
if err != nil {
return nil, errors.Wrap(err, "custody column subnets")
}
// Store the subnets in a file.
data, err := json.Marshal(subnets)
if err != nil {
return nil, errors.Wrap(err, "marshal subnets")
}
if err := file.WriteFile(subnetsPath, data); err != nil {
return nil, errors.Wrap(err, "write file")
}
return privKey, nil
}
// storePrivateKey stores a private key to a file.
func storePrivateKey(privKey crypto.PrivKey, destFilePath string) error {
// Get the raw bytes of the private key.
rawbytes, err := privKey.Raw()
if err != nil {
return errors.Wrap(err, "raw")
}
// Encode the raw bytes to hex.
dst := make([]byte, hex.EncodedLen(len(rawbytes)))
hex.Encode(dst, rawbytes)
if err := file.WriteFile(destFilePath, dst); err != nil {
return errors.Wrapf(err, "write file: %s", destFilePath)
}
return err
}
// randomPrivKey generates a random private key.
func randomPrivKey(datadir string) (crypto.PrivKey, error) {
if features.Get().EnablePeerDAS {
// Check if the file containing the custody column subnets exists.
subnetsPath := path.Join(datadir, custodyColumnSubnetsPath)
exists, err := file.Exists(subnetsPath, file.Regular)
if err != nil {
return nil, errors.Wrap(err, "exists")
}
// If the file does not exist, generate a new private key, compute the subnets and store them.
if !exists {
priv, err := privateKeyWithoutConstraint(subnetsPath)
if err != nil {
return nil, errors.Wrap(err, "generate private without constraint")
}
return priv, nil
}
// If the file exists, read the subnets and generate a new private key.
priv, err := privateKeyWithConstraint(subnetsPath)
if err != nil {
return nil, errors.Wrap(err, "generate private key with constraint for PeerDAS")
}
return priv, nil
}
privKey, _, err := crypto.GenerateSecp256k1Key(rand.Reader)
if err != nil {
return nil, errors.Wrap(err, "generate SECP256K1 key")
}
return privKey, err
}
// privKey determines a private key for p2p networking from the p2p service's
// configuration struct. If no key is found, it generates a new one.
func privKey(cfg *Config) (*ecdsa.PrivateKey, error) {
defaultKeyPath := path.Join(cfg.DataDir, keyPath)
privateKeyPath := cfg.PrivateKey
// PrivateKey cli flag takes highest precedence.
// PrivateKey CLI flag takes highest precedence.
if privateKeyPath != "" {
return privKeyFromFile(cfg.PrivateKey)
}
// Default keys have the next highest precedence, if they exist.
_, err := os.Stat(defaultKeyPath)
defaultKeysExist := !os.IsNotExist(err)
if err != nil && defaultKeysExist {
return nil, err
defaultKeysExist, err := file.Exists(defaultKeyPath, file.Regular)
if err != nil {
return nil, errors.Wrap(err, "exists")
}
if defaultKeysExist {
log.WithField("filePath", defaultKeyPath).Info("Reading static P2P private key from a file. To generate a new random private key at every start, please remove this file.")
return privKeyFromFile(defaultKeyPath)
}
// There are no keys on the filesystem, so we need to generate one.
priv, _, err := crypto.GenerateSecp256k1Key(rand.Reader)
// Generate a new (possibly contrained) random private key.
priv, err := randomPrivKey(cfg.DataDir)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "random private key")
}
// If the StaticPeerID flag is not set, return the private key.
@@ -78,21 +293,19 @@ func privKey(cfg *Config) (*ecdsa.PrivateKey, error) {
// Save the generated key as the default key, so that it will be used by
// default on the next node start.
rawbytes, err := priv.Raw()
if err != nil {
return nil, err
log.WithField("file", defaultKeyPath).Info("Wrote network key to")
if err := storePrivateKey(priv, defaultKeyPath); err != nil {
return nil, errors.Wrap(err, "store private key")
}
dst := make([]byte, hex.EncodedLen(len(rawbytes)))
hex.Encode(dst, rawbytes)
if err := file.WriteFile(defaultKeyPath, dst); err != nil {
return nil, err
}
log.Info("Wrote network key to file")
// Read the key from the defaultKeyPath file just written
// for the strongest guarantee that the next start will be the same as this one.
return privKeyFromFile(defaultKeyPath)
privKey, err := privKeyFromFile(defaultKeyPath)
if err != nil {
return nil, errors.Wrap(err, "private key from file")
}
return privKey, nil
}
// Retrieves a p2p networking private key from a file path.
@@ -173,3 +386,23 @@ func verifyConnectivity(addr string, port uint, protocol string) {
}
}
}
func ConvertPeerIDToNodeID(pid peer.ID) (enode.ID, error) {
// Retrieve the public key object of the peer under "crypto" form.
pubkeyObjCrypto, err := pid.ExtractPublicKey()
if err != nil {
return [32]byte{}, errors.Wrap(err, "extract public key")
}
// Extract the bytes representation of the public key.
compressedPubKeyBytes, err := pubkeyObjCrypto.Raw()
if err != nil {
return [32]byte{}, errors.Wrap(err, "public key raw")
}
// Retrieve the public key object of the peer under "SECP256K1" form.
pubKeyObjSecp256k1, err := btcec.ParsePubKey(compressedPubKeyBytes)
if err != nil {
return [32]byte{}, errors.Wrap(err, "parse public key")
}
newPubkey := &ecdsa.PublicKey{Curve: gCrypto.S256(), X: pubKeyObjSecp256k1.X(), Y: pubKeyObjSecp256k1.Y()}
return enode.PubkeyToIDV4(newPubkey), nil
}

View File

@@ -6,12 +6,110 @@ import (
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/p2p/enode"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/config/params"
ecdsaprysm "github.com/prysmaticlabs/prysm/v5/crypto/ecdsa"
"github.com/prysmaticlabs/prysm/v5/crypto/rand"
"github.com/prysmaticlabs/prysm/v5/testing/assert"
"github.com/prysmaticlabs/prysm/v5/testing/require"
logTest "github.com/sirupsen/logrus/hooks/test"
)
// generateRandomSubnets generates a set of `count` random subnets.
func generateRandomSubnets(requestedCount, totalSubnetsCount uint64) map[uint64]bool {
// Populate all the subnets.
subnets := make(map[uint64]bool, totalSubnetsCount)
for i := uint64(0); i < totalSubnetsCount; i++ {
subnets[i] = true
}
// Get a random generator.
randGen := rand.NewGenerator()
// Randomly delete subnets until we have the desired count.
for uint64(len(subnets)) > requestedCount {
// Get a random subnet.
subnet := randGen.Uint64() % totalSubnetsCount
// Delete the subnet.
delete(subnets, subnet)
}
return subnets
}
func TestRandomPrivKeyWithConstraint(t *testing.T) {
// Get the total number of subnets.
totalSubnetsCount := params.BeaconConfig().DataColumnSidecarSubnetCount
// We generate only tests for a low and high number of subnets to minimize computation, as explained here:
// https://hackmd.io/@6-HLeMXARN2tdFLKKcqrxw/BJVSxU7VC
testCases := []struct {
name string
expectedSubnetsCount uint64
expectedError bool
}{
{
name: "0 subnet - n subnets",
expectedSubnetsCount: 0,
},
{
name: "1 subnet - n-1 subnets",
expectedSubnetsCount: 1,
},
{
name: "2 subnets - n-2 subnets",
expectedSubnetsCount: 2,
},
{
name: "3 subnets - n-3 subnets",
expectedSubnetsCount: 3,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
expectedSubnetsList := []map[uint64]bool{
generateRandomSubnets(tc.expectedSubnetsCount, totalSubnetsCount),
generateRandomSubnets(totalSubnetsCount-tc.expectedSubnetsCount, totalSubnetsCount),
}
for _, expectedSubnets := range expectedSubnetsList {
// Determine the number of expected subnets.
expectedSubnetsCount := uint64(len(expectedSubnets))
// Determine the private key that matches the expected subnets.
privateKey, iterationsCount, _, err := randomPrivKeyWithSubnets(expectedSubnets)
require.NoError(t, err)
// Sanity check the number of iterations.
assert.Equal(t, true, iterationsCount > 0)
// Compute the node ID from the public key.
ecdsaPrivKey, err := ecdsaprysm.ConvertFromInterfacePrivKey(privateKey)
require.NoError(t, err)
nodeID := enode.PubkeyToIDV4(&ecdsaPrivKey.PublicKey)
// Retrieve the subnets from the node ID.
actualSubnets, err := peerdas.CustodyColumnSubnets(nodeID, expectedSubnetsCount)
require.NoError(t, err)
// Determine the number of actual subnets.
actualSubnetsCounts := uint64(len(actualSubnets))
// Check the count of the actual subnets against the expected subnets.
assert.Equal(t, expectedSubnetsCount, actualSubnetsCounts)
// Check the actual subnets against the expected subnets.
for _, subnet := range actualSubnets {
assert.Equal(t, true, expectedSubnets[subnet])
}
}
})
}
}
// Test `verifyConnectivity` function by trying to connect to google.com (successfully)
// and then by connecting to an unreachable IP and ensuring that a log is emitted
func TestVerifyConnectivity(t *testing.T) {

View File

@@ -1,3 +1,5 @@
# gazelle:ignore
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
@@ -44,6 +46,7 @@ go_library(
"//beacon-chain/core/feed/operation:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",
@@ -88,6 +91,7 @@ go_library(
"//runtime/version:go_default_library",
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_c_kzg_4844//bindings/go:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_golang_protobuf//ptypes/empty",
@@ -173,7 +177,6 @@ common_deps = [
"@org_golang_google_protobuf//types/known/emptypb:go_default_library",
]
# gazelle:ignore
go_test(
name = "go_default_test",
timeout = "moderate",

View File

@@ -7,6 +7,7 @@ import (
"sync"
"time"
cKzg4844 "github.com/ethereum/c-kzg-4844/bindings/go"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
emptypb "github.com/golang/protobuf/ptypes/empty"
@@ -19,9 +20,12 @@ import (
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/kv"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -261,7 +265,15 @@ func (vs *Server) BuildBlockParallel(ctx context.Context, sBlk interfaces.Signed
}
// ProposeBeaconBlock handles the proposal of beacon blocks.
// TODO: Add tests
func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error) {
var (
blobSidecars []*ethpb.BlobSidecar
dataColumnSideCars []*ethpb.DataColumnSidecar
)
isPeerDASEnabled := features.Get().EnablePeerDAS
ctx, span := trace.StartSpan(ctx, "ProposerServer.ProposeBeaconBlock")
defer span.End()
@@ -274,14 +286,16 @@ func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSign
return nil, status.Errorf(codes.InvalidArgument, "%s: %v", "decode block failed", err)
}
var sidecars []*ethpb.BlobSidecar
if block.IsBlinded() {
block, sidecars, err = vs.handleBlindedBlock(ctx, block)
block, blobSidecars, dataColumnSideCars, err = vs.handleBlindedBlock(ctx, block, isPeerDASEnabled)
if err != nil {
return nil, status.Errorf(codes.Internal, "%s: %v", "handle blinded block", err)
}
} else {
sidecars, err = vs.handleUnblindedBlock(block, req)
}
if err != nil {
return nil, status.Errorf(codes.Internal, "%s: %v", "handle block failed", err)
blobSidecars, dataColumnSideCars, err = handleUnblindedBlock(block, req, isPeerDASEnabled)
if err != nil {
return nil, status.Errorf(codes.Internal, "%s: %v", "handle unblided block", err)
}
}
root, err := block.Block().HashTreeRoot()
@@ -302,8 +316,14 @@ func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSign
errChan <- nil
}()
if err := vs.broadcastAndReceiveBlobs(ctx, sidecars, root); err != nil {
return nil, status.Errorf(codes.Internal, "Could not broadcast/receive blobs: %v", err)
if isPeerDASEnabled {
if err := vs.broadcastAndReceiveDataColumns(ctx, dataColumnSideCars, root); err != nil {
return nil, status.Errorf(codes.Internal, "Could not broadcast/receive data columns: %v", err)
}
} else {
if err := vs.broadcastAndReceiveBlobs(ctx, blobSidecars, root); err != nil {
return nil, status.Errorf(codes.Internal, "Could not broadcast/receive blobs: %v", err)
}
}
wg.Wait()
@@ -315,47 +335,83 @@ func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSign
}
// handleBlindedBlock processes blinded beacon blocks.
func (vs *Server) handleBlindedBlock(ctx context.Context, block interfaces.SignedBeaconBlock) (interfaces.SignedBeaconBlock, []*ethpb.BlobSidecar, error) {
func (vs *Server) handleBlindedBlock(ctx context.Context, block interfaces.SignedBeaconBlock, isPeerDASEnabled bool) (interfaces.SignedBeaconBlock, []*ethpb.BlobSidecar, []*ethpb.DataColumnSidecar, error) {
if block.Version() < version.Bellatrix {
return nil, nil, errors.New("pre-Bellatrix blinded block")
return nil, nil, nil, errors.New("pre-Bellatrix blinded block")
}
if vs.BlockBuilder == nil || !vs.BlockBuilder.Configured() {
return nil, nil, errors.New("unconfigured block builder")
return nil, nil, nil, errors.New("unconfigured block builder")
}
copiedBlock, err := block.Copy()
if err != nil {
return nil, nil, err
return nil, nil, nil, errors.Wrap(err, "block copy")
}
payload, bundle, err := vs.BlockBuilder.SubmitBlindedBlock(ctx, block)
if err != nil {
return nil, nil, errors.Wrap(err, "submit blinded block failed")
return nil, nil, nil, errors.Wrap(err, "submit blinded block")
}
if err := copiedBlock.Unblind(payload); err != nil {
return nil, nil, errors.Wrap(err, "unblind failed")
return nil, nil, nil, errors.Wrap(err, "unblind")
}
sidecars, err := unblindBlobsSidecars(copiedBlock, bundle)
if isPeerDASEnabled {
dataColumnSideCars, err := unblindDataColumnsSidecars(copiedBlock, bundle)
if err != nil {
return nil, nil, nil, errors.Wrap(err, "unblind data columns sidecars")
}
return copiedBlock, nil, dataColumnSideCars, nil
}
blobSidecars, err := unblindBlobsSidecars(copiedBlock, bundle)
if err != nil {
return nil, nil, errors.Wrap(err, "unblind sidecars failed")
return nil, nil, nil, errors.Wrap(err, "unblind blobs sidecars")
}
return copiedBlock, sidecars, nil
return copiedBlock, blobSidecars, nil, nil
}
// handleUnblindedBlock processes unblinded beacon blocks.
func (vs *Server) handleUnblindedBlock(block interfaces.SignedBeaconBlock, req *ethpb.GenericSignedBeaconBlock) ([]*ethpb.BlobSidecar, error) {
func handleUnblindedBlock(block interfaces.SignedBeaconBlock, req *ethpb.GenericSignedBeaconBlock, isPeerDASEnabled bool) ([]*ethpb.BlobSidecar, []*ethpb.DataColumnSidecar, error) {
dbBlockContents := req.GetDeneb()
if dbBlockContents == nil {
return nil, nil
return nil, nil, nil
}
return BuildBlobSidecars(block, dbBlockContents.Blobs, dbBlockContents.KzgProofs)
if isPeerDASEnabled {
// Convert blobs from slices to array.
blobs := make([]cKzg4844.Blob, 0, len(dbBlockContents.Blobs))
for _, blob := range dbBlockContents.Blobs {
if len(blob) != cKzg4844.BytesPerBlob {
return nil, nil, errors.Errorf("invalid blob size. expected %d bytes, got %d bytes", cKzg4844.BytesPerBlob, len(blob))
}
blobs = append(blobs, cKzg4844.Blob(blob))
}
dataColumnSideCars, err := peerdas.DataColumnSidecars(block, blobs)
if err != nil {
return nil, nil, errors.Wrap(err, "data column sidecars")
}
return nil, dataColumnSideCars, nil
}
blobSidecars, err := BuildBlobSidecars(block, dbBlockContents.Blobs, dbBlockContents.KzgProofs)
if err != nil {
return nil, nil, errors.Wrap(err, "build blob sidecars")
}
return blobSidecars, nil, nil
}
// broadcastReceiveBlock broadcasts a block and handles its reception.
func (vs *Server) broadcastReceiveBlock(ctx context.Context, block interfaces.SignedBeaconBlock, root [32]byte) error {
func (vs *Server) broadcastReceiveBlock(ctx context.Context, block interfaces.SignedBeaconBlock, root [fieldparams.RootLength]byte) error {
protoBlock, err := block.Proto()
if err != nil {
return errors.Wrap(err, "protobuf conversion failed")
@@ -371,7 +427,7 @@ func (vs *Server) broadcastReceiveBlock(ctx context.Context, block interfaces.Si
}
// broadcastAndReceiveBlobs handles the broadcasting and reception of blob sidecars.
func (vs *Server) broadcastAndReceiveBlobs(ctx context.Context, sidecars []*ethpb.BlobSidecar, root [32]byte) error {
func (vs *Server) broadcastAndReceiveBlobs(ctx context.Context, sidecars []*ethpb.BlobSidecar, root [fieldparams.RootLength]byte) error {
eg, eCtx := errgroup.WithContext(ctx)
for i, sc := range sidecars {
// Copy the iteration instance to a local variable to give each go-routine its own copy to play with.
@@ -400,6 +456,53 @@ func (vs *Server) broadcastAndReceiveBlobs(ctx context.Context, sidecars []*ethp
return eg.Wait()
}
// broadcastAndReceiveDataColumns handles the broadcasting and reception of data columns sidecars.
func (vs *Server) broadcastAndReceiveDataColumns(ctx context.Context, sidecars []*ethpb.DataColumnSidecar, root [fieldparams.RootLength]byte) error {
eg, _ := errgroup.WithContext(ctx)
dataColumnsWithholdCount := features.Get().DataColumnsWithholdCount
for i, sd := range sidecars {
// Copy the iteration instance to a local variable to give each go-routine its own copy to play with.
// See https://golang.org/doc/faq#closures_and_goroutines for more details.
colIdx, sidecar := i, sd
eg.Go(func() error {
// Compute the subnet index based on the column index.
subnet := uint64(colIdx) % params.BeaconConfig().DataColumnSidecarSubnetCount
if colIdx < dataColumnsWithholdCount {
log.WithFields(logrus.Fields{
"root": fmt.Sprintf("%#x", root),
"subnet": subnet,
"dataColumnIndex": colIdx,
}).Warning("Withholding data column")
} else {
if err := vs.P2P.BroadcastDataColumn(ctx, subnet, sidecar); err != nil {
return errors.Wrap(err, "broadcast data column")
}
}
roDataColumn, err := blocks.NewRODataColumnWithRoot(sidecar, root)
if err != nil {
return errors.Wrap(err, "new read-only data column with root")
}
verifiedRODataColumn := blocks.NewVerifiedRODataColumn(roDataColumn)
if err := vs.DataColumnReceiver.ReceiveDataColumn(ctx, verifiedRODataColumn); err != nil {
return errors.Wrap(err, "receive data column")
}
vs.OperationNotifier.OperationFeed().Send(&feed.Event{
Type: operation.DataColumnSidecarReceived,
Data: &operation.DataColumnSidecarReceivedData{DataColumn: &verifiedRODataColumn},
})
return nil
})
}
return eg.Wait()
}
// PrepareBeaconProposer caches and updates the fee recipient for the given proposer.
func (vs *Server) PrepareBeaconProposer(
_ context.Context, request *ethpb.PrepareBeaconProposerRequest,

View File

@@ -72,7 +72,6 @@ func (vs *Server) getLocalPayload(ctx context.Context, blk interfaces.ReadOnlyBe
}
setFeeRecipientIfBurnAddress(&val)
var err error
if ok && payloadId != [8]byte{} {
// Payload ID is cache hit. Return the cached payload ID.
var pid primitives.PayloadID

View File

@@ -943,7 +943,7 @@ func TestProposer_ProposeBlock_OK(t *testing.T) {
return &ethpb.GenericSignedBeaconBlock{Block: blk}
},
useBuilder: true,
err: "unblind sidecars failed: commitment value doesn't match block",
err: "unblind blobs sidecars: commitment value doesn't match block",
},
}

View File

@@ -67,6 +67,7 @@ type Server struct {
SyncCommitteePool synccommittee.Pool
BlockReceiver blockchain.BlockReceiver
BlobReceiver blockchain.BlobReceiver
DataColumnReceiver blockchain.DataColumnReceiver
MockEth1Votes bool
Eth1BlockFetcher execution.POWBlockFetcher
PendingDepositsFetcher depositsnapshot.PendingDepositsFetcher

View File

@@ -3,7 +3,10 @@ package validator
import (
"bytes"
cKzg4844 "github.com/ethereum/c-kzg-4844/bindings/go"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
consensusblocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
@@ -68,3 +71,29 @@ func unblindBlobsSidecars(block interfaces.SignedBeaconBlock, bundle *enginev1.B
}
return sidecars, nil
}
// TODO: Add tests
func unblindDataColumnsSidecars(block interfaces.SignedBeaconBlock, bundle *enginev1.BlobsBundle) ([]*ethpb.DataColumnSidecar, error) {
// Check if the block is at least a Deneb block.
if block.Version() < version.Deneb {
return nil, nil
}
// Convert blobs from slices to array.
blobs := make([]cKzg4844.Blob, 0, len(bundle.Blobs))
for _, blob := range bundle.Blobs {
if len(blob) != cKzg4844.BytesPerBlob {
return nil, errors.Errorf("invalid blob size. expected %d bytes, got %d bytes", cKzg4844.BytesPerBlob, len(blob))
}
blobs = append(blobs, cKzg4844.Blob(blob))
}
// Retrieve data columns from blobs.
dataColumnSidecars, err := peerdas.DataColumnSidecars(block, blobs)
if err != nil {
return nil, errors.Wrap(err, "data column sidecars")
}
return dataColumnSidecars, nil
}

View File

@@ -109,6 +109,7 @@ type Config struct {
AttestationReceiver blockchain.AttestationReceiver
BlockReceiver blockchain.BlockReceiver
BlobReceiver blockchain.BlobReceiver
DataColumnReceiver blockchain.DataColumnReceiver
ExecutionChainService execution.Chain
ChainStartFetcher execution.ChainStartFetcher
ExecutionChainInfoFetcher execution.ChainInfoFetcher
@@ -251,6 +252,7 @@ func NewService(ctx context.Context, cfg *Config) *Service {
P2P: s.cfg.Broadcaster,
BlockReceiver: s.cfg.BlockReceiver,
BlobReceiver: s.cfg.BlobReceiver,
DataColumnReceiver: s.cfg.DataColumnReceiver,
MockEth1Votes: s.cfg.MockEth1Votes,
Eth1BlockFetcher: s.cfg.ExecutionChainService,
PendingDepositsFetcher: s.cfg.PendingDepositFetcher,

View File

@@ -7,6 +7,8 @@ go_library(
"block_batcher.go",
"broadcast_bls_changes.go",
"context.go",
"data_columns_reconstruct.go",
"data_columns_sampling.go",
"deadlines.go",
"decode_pubsub.go",
"doc.go",
@@ -25,6 +27,8 @@ go_library(
"rpc_blob_sidecars_by_range.go",
"rpc_blob_sidecars_by_root.go",
"rpc_chunked_response.go",
"rpc_data_column_sidecars_by_range.go",
"rpc_data_column_sidecars_by_root.go",
"rpc_goodbye.go",
"rpc_metadata.go",
"rpc_ping.go",
@@ -37,6 +41,7 @@ go_library(
"subscriber_beacon_blocks.go",
"subscriber_blob_sidecar.go",
"subscriber_bls_to_execution_change.go",
"subscriber_data_column_sidecar.go",
"subscriber_handlers.go",
"subscriber_sync_committee_message.go",
"subscriber_sync_contribution_proof.go",
@@ -47,6 +52,7 @@ go_library(
"validate_beacon_blocks.go",
"validate_blob.go",
"validate_bls_to_execution_change.go",
"validate_data_column.go",
"validate_proposer_slashing.go",
"validate_sync_committee_message.go",
"validate_sync_contribution_proof.go",
@@ -71,6 +77,7 @@ go_library(
"//beacon-chain/core/feed/operation:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/core/transition/interop:go_default_library",
@@ -119,7 +126,11 @@ go_library(
"//runtime/version:go_default_library",
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_btcsuite_btcd_btcec_v2//:go_default_library",
"@com_github_ethereum_c_kzg_4844//bindings/go:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_ethereum_go_ethereum//common/math:go_default_library",
"@com_github_ethereum_go_ethereum//crypto:go_default_library",
"@com_github_hashicorp_golang_lru//:go_default_library",
"@com_github_libp2p_go_libp2p//core:go_default_library",
"@com_github_libp2p_go_libp2p//core/host:go_default_library",

View File

@@ -0,0 +1,336 @@
package sync
import (
"context"
"fmt"
"sort"
"time"
cKzg4844 "github.com/ethereum/c-kzg-4844/bindings/go"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"github.com/sirupsen/logrus"
)
const broadCastMissingDataColumnsTimeIntoSlot = 3 * time.Second
// recoverBlobs recovers the blobs from the data column sidecars.
func recoverBlobs(
dataColumnSideCars []*ethpb.DataColumnSidecar,
columnsCount int,
blockRoot [fieldparams.RootLength]byte,
) ([]cKzg4844.Blob, error) {
if len(dataColumnSideCars) == 0 {
return nil, errors.New("no data column sidecars")
}
// Check if all columns have the same length.
blobCount := len(dataColumnSideCars[0].DataColumn)
for _, sidecar := range dataColumnSideCars {
length := len(sidecar.DataColumn)
if length != blobCount {
return nil, errors.New("columns do not have the same length")
}
}
recoveredBlobs := make([]cKzg4844.Blob, 0, blobCount)
for blobIndex := 0; blobIndex < blobCount; blobIndex++ {
start := time.Now()
cellsId := make([]uint64, 0, columnsCount)
cKzgCells := make([]cKzg4844.Cell, 0, columnsCount)
for _, sidecar := range dataColumnSideCars {
// Build the cell ids.
cellsId = append(cellsId, sidecar.ColumnIndex)
// Get the cell.
column := sidecar.DataColumn
cell := column[blobIndex]
// Transform the cell as a cKzg cell.
var ckzgCell cKzg4844.Cell
for i := 0; i < cKzg4844.FieldElementsPerCell; i++ {
copy(ckzgCell[i][:], cell[32*i:32*(i+1)])
}
cKzgCells = append(cKzgCells, ckzgCell)
}
// Recover the blob.
recoveredCells, err := cKzg4844.RecoverAllCells(cellsId, cKzgCells)
if err != nil {
return nil, errors.Wrapf(err, "recover all cells for blob %d", blobIndex)
}
recoveredBlob, err := cKzg4844.CellsToBlob(recoveredCells)
if err != nil {
return nil, errors.Wrapf(err, "cells to blob for blob %d", blobIndex)
}
recoveredBlobs = append(recoveredBlobs, recoveredBlob)
log.WithFields(logrus.Fields{
"elapsed": time.Since(start),
"index": blobIndex,
"root": fmt.Sprintf("%x", blockRoot),
}).Debug("Recovered blob")
}
return recoveredBlobs, nil
}
func (s *Service) reconstructDataColumns(ctx context.Context, verifiedRODataColumn blocks.VerifiedRODataColumn) error {
// Lock to prevent concurrent reconstruction.
s.dataColumsnReconstructionLock.Lock()
defer s.dataColumsnReconstructionLock.Unlock()
// Get the block root.
blockRoot := verifiedRODataColumn.BlockRoot()
// Get the columns we store.
storedDataColumns, err := s.cfg.blobStorage.ColumnIndices(blockRoot)
if err != nil {
return errors.Wrap(err, "columns indices")
}
storedColumnsCount := len(storedDataColumns)
numberOfColumns := fieldparams.NumberOfColumns
// If less than half of the columns are stored, reconstruction is not possible.
// If all columns are stored, no need to reconstruct.
if storedColumnsCount < numberOfColumns/2 || storedColumnsCount == numberOfColumns {
return nil
}
// Retrieve the custodied columns.
custodiedSubnetCount := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodiedSubnetCount = params.BeaconConfig().DataColumnSidecarSubnetCount
}
custodiedColumns, err := peerdas.CustodyColumns(s.cfg.p2p.NodeID(), custodiedSubnetCount)
if err != nil {
return errors.Wrap(err, "custodied columns")
}
// Load the data columns sidecars.
dataColumnSideCars := make([]*ethpb.DataColumnSidecar, 0, storedColumnsCount)
for index := range storedDataColumns {
dataColumnSidecar, err := s.cfg.blobStorage.GetColumn(blockRoot, index)
if err != nil {
return errors.Wrap(err, "get column")
}
dataColumnSideCars = append(dataColumnSideCars, dataColumnSidecar)
}
// Recover blobs.
recoveredBlobs, err := recoverBlobs(dataColumnSideCars, storedColumnsCount, blockRoot)
if err != nil {
return errors.Wrap(err, "recover blobs")
}
// Reconstruct the data columns sidecars.
dataColumnSidecars, err := peerdas.DataColumnSidecarsForReconstruct(
verifiedRODataColumn.KzgCommitments,
verifiedRODataColumn.SignedBlockHeader,
verifiedRODataColumn.KzgCommitmentsInclusionProof,
recoveredBlobs,
)
if err != nil {
return errors.Wrap(err, "data column sidecars")
}
// Save the data columns sidecars in the database.
for _, dataColumnSidecar := range dataColumnSidecars {
shouldSave := custodiedColumns[dataColumnSidecar.ColumnIndex]
if !shouldSave {
// We do not custody this column, so we dot not need to save it.
continue
}
roDataColumn, err := blocks.NewRODataColumnWithRoot(dataColumnSidecar, blockRoot)
if err != nil {
return errors.Wrap(err, "new read-only data column with root")
}
verifiedRoDataColumn := blocks.NewVerifiedRODataColumn(roDataColumn)
if err := s.cfg.blobStorage.SaveDataColumn(verifiedRoDataColumn); err != nil {
return errors.Wrap(err, "save column")
}
}
log.WithField("root", fmt.Sprintf("%x", blockRoot)).Debug("Data columns reconstructed and saved successfully")
// Schedule the broadcast.
if err := s.scheduleReconstructedDataColumnsBroadcast(ctx, blockRoot, verifiedRODataColumn); err != nil {
return errors.Wrap(err, "schedule reconstructed data columns broadcast")
}
return nil
}
func (s *Service) scheduleReconstructedDataColumnsBroadcast(
ctx context.Context,
blockRoot [fieldparams.RootLength]byte,
dataColumn blocks.VerifiedRODataColumn,
) error {
// Retrieve the slot of the block.
slot := dataColumn.Slot()
// Get the time corresponding to the start of the slot.
slotStart, err := slots.ToTime(uint64(s.cfg.chain.GenesisTime().Unix()), slot)
if err != nil {
return errors.Wrap(err, "to time")
}
// Compute when to broadcast the missing data columns.
broadcastTime := slotStart.Add(broadCastMissingDataColumnsTimeIntoSlot)
// Compute the waiting time. This could be negative. In such a case, broadcast immediately.
waitingTime := time.Until(broadcastTime)
time.AfterFunc(waitingTime, func() {
s.dataColumsnReconstructionLock.Lock()
defer s.deleteReceivedDataColumns(blockRoot)
defer s.dataColumsnReconstructionLock.Unlock()
// Get the received by gossip data columns.
receivedDataColumns := s.receivedDataColumns(blockRoot)
if receivedDataColumns == nil {
log.WithField("root", fmt.Sprintf("%x", blockRoot)).Error("No received data columns")
}
// Get the data columns we should store.
custodiedSubnetCount := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodiedSubnetCount = params.BeaconConfig().DataColumnSidecarSubnetCount
}
custodiedDataColumns, err := peerdas.CustodyColumns(s.cfg.p2p.NodeID(), custodiedSubnetCount)
if err != nil {
log.WithError(err).Error("Custody columns")
}
// Get the data columns we actually store.
storedDataColumns, err := s.cfg.blobStorage.ColumnIndices(blockRoot)
if err != nil {
log.WithField("root", fmt.Sprintf("%x", blockRoot)).WithError(err).Error("Columns indices")
return
}
// Compute the missing data columns (data columns we should custody but we do not have received via gossip.)
missingColumns := make(map[uint64]bool, len(custodiedDataColumns))
for column := range custodiedDataColumns {
if ok := receivedDataColumns[column]; !ok {
missingColumns[column] = true
}
}
// Exit early if there are no missing data columns.
// This is the happy path.
if len(missingColumns) == 0 {
return
}
for column := range missingColumns {
if ok := storedDataColumns[column]; !ok {
// This column was not received nor reconstructed. This should not happen.
log.WithFields(logrus.Fields{
"root": fmt.Sprintf("%x", blockRoot),
"slot": slot,
"column": column,
}).Error("Data column not received nor reconstructed.")
continue
}
// Get the non received but reconstructed data column.
dataColumnSidecar, err := s.cfg.blobStorage.GetColumn(blockRoot, column)
if err != nil {
log.WithError(err).Error("Get column")
continue
}
// Compute the subnet for this column.
subnet := column % params.BeaconConfig().DataColumnSidecarSubnetCount
// Broadcast the missing data column.
if err := s.cfg.p2p.BroadcastDataColumn(ctx, subnet, dataColumnSidecar); err != nil {
log.WithError(err).Error("Broadcast data column")
}
}
// Get the missing data columns under sorted form.
missingColumnsList := make([]uint64, 0, len(missingColumns))
for column := range missingColumns {
missingColumnsList = append(missingColumnsList, column)
}
// Sort the missing data columns.
sort.Slice(missingColumnsList, func(i, j int) bool {
return missingColumnsList[i] < missingColumnsList[j]
})
log.WithFields(logrus.Fields{
"root": fmt.Sprintf("%x", blockRoot),
"slot": slot,
"timeIntoSlot": broadCastMissingDataColumnsTimeIntoSlot,
"columns": missingColumnsList,
}).Debug("Broadcasting not seen via gossip but reconstructed data columns.")
})
return nil
}
// setReceivedDataColumn marks the data column for a given root as received.
func (s *Service) setReceivedDataColumn(root [fieldparams.RootLength]byte, columnIndex uint64) {
s.receivedDataColumnsFromRootLock.Lock()
defer s.receivedDataColumnsFromRootLock.Unlock()
// Get all the received data columns for this root.
receivedDataColumns, ok := s.receivedDataColumnsFromRoot[root]
if !ok {
// Create the map for this block root if needed.
receivedDataColumns = make(map[uint64]bool, params.BeaconConfig().NumberOfColumns)
s.receivedDataColumnsFromRoot[root] = receivedDataColumns
}
// Mark the data column as received.
receivedDataColumns[columnIndex] = true
}
// receivedDataColumns returns the received data columns for a given root.
func (s *Service) receivedDataColumns(root [fieldparams.RootLength]byte) map[uint64]bool {
s.receivedDataColumnsFromRootLock.RLock()
defer s.receivedDataColumnsFromRootLock.RUnlock()
// Get all the received data columns for this root.
receivedDataColumns, ok := s.receivedDataColumnsFromRoot[root]
if !ok {
return nil
}
// Copy the received data columns.
copied := make(map[uint64]bool, len(receivedDataColumns))
for column, received := range receivedDataColumns {
copied[column] = received
}
return copied
}
// deleteReceivedDataColumns deletes the received data columns for a given root.
func (s *Service) deleteReceivedDataColumns(root [fieldparams.RootLength]byte) {
s.receivedDataColumnsFromRootLock.Lock()
defer s.receivedDataColumnsFromRootLock.Unlock()
delete(s.receivedDataColumnsFromRoot, root)
}

View File

@@ -0,0 +1,303 @@
package sync
import (
"context"
"fmt"
"sort"
"github.com/btcsuite/btcd/btcec/v2"
"github.com/ethereum/go-ethereum/common/math"
"github.com/ethereum/go-ethereum/crypto"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/crypto/rand"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/sirupsen/logrus"
)
// reandomIntegers returns a map of `count` random integers in the range [0, max[.
func randomIntegers(count uint64, max uint64) map[uint64]bool {
result := make(map[uint64]bool, count)
randGenerator := rand.NewGenerator()
for uint64(len(result)) < count {
n := randGenerator.Uint64() % max
result[n] = true
}
return result
}
// sortedListFromMap returns a sorted list of keys from a map.
func sortedListFromMap(m map[uint64]bool) []uint64 {
result := make([]uint64, 0, len(m))
for k := range m {
result = append(result, k)
}
sort.Slice(result, func(i, j int) bool {
return result[i] < result[j]
})
return result
}
// extractNodeID extracts the node ID from a peer ID.
func extractNodeID(pid peer.ID) ([32]byte, error) {
var nodeID [32]byte
// Retrieve the public key object of the peer under "crypto" form.
pubkeyObjCrypto, err := pid.ExtractPublicKey()
if err != nil {
return nodeID, errors.Wrap(err, "extract public key")
}
// Extract the bytes representation of the public key.
compressedPubKeyBytes, err := pubkeyObjCrypto.Raw()
if err != nil {
return nodeID, errors.Wrap(err, "public key raw")
}
// Retrieve the public key object of the peer under "SECP256K1" form.
pubKeyObjSecp256k1, err := btcec.ParsePubKey(compressedPubKeyBytes)
if err != nil {
return nodeID, errors.Wrap(err, "parse public key")
}
// Concatenate the X and Y coordinates represented in bytes.
buf := make([]byte, 64)
math.ReadBits(pubKeyObjSecp256k1.X(), buf[:32])
math.ReadBits(pubKeyObjSecp256k1.Y(), buf[32:])
// Get the node ID by hashing the concatenated X and Y coordinates.
nodeIDBytes := crypto.Keccak256(buf)
copy(nodeID[:], nodeIDBytes)
return nodeID, nil
}
// sampleDataColumnFromPeer samples data columns from a peer.
// It returns the missing columns after sampling.
func (s *Service) sampleDataColumnFromPeer(
pid peer.ID,
columnsToSample map[uint64]bool,
requestedRoot [fieldparams.RootLength]byte,
) (map[uint64]bool, error) {
// Define missing columns.
missingColumns := make(map[uint64]bool, len(columnsToSample))
for index := range columnsToSample {
missingColumns[index] = true
}
// Retrieve the custody count of the peer.
peerCustodiedSubnetCount := s.cfg.p2p.CustodyCountFromRemotePeer(pid)
// Extract the node ID from the peer ID.
nodeID, err := extractNodeID(pid)
if err != nil {
return nil, errors.Wrap(err, "extract node ID")
}
// Determine which columns the peer should custody.
peerCustodiedColumns, err := peerdas.CustodyColumns(nodeID, peerCustodiedSubnetCount)
if err != nil {
return nil, errors.Wrap(err, "custody columns")
}
peerCustodiedColumnsList := sortedListFromMap(peerCustodiedColumns)
// Compute the intersection of the columns to sample and the columns the peer should custody.
peerRequestedColumns := make(map[uint64]bool, len(columnsToSample))
for column := range columnsToSample {
if peerCustodiedColumns[column] {
peerRequestedColumns[column] = true
}
}
peerRequestedColumnsList := sortedListFromMap(peerRequestedColumns)
// Get the data column identifiers to sample from this peer.
dataColumnIdentifiers := make(types.BlobSidecarsByRootReq, 0, len(peerRequestedColumns))
for index := range peerRequestedColumns {
dataColumnIdentifiers = append(dataColumnIdentifiers, &eth.BlobIdentifier{
BlockRoot: requestedRoot[:],
Index: index,
})
}
// Return early if there are no data columns to sample.
if len(dataColumnIdentifiers) == 0 {
log.WithFields(logrus.Fields{
"peerID": pid,
"custodiedColumns": peerCustodiedColumnsList,
"requestedColumns": peerRequestedColumnsList,
}).Debug("Peer does not custody any of the requested columns")
return columnsToSample, nil
}
// Sample data columns.
roDataColumns, err := SendDataColumnSidecarByRoot(s.ctx, s.cfg.clock, s.cfg.p2p, pid, s.ctxMap, &dataColumnIdentifiers)
if err != nil {
return nil, errors.Wrap(err, "send data column sidecar by root")
}
peerRetrievedColumns := make(map[uint64]bool, len(roDataColumns))
// Remove retrieved items from rootsByDataColumnIndex.
for _, roDataColumn := range roDataColumns {
retrievedColumn := roDataColumn.ColumnIndex
actualRoot := roDataColumn.BlockRoot()
if actualRoot != requestedRoot {
// TODO: Should we decrease the peer score here?
log.WithFields(logrus.Fields{
"peerID": pid,
"requestedRoot": fmt.Sprintf("%#x", requestedRoot),
"actualRoot": fmt.Sprintf("%#x", actualRoot),
}).Warning("Actual root does not match requested root")
continue
}
peerRetrievedColumns[retrievedColumn] = true
if !columnsToSample[retrievedColumn] {
// TODO: Should we decrease the peer score here?
log.WithFields(logrus.Fields{
"peerID": pid,
"retrievedColumn": retrievedColumn,
"requestedColumns": peerRequestedColumnsList,
}).Warning("Retrieved column is was not requested")
}
delete(missingColumns, retrievedColumn)
}
peerRetrievedColumnsList := sortedListFromMap(peerRetrievedColumns)
remainingMissingColumnsList := sortedListFromMap(missingColumns)
log.WithFields(logrus.Fields{
"peerID": pid,
"custodiedColumns": peerCustodiedColumnsList,
"requestedColumns": peerRequestedColumnsList,
"retrievedColumns": peerRetrievedColumnsList,
"remainingMissingColumns": remainingMissingColumnsList,
}).Debug("Peer data column sampling summary")
return missingColumns, nil
}
// sampleDataColumns samples data columns from active peers.
func (s *Service) sampleDataColumns(requestedRoot [fieldparams.RootLength]byte, samplesCount uint64) error {
// Determine `samplesCount` random column indexes.
requestedColumns := randomIntegers(samplesCount, params.BeaconConfig().NumberOfColumns)
missingColumns := make(map[uint64]bool, len(requestedColumns))
for index := range requestedColumns {
missingColumns[index] = true
}
// Get the active peers from the p2p service.
activePeers := s.cfg.p2p.Peers().Active()
var err error
// Sampling is done sequentially peer by peer.
// TODO: Add parallelism if (probably) needed.
for _, pid := range activePeers {
// Early exit if all needed columns are already sampled. (This is the happy path.)
if len(missingColumns) == 0 {
break
}
// Sample data columns from the peer.
missingColumns, err = s.sampleDataColumnFromPeer(pid, missingColumns, requestedRoot)
if err != nil {
return errors.Wrap(err, "sample data column from peer")
}
}
requestedColumnsList := sortedListFromMap(requestedColumns)
if len(missingColumns) == 0 {
log.WithField("requestedColumns", requestedColumnsList).Debug("Successfully sampled all requested columns")
return nil
}
missingColumnsList := sortedListFromMap(missingColumns)
log.WithFields(logrus.Fields{
"requestedColumns": requestedColumnsList,
"missingColumns": missingColumnsList,
}).Warning("Failed to sample some requested columns")
return nil
}
func (s *Service) dataColumnSampling(ctx context.Context) {
// Create a subscription to the state feed.
stateChannel := make(chan *feed.Event, 1)
stateSub := s.cfg.stateNotifier.StateFeed().Subscribe(stateChannel)
// Unsubscribe from the state feed when the function returns.
defer stateSub.Unsubscribe()
for {
select {
case e := <-stateChannel:
if e.Type != statefeed.BlockProcessed {
continue
}
data, ok := e.Data.(*statefeed.BlockProcessedData)
if !ok {
log.Error("Event feed data is not of type *statefeed.BlockProcessedData")
continue
}
if !data.Verified {
// We only process blocks that have been verified
log.Error("Data is not verified")
continue
}
if data.SignedBlock.Version() < version.Deneb {
log.Debug("Pre Deneb block, skipping data column sampling")
continue
}
// Get the commitments for this block.
commitments, err := data.SignedBlock.Block().Body().BlobKzgCommitments()
if err != nil {
log.WithError(err).Error("Failed to get blob KZG commitments")
continue
}
// Skip if there are no commitments.
if len(commitments) == 0 {
log.Debug("No commitments in block, skipping data column sampling")
continue
}
dataColumnSamplingCount := params.BeaconConfig().SamplesPerSlot
// Sample data columns.
if err := s.sampleDataColumns(data.BlockRoot, dataColumnSamplingCount); err != nil {
log.WithError(err).Error("Failed to sample data columns")
}
case <-s.ctx.Done():
log.Debug("Context closed, exiting goroutine")
return
case err := <-stateSub.Err():
log.WithError(err).Error("Subscription to state feed failed")
}
}
}

View File

@@ -39,6 +39,8 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.SyncCommitteeMessage{})]
case strings.Contains(topic, p2p.GossipBlobSidecarMessage):
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.BlobSidecar{})]
case strings.Contains(topic, p2p.GossipDataColumnSidecarMessage):
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.DataColumnSidecar{})]
}
base := p2p.GossipTopicMappings(topic, 0)
@@ -63,7 +65,7 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
}
// Replaces our fork digest with the formatter.
func (_ *Service) replaceForkDigest(topic string) (string, error) {
func (*Service) replaceForkDigest(topic string) (string, error) {
subStrings := strings.Split(topic, "/")
if len(subStrings) != 4 {
return "", errInvalidTopic

View File

@@ -20,6 +20,7 @@ go_library(
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/core/feed/block:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/das:go_default_library",
"//beacon-chain/db:go_default_library",
@@ -32,6 +33,8 @@ go_library(
"//beacon-chain/sync/verify:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",

View File

@@ -10,6 +10,7 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
@@ -18,6 +19,7 @@ import (
prysmsync "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/verify"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
blocks2 "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
@@ -316,11 +318,19 @@ func (f *blocksFetcher) handleRequest(ctx context.Context, start primitives.Slot
response.bwb, response.pid, response.err = f.fetchBlocksFromPeer(ctx, start, count, peers)
if response.err == nil {
bwb, err := f.fetchBlobsFromPeer(ctx, response.bwb, response.pid, peers)
if err != nil {
response.err = err
if features.Get().EnablePeerDAS {
bwb, err := f.fetchColumnsFromPeer(ctx, response.bwb, response.pid, peers)
if err != nil {
response.err = err
}
response.bwb = bwb
} else {
bwb, err := f.fetchBlobsFromPeer(ctx, response.bwb, response.pid, peers)
if err != nil {
response.err = err
}
response.bwb = bwb
}
response.bwb = bwb
}
return response
}
@@ -465,6 +475,16 @@ func (r *blobRange) Request() *p2ppb.BlobSidecarsByRangeRequest {
}
}
func (r *blobRange) RequestDataColumns() *p2ppb.DataColumnSidecarsByRangeRequest {
if r == nil {
return nil
}
return &p2ppb.DataColumnSidecarsByRangeRequest{
StartSlot: r.low,
Count: uint64(r.high.SubSlot(r.low)) + 1,
}
}
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")
@@ -490,6 +510,28 @@ func verifyAndPopulateBlobs(bwb []blocks2.BlockWithROBlobs, blobs []blocks.ROBlo
return bwb, nil
}
func verifyAndPopulateColumns(bwb []blocks2.BlockWithROBlobs, columns []blocks.RODataColumn, req *p2ppb.DataColumnSidecarsByRangeRequest, bss filesystem.BlobStorageSummarizer) ([]blocks2.BlockWithROBlobs, error) {
columnsByRoot := make(map[[32]byte][]blocks.RODataColumn)
for i := range columns {
if columns[i].Slot() < req.StartSlot {
continue
}
br := columns[i].BlockRoot()
columnsByRoot[br] = append(columnsByRoot[br], columns[i])
}
for i := range bwb {
bwi, err := populateBlockWithColumns(bwb[i], columnsByRoot[bwb[i].Block.Root()], req, bss)
if err != nil {
if errors.Is(err, errDidntPopulate) {
continue
}
return bwb, err
}
bwb[i] = bwi
}
return bwb, nil
}
var errDidntPopulate = errors.New("skipping population of block")
func populateBlock(bw blocks2.BlockWithROBlobs, blobs []blocks.ROBlob, req *p2ppb.BlobSidecarsByRangeRequest, bss filesystem.BlobStorageSummarizer) (blocks2.BlockWithROBlobs, error) {
@@ -520,6 +562,31 @@ func populateBlock(bw blocks2.BlockWithROBlobs, blobs []blocks.ROBlob, req *p2pp
return bw, nil
}
func populateBlockWithColumns(bw blocks2.BlockWithROBlobs, columns []blocks.RODataColumn, req *p2ppb.DataColumnSidecarsByRangeRequest, bss filesystem.BlobStorageSummarizer) (blocks2.BlockWithROBlobs, error) {
blk := bw.Block
if blk.Version() < version.Deneb || blk.Block().Slot() < req.StartSlot {
return bw, errDidntPopulate
}
commits, err := blk.Block().Body().BlobKzgCommitments()
if err != nil {
return bw, errDidntPopulate
}
if len(commits) == 0 {
return bw, errDidntPopulate
}
colsPersub := params.BeaconConfig().NumberOfColumns / params.BeaconConfig().DataColumnSidecarSubnetCount
if len(columns) != int(params.BeaconConfig().CustodyRequirement*colsPersub) {
return bw, errors.Errorf("unequal custodied columns provided, got %d instead of %d", len(columns), int(params.BeaconConfig().CustodyRequirement))
}
for ci := range columns {
if err := verify.ColumnAlignsWithBlock(columns[ci], blk); err != nil {
return bw, err
}
}
bw.Columns = columns
return bw, nil
}
func missingCommitError(root [32]byte, slot primitives.Slot, missing [][]byte) error {
missStr := make([]string, 0, len(missing))
for k := range missing {
@@ -571,6 +638,71 @@ func (f *blocksFetcher) fetchBlobsFromPeer(ctx context.Context, bwb []blocks2.Bl
return nil, errNoPeersAvailable
}
// fetchColumnsFromPeer fetches blocks from a single randomly selected peer.
func (f *blocksFetcher) fetchColumnsFromPeer(ctx context.Context, bwb []blocks2.BlockWithROBlobs, pid peer.ID, peers []peer.ID) ([]blocks2.BlockWithROBlobs, error) {
ctx, span := trace.StartSpan(ctx, "initialsync.fetchColumnsFromPeer")
defer span.End()
if slots.ToEpoch(f.clock.CurrentSlot()) < params.BeaconConfig().DenebForkEpoch {
return bwb, nil
}
columnWindowStart, err := prysmsync.DataColumnsRPCMinValidSlot(f.clock.CurrentSlot())
if err != nil {
return nil, err
}
// Construct request message based on observed interval of blocks in need of columns.
req := countCommitments(bwb, columnWindowStart).blobRange(f.bs).RequestDataColumns()
if req == nil {
return bwb, nil
}
// Construct request message based on required custodied columns.
custodyCols, err := peerdas.CustodyColumns(f.p2p.NodeID(), params.BeaconConfig().CustodyRequirement)
if err != nil {
return nil, err
}
colIdxs := make([]uint64, 0, len(custodyCols))
for c := range custodyCols {
colIdxs = append(colIdxs, c)
}
req.Columns = colIdxs
peers = f.filterPeers(ctx, peers, peersPercentagePerRequest)
// We dial the initial peer first to ensure that we get the desired set of columns.
wantedPeers := append([]peer.ID{pid}, peers...)
bestPeers := f.hasSufficientBandwidth(wantedPeers, req.Count)
// We append the best peers to the front so that higher capacity
// peers are dialed first. If all of them fail, we fallback to the
// initial peer we wanted to request blobs from.
peers = append(bestPeers, pid)
for i := 0; i < len(peers); i++ {
p := peers[i]
nid, err := p2p.ConvertPeerIDToNodeID(pid)
if err != nil {
return nil, err
}
remoteCustody, err := peerdas.CustodyColumns(nid, params.BeaconConfig().CustodyRequirement)
if err != nil {
return nil, err
}
if !remotePeerHasCustody(req.Columns, remoteCustody) {
// TODO: For easier interop we do not skip for now
log.Warnf("Remote peer %s does not have wanted columns", p.String())
}
columns, err := f.requestColumns(ctx, req, p)
if err != nil {
log.WithField("peer", p).WithError(err).Debug("Could not request data columns by range from peer")
continue
}
f.p2p.Peers().Scorers().BlockProviderScorer().Touch(p)
robs, err := verifyAndPopulateColumns(bwb, columns, req, f.bs)
if err != nil {
log.WithField("peer", p).WithError(err).Debug("Invalid DataColumnByRange response")
continue
}
return robs, err
}
return nil, errNoPeersAvailable
}
// requestBlocks is a wrapper for handling BeaconBlocksByRangeRequest requests/streams.
func (f *blocksFetcher) requestBlocks(
ctx context.Context,
@@ -625,9 +757,38 @@ func (f *blocksFetcher) requestBlobs(ctx context.Context, req *p2ppb.BlobSidecar
}
f.rateLimiter.Add(pid.String(), int64(req.Count))
l.Unlock()
return prysmsync.SendBlobsByRangeRequest(ctx, f.clock, f.p2p, pid, f.ctxMap, req)
}
func (f *blocksFetcher) requestColumns(ctx context.Context, req *p2ppb.DataColumnSidecarsByRangeRequest, pid peer.ID) ([]blocks.RODataColumn, error) {
if ctx.Err() != nil {
return nil, ctx.Err()
}
l := f.peerLock(pid)
l.Lock()
log.WithFields(logrus.Fields{
"peer": pid,
"start": req.StartSlot,
"count": req.Count,
"capacity": f.rateLimiter.Remaining(pid.String()),
"score": f.p2p.Peers().Scorers().BlockProviderScorer().FormatScorePretty(pid),
}).Debug("Requesting Columns")
// We're intentionally abusing the block rate limit here, treating data column requests as if they were block requests.
// Since column requests take more bandwidth than blocks, we should improve how we account for the different kinds
// of requests, more in proportion to the cost of serving them.
if f.rateLimiter.Remaining(pid.String()) < int64(req.Count) {
if err := f.waitForBandwidth(pid, req.Count); err != nil {
l.Unlock()
return nil, err
}
}
f.rateLimiter.Add(pid.String(), int64(req.Count))
l.Unlock()
return prysmsync.SendDataColumnsByRangeRequest(ctx, f.clock, f.p2p, pid, f.ctxMap, req)
}
// requestBlocksByRoot is a wrapper for handling BeaconBlockByRootsReq requests/streams.
func (f *blocksFetcher) requestBlocksByRoot(
ctx context.Context,
@@ -728,3 +889,12 @@ func dedupPeers(peers []peer.ID) []peer.ID {
}
return newPeerList
}
func remotePeerHasCustody(wantedIdxs []uint64, remoteCustMap map[uint64]bool) bool {
for _, wIdx := range wantedIdxs {
if !remoteCustMap[wIdx] {
return false
}
}
return true
}

View File

@@ -8,6 +8,7 @@ import (
"github.com/pkg/errors"
p2pTypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -236,18 +237,18 @@ func (f *blocksFetcher) findForkWithPeer(ctx context.Context, pid peer.ID, slot
Count: reqCount,
Step: 1,
}
blocks, err := f.requestBlocks(ctx, req, pid)
reqBlocks, err := f.requestBlocks(ctx, req, pid)
if err != nil {
return nil, fmt.Errorf("cannot fetch blocks: %w", err)
}
if len(blocks) == 0 {
if len(reqBlocks) == 0 {
return nil, errNoAlternateBlocks
}
// If the first block is not connected to the current canonical chain, we'll stop processing this batch.
// Instead, we'll work backwards from the first block until we find a common ancestor,
// and then begin processing from there.
first := blocks[0]
first := reqBlocks[0]
if !f.chain.HasBlock(ctx, first.Block().ParentRoot()) {
// Backtrack on a root, to find a common ancestor from which we can resume syncing.
fork, err := f.findAncestor(ctx, pid, first)
@@ -260,8 +261,8 @@ func (f *blocksFetcher) findForkWithPeer(ctx context.Context, pid peer.ID, slot
// Traverse blocks, and if we've got one that doesn't have parent in DB, backtrack on it.
// Note that we start from the second element in the array, because we know that the first element is in the db,
// otherwise we would have gone into the findAncestor early return path above.
for i := 1; i < len(blocks); i++ {
block := blocks[i]
for i := 1; i < len(reqBlocks); i++ {
block := reqBlocks[i]
parentRoot := block.Block().ParentRoot()
// Step through blocks until we find one that is not in the chain. The goal is to find the point where the
// chain observed in the peer diverges from the locally known chain, and then collect up the remainder of the
@@ -274,16 +275,25 @@ func (f *blocksFetcher) findForkWithPeer(ctx context.Context, pid peer.ID, slot
"slot": block.Block().Slot(),
"root": fmt.Sprintf("%#x", parentRoot),
}).Debug("Block with unknown parent root has been found")
altBlocks, err := sortedBlockWithVerifiedBlobSlice(blocks[i-1:])
altBlocks, err := sortedBlockWithVerifiedBlobSlice(reqBlocks[i-1:])
if err != nil {
return nil, errors.Wrap(err, "invalid blocks received in findForkWithPeer")
}
var bwb []blocks.BlockWithROBlobs
if features.Get().EnablePeerDAS {
bwb, err = f.fetchColumnsFromPeer(ctx, altBlocks, pid, []peer.ID{pid})
if err != nil {
return nil, errors.Wrap(err, "unable to retrieve blobs for blocks found in findForkWithPeer")
}
} else {
bwb, err = f.fetchBlobsFromPeer(ctx, altBlocks, pid, []peer.ID{pid})
if err != nil {
return nil, errors.Wrap(err, "unable to retrieve blobs for blocks found in findForkWithPeer")
}
}
// We need to fetch the blobs for the given alt-chain if any exist, so that we can try to verify and import
// the blocks.
bwb, err := f.fetchBlobsFromPeer(ctx, altBlocks, pid, []peer.ID{pid})
if err != nil {
return nil, errors.Wrap(err, "unable to retrieve blobs for blocks found in findForkWithPeer")
}
// The caller will use the BlocksWith VerifiedBlobs in bwb as the starting point for
// round-robin syncing the alternate chain.
return &forkData{peer: pid, bwb: bwb}, nil
@@ -302,9 +312,16 @@ func (f *blocksFetcher) findAncestor(ctx context.Context, pid peer.ID, b interfa
if err != nil {
return nil, errors.Wrap(err, "received invalid blocks in findAncestor")
}
bwb, err = f.fetchBlobsFromPeer(ctx, bwb, pid, []peer.ID{pid})
if err != nil {
return nil, errors.Wrap(err, "unable to retrieve blobs for blocks found in findAncestor")
if features.Get().EnablePeerDAS {
bwb, err = f.fetchColumnsFromPeer(ctx, bwb, pid, []peer.ID{pid})
if err != nil {
return nil, errors.Wrap(err, "unable to retrieve columns for blocks found in findAncestor")
}
} else {
bwb, err = f.fetchBlobsFromPeer(ctx, bwb, pid, []peer.ID{pid})
if err != nil {
return nil, errors.Wrap(err, "unable to retrieve blobs for blocks found in findAncestor")
}
}
return &forkData{
peer: pid,

View File

@@ -14,6 +14,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -172,27 +173,52 @@ func (s *Service) processFetchedDataRegSync(
if len(bwb) == 0 {
return
}
bv := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncSidecarRequirements)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, bv)
batchFields := logrus.Fields{
"firstSlot": data.bwb[0].Block.Block().Slot(),
"firstUnprocessed": bwb[0].Block.Block().Slot(),
}
for _, b := range bwb {
if err := avs.Persist(s.clock.CurrentSlot(), b.Blobs...); err != nil {
log.WithError(err).WithFields(batchFields).WithFields(syncFields(b.Block)).Warn("Batch failure due to BlobSidecar issues")
return
if features.Get().EnablePeerDAS {
avs := das.NewLazilyPersistentStoreColumn(s.cfg.BlobStorage, emptyVerifier{}, s.cfg.P2P.NodeID())
batchFields := logrus.Fields{
"firstSlot": data.bwb[0].Block.Block().Slot(),
"firstUnprocessed": bwb[0].Block.Block().Slot(),
}
if err := s.processBlock(ctx, genesis, b, s.cfg.Chain.ReceiveBlock, avs); err != nil {
switch {
case errors.Is(err, errParentDoesNotExist):
log.WithFields(batchFields).WithField("missingParent", fmt.Sprintf("%#x", b.Block.Block().ParentRoot())).
WithFields(syncFields(b.Block)).Debug("Could not process batch blocks due to missing parent")
for _, b := range bwb {
if err := avs.PersistColumns(s.clock.CurrentSlot(), b.Columns...); err != nil {
log.WithError(err).WithFields(batchFields).WithFields(syncFields(b.Block)).Warn("Batch failure due to DataColumnSidecar issues")
return
default:
log.WithError(err).WithFields(batchFields).WithFields(syncFields(b.Block)).Warn("Block processing failure")
}
if err := s.processBlock(ctx, genesis, b, s.cfg.Chain.ReceiveBlock, avs); err != nil {
switch {
case errors.Is(err, errParentDoesNotExist):
log.WithFields(batchFields).WithField("missingParent", fmt.Sprintf("%#x", b.Block.Block().ParentRoot())).
WithFields(syncFields(b.Block)).Debug("Could not process batch blocks due to missing parent")
return
default:
log.WithError(err).WithFields(batchFields).WithFields(syncFields(b.Block)).Warn("Block processing failure")
return
}
}
}
} else {
bv := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncSidecarRequirements)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, bv)
batchFields := logrus.Fields{
"firstSlot": data.bwb[0].Block.Block().Slot(),
"firstUnprocessed": bwb[0].Block.Block().Slot(),
}
for _, b := range bwb {
if err := avs.Persist(s.clock.CurrentSlot(), b.Blobs...); err != nil {
log.WithError(err).WithFields(batchFields).WithFields(syncFields(b.Block)).Warn("Batch failure due to BlobSidecar issues")
return
}
if err := s.processBlock(ctx, genesis, b, s.cfg.Chain.ReceiveBlock, avs); err != nil {
switch {
case errors.Is(err, errParentDoesNotExist):
log.WithFields(batchFields).WithField("missingParent", fmt.Sprintf("%#x", b.Block.Block().ParentRoot())).
WithFields(syncFields(b.Block)).Debug("Could not process batch blocks due to missing parent")
return
default:
log.WithError(err).WithFields(batchFields).WithFields(syncFields(b.Block)).Warn("Block processing failure")
return
}
}
}
}
}
@@ -330,20 +356,34 @@ func (s *Service) processBatchedBlocks(ctx context.Context, genesis time.Time,
return fmt.Errorf("%w: %#x (in processBatchedBlocks, slot=%d)",
errParentDoesNotExist, first.Block().ParentRoot(), first.Block().Slot())
}
bv := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncSidecarRequirements)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, bv)
s.logBatchSyncStatus(genesis, first, len(bwb))
for _, bb := range bwb {
if len(bb.Blobs) == 0 {
continue
var aStore das.AvailabilityStore
if features.Get().EnablePeerDAS {
avs := das.NewLazilyPersistentStoreColumn(s.cfg.BlobStorage, emptyVerifier{}, s.cfg.P2P.NodeID())
s.logBatchSyncStatus(genesis, first, len(bwb))
for _, bb := range bwb {
if len(bb.Columns) == 0 {
continue
}
if err := avs.PersistColumns(s.clock.CurrentSlot(), bb.Columns...); err != nil {
return err
}
}
if err := avs.Persist(s.clock.CurrentSlot(), bb.Blobs...); err != nil {
return err
aStore = avs
} else {
bv := verification.NewBlobBatchVerifier(s.newBlobVerifier, verification.InitsyncSidecarRequirements)
avs := das.NewLazilyPersistentStore(s.cfg.BlobStorage, bv)
s.logBatchSyncStatus(genesis, first, len(bwb))
for _, bb := range bwb {
if len(bb.Blobs) == 0 {
continue
}
if err := avs.Persist(s.clock.CurrentSlot(), bb.Blobs...); err != nil {
return err
}
}
aStore = avs
}
return bFunc(ctx, blocks.BlockWithROBlobsSlice(bwb).ROBlocks(), avs)
return bFunc(ctx, blocks.BlockWithROBlobsSlice(bwb).ROBlocks(), aStore)
}
// updatePeerScorerStats adjusts monitored metrics for a peer.
@@ -380,3 +420,15 @@ func (s *Service) isProcessedBlock(ctx context.Context, blk blocks.ROBlock) bool
}
return false
}
type emptyVerifier struct {
}
func (_ emptyVerifier) VerifiedRODataColumns(_ context.Context, _ blocks.ROBlock, cols []blocks.RODataColumn) ([]blocks.VerifiedRODataColumn, error) {
var verCols []blocks.VerifiedRODataColumn
for _, col := range cols {
vCol := blocks.NewVerifiedRODataColumn(col)
verCols = append(verCols, vCol)
}
return verCols, nil
}

View File

@@ -15,6 +15,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/das"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
@@ -24,6 +25,8 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/crypto/rand"
@@ -184,9 +187,16 @@ func (s *Service) Start() {
log.WithError(err).Error("Error waiting for minimum number of peers")
return
}
if err := s.fetchOriginBlobs(peers); err != nil {
log.WithError(err).Error("Failed to fetch missing blobs for checkpoint origin")
return
if features.Get().EnablePeerDAS {
if err := s.fetchOriginColumns(peers); err != nil {
log.WithError(err).Error("Failed to fetch missing columns for checkpoint origin")
return
}
} else {
if err := s.fetchOriginBlobs(peers); err != nil {
log.WithError(err).Error("Failed to fetch missing blobs for checkpoint origin")
return
}
}
if err := s.roundRobinSync(gt); err != nil {
if errors.Is(s.ctx.Err(), context.Canceled) {
@@ -306,6 +316,59 @@ func missingBlobRequest(blk blocks.ROBlock, store *filesystem.BlobStorage) (p2pt
return req, nil
}
func (s *Service) missingColumnRequest(roBlock blocks.ROBlock, store *filesystem.BlobStorage) (p2ptypes.BlobSidecarsByRootReq, error) {
// No columns for pre-Deneb blocks.
if roBlock.Version() < version.Deneb {
return nil, nil
}
// Get the block root.
blockRoot := roBlock.Root()
// Get the commitments from the block.
commitments, err := roBlock.Block().Body().BlobKzgCommitments()
if err != nil {
return nil, errors.Wrap(err, "failed to get blob KZG commitments")
}
// Return early if there are no commitments.
if len(commitments) == 0 {
return nil, nil
}
// Check which columns are already on disk.
storedColumns, err := store.ColumnIndices(blockRoot)
if err != nil {
return nil, errors.Wrapf(err, "error checking existing blobs for checkpoint sync block root %#x", blockRoot)
}
// Get the number of columns we should custody.
custodyRequirement := params.BeaconConfig().CustodyRequirement
if features.Get().EnablePeerDAS {
custodyRequirement = fieldparams.NumberOfColumns
}
// Get our node ID.
nodeID := s.cfg.P2P.NodeID()
// Get the custodied columns.
custodiedColumns, err := peerdas.CustodyColumns(nodeID, custodyRequirement)
if err != nil {
return nil, errors.Wrap(err, "custody columns")
}
// Build blob sidecars by root requests based on missing columns.
req := make(p2ptypes.BlobSidecarsByRootReq, 0, len(commitments))
for columnIndex := range custodiedColumns {
isColumnAvailable := storedColumns[columnIndex]
if !isColumnAvailable {
req = append(req, &eth.BlobIdentifier{BlockRoot: blockRoot[:], Index: columnIndex})
}
}
return req, nil
}
func (s *Service) fetchOriginBlobs(pids []peer.ID) error {
r, err := s.cfg.DB.OriginCheckpointBlockRoot(s.ctx)
if errors.Is(err, db.ErrNotFoundOriginBlockRoot) {
@@ -356,6 +419,59 @@ func (s *Service) fetchOriginBlobs(pids []peer.ID) error {
return fmt.Errorf("no connected peer able to provide blobs for checkpoint sync block %#x", r)
}
func (s *Service) fetchOriginColumns(pids []peer.ID) error {
r, err := s.cfg.DB.OriginCheckpointBlockRoot(s.ctx)
if errors.Is(err, db.ErrNotFoundOriginBlockRoot) {
return nil
}
blk, err := s.cfg.DB.Block(s.ctx, r)
if err != nil {
log.WithField("root", fmt.Sprintf("%#x", r)).Error("Block for checkpoint sync origin root not found in db")
return err
}
if !params.WithinDAPeriod(slots.ToEpoch(blk.Block().Slot()), slots.ToEpoch(s.clock.CurrentSlot())) {
return nil
}
rob, err := blocks.NewROBlockWithRoot(blk, r)
if err != nil {
return err
}
req, err := s.missingColumnRequest(rob, s.cfg.BlobStorage)
if err != nil {
return err
}
if len(req) == 0 {
log.WithField("root", fmt.Sprintf("%#x", r)).Debug("All columns for checkpoint block are present")
return nil
}
shufflePeers(pids)
pids, err = s.cfg.P2P.GetValidCustodyPeers(pids)
if err != nil {
return err
}
for i := range pids {
sidecars, err := sync.SendDataColumnSidecarByRoot(s.ctx, s.clock, s.cfg.P2P, pids[i], s.ctxMap, &req)
if err != nil {
continue
}
if len(sidecars) != len(req) {
continue
}
avs := das.NewLazilyPersistentStoreColumn(s.cfg.BlobStorage, emptyVerifier{}, s.cfg.P2P.NodeID())
current := s.clock.CurrentSlot()
if err := avs.PersistColumns(current, sidecars...); err != nil {
return err
}
if err := avs.IsDataAvailable(s.ctx, current, rob); err != nil {
log.WithField("root", fmt.Sprintf("%#x", r)).WithField("peerID", pids[i]).Warn("Columns from peer for origin block were unusable")
continue
}
log.WithField("nColumns", len(sidecars)).WithField("root", fmt.Sprintf("%#x", r)).Info("Successfully downloaded blobs for checkpoint sync block")
return nil
}
return fmt.Errorf("no connected peer able to provide columns for checkpoint sync block %#x", r)
}
func shufflePeers(pids []peer.ID) {
rg := rand.NewGenerator()
rg.Shuffle(len(pids), func(i, j int) {

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/async"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -203,20 +204,40 @@ func (s *Service) processAndBroadcastBlock(ctx context.Context, b interfaces.Rea
return err
}
}
request, err := s.pendingBlobsRequestForBlock(blkRoot, b)
if err != nil {
return err
}
if len(request) > 0 {
peers := s.getBestPeers()
peerCount := len(peers)
if peerCount == 0 {
return errors.Wrapf(errNoPeersForPending, "block root=%#x", blkRoot)
}
if err := s.sendAndSaveBlobSidecars(ctx, request, peers[rand.NewGenerator().Int()%peerCount], b); err != nil {
if features.Get().EnablePeerDAS {
request, err := s.pendingDataColumnRequestForBlock(blkRoot, b)
if err != nil {
return err
}
if len(request) > 0 {
peers := s.getBestPeers()
peers, err = s.cfg.p2p.GetValidCustodyPeers(peers)
if err != nil {
return err
}
peerCount := len(peers)
if peerCount == 0 {
return errors.Wrapf(errNoPeersForPending, "block root=%#x", blkRoot)
}
if err := s.sendAndSaveDataColumnSidecars(ctx, request, peers[rand.NewGenerator().Int()%peerCount], b); err != nil {
return err
}
}
} else {
request, err := s.pendingBlobsRequestForBlock(blkRoot, b)
if err != nil {
return err
}
if len(request) > 0 {
peers := s.getBestPeers()
peerCount := len(peers)
if peerCount == 0 {
return errors.Wrapf(errNoPeersForPending, "block root=%#x", blkRoot)
}
if err := s.sendAndSaveBlobSidecars(ctx, request, peers[rand.NewGenerator().Int()%peerCount], b); err != nil {
return err
}
}
}
if err := s.cfg.chain.ReceiveBlock(ctx, b, blkRoot, nil); err != nil {

View File

@@ -10,6 +10,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/params"
leakybucket "github.com/prysmaticlabs/prysm/v5/container/leaky-bucket"
"github.com/sirupsen/logrus"
"github.com/trailofbits/go-mutexasserts"
@@ -42,9 +43,9 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
allowedBlocksPerSecond := float64(flags.Get().BlockBatchLimit)
allowedBlocksBurst := int64(flags.Get().BlockBatchLimitBurstFactor * flags.Get().BlockBatchLimit)
// Initialize blob limits.
allowedBlobsPerSecond := float64(flags.Get().BlobBatchLimit)
allowedBlobsBurst := int64(flags.Get().BlobBatchLimitBurstFactor * flags.Get().BlobBatchLimit)
// Initialize data column limits.
allowedDataColumnsPerSecond := float64(flags.Get().BlobBatchLimit * int(params.BeaconConfig().CustodyRequirement))
allowedDataColumnsBurst := int64(flags.Get().BlobBatchLimitBurstFactor * flags.Get().BlobBatchLimit * int(params.BeaconConfig().CustodyRequirement))
// Set topic map for all rpc topics.
topicMap := make(map[string]*leakybucket.Collector, len(p2p.RPCTopicMappings))
@@ -64,7 +65,9 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
blockCollectorV2 := leakybucket.NewCollector(allowedBlocksPerSecond, allowedBlocksBurst, blockBucketPeriod, false /* deleteEmptyBuckets */)
// for BlobSidecarsByRoot and BlobSidecarsByRange
blobCollector := leakybucket.NewCollector(allowedBlobsPerSecond, allowedBlobsBurst, blockBucketPeriod, false)
blobCollector := leakybucket.NewCollector(allowedDataColumnsPerSecond, allowedDataColumnsBurst, blockBucketPeriod, false)
// for DataColumnSidecarsByRoot and DataColumnSidecarsByRange
columnCollector := leakybucket.NewCollector(allowedDataColumnsPerSecond, allowedDataColumnsBurst, blockBucketPeriod, false)
// BlocksByRoots requests
topicMap[addEncoding(p2p.RPCBlocksByRootTopicV1)] = blockCollector
@@ -79,6 +82,11 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
// BlobSidecarsByRangeV1
topicMap[addEncoding(p2p.RPCBlobSidecarsByRangeTopicV1)] = blobCollector
// DataColumnSidecarsByRootV1
topicMap[addEncoding(p2p.RPCDataColumnSidecarsByRootTopicV1)] = columnCollector
// DataColumnSidecarsByRangeV1
topicMap[addEncoding(p2p.RPCDataColumnSidecarsByRangeTopicV1)] = columnCollector
// General topic for all rpc requests.
topicMap[rpcLimiterTopic] = leakybucket.NewCollector(5, defaultBurstLimit*2, leakyBucketPeriod, false /* deleteEmptyBuckets */)

View File

@@ -18,7 +18,7 @@ import (
func TestNewRateLimiter(t *testing.T) {
rlimiter := newRateLimiter(mockp2p.NewTestP2P(t))
assert.Equal(t, len(rlimiter.limiterMap), 12, "correct number of topics not registered")
assert.Equal(t, 14, len(rlimiter.limiterMap), "correct number of topics not registered")
}
func TestNewRateLimiter_FreeCorrectly(t *testing.T) {

View File

@@ -13,6 +13,7 @@ import (
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
"github.com/prysmaticlabs/prysm/v5/time/slots"
@@ -98,6 +99,17 @@ func (s *Service) registerRPCHandlersAltair() {
}
func (s *Service) registerRPCHandlersDeneb() {
if features.Get().EnablePeerDAS {
s.registerRPC(
p2p.RPCDataColumnSidecarsByRootTopicV1,
s.dataColumnSidecarByRootRPCHandler,
)
s.registerRPC(
p2p.RPCDataColumnSidecarsByRangeTopicV1,
s.dataColumnSidecarsByRangeRPCHandler,
)
return
}
s.registerRPC(
p2p.RPCBlobSidecarsByRangeTopicV1,
s.blobSidecarsByRangeRPCHandler,

View File

@@ -7,10 +7,13 @@ import (
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/verify"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
@@ -55,15 +58,28 @@ func (s *Service) sendRecentBeaconBlocksRequest(ctx context.Context, requests *t
if err != nil {
return err
}
request, err := s.pendingBlobsRequestForBlock(blkRoot, blk)
if err != nil {
return err
}
if len(request) == 0 {
continue
}
if err := s.sendAndSaveBlobSidecars(ctx, request, id, blk); err != nil {
return err
if features.Get().EnablePeerDAS {
request, err := s.pendingDataColumnRequestForBlock(blkRoot, blk)
if err != nil {
return errors.Wrap(err, "pending data column request for block")
}
if len(request) == 0 {
continue
}
if err := s.sendAndSaveDataColumnSidecars(ctx, request, id, blk); err != nil {
return errors.Wrap(err, "send and save data column sidecars")
}
} else {
request, err := s.pendingBlobsRequestForBlock(blkRoot, blk)
if err != nil {
return errors.Wrap(err, "pending blobs request for block")
}
if len(request) == 0 {
continue
}
if err := s.sendAndSaveBlobSidecars(ctx, request, id, blk); err != nil {
return errors.Wrap(err, "send and save blob sidecars")
}
}
}
return err
@@ -170,6 +186,36 @@ func (s *Service) sendAndSaveBlobSidecars(ctx context.Context, request types.Blo
return nil
}
func (s *Service) sendAndSaveDataColumnSidecars(ctx context.Context, request types.BlobSidecarsByRootReq, peerID peer.ID, block interfaces.ReadOnlySignedBeaconBlock) error {
if len(request) == 0 {
return nil
}
sidecars, err := SendDataColumnSidecarByRoot(ctx, s.cfg.clock, s.cfg.p2p, peerID, s.ctxMap, &request)
if err != nil {
return err
}
RoBlock, err := blocks.NewROBlock(block)
if err != nil {
return err
}
for _, sidecar := range sidecars {
if err := verify.ColumnAlignsWithBlock(sidecar, RoBlock); err != nil {
return err
}
log.WithFields(columnFields(sidecar)).Debug("Received data column sidecar RPC")
}
for i := range sidecars {
verfiedCol := blocks.NewVerifiedRODataColumn(sidecars[i])
if err := s.cfg.blobStorage.SaveDataColumn(verfiedCol); err != nil {
return err
}
}
return nil
}
func (s *Service) pendingBlobsRequestForBlock(root [32]byte, b interfaces.ReadOnlySignedBeaconBlock) (types.BlobSidecarsByRootReq, error) {
if b.Version() < version.Deneb {
return nil, nil // Block before deneb has no blob.
@@ -181,7 +227,27 @@ func (s *Service) pendingBlobsRequestForBlock(root [32]byte, b interfaces.ReadOn
if len(cc) == 0 {
return nil, nil
}
return s.constructPendingBlobsRequest(root, len(cc))
blobIdentifiers, err := s.constructPendingBlobsRequest(root, len(cc))
if err != nil {
return nil, errors.Wrap(err, "construct pending blobs request")
}
return blobIdentifiers, nil
}
func (s *Service) pendingDataColumnRequestForBlock(root [32]byte, b interfaces.ReadOnlySignedBeaconBlock) (types.BlobSidecarsByRootReq, error) {
if b.Version() < version.Deneb {
return nil, nil // Block before deneb has no blob.
}
cc, err := b.Block().Body().BlobKzgCommitments()
if err != nil {
return nil, err
}
if len(cc) == 0 {
return nil, nil
}
return s.constructPendingColumnRequest(root)
}
// constructPendingBlobsRequest creates a request for BlobSidecars by root, considering blobs already in DB.
@@ -191,12 +257,43 @@ func (s *Service) constructPendingBlobsRequest(root [32]byte, commitments int) (
}
stored, err := s.cfg.blobStorage.Indices(root)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "indices")
}
return requestsForMissingIndices(stored, commitments, root), nil
}
func (s *Service) constructPendingColumnRequest(root [32]byte) (types.BlobSidecarsByRootReq, error) {
// Retrieve the storedColumns columns for the current root.
storedColumns, err := s.cfg.blobStorage.ColumnIndices(root)
if err != nil {
return nil, errors.Wrap(err, "column indices")
}
// Compute how many subnets we should custody.
custodiedSubnetCount := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodiedSubnetCount = params.BeaconConfig().DataColumnSidecarSubnetCount
}
// Retrieve the columns we should custody.
custodiedColumns, err := peerdas.CustodyColumns(s.cfg.p2p.NodeID(), custodiedSubnetCount)
if err != nil {
return nil, errors.Wrap(err, "custody columns")
}
// Build the request for the missing columns.
req := make(types.BlobSidecarsByRootReq, 0, len(custodiedColumns))
for column := range custodiedColumns {
isColumnStored := storedColumns[column]
if !isColumnStored {
req = append(req, &eth.BlobIdentifier{Index: column, BlockRoot: root[:]})
}
}
return req, nil
}
// requestsForMissingIndices constructs a slice of BlobIdentifiers that are missing from
// local storage, based on a mapping that represents which indices are locally stored,
// and the highest expected index.

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/network/forks"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
@@ -181,3 +182,22 @@ func WriteBlobSidecarChunk(stream libp2pcore.Stream, tor blockchain.TemporalOrac
_, err = encoding.EncodeWithMaxLength(stream, sidecar)
return err
}
// WriteDataColumnSidecarChunk writes data column chunk object to stream.
// response_chunk ::= <result> | <context-bytes> | <encoding-dependent-header> | <encoded-payload>
func WriteDataColumnSidecarChunk(stream libp2pcore.Stream, tor blockchain.TemporalOracle, encoding encoder.NetworkEncoding, sidecar *ethpb.DataColumnSidecar) error {
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return err
}
valRoot := tor.GenesisValidatorsRoot()
ctxBytes, err := forks.ForkDigestFromEpoch(slots.ToEpoch(sidecar.SignedBlockHeader.Header.Slot), valRoot[:])
if err != nil {
return err
}
if err := writeContextToStream(ctxBytes[:], stream); err != nil {
return err
}
_, err = encoding.EncodeWithMaxLength(stream, sidecar)
return err
}

View File

@@ -0,0 +1,196 @@
package sync
import (
"context"
"time"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"go.opencensus.io/trace"
)
func (s *Service) streamDataColumnBatch(ctx context.Context, batch blockBatch, wQuota uint64, wantedIndexes map[uint64]bool, stream libp2pcore.Stream) (uint64, error) {
// Defensive check to guard against underflow.
if wQuota == 0 {
return 0, nil
}
_, span := trace.StartSpan(ctx, "sync.streamDataColumnBatch")
defer span.End()
for _, b := range batch.canonical() {
root := b.Root()
idxs, err := s.cfg.blobStorage.ColumnIndices(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 i, l := uint64(0), uint64(len(idxs)); i < l; i++ {
// index not available or unwanted, skip
if !idxs[i] || !wantedIndexes[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.GetColumn(b.Root(), i)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
return wQuota, errors.Wrapf(err, "could not retrieve data column sidecar: index %d, block root %#x", i, root)
}
SetStreamWriteDeadline(stream, defaultWriteDuration)
if chunkErr := WriteDataColumnSidecarChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), sc); chunkErr != nil {
log.WithError(chunkErr).Debug("Could not send a chunked response")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, chunkErr)
return wQuota, chunkErr
}
s.rateLimiter.add(stream, 1)
wQuota -= 1
// Stop streaming results once the quota of writes for the request is consumed.
if wQuota == 0 {
return 0, nil
}
}
}
return wQuota, nil
}
// dataColumnSidecarsByRangeRPCHandler looks up the request data columns from the database from a given start slot index
func (s *Service) dataColumnSidecarsByRangeRPCHandler(ctx context.Context, msg interface{}, stream libp2pcore.Stream) error {
var err error
ctx, span := trace.StartSpan(ctx, "sync.DataColumnSidecarsByRangeHandler")
defer span.End()
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
SetRPCStreamDeadlines(stream)
log := log.WithField("handler", p2p.DataColumnSidecarsByRangeName[1:]) // slice the leading slash off the name var
r, ok := msg.(*pb.DataColumnSidecarsByRangeRequest)
if !ok {
return errors.New("message is not type *pb.DataColumnSidecarsByRangeRequest")
}
if err := s.rateLimiter.validateRequest(stream, 1); err != nil {
return err
}
rp, err := validateDataColumnsByRange(r, s.cfg.chain.CurrentSlot())
if err != nil {
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
tracing.AnnotateError(span, err)
return err
}
// Ticker to stagger out large requests.
ticker := time.NewTicker(time.Second)
defer ticker.Stop()
batcher, err := newBlockRangeBatcher(rp, s.cfg.beaconDB, s.rateLimiter, s.cfg.chain.IsCanonical, ticker)
if err != nil {
log.WithError(err).Info("error in DataColumnSidecarsByRange batch")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)
return err
}
// Derive the wanted columns for the request.
wantedColumns := map[uint64]bool{}
for _, c := range r.Columns {
wantedColumns[c] = true
}
var batch blockBatch
wQuota := params.BeaconConfig().MaxRequestDataColumnSidecars
for batch, ok = batcher.next(ctx, stream); ok; batch, ok = batcher.next(ctx, stream) {
batchStart := time.Now()
wQuota, err = s.streamDataColumnBatch(ctx, batch, wQuota, wantedColumns, stream)
rpcBlobsByRangeResponseLatency.Observe(float64(time.Since(batchStart).Milliseconds()))
if err != nil {
return err
}
// once we have written MAX_REQUEST_BLOB_SIDECARS, we're done serving the request
if wQuota == 0 {
break
}
}
if err := batch.error(); err != nil {
log.WithError(err).Debug("error in DataColumnSidecarsByRange batch")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)
return err
}
closeStream(stream, log)
return nil
}
// Set the count limit to the number of blobs in a batch.
func columnBatchLimit() uint64 {
return uint64(flags.Get().BlockBatchLimit) / fieldparams.MaxBlobsPerBlock
}
// TODO: Generalize between data columns and blobs, while the validation parameters used are different they
// are the same value in the config. Can this be safely abstracted ?
func validateDataColumnsByRange(r *pb.DataColumnSidecarsByRangeRequest, current primitives.Slot) (rangeParams, error) {
if r.Count == 0 {
return rangeParams{}, errors.Wrap(p2ptypes.ErrInvalidRequest, "invalid request Count parameter")
}
rp := rangeParams{
start: r.StartSlot,
size: r.Count,
}
// Peers may overshoot the current slot when in initial sync, so we don't want to penalize them by treating the
// request as an error. So instead we return a set of params that acts as a noop.
if rp.start > current {
return rangeParams{start: current, end: current, size: 0}, nil
}
var err error
rp.end, err = rp.start.SafeAdd(rp.size - 1)
if err != nil {
return rangeParams{}, errors.Wrap(p2ptypes.ErrInvalidRequest, "overflow start + count -1")
}
maxRequest := params.MaxRequestBlock(slots.ToEpoch(current))
// Allow some wiggle room, up to double the MaxRequestBlocks past the current slot,
// to give nodes syncing close to the head of the chain some margin for error.
maxStart, err := current.SafeAdd(maxRequest * 2)
if err != nil {
return rangeParams{}, errors.Wrap(p2ptypes.ErrInvalidRequest, "current + maxRequest * 2 > max uint")
}
// Clients MUST keep a record of signed data column sidecars seen on the epoch range
// [max(current_epoch - MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS, DENEB_FORK_EPOCH), current_epoch]
// where current_epoch is defined by the current wall-clock time,
// and clients MUST support serving requests of data columns on this range.
minStartSlot, err := DataColumnsRPCMinValidSlot(current)
if err != nil {
return rangeParams{}, errors.Wrap(p2ptypes.ErrInvalidRequest, "DataColumnsRPCMinValidSlot error")
}
if rp.start > maxStart {
return rangeParams{}, errors.Wrap(p2ptypes.ErrInvalidRequest, "start > maxStart")
}
if rp.start < minStartSlot {
rp.start = minStartSlot
}
if rp.end > current {
rp.end = current
}
if rp.end < rp.start {
rp.end = rp.start
}
limit := columnBatchLimit()
if limit > maxRequest {
limit = maxRequest
}
if rp.size > limit {
rp.size = limit
}
return rp, nil
}

View File

@@ -0,0 +1,229 @@
package sync
import (
"context"
"fmt"
"math"
"sort"
"time"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"github.com/sirupsen/logrus"
"go.opencensus.io/trace"
)
func (s *Service) dataColumnSidecarByRootRPCHandler(ctx context.Context, msg interface{}, stream libp2pcore.Stream) error {
ctx, span := trace.StartSpan(ctx, "sync.dataColumnSidecarByRootRPCHandler")
defer span.End()
ctx, cancel := context.WithTimeout(ctx, ttfbTimeout)
defer cancel()
SetRPCStreamDeadlines(stream)
log := log.WithField("handler", p2p.DataColumnSidecarsByRootName[1:]) // slice the leading slash off the name var
// We use the same type as for blobs as they are the same data structure.
// TODO: Make the type naming more generic to be extensible to data columns
ref, ok := msg.(*types.BlobSidecarsByRootReq)
if !ok {
return errors.New("message is not type BlobSidecarsByRootReq")
}
requestedColumnIdents := *ref
if err := validateDataColummnsByRootRequest(requestedColumnIdents); err != nil {
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
return errors.Wrap(err, "validate data columns by root request")
}
// Sort the identifiers so that requests for the same blob root will be adjacent, minimizing db lookups.
sort.Sort(requestedColumnIdents)
requestedColumnsList := make([]uint64, 0, len(requestedColumnIdents))
for _, ident := range requestedColumnIdents {
requestedColumnsList = append(requestedColumnsList, ident.Index)
}
// TODO: Customize data column batches too
batchSize := flags.Get().BlobBatchLimit
var ticker *time.Ticker
if len(requestedColumnIdents) > batchSize {
ticker = time.NewTicker(time.Second)
}
// Compute the oldest slot we'll allow a peer to request, based on the current slot.
cs := s.cfg.clock.CurrentSlot()
minReqSlot, err := DataColumnsRPCMinValidSlot(cs)
if err != nil {
return errors.Wrapf(err, "unexpected error computing min valid blob request slot, current_slot=%d", cs)
}
// Compute all custodied subnets.
custodiedSubnets := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
custodiedSubnets = params.BeaconConfig().DataColumnSidecarSubnetCount
}
// Compute all custodied columns.
custodiedColumns, err := peerdas.CustodyColumns(s.cfg.p2p.NodeID(), custodiedSubnets)
if err != nil {
log.WithError(err).Errorf("unexpected error retrieving the node id")
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
return errors.Wrap(err, "custody columns")
}
custodiedColumnsList := make([]uint64, 0, len(custodiedColumns))
for column := range custodiedColumns {
custodiedColumnsList = append(custodiedColumnsList, column)
}
// Sort the custodied columns by index.
sort.Slice(custodiedColumnsList, func(i, j int) bool {
return custodiedColumnsList[i] < custodiedColumnsList[j]
})
log.WithFields(logrus.Fields{
"custodied": custodiedColumnsList,
"requested": requestedColumnsList,
"custodiedCount": len(custodiedColumnsList),
"requestedCount": len(requestedColumnsList),
}).Debug("Received data column sidecar by root request")
// Subscribe to the data column feed.
rootIndexChan := make(chan filesystem.RootIndexPair)
subscription := s.cfg.blobStorage.DataColumnFeed.Subscribe(rootIndexChan)
defer subscription.Unsubscribe()
for i := range requestedColumnIdents {
if err := ctx.Err(); err != nil {
closeStream(stream, log)
return errors.Wrap(err, "context error")
}
// Throttle request processing to no more than batchSize/sec.
if ticker != nil && i != 0 && i%batchSize == 0 {
for {
select {
case <-ticker.C:
log.Debug("Throttling data column sidecar request")
case <-ctx.Done():
log.Debug("Context closed, exiting routine")
return nil
}
}
}
s.rateLimiter.add(stream, 1)
requestedRoot, requestedIndex := bytesutil.ToBytes32(requestedColumnIdents[i].BlockRoot), requestedColumnIdents[i].Index
// Decrease the peer's score if it requests a column that is not custodied.
isCustodied := custodiedColumns[requestedIndex]
if !isCustodied {
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.writeErrorResponseToStream(responseCodeInvalidRequest, types.ErrInvalidColumnIndex.Error(), stream)
return types.ErrInvalidColumnIndex
}
// TODO: Differentiate between blobs and columns for our storage engine
// If the data column is nil, it means it is not yet available in the db.
// We wait for it to be available.
// Retrieve the data column from the database.
dataColumnSidecar, err := s.cfg.blobStorage.GetColumn(requestedRoot, requestedIndex)
if err != nil && !db.IsNotFound(err) {
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
return errors.Wrap(err, "get column")
}
if err != nil && db.IsNotFound(err) {
fields := logrus.Fields{
"root": fmt.Sprintf("%#x", requestedRoot),
"index": requestedIndex,
}
log.WithFields(fields).Debug("Peer requested data column sidecar by root not found in db, waiting for it to be available")
loop:
for {
select {
case receivedRootIndex := <-rootIndexChan:
if receivedRootIndex.Root == requestedRoot && receivedRootIndex.Index == requestedIndex {
// This is the data column we are looking for.
log.WithFields(fields).Debug("Data column sidecar by root is now available in the db")
break loop
}
case <-ctx.Done():
closeStream(stream, log)
return errors.Errorf("context closed while waiting for data column with root %#x and index %d", requestedRoot, requestedIndex)
}
}
// Retrieve the data column from the db.
dataColumnSidecar, err = s.cfg.blobStorage.GetColumn(requestedRoot, requestedIndex)
if err != nil {
// This time, no error (even not found error) should be returned.
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
return errors.Wrap(err, "get column")
}
}
// 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 data column in the response.
// note: we are deviating from the spec to allow requests for data column that are before minimum_request_epoch,
// up to the beginning of the retention period.
if dataColumnSidecar.SignedBlockHeader.Header.Slot < minReqSlot {
s.writeErrorResponseToStream(responseCodeResourceUnavailable, types.ErrDataColumnLTMinRequest.Error(), stream)
log.WithError(types.ErrDataColumnLTMinRequest).
Debugf("requested data column for block %#x before minimum_request_epoch", requestedColumnIdents[i].BlockRoot)
return types.ErrDataColumnLTMinRequest
}
SetStreamWriteDeadline(stream, defaultWriteDuration)
if chunkErr := WriteDataColumnSidecarChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), dataColumnSidecar); chunkErr != nil {
log.WithError(chunkErr).Debug("Could not send a chunked response")
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, chunkErr)
return chunkErr
}
}
closeStream(stream, log)
return nil
}
func validateDataColummnsByRootRequest(colIdents types.BlobSidecarsByRootReq) error {
if uint64(len(colIdents)) > params.BeaconConfig().MaxRequestDataColumnSidecars {
return types.ErrMaxDataColumnReqExceeded
}
return nil
}
func DataColumnsRPCMinValidSlot(current primitives.Slot) (primitives.Slot, error) {
// Avoid overflow if we're running on a config where deneb is set to far future epoch.
if params.BeaconConfig().DenebForkEpoch == math.MaxUint64 || !features.Get().EnablePeerDAS {
return primitives.Slot(math.MaxUint64), nil
}
minReqEpochs := params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest
currEpoch := slots.ToEpoch(current)
minStart := params.BeaconConfig().DenebForkEpoch
if currEpoch > minReqEpochs && currEpoch-minReqEpochs > minStart {
minStart = currEpoch - minReqEpochs
}
return slots.EpochStart(minStart)
}

View File

@@ -18,7 +18,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
@@ -38,6 +37,7 @@ var (
errChunkResponseIndexNotAsc = errors.Wrap(ErrInvalidFetchedData, "blob indices for a block must start at 0 and increase by 1")
errUnrequested = errors.Wrap(ErrInvalidFetchedData, "received BlobSidecar in response that was not requested")
errBlobResponseOutOfBounds = errors.Wrap(ErrInvalidFetchedData, "received BlobSidecar with slot outside BlobSidecarsByRangeRequest bounds")
errDataColumnResponseOutOfBounds = errors.Wrap(ErrInvalidFetchedData, "received DataColumnSidecar with slot outside DataColumnSidecarsByRangeRequest bounds")
errChunkResponseBlockMismatch = errors.Wrap(ErrInvalidFetchedData, "blob block details do not match")
errChunkResponseParentMismatch = errors.Wrap(ErrInvalidFetchedData, "parent root for response element doesn't match previous element root")
)
@@ -208,10 +208,173 @@ func SendBlobSidecarByRoot(
return readChunkEncodedBlobs(stream, p2pApi.Encoding(), ctxMap, blobValidatorFromRootReq(req), max)
}
func SendDataColumnSidecarByRoot(
ctx context.Context,
tor blockchain.TemporalOracle,
p2pApi p2p.P2P,
pid peer.ID,
ctxMap ContextByteVersions,
req *p2ptypes.BlobSidecarsByRootReq,
) ([]blocks.RODataColumn, error) {
reqCount := uint64(len(*req))
maxRequestDataColumnSideCar := params.BeaconConfig().MaxRequestDataColumnSidecars
// Verify that the request count is within the maximum allowed.
if reqCount > maxRequestDataColumnSideCar {
return nil, errors.Wrapf(p2ptypes.ErrMaxDataColumnReqExceeded, "current: %d, max: %d", reqCount, maxRequestDataColumnSideCar)
}
// Get the topic for the request.
topic, err := p2p.TopicFromMessage(p2p.DataColumnSidecarsByRootName, slots.ToEpoch(tor.CurrentSlot()))
if err != nil {
return nil, errors.Wrap(err, "topic from message")
}
// Send the request to the peer.
log.WithField("topic", topic).Debug("Sending data column sidecar request")
stream, err := p2pApi.Send(ctx, req, topic, pid)
if err != nil {
return nil, errors.Wrap(err, "send")
}
// Close the stream when done.
defer closeStream(stream, log)
// Read the data column sidecars from the stream.
roDataColumns := make([]blocks.RODataColumn, 0, reqCount)
for i := uint64(0); ; /* no stop condition */ i++ {
roDataColumn, err := readChunkedDataColumnSideCar(stream, p2pApi, ctxMap, []DataColumnResponseValidation{dataColumnValidatorFromRootReq(req)})
if errors.Is(err, io.EOF) {
// End of stream.
break
}
if err != nil {
return nil, errors.Wrap(err, "read chunked data column sidecar")
}
if i >= reqCount {
// The response MUST contain no more than `reqCount` blocks.
// (`reqCount` is already capped by `maxRequestDataColumnSideCar`.)
return nil, errors.Wrap(ErrInvalidFetchedData, "response contains more data column sidecars than requested")
}
roDataColumns = append(roDataColumns, *roDataColumn)
}
return roDataColumns, nil
}
func SendDataColumnsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle, p2pApi p2p.P2P, pid peer.ID, ctxMap ContextByteVersions, req *pb.DataColumnSidecarsByRangeRequest) ([]blocks.RODataColumn, error) {
topic, err := p2p.TopicFromMessage(p2p.DataColumnSidecarsByRangeName, slots.ToEpoch(tor.CurrentSlot()))
if err != nil {
return nil, err
}
log.WithFields(logrus.Fields{
"topic": topic,
"startSlot": req.StartSlot,
"count": req.Count,
}).Debug("Sending data column by range request")
stream, err := p2pApi.Send(ctx, req, topic, pid)
if err != nil {
return nil, err
}
defer closeStream(stream, log)
max := params.BeaconConfig().MaxRequestDataColumnSidecars
if max > req.Count*fieldparams.NumberOfColumns {
max = req.Count * fieldparams.NumberOfColumns
}
vfuncs := []DataColumnResponseValidation{dataColumnValidatorFromRangeReq(req), dataColumnIndexValidatorFromRangeReq(req)}
// Read the data column sidecars from the stream.
roDataColumns := make([]blocks.RODataColumn, 0, max)
for i := uint64(0); ; /* no stop condition */ i++ {
roDataColumn, err := readChunkedDataColumnSideCar(stream, p2pApi, ctxMap, vfuncs)
if errors.Is(err, io.EOF) {
// End of stream.
break
}
if err != nil {
return nil, errors.Wrap(err, "read chunked data column sidecar")
}
if i >= max {
// The response MUST contain no more than `reqCount` blocks.
// (`reqCount` is already capped by `maxRequestDataColumnSideCar`.)
return nil, errors.Wrap(ErrInvalidFetchedData, "response contains more data column sidecars than maximum")
}
roDataColumns = append(roDataColumns, *roDataColumn)
}
return roDataColumns, nil
}
func readChunkedDataColumnSideCar(
stream network.Stream,
p2pApi p2p.P2P,
ctxMap ContextByteVersions,
validation []DataColumnResponseValidation,
) (*blocks.RODataColumn, error) {
// Read the status code from the stream.
statusCode, errMessage, err := ReadStatusCode(stream, p2pApi.Encoding())
if err != nil {
return nil, errors.Wrap(err, "read status code")
}
if statusCode != 0 {
return nil, errors.Wrap(errBlobChunkedReadFailure, errMessage)
}
// Retrieve the fork digest.
ctxBytes, err := readContextFromStream(stream)
if err != nil {
return nil, errors.Wrap(err, "read context from stream")
}
// Check if the fork digest is recognized.
v, ok := ctxMap[bytesutil.ToBytes4(ctxBytes)]
if !ok {
return nil, errors.Errorf("unrecognized fork digest %#x", ctxBytes)
}
// Check if we are on debeb.
// Only deneb is supported at this time, because we lack a fork-spanning interface/union type for blobs.
if v != version.Deneb {
return nil, errors.Errorf("unexpected context bytes for deneb DataColumnSidecar, ctx=%#x, v=%v", ctxBytes, v)
}
// Decode the data column sidecar from the stream.
dataColumnSidecar := new(pb.DataColumnSidecar)
if err := p2pApi.Encoding().DecodeWithMaxLength(stream, dataColumnSidecar); err != nil {
return nil, errors.Wrap(err, "failed to decode the protobuf-encoded BlobSidecar message from RPC chunk stream")
}
// Create a read-only data column from the data column sidecar.
roDataColumn, err := blocks.NewRODataColumn(dataColumnSidecar)
if err != nil {
return nil, errors.Wrap(err, "new read only data column")
}
for _, val := range validation {
if err := val(roDataColumn); err != nil {
return nil, err
}
}
return &roDataColumn, nil
}
// BlobResponseValidation represents a function that can validate aspects of a single unmarshaled blob
// that was received from a peer in response to an rpc request.
type BlobResponseValidation func(blocks.ROBlob) error
// DataColumnResponseValidation represents a function that can validate aspects of a single unmarshaled data column
// that was received from a peer in response to an rpc request.
type DataColumnResponseValidation func(column blocks.RODataColumn) error
func composeBlobValidations(vf ...BlobResponseValidation) BlobResponseValidation {
return func(blob blocks.ROBlob) error {
for i := range vf {
@@ -308,6 +471,52 @@ func blobValidatorFromRangeReq(req *pb.BlobSidecarsByRangeRequest) BlobResponseV
}
}
func dataColumnValidatorFromRootReq(req *p2ptypes.BlobSidecarsByRootReq) DataColumnResponseValidation {
columnIds := make(map[[32]byte]map[uint64]bool)
for _, sc := range *req {
blockRoot := bytesutil.ToBytes32(sc.BlockRoot)
if columnIds[blockRoot] == nil {
columnIds[blockRoot] = make(map[uint64]bool)
}
columnIds[blockRoot][sc.Index] = true
}
return func(sc blocks.RODataColumn) error {
columnIndices := columnIds[sc.BlockRoot()]
if columnIndices == nil {
return errors.Wrapf(errUnrequested, "root=%#x", sc.BlockRoot())
}
requested := columnIndices[sc.ColumnIndex]
if !requested {
return errors.Wrapf(errUnrequested, "root=%#x index=%d", sc.BlockRoot(), sc.ColumnIndex)
}
return nil
}
}
func dataColumnIndexValidatorFromRangeReq(req *pb.DataColumnSidecarsByRangeRequest) DataColumnResponseValidation {
columnIds := make(map[uint64]bool)
for _, col := range req.Columns {
columnIds[col] = true
}
return func(sc blocks.RODataColumn) error {
requested := columnIds[sc.ColumnIndex]
if !requested {
return errors.Wrapf(errUnrequested, "root=%#x index=%d", sc.BlockRoot(), sc.ColumnIndex)
}
return nil
}
}
func dataColumnValidatorFromRangeReq(req *pb.DataColumnSidecarsByRangeRequest) DataColumnResponseValidation {
end := req.StartSlot + primitives.Slot(req.Count)
return func(sc blocks.RODataColumn) error {
if sc.Slot() < req.StartSlot || sc.Slot() >= end {
return errors.Wrapf(errDataColumnResponseOutOfBounds, "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) ([]blocks.ROBlob, error) {
sidecars := make([]blocks.ROBlob, 0)
// Attempt an extra read beyond max to check if the peer is violating the spec by
@@ -334,7 +543,7 @@ func readChunkEncodedBlobs(stream network.Stream, encoding encoder.NetworkEncodi
func readChunkedBlobSidecar(stream network.Stream, encoding encoder.NetworkEncoding, ctxMap ContextByteVersions, vf BlobResponseValidation) (blocks.ROBlob, error) {
var b blocks.ROBlob
pb := &ethpb.BlobSidecar{}
pb := &pb.BlobSidecar{}
decode := encoding.DecodeWithMaxLength
var (
code uint8

View File

@@ -36,6 +36,8 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/backfill/coverage"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
lruwrpr "github.com/prysmaticlabs/prysm/v5/cache/lru"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -49,17 +51,20 @@ import (
var _ runtime.Service = (*Service)(nil)
const rangeLimit uint64 = 1024
const seenBlockSize = 1000
const seenBlobSize = seenBlockSize * 4 // Each block can have max 4 blobs. Worst case 164kB for cache.
const seenUnaggregatedAttSize = 20000
const seenAggregatedAttSize = 16384
const seenSyncMsgSize = 1000 // Maximum of 512 sync committee members, 1000 is a safe amount.
const seenSyncContributionSize = 512 // Maximum of SYNC_COMMITTEE_SIZE as specified by the spec.
const seenExitSize = 100
const seenProposerSlashingSize = 100
const badBlockSize = 1000
const syncMetricsInterval = 10 * time.Second
const (
rangeLimit uint64 = 1024
seenBlockSize = 1000
seenBlobSize = seenBlockSize * 6 // Each block can have max 6 blobs.
seenDataColumnSize = seenBlockSize * 128 // Each block can have max 128 data columns.
seenUnaggregatedAttSize = 20000
seenAggregatedAttSize = 16384
seenSyncMsgSize = 1000 // Maximum of 512 sync committee members, 1000 is a safe amount.
seenSyncContributionSize = 512 // Maximum of SYNC_COMMITTEE_SIZE as specified by the spec.
seenExitSize = 100
seenProposerSlashingSize = 100
badBlockSize = 1000
syncMetricsInterval = 10 * time.Second
)
var (
// Seconds in one epoch.
@@ -102,6 +107,7 @@ type config struct {
type blockchainService interface {
blockchain.BlockReceiver
blockchain.BlobReceiver
blockchain.DataColumnReceiver
blockchain.HeadFetcher
blockchain.FinalizationFetcher
blockchain.ForkFetcher
@@ -133,6 +139,8 @@ type Service struct {
seenBlockCache *lru.Cache
seenBlobLock sync.RWMutex
seenBlobCache *lru.Cache
seenDataColumnLock sync.RWMutex
seenDataColumnCache *lru.Cache
seenAggregatedAttestationLock sync.RWMutex
seenAggregatedAttestationCache *lru.Cache
seenUnAggregatedAttestationLock sync.RWMutex
@@ -157,6 +165,9 @@ type Service struct {
verifierWaiter *verification.InitializerWaiter
newBlobVerifier verification.NewBlobVerifier
availableBlocker coverage.AvailableBlocker
dataColumsnReconstructionLock sync.Mutex
receivedDataColumnsFromRoot map[[fieldparams.RootLength]byte]map[uint64]bool
receivedDataColumnsFromRootLock sync.RWMutex
ctxMap ContextByteVersions
}
@@ -165,14 +176,15 @@ func NewService(ctx context.Context, opts ...Option) *Service {
c := gcache.New(pendingBlockExpTime /* exp time */, 0 /* disable janitor */)
ctx, cancel := context.WithCancel(ctx)
r := &Service{
ctx: ctx,
cancel: cancel,
chainStarted: abool.New(),
cfg: &config{clock: startup.NewClock(time.Unix(0, 0), [32]byte{})},
slotToPendingBlocks: c,
seenPendingBlocks: make(map[[32]byte]bool),
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
signatureChan: make(chan *signatureVerifier, verifierLimit),
ctx: ctx,
cancel: cancel,
chainStarted: abool.New(),
cfg: &config{clock: startup.NewClock(time.Unix(0, 0), [32]byte{})},
slotToPendingBlocks: c,
seenPendingBlocks: make(map[[32]byte]bool),
blkRootToPendingAtts: make(map[[32]byte][]*ethpb.SignedAggregateAttestationAndProof),
signatureChan: make(chan *signatureVerifier, verifierLimit),
receivedDataColumnsFromRoot: make(map[[32]byte]map[uint64]bool),
}
for _, opt := range opts {
if err := opt(r); err != nil {
@@ -239,6 +251,11 @@ func (s *Service) Start() {
// Update sync metrics.
async.RunEvery(s.ctx, syncMetricsInterval, s.updateMetrics)
// Run data column sampling
if features.Get().EnablePeerDAS {
go s.dataColumnSampling(s.ctx)
}
}
// Stop the regular sync service.
@@ -276,6 +293,7 @@ func (s *Service) Status() error {
func (s *Service) initCaches() {
s.seenBlockCache = lruwrpr.New(seenBlockSize)
s.seenBlobCache = lruwrpr.New(seenBlobSize)
s.seenDataColumnCache = lruwrpr.New(seenDataColumnSize)
s.seenAggregatedAttestationCache = lruwrpr.New(seenAggregatedAttSize)
s.seenUnAggregatedAttestationCache = lruwrpr.New(seenUnaggregatedAttSize)
s.seenSyncMessageCache = lruwrpr.New(seenSyncMsgSize)

View File

@@ -137,13 +137,32 @@ func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
// New Gossip Topic in Deneb
if epoch >= params.BeaconConfig().DenebForkEpoch {
s.subscribeStaticWithSubnets(
p2p.BlobSubnetTopicFormat,
s.validateBlob, /* validator */
s.blobSubscriber, /* message handler */
digest,
params.BeaconConfig().BlobsidecarSubnetCount,
)
if features.Get().EnablePeerDAS {
if flags.Get().SubscribeToAllSubnets {
s.subscribeStaticWithSubnets(
p2p.DataColumnSubnetTopicFormat,
s.validateDataColumn, /* validator */
s.dataColumnSubscriber, /* message handler */
digest,
params.BeaconConfig().DataColumnSidecarSubnetCount,
)
} else {
s.subscribeDynamicWithColumnSubnets(
p2p.DataColumnSubnetTopicFormat,
s.validateDataColumn, /* validator */
s.dataColumnSubscriber, /* message handler */
digest,
)
}
} else {
s.subscribeStaticWithSubnets(
p2p.BlobSubnetTopicFormat,
s.validateBlob, /* validator */
s.blobSubscriber, /* message handler */
digest,
params.BeaconConfig().BlobsidecarSubnetCount,
)
}
}
}
@@ -646,6 +665,87 @@ func (s *Service) subscribeDynamicWithSyncSubnets(
}()
}
// subscribe missing subnets for our persistent columns.
func (s *Service) subscribeColumnSubnet(
subscriptions map[uint64]*pubsub.Subscription,
idx uint64,
digest [4]byte,
validate wrappedVal,
handle subHandler,
) {
// do not subscribe if we have no peers in the same
// subnet
topic := p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.DataColumnSidecar{})]
subnetTopic := fmt.Sprintf(topic, digest, idx)
// check if subscription exists and if not subscribe the relevant subnet.
if _, exists := subscriptions[idx]; !exists {
subscriptions[idx] = s.subscribeWithBase(subnetTopic, validate, handle)
}
if !s.validPeersExist(subnetTopic) {
log.Debugf("No peers found subscribed to column gossip subnet with "+
"column index %d. Searching network for peers subscribed to the subnet.", idx)
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
if err != nil {
log.WithError(err).Debug("Could not search for peers")
}
}
}
func (s *Service) subscribeDynamicWithColumnSubnets(
topicFormat string,
validate wrappedVal,
handle subHandler,
digest [4]byte,
) {
genRoot := s.cfg.clock.GenesisValidatorsRoot()
_, e, err := forks.RetrieveForkDataFromDigest(digest, genRoot[:])
if err != nil {
panic(err)
}
base := p2p.GossipTopicMappings(topicFormat, e)
if base == nil {
panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topicFormat))
}
subscriptions := make(map[uint64]*pubsub.Subscription, params.BeaconConfig().DataColumnSidecarSubnetCount)
genesis := s.cfg.clock.GenesisTime()
ticker := slots.NewSlotTicker(genesis, params.BeaconConfig().SecondsPerSlot)
go func() {
for {
select {
case <-s.ctx.Done():
ticker.Done()
return
case <-ticker.C():
if s.chainStarted.IsSet() && s.cfg.initialSync.Syncing() {
continue
}
valid, err := isDigestValid(digest, genesis, genRoot)
if err != nil {
log.Error(err)
continue
}
if !valid {
log.Warnf("Column subnets with digest %#x are no longer valid, unsubscribing from all of them.", digest)
// Unsubscribes from all our current subnets.
s.reValidateSubscriptions(subscriptions, []uint64{}, topicFormat, digest)
ticker.Done()
return
}
wantedSubs := s.retrieveActiveColumnSubnets()
// Resize as appropriate.
s.reValidateSubscriptions(subscriptions, wantedSubs, topicFormat, digest)
// subscribe desired column subnets.
for _, idx := range wantedSubs {
s.subscribeColumnSubnet(subscriptions, idx, digest, validate, handle)
}
}
}
}()
}
// lookup peers for attester specific subnets.
func (s *Service) lookupAttesterSubnets(digest [4]byte, idx uint64) {
topic := p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.Attestation{})]
@@ -697,6 +797,14 @@ func (*Service) retrieveActiveSyncSubnets(currEpoch primitives.Epoch) []uint64 {
return slice.SetUint64(subs)
}
func (*Service) retrieveActiveColumnSubnets() []uint64 {
subs, ok, _ := cache.ColumnSubnetIDs.GetColumnSubnets()
if !ok {
return nil
}
return subs
}
// filters out required peers for the node to function, not
// pruning peers who are in our attestation subnets.
func (s *Service) filterNeededPeers(pids []peer.ID) []peer.ID {

View File

@@ -36,11 +36,11 @@ func (s *Service) committeeIndexBeaconAttestationSubscriber(_ context.Context, m
return s.cfg.attPool.SaveUnaggregatedAttestation(a)
}
func (_ *Service) persistentSubnetIndices() []uint64 {
func (*Service) persistentSubnetIndices() []uint64 {
return cache.SubnetIDs.GetAllSubnets()
}
func (_ *Service) aggregatorSubnetIndices(currentSlot primitives.Slot) []uint64 {
func (*Service) aggregatorSubnetIndices(currentSlot primitives.Slot) []uint64 {
endEpoch := slots.ToEpoch(currentSlot) + 1
endSlot := params.BeaconConfig().SlotsPerEpoch.Mul(uint64(endEpoch))
var commIds []uint64
@@ -50,7 +50,7 @@ func (_ *Service) aggregatorSubnetIndices(currentSlot primitives.Slot) []uint64
return slice.SetUint64(commIds)
}
func (_ *Service) attesterSubnetIndices(currentSlot primitives.Slot) []uint64 {
func (*Service) attesterSubnetIndices(currentSlot primitives.Slot) []uint64 {
endEpoch := slots.ToEpoch(currentSlot) + 1
endSlot := params.BeaconConfig().SlotsPerEpoch.Mul(uint64(endEpoch))
var commIds []uint64

View File

@@ -13,7 +13,7 @@ import (
func (s *Service) blobSubscriber(ctx context.Context, msg proto.Message) error {
b, ok := msg.(blocks.VerifiedROBlob)
if !ok {
return fmt.Errorf("message was not type blocks.ROBlob, type=%T", msg)
return fmt.Errorf("message was not type blocks.VerifiedROBlob, type=%T", msg)
}
s.setSeenBlobIndex(b.Slot(), b.ProposerIndex(), b.Index)

View File

@@ -0,0 +1,40 @@
package sync
import (
"context"
"fmt"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
opfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"google.golang.org/protobuf/proto"
)
func (s *Service) dataColumnSubscriber(ctx context.Context, msg proto.Message) error {
dc, ok := msg.(blocks.VerifiedRODataColumn)
if !ok {
return fmt.Errorf("message was not type blocks.VerifiedRODataColumn, type=%T", msg)
}
s.setSeenDataColumnIndex(dc.SignedBlockHeader.Header.Slot, dc.SignedBlockHeader.Header.ProposerIndex, dc.ColumnIndex)
s.setReceivedDataColumn(dc.BlockRoot(), dc.ColumnIndex)
if err := s.cfg.chain.ReceiveDataColumn(ctx, dc); err != nil {
return errors.Wrap(err, "receive data column")
}
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
Type: opfeed.DataColumnSidecarReceived,
Data: &opfeed.DataColumnSidecarReceivedData{
DataColumn: &dc,
},
})
// Reconstruct the data columns if needed.
if err := s.reconstructDataColumns(ctx, dc); err != nil {
return errors.Wrap(err, "reconstruct data columns")
}
return nil
}

View File

@@ -169,6 +169,16 @@ func blobFields(b blocks.ROBlob) logrus.Fields {
}
}
func columnFields(b blocks.RODataColumn) logrus.Fields {
return logrus.Fields{
"slot": b.Slot(),
"proposerIndex": b.ProposerIndex(),
"blockRoot": fmt.Sprintf("%#x", b.BlockRoot()),
"kzgCommitments": fmt.Sprintf("%#x", b.KzgCommitments),
"columnIndex": b.ColumnIndex,
}
}
func computeSubnetForBlobSidecar(index uint64) uint64 {
return index % params.BeaconConfig().BlobsidecarSubnetCount
}

View File

@@ -0,0 +1,170 @@
package sync
import (
"context"
"fmt"
"strings"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
coreBlocks "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
prysmTime "github.com/prysmaticlabs/prysm/v5/time"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"github.com/sirupsen/logrus"
)
func (s *Service) validateDataColumn(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
receivedTime := prysmTime.Now()
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
if msg.Topic == nil {
return pubsub.ValidationReject, errInvalidTopic
}
m, err := s.decodePubsubMessage(msg)
if err != nil {
log.WithError(err).Error("Failed to decode message")
return pubsub.ValidationReject, err
}
ds, ok := m.(*eth.DataColumnSidecar)
if !ok {
log.WithField("message", m).Error("Message is not of type *eth.DataColumnSidecar")
return pubsub.ValidationReject, errWrongMessage
}
if ds.ColumnIndex >= params.BeaconConfig().NumberOfColumns {
return pubsub.ValidationReject, errors.Errorf("invalid column index provided, got %d", ds.ColumnIndex)
}
want := fmt.Sprintf("data_column_sidecar_%d", computeSubnetForColumnSidecar(ds.ColumnIndex))
if !strings.Contains(*msg.Topic, want) {
log.Debug("Column Sidecar index does not match topic")
return pubsub.ValidationReject, fmt.Errorf("wrong topic name: %s", *msg.Topic)
}
if err := slots.VerifyTime(uint64(s.cfg.clock.GenesisTime().Unix()), ds.SignedBlockHeader.Header.Slot, params.BeaconConfig().MaximumGossipClockDisparityDuration()); err != nil {
log.WithError(err).Debug("Ignored sidecar: could not verify slot time")
return pubsub.ValidationIgnore, nil
}
cp := s.cfg.chain.FinalizedCheckpt()
startSlot, err := slots.EpochStart(cp.Epoch)
if err != nil {
log.WithError(err).Debug("Ignored column sidecar: could not calculate epoch start slot")
return pubsub.ValidationIgnore, nil
}
if startSlot >= ds.SignedBlockHeader.Header.Slot {
err := fmt.Errorf("finalized slot %d greater or equal to block slot %d", startSlot, ds.SignedBlockHeader.Header.Slot)
log.Debug(err)
return pubsub.ValidationIgnore, err
}
// Handle sidecar when the parent is unknown.
if !s.cfg.chain.HasBlock(ctx, [32]byte(ds.SignedBlockHeader.Header.ParentRoot)) {
err := errors.Errorf("unknown parent for data column sidecar with slot %d and parent root %#x", ds.SignedBlockHeader.Header.Slot, ds.SignedBlockHeader.Header.ParentRoot)
log.WithError(err).Debug("Could not identify parent for data column sidecar")
return pubsub.ValidationIgnore, err
}
if s.hasBadBlock([32]byte(ds.SignedBlockHeader.Header.ParentRoot)) {
bRoot, err := ds.SignedBlockHeader.Header.HashTreeRoot()
if err != nil {
return pubsub.ValidationIgnore, err
}
s.setBadBlock(ctx, bRoot)
return pubsub.ValidationReject, errors.Errorf("column sidecar with bad parent provided")
}
parentSlot, err := s.cfg.chain.RecentBlockSlot([32]byte(ds.SignedBlockHeader.Header.ParentRoot))
if err != nil {
return pubsub.ValidationIgnore, err
}
if ds.SignedBlockHeader.Header.Slot <= parentSlot {
return pubsub.ValidationReject, errors.Errorf("invalid column sidecar slot: %d", ds.SignedBlockHeader.Header.Slot)
}
if !s.cfg.chain.InForkchoice([32]byte(ds.SignedBlockHeader.Header.ParentRoot)) {
return pubsub.ValidationReject, blockchain.ErrNotDescendantOfFinalized
}
if err := blocks.VerifyKZGInclusionProofColumn(ds); err != nil {
return pubsub.ValidationReject, err
}
verified, err := peerdas.VerifyDataColumnSidecarKZGProofs(ds)
if err != nil {
return pubsub.ValidationReject, err
}
if !verified {
return pubsub.ValidationReject, errors.New("failed to verify kzg proof of column")
}
parentState, err := s.cfg.stateGen.StateByRoot(ctx, [32]byte(ds.SignedBlockHeader.Header.ParentRoot))
if err != nil {
return pubsub.ValidationIgnore, err
}
if err := coreBlocks.VerifyBlockHeaderSignatureUsingCurrentFork(parentState, ds.SignedBlockHeader); err != nil {
return pubsub.ValidationReject, err
}
// In the event the block is more than an epoch ahead from its
// parent state, we have to advance the state forward.
parentRoot := ds.SignedBlockHeader.Header.ParentRoot
parentState, err = transition.ProcessSlotsUsingNextSlotCache(ctx, parentState, parentRoot, ds.SignedBlockHeader.Header.Slot)
if err != nil {
return pubsub.ValidationIgnore, err
}
idx, err := helpers.BeaconProposerIndex(ctx, parentState)
if err != nil {
return pubsub.ValidationIgnore, err
}
if ds.SignedBlockHeader.Header.ProposerIndex != idx {
return pubsub.ValidationReject, errors.New("incorrect proposer index")
}
startTime, err := slots.ToTime(uint64(s.cfg.chain.GenesisTime().Unix()), ds.SignedBlockHeader.Header.Slot)
if err != nil {
return pubsub.ValidationIgnore, err
}
sinceSlotStartTime := receivedTime.Sub(startTime)
validationTime := s.cfg.clock.Now().Sub(receivedTime)
log.WithFields(logrus.Fields{
"sinceSlotStartTime": sinceSlotStartTime,
"validationTime": validationTime,
"columnIndex": ds.ColumnIndex,
}).Debug("Received data column sidecar")
// TODO: Transform this whole function so it looks like to the `validateBlob`
// with the tiny verifiers inside.
roDataColumn, err := blocks.NewRODataColumn(ds)
if err != nil {
return pubsub.ValidationReject, errors.Wrap(err, "new RO data columns")
}
verifiedRODataColumn := blocks.NewVerifiedRODataColumn(roDataColumn)
msg.ValidatorData = verifiedRODataColumn
return pubsub.ValidationAccept, nil
}
// Sets the data column with the same slot, proposer index, and data column index as seen.
func (s *Service) setSeenDataColumnIndex(slot primitives.Slot, proposerIndex primitives.ValidatorIndex, index uint64) {
s.seenDataColumnLock.Lock()
defer s.seenDataColumnLock.Unlock()
b := append(bytesutil.Bytes32(uint64(slot)), bytesutil.Bytes32(uint64(proposerIndex))...)
b = append(b, bytesutil.Bytes32(index)...)
s.seenDataColumnCache.Add(string(b), true)
}
func computeSubnetForColumnSidecar(colIdx uint64) uint64 {
return colIdx % params.BeaconConfig().DataColumnSidecarSubnetCount
}

View File

@@ -1,6 +1,8 @@
package verify
import (
"reflect"
"github.com/pkg/errors"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
@@ -9,10 +11,17 @@ import (
)
var (
errBlobVerification = errors.New("unable to verify blobs")
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")
errBlobVerification = errors.New("unable to verify blobs")
errColumnVerification = errors.New("unable to verify column")
ErrIncorrectBlobIndex = errors.New("incorrect blob index")
ErrIncorrectColumnIndex = errors.New("incorrect column index")
ErrBlobBlockMisaligned = errors.Wrap(errBlobVerification, "root of block header in blob sidecar does not match block root")
ErrColumnBlockMisaligned = errors.Wrap(errColumnVerification, "root of block header in column sidecar does not match block root")
ErrMismatchedBlobCommitments = errors.Wrap(errBlobVerification, "commitments at given slot, root and index do not match")
ErrMismatchedColumnCommitments = errors.Wrap(errColumnVerification, "commitments at given slot, root and index do not match")
)
// BlobAlignsWithBlock verifies if the blob aligns with the block.
@@ -41,3 +50,26 @@ func BlobAlignsWithBlock(blob blocks.ROBlob, block blocks.ROBlock) error {
}
return nil
}
func ColumnAlignsWithBlock(col blocks.RODataColumn, block blocks.ROBlock) error {
if block.Version() < version.Deneb {
return nil
}
if col.ColumnIndex >= fieldparams.NumberOfColumns {
return errors.Wrapf(ErrIncorrectColumnIndex, "index %d exceeds NUMBERS_OF_COLUMN %d", col.ColumnIndex, fieldparams.NumberOfColumns)
}
if col.BlockRoot() != block.Root() {
return ErrColumnBlockMisaligned
}
// Verify commitment byte values match
commits, err := block.Block().Body().BlobKzgCommitments()
if err != nil {
return err
}
if !reflect.DeepEqual(commits, col.KzgCommitments) {
return errors.Wrapf(ErrMismatchedColumnCommitments, "commitment %#v != block commitment %#v for block root %#x at slot %d ", col.KzgCommitments, commits, block.Root(), col.Slot())
}
return nil
}

View File

@@ -72,6 +72,11 @@ type Flags struct {
PrepareAllPayloads bool // PrepareAllPayloads informs the engine to prepare a block on every slot.
// BlobSaveFsync requires blob saving to block on fsync to ensure blobs are durably persisted before passing DA.
BlobSaveFsync bool
// EnablePeerDAS enables running the node with the experimental data availability sampling scheme.
EnablePeerDAS bool
// DataColumnsWithholdCount specifies the likelihood of withholding a data column sidecar when proposing a block (percentage)
DataColumnsWithholdCount int
SaveInvalidBlock bool // SaveInvalidBlock saves invalid block to temp.
SaveInvalidBlob bool // SaveInvalidBlob saves invalid blob to temp.
@@ -254,6 +259,16 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
logEnabled(EnableQUIC)
cfg.EnableQUIC = true
}
cfg.EnablePeerDAS = true
if ctx.IsSet(EnablePeerDAS.Name) {
logEnabled(EnablePeerDAS)
cfg.EnablePeerDAS = true
}
if ctx.IsSet(DataColumnsWithholdCount.Name) {
logEnabled(DataColumnsWithholdCount)
cfg.DataColumnsWithholdCount = ctx.Int(DataColumnsWithholdCount.Name)
}
if ctx.IsSet(eip6110ValidatorCache.Name) {
logEnabled(eip6110ValidatorCache)

View File

@@ -171,6 +171,18 @@ var (
Name: "eip6110-validator-cache",
Usage: "Enables the EIP-6110 validator cache.",
}
// EnablePeerDAS is a flag for enabling the peer data availability sampling.
EnablePeerDAS = &cli.BoolFlag{
Name: "peer-das",
Usage: "Enables Prysm to run with the experimental peer data availability sampling scheme.",
}
// DataColumnsWithholdCount is a flag for withholding data columns when proposing a block.
DataColumnsWithholdCount = &cli.IntFlag{
Name: "data-columns-withhold-count",
Usage: "Number of columns to withhold when proposing a block. DO NOT USE IN PRODUCTION.",
Value: 0,
Hidden: true,
}
)
// devModeFlags holds list of flags that are set when development mode is on.
@@ -229,6 +241,8 @@ var BeaconChainFlags = append(deprecatedBeaconFlags, append(deprecatedFlags, []c
BlobSaveFsync,
EnableQUIC,
eip6110ValidatorCache,
EnablePeerDAS,
DataColumnsWithholdCount,
}...)...)
// E2EBeaconChainFlags contains a list of the beacon chain feature flags to be tested in E2E.

View File

@@ -38,4 +38,5 @@ const (
PendingConsolidationsLimit = 262144 // Maximum number of pending consolidations in the beacon state.
MaxDepositReceiptsPerPayload = 8192 // Maximum number of deposit receipts in an execution payload.
MaxWithdrawalRequestsPerPayload = 16 // Maximum number of execution layer withdrawal requests in an execution payload.
NumberOfColumns = 128 // NumberOfColumns refers to the specified number of data columns that can exist in a network.
)

View File

@@ -38,4 +38,5 @@ const (
PendingConsolidationsLimit = 64 // Maximum number of pending consolidations in the beacon state.
MaxDepositReceiptsPerPayload = 4 // Maximum number of deposit receipts in an execution payload.
MaxWithdrawalRequestsPerPayload = 2 // Maximum number of execution layer withdrawal requests in an execution payload.
NumberOfColumns = 128 // NumberOfColumns refers to the specified number of data columns that can exist in a network.
)

View File

@@ -252,6 +252,13 @@ type BeaconChainConfig struct {
MaxWithdrawalRequestsPerPayload uint64 `yaml:"MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD" spec:"true"` // MaxWithdrawalRequestsPerPayload is the maximum number of execution layer withdrawal requests in each payload.
UnsetDepositReceiptsStartIndex uint64 `yaml:"UNSET_DEPOSIT_RECEIPTS_START_INDEX" spec:"true"` // UnsetDepositReceiptsStartIndex is used to check the start index for eip6110
// PeerDAS Values
SamplesPerSlot uint64 `yaml:"SAMPLES_PER_SLOT"` // SamplesPerSlot refers to the humber of random samples a node queries per slot.
CustodyRequirement uint64 `yaml:"CUSTODY_REQUIREMENT"` // CustodyRequirement refers to the minimum amount of subnets a peer must custody and serve samples from.
MinEpochsForDataColumnSidecarsRequest primitives.Epoch `yaml:"MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS"` // MinEpochsForDataColumnSidecarsRequest is the minimum number of epochs the node will keep the data columns for.
MaxCellsInExtendedMatrix uint64 `yaml:"MAX_CELLS_IN_EXTENDED_MATRIX" spec:"true"` // MaxCellsInExtendedMatrix is the full data of one-dimensional erasure coding extended blobs (in row major format).
NumberOfColumns uint64 `yaml:"NUMBER_OF_COLUMNS" spec:"true"` // NumberOfColumns in the extended data matrix.
// Networking Specific Parameters
GossipMaxSize uint64 `yaml:"GOSSIP_MAX_SIZE" spec:"true"` // GossipMaxSize is the maximum allowed size of uncompressed gossip messages.
MaxChunkSize uint64 `yaml:"MAX_CHUNK_SIZE" spec:"true"` // MaxChunkSize is the maximum allowed size of uncompressed req/resp chunked responses.
@@ -269,10 +276,6 @@ type BeaconChainConfig struct {
AttestationSubnetPrefixBits uint64 `yaml:"ATTESTATION_SUBNET_PREFIX_BITS" spec:"true"` // AttestationSubnetPrefixBits is defined as (ceillog2(ATTESTATION_SUBNET_COUNT) + ATTESTATION_SUBNET_EXTRA_BITS).
SubnetsPerNode uint64 `yaml:"SUBNETS_PER_NODE" spec:"true"` // SubnetsPerNode is the number of long-lived subnets a beacon node should be subscribed to.
NodeIdBits uint64 `yaml:"NODE_ID_BITS" spec:"true"` // NodeIdBits defines the bit length of a node id.
// PeerDAS
NumberOfColumns uint64 `yaml:"NUMBER_OF_COLUMNS" spec:"true"` // NumberOfColumns in the extended data matrix.
MaxCellsInExtendedMatrix uint64 `yaml:"MAX_CELLS_IN_EXTENDED_MATRIX" spec:"true"` // MaxCellsInExtendedMatrix is the full data of one-dimensional erasure coding extended blobs (in row major format).
}
// InitializeForkSchedule initializes the schedules forks baked into the config.

View File

@@ -292,6 +292,13 @@ var mainnetBeaconConfig = &BeaconChainConfig{
MaxWithdrawalRequestsPerPayload: 16,
UnsetDepositReceiptsStartIndex: math.MaxUint64,
// PeerDAS
SamplesPerSlot: 8,
CustodyRequirement: 1,
MinEpochsForDataColumnSidecarsRequest: 4096,
NumberOfColumns: 128,
MaxCellsInExtendedMatrix: 768,
// Values related to networking parameters.
GossipMaxSize: 10 * 1 << 20, // 10 MiB
MaxChunkSize: 10 * 1 << 20, // 10 MiB
@@ -309,10 +316,6 @@ var mainnetBeaconConfig = &BeaconChainConfig{
AttestationSubnetPrefixBits: 6,
SubnetsPerNode: 2,
NodeIdBits: 256,
// PeerDAS
NumberOfColumns: 128,
MaxCellsInExtendedMatrix: 768,
}
// MainnetTestConfig provides a version of the mainnet config that has a different name

View File

@@ -11,6 +11,7 @@ go_library(
"proto.go",
"roblob.go",
"roblock.go",
"rodatacolumn.go",
"setters.go",
"types.go",
],

View File

@@ -8,6 +8,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/container/trie"
"github.com/prysmaticlabs/prysm/v5/encoding/ssz"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
@@ -47,6 +48,35 @@ func VerifyKZGInclusionProof(blob ROBlob) error {
return nil
}
// VerifyKZGInclusionProofColumn verifies the Merkle proof in a data column sidecar against
// the beacon block body root.
func VerifyKZGInclusionProofColumn(sc *ethpb.DataColumnSidecar) error {
if sc.SignedBlockHeader == nil {
return errNilBlockHeader
}
if sc.SignedBlockHeader.Header == nil {
return errNilBlockHeader
}
root := sc.SignedBlockHeader.Header.BodyRoot
if len(root) != field_params.RootLength {
return errInvalidBodyRoot
}
leaves := leavesFromCommitments(sc.KzgCommitments)
sparse, err := trie.GenerateTrieFromItems(leaves, field_params.LogMaxBlobCommitments)
if err != nil {
return err
}
rt, err := sparse.HashTreeRoot()
if err != nil {
return err
}
verified := trie.VerifyMerkleProof(root, rt[:], kzgPosition, sc.KzgCommitmentsInclusionProof)
if !verified {
return errInvalidInclusionProof
}
return nil
}
// MerkleProofKZGCommitment constructs a Merkle proof of inclusion of the KZG
// commitment of index `index` into the Beacon Block with the given `body`
func MerkleProofKZGCommitment(body interfaces.ReadOnlyBeaconBlockBody, index int) ([][]byte, error) {
@@ -80,6 +110,35 @@ func MerkleProofKZGCommitment(body interfaces.ReadOnlyBeaconBlockBody, index int
return proof, nil
}
// MerkleProofKZGCommitments constructs a Merkle proof of inclusion of the KZG
// commitments into the Beacon Block with the given `body`
func MerkleProofKZGCommitments(body interfaces.ReadOnlyBeaconBlockBody) ([][]byte, error) {
bodyVersion := body.Version()
if bodyVersion < version.Deneb {
return nil, errUnsupportedBeaconBlockBody
}
membersRoots, err := topLevelRoots(body)
if err != nil {
return nil, errors.Wrap(err, "top level roots")
}
sparse, err := trie.GenerateTrieFromItems(membersRoots, logBodyLength)
if err != nil {
return nil, errors.Wrap(err, "generate trie from items")
}
proof, err := sparse.MerkleProof(kzgPosition)
if err != nil {
return nil, errors.Wrap(err, "merkle proof")
}
// Remove the last element as it is a mix in with the number of
// elements in the trie.
proof = proof[:len(proof)-1]
return proof, nil
}
// leavesFromCommitments hashes each commitment to construct a slice of roots
func leavesFromCommitments(commitments [][]byte) [][]byte {
leaves := make([][]byte, len(commitments))

View File

@@ -98,9 +98,11 @@ func (s ROBlockSlice) Len() int {
// BlockWithROBlobs is a wrapper that collects the block and blob values together.
// This is helpful because these values are collated from separate RPC requests.
// TODO: Use a more generic name
type BlockWithROBlobs struct {
Block ROBlock
Blobs []ROBlob
Block ROBlock
Blobs []ROBlob
Columns []RODataColumn
}
// BlockWithROBlobsSlice gives convenient access to getting a slice of just the ROBlocks,

View File

@@ -0,0 +1,122 @@
package blocks
import (
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
// RODataColumn represents a read-only data column sidecar with its block root.
type RODataColumn struct {
*ethpb.DataColumnSidecar
root [fieldparams.RootLength]byte
}
func roDataColumnNilCheck(dc *ethpb.DataColumnSidecar) error {
// Check if the data column is nil.
if dc == nil {
return errNilDataColumn
}
// Check if the data column header is nil.
if dc.SignedBlockHeader == nil || dc.SignedBlockHeader.Header == nil {
return errNilBlockHeader
}
// Check if the data column signature is nil.
if len(dc.SignedBlockHeader.Signature) == 0 {
return errMissingBlockSignature
}
return nil
}
// NewRODataColumnWithRoot creates a new RODataColumn with a given root.
// TODO: Add test
func NewRODataColumnWithRoot(dc *ethpb.DataColumnSidecar, root [fieldparams.RootLength]byte) (RODataColumn, error) {
// Check if the data column is nil.
if err := roDataColumnNilCheck(dc); err != nil {
return RODataColumn{}, err
}
return RODataColumn{DataColumnSidecar: dc, root: root}, nil
}
// NewRODataColumn creates a new RODataColumn by computing the HashTreeRoot of the header.
// TODO: Add test
func NewRODataColumn(dc *ethpb.DataColumnSidecar) (RODataColumn, error) {
if err := roDataColumnNilCheck(dc); err != nil {
return RODataColumn{}, err
}
root, err := dc.SignedBlockHeader.Header.HashTreeRoot()
if err != nil {
return RODataColumn{}, err
}
return RODataColumn{DataColumnSidecar: dc, root: root}, nil
}
// BlockRoot returns the root of the block.
// TODO: Add test
func (dc *RODataColumn) BlockRoot() [fieldparams.RootLength]byte {
return dc.root
}
// Slot returns the slot of the data column sidecar.
// TODO: Add test
func (dc *RODataColumn) Slot() primitives.Slot {
return dc.SignedBlockHeader.Header.Slot
}
// ParentRoot returns the parent root of the data column sidecar.
// TODO: Add test
func (dc *RODataColumn) ParentRoot() [fieldparams.RootLength]byte {
return bytesutil.ToBytes32(dc.SignedBlockHeader.Header.ParentRoot)
}
// ParentRootSlice returns the parent root of the data column sidecar as a byte slice.
// TODO: Add test
func (dc *RODataColumn) ParentRootSlice() []byte {
return dc.SignedBlockHeader.Header.ParentRoot
}
// BodyRoot returns the body root of the data column sidecar.
// TODO: Add test
func (dc *RODataColumn) BodyRoot() [fieldparams.RootLength]byte {
return bytesutil.ToBytes32(dc.SignedBlockHeader.Header.BodyRoot)
}
// ProposerIndex returns the proposer index of the data column sidecar.
// TODO: Add test
func (dc *RODataColumn) ProposerIndex() primitives.ValidatorIndex {
return dc.SignedBlockHeader.Header.ProposerIndex
}
// BlockRootSlice returns the block root as a byte slice. This is often more convenient/concise
// than setting a tmp var to BlockRoot(), just so that it can be sliced.
// TODO: Add test
func (dc *RODataColumn) BlockRootSlice() []byte {
return dc.root[:]
}
// RODataColumn is a custom type for a []RODataColumn, allowing methods to be defined that act on a slice of RODataColumn.
type RODataColumnSlice []RODataColumn
// Protos is a helper to make a more concise conversion from []RODataColumn->[]*ethpb.DataColumnSidecar.
func (s RODataColumnSlice) Protos() []*ethpb.DataColumnSidecar {
pb := make([]*ethpb.DataColumnSidecar, len(s))
for i := range s {
pb[i] = s[i].DataColumnSidecar
}
return pb
}
// VerifiedRODataColumn represents an RODataColumn that has undergone full verification (eg block sig, inclusion proof, commitment check).
type VerifiedRODataColumn struct {
RODataColumn
}
// NewVerifiedRODataColumn "upgrades" an RODataColumn to a VerifiedRODataColumn. This method should only be used by the verification package.
func NewVerifiedRODataColumn(rodc RODataColumn) VerifiedRODataColumn {
return VerifiedRODataColumn{RODataColumn: rodc}
}

View File

@@ -28,6 +28,7 @@ var (
// ErrUnsupportedVersion for beacon block methods.
ErrUnsupportedVersion = errors.New("unsupported beacon block version")
errNilBlob = errors.New("received nil blob sidecar")
errNilDataColumn = errors.New("received nil data column sidecar")
errNilBlock = errors.New("received nil beacon block")
errNilBlockBody = errors.New("received nil beacon block body")
errIncorrectBlockVersion = errors.New(incorrectBlockVersion)

View File

@@ -860,8 +860,8 @@ def prysm_deps():
importpath = "github.com/ethereum/c-kzg-4844",
patch_args = ["-p1"],
patches = ["//third_party:com_github_ethereum_c_kzg_4844.patch"],
sum = "h1:3MS1s4JtA868KpJxroZoepdV0ZKBp3u/O5HcZ7R3nlY=",
version = "v0.4.0",
sum = "h1:EV64oiDZGl97cptCieq1X7KrumSbP4MhmKg0/ll65wo=",
version = "v1.0.2-0.20240507203752-26d3b4156f7a",
)
go_repository(
name = "com_github_ethereum_go_ethereum",

4
go.mod
View File

@@ -17,7 +17,9 @@ require (
github.com/dgraph-io/ristretto v0.0.4-0.20210318174700-74754f61e018
github.com/dustin/go-humanize v1.0.0
github.com/emicklei/dot v0.11.0
github.com/ethereum/c-kzg-4844 v1.0.2-0.20240507203752-26d3b4156f7a
github.com/ethereum/go-ethereum v1.13.5
github.com/ferranbt/fastssz v0.0.0-20210120143747-11b9eff30ea9
github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5
github.com/fsnotify/fsnotify v1.6.0
github.com/ghodss/yaml v1.0.0
@@ -134,8 +136,6 @@ require (
github.com/docker/go-units v0.5.0 // indirect
github.com/dop251/goja v0.0.0-20230806174421-c933cf95e127 // indirect
github.com/elastic/gosigar v0.14.2 // indirect
github.com/ethereum/c-kzg-4844 v0.4.0 // indirect
github.com/ferranbt/fastssz v0.0.0-20210120143747-11b9eff30ea9 // indirect
github.com/flynn/noise v1.1.0 // indirect
github.com/francoispqt/gojay v1.2.13 // indirect
github.com/getsentry/sentry-go v0.25.0 // indirect

4
go.sum
View File

@@ -262,8 +262,8 @@ github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5y
github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk=
github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
github.com/ethereum/c-kzg-4844 v0.4.0 h1:3MS1s4JtA868KpJxroZoepdV0ZKBp3u/O5HcZ7R3nlY=
github.com/ethereum/c-kzg-4844 v0.4.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0=
github.com/ethereum/c-kzg-4844 v1.0.2-0.20240507203752-26d3b4156f7a h1:EV64oiDZGl97cptCieq1X7KrumSbP4MhmKg0/ll65wo=
github.com/ethereum/c-kzg-4844 v1.0.2-0.20240507203752-26d3b4156f7a/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0=
github.com/ethereum/go-ethereum v1.13.5 h1:U6TCRciCqZRe4FPXmy1sMGxTfuk8P7u2UoinF3VbaFk=
github.com/ethereum/go-ethereum v1.13.5/go.mod h1:yMTu38GSuyxaYzQMViqNmQ1s3cE84abZexQmTgenWk0=
github.com/evanphx/json-patch v4.9.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=

View File

@@ -50,8 +50,8 @@ ssz_gen_marshal(
go_proto = ":go_proto",
includes = [
"//consensus-types/primitives:go_default_library",
"//proto/engine/v1:go_default_library",
"//math:go_default_library",
"//proto/engine/v1:go_default_library",
],
objs = [
"BeaconBlockAltair",
@@ -90,6 +90,7 @@ ssz_gen_marshal(
"SignedContributionAndProof",
"BeaconBlocksByRangeRequest",
"BlobSidecarsByRangeRequest",
"DataColumnSidecarsByRangeRequest",
"ENRForkID",
"MetaDataV0",
"MetaDataV1",
@@ -150,6 +151,7 @@ ssz_gen_marshal(
"Consolidation",
"SignedConsolidation",
"PendingConsolidation",
"DataColumnSidecar",
],
)
@@ -253,6 +255,7 @@ ssz_proto_files(
"beacon_block.proto",
"beacon_state.proto",
"blobs.proto",
"data_columns.proto",
"sync_committee.proto",
"withdrawals.proto",
],

Some files were not shown because too many files have changed in this diff Show More