mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-10 05:47:59 -05:00
Compare commits
13 Commits
electra-up
...
peerDASHac
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
ed55928153 | ||
|
|
c3598c6939 | ||
|
|
f2d956d688 | ||
|
|
a5af4d8a90 | ||
|
|
5976319a85 | ||
|
|
3fdb85a37d | ||
|
|
6c22814ed1 | ||
|
|
f2019dd053 | ||
|
|
92de4a1274 | ||
|
|
e498183d74 | ||
|
|
9a98e194c0 | ||
|
|
ebcc1404b6 | ||
|
|
2bef9563a1 |
1
.bazelrc
1
.bazelrc
@@ -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
|
||||
|
||||
@@ -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",
|
||||
@@ -157,6 +159,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",
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,6 +6,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
|
||||
"github.com/sirupsen/logrus"
|
||||
"go.opencensus.io/trace"
|
||||
|
||||
@@ -514,12 +515,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
|
||||
}
|
||||
@@ -591,6 +615,86 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
colMap, err := peerdas.CustodyColumns(s.cfg.P2P.NodeID(), params.BeaconConfig().CustodyRequirement)
|
||||
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
|
||||
}
|
||||
// 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.
|
||||
if len(missing) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// The gossip handler for data columns writes the index of each verified data column referencing the given
|
||||
// root to the channel returned by blobNotifiers.forRoot.
|
||||
nc := s.blobNotifiers.forRoot(root)
|
||||
|
||||
// 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(daCheckLogFields(root, signed.Block().Slot(), expected, len(missing))).
|
||||
Error("Still waiting for DA check at slot end.")
|
||||
})
|
||||
defer nst.Stop()
|
||||
}
|
||||
for {
|
||||
select {
|
||||
case idx := <-nc:
|
||||
// Delete each index seen in the notification channel.
|
||||
delete(missing, idx)
|
||||
// Read from the channel until there are no more missing sidecars.
|
||||
if len(missing) > 0 {
|
||||
continue
|
||||
}
|
||||
// Once all sidecars have been observed, clean up the notification channel.
|
||||
s.blobNotifiers.delete(root)
|
||||
return nil
|
||||
case <-ctx.Done():
|
||||
return errors.Wrapf(ctx.Err(), "context deadline waiting for blob sidecars slot: %d, BlockRoot: %#x", block.Slot(), root)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func daCheckLogFields(root [32]byte, slot primitives.Slot, expected, missing int) logrus.Fields {
|
||||
return logrus.Fields{
|
||||
"slot": slot,
|
||||
|
||||
@@ -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, slashings *ethpb.AttesterSlashing)
|
||||
|
||||
18
beacon-chain/blockchain/receive_data_column.go
Normal file
18
beacon-chain/blockchain/receive_data_column.go
Normal file
@@ -0,0 +1,18 @@
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"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 err
|
||||
}
|
||||
|
||||
// TODO use a custom event or new method of for data columns. For speed
|
||||
// we are simply reusing blob paths here.
|
||||
s.sendNewBlobEvent(ds.BlockRoot(), uint64(ds.SignedBlockHeader.Header.Slot))
|
||||
return nil
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -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),
|
||||
|
||||
@@ -18,6 +18,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"
|
||||
@@ -44,6 +45,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
|
||||
@@ -64,6 +70,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) {
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
1
beacon-chain/cache/BUILD.bazel
vendored
1
beacon-chain/cache/BUILD.bazel
vendored
@@ -7,6 +7,7 @@ go_library(
|
||||
"active_balance_disabled.go", # keep
|
||||
"attestation_data.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
65
beacon-chain/cache/column_subnet_ids.go
vendored
Normal 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()
|
||||
}
|
||||
@@ -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.
|
||||
|
||||
@@ -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.AttesterSlashing
|
||||
}
|
||||
|
||||
type DataColumnSidecarReceivedData struct {
|
||||
DataColumn *blocks.VerifiedRODataColumn
|
||||
}
|
||||
|
||||
21
beacon-chain/core/peerdas/BUILD.bazel
Normal file
21
beacon-chain/core/peerdas/BUILD.bazel
Normal file
@@ -0,0 +1,21 @@
|
||||
load("@prysm//tools/go:def.bzl", "go_library")
|
||||
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = ["helpers.go"],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//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",
|
||||
],
|
||||
)
|
||||
240
beacon-chain/core/peerdas/helpers.go
Normal file
240
beacon-chain/core/peerdas/helpers.go
Normal file
@@ -0,0 +1,240 @@
|
||||
package peerdas
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
|
||||
cKzg4844 "github.com/ethereum/c-kzg-4844/bindings/go"
|
||||
"github.com/holiman/uint256"
|
||||
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
errors "github.com/pkg/errors"
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/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"
|
||||
)
|
||||
|
||||
const (
|
||||
// Bytes per cell
|
||||
bytesPerCell = cKzg4844.FieldElementsPerCell * cKzg4844.BytesPerFieldElement
|
||||
|
||||
// Number of cells in the extended matrix
|
||||
extendedMatrixSize = fieldparams.MaxBlobsPerBlock * cKzg4844.CellsPerExtBlob
|
||||
)
|
||||
|
||||
type (
|
||||
extendedMatrix []cKzg4844.Cell
|
||||
|
||||
cellCoordinate struct {
|
||||
blobIndex uint64
|
||||
cellID uint64
|
||||
}
|
||||
)
|
||||
|
||||
var (
|
||||
errCustodySubnetCountTooLarge = errors.New("custody subnet count larger than data column sidecar subnet count")
|
||||
errCellNotFound = errors.New("cell not found (should never happen)")
|
||||
)
|
||||
|
||||
// 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
|
||||
}
|
||||
|
||||
func CustodyColumnSubnets(nodeId enode.ID, custodySubnetCount uint64) (map[uint64]bool, 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
|
||||
}
|
||||
|
||||
// First, compute the subnet IDs that the node should participate in.
|
||||
subnetIds := make(map[uint64]bool, custodySubnetCount)
|
||||
|
||||
for i := uint64(0); uint64(len(subnetIds)) < custodySubnetCount; i++ {
|
||||
nodeIdUInt256, nextNodeIdUInt256 := new(uint256.Int), new(uint256.Int)
|
||||
nodeIdUInt256.SetBytes(nodeId.Bytes())
|
||||
nextNodeIdUInt256.Add(nodeIdUInt256, uint256.NewInt(i))
|
||||
nextNodeIdUInt64 := nextNodeIdUInt256.Uint64()
|
||||
nextNodeId := bytesutil.Uint64ToBytesLittleEndian(nextNodeIdUInt64)
|
||||
|
||||
hashedNextNodeId := hash.Hash(nextNodeId)
|
||||
subnetId := binary.LittleEndian.Uint64(hashedNextNodeId[:8]) % dataColumnSidecarSubnetCount
|
||||
|
||||
if _, exists := subnetIds[subnetId]; !exists {
|
||||
subnetIds[subnetId] = true
|
||||
}
|
||||
}
|
||||
|
||||
return subnetIds, nil
|
||||
}
|
||||
|
||||
// ComputeExtendedMatrix computes the extended matrix from the blobs.
|
||||
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#compute_extended_matrix
|
||||
func ComputeExtendedMatrix(blobs []cKzg4844.Blob) (extendedMatrix, error) {
|
||||
matrix := make(extendedMatrix, 0, extendedMatrixSize)
|
||||
|
||||
for i := range blobs {
|
||||
// Chunk a non-extended blob into cells representing the corresponding extended blob.
|
||||
blob := &blobs[i]
|
||||
cells, err := cKzg4844.ComputeCells(blob)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "compute cells for blob")
|
||||
}
|
||||
|
||||
matrix = append(matrix, cells[:]...)
|
||||
}
|
||||
|
||||
return matrix, nil
|
||||
}
|
||||
|
||||
// RecoverMatrix recovers the extended matrix from some cells.
|
||||
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#recover_matrix
|
||||
func RecoverMatrix(cellFromCoordinate map[cellCoordinate]cKzg4844.Cell, blobCount uint64) (extendedMatrix, error) {
|
||||
matrix := make(extendedMatrix, 0, extendedMatrixSize)
|
||||
|
||||
for blobIndex := uint64(0); blobIndex < blobCount; blobIndex++ {
|
||||
// Filter all cells that belong to the current blob.
|
||||
cellIds := make([]uint64, 0, cKzg4844.CellsPerExtBlob)
|
||||
for coordinate := range cellFromCoordinate {
|
||||
if coordinate.blobIndex == blobIndex {
|
||||
cellIds = append(cellIds, coordinate.cellID)
|
||||
}
|
||||
}
|
||||
|
||||
// Retrieve cells corresponding to all `cellIds`.
|
||||
cellIdsCount := len(cellIds)
|
||||
|
||||
cells := make([]cKzg4844.Cell, 0, cellIdsCount)
|
||||
for _, cellId := range cellIds {
|
||||
coordinate := cellCoordinate{blobIndex: blobIndex, cellID: cellId}
|
||||
cell, ok := cellFromCoordinate[coordinate]
|
||||
if !ok {
|
||||
return matrix, errCellNotFound
|
||||
}
|
||||
|
||||
cells = append(cells, cell)
|
||||
}
|
||||
|
||||
// Recover all cells.
|
||||
allCellsForRow, err := cKzg4844.RecoverAllCells(cellIds, cells)
|
||||
if err != nil {
|
||||
return matrix, errors.Wrap(err, "recover all cells")
|
||||
}
|
||||
|
||||
matrix = append(matrix, allCellsForRow[:]...)
|
||||
}
|
||||
|
||||
return matrix, nil
|
||||
}
|
||||
|
||||
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#recover_matrix
|
||||
func DataColumnSidecars(signedBlock interfaces.SignedBeaconBlock, blobs []cKzg4844.Blob) ([]*ethpb.DataColumnSidecar, error) {
|
||||
blobsCount := len(blobs)
|
||||
|
||||
// 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.ComputeCellsAndProofs(blob)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "compute cells and proofs")
|
||||
}
|
||||
|
||||
cells = append(cells, blobCells)
|
||||
proofs = append(proofs, blobProofs)
|
||||
}
|
||||
|
||||
// Get the column sidecars.
|
||||
sidecars := make([]*ethpb.DataColumnSidecar, 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 {
|
||||
cellBytes = append(cellBytes, fieldElement[:]...)
|
||||
}
|
||||
|
||||
columnBytes = append(columnBytes, cellBytes)
|
||||
}
|
||||
|
||||
kzgProofOfColumnBytes := make([][]byte, 0, blobsCount)
|
||||
for _, kzgProof := range kzgProofOfColumn {
|
||||
kzgProofOfColumnBytes = append(kzgProofOfColumnBytes, kzgProof[:])
|
||||
}
|
||||
|
||||
sidecar := ðpb.DataColumnSidecar{
|
||||
ColumnIndex: columnIndex,
|
||||
DataColumn: columnBytes,
|
||||
KzgCommitments: blobKzgCommitments,
|
||||
KzgProof: kzgProofOfColumnBytes,
|
||||
SignedBlockHeader: signedBlockHeader,
|
||||
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
|
||||
}
|
||||
|
||||
sidecars = append(sidecars, sidecar)
|
||||
}
|
||||
|
||||
return sidecars, nil
|
||||
}
|
||||
@@ -218,6 +218,101 @@ 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.Debug("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
|
||||
// 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.
|
||||
@@ -243,6 +338,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 := ðpb.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()
|
||||
@@ -286,6 +395,41 @@ 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) ([fieldparams.NumberOfColumns]bool, error) {
|
||||
var mask [fieldparams.NumberOfColumns]bool
|
||||
rootDir := blobNamer{root: root}.dir()
|
||||
entries, err := afero.ReadDir(bs.fs, rootDir)
|
||||
if err != nil {
|
||||
if os.IsNotExist(err) {
|
||||
return mask, nil
|
||||
}
|
||||
return mask, err
|
||||
}
|
||||
for i := range entries {
|
||||
if entries[i].IsDir() {
|
||||
continue
|
||||
}
|
||||
name := entries[i].Name()
|
||||
if !strings.HasSuffix(name, sszExt) {
|
||||
continue
|
||||
}
|
||||
parts := strings.Split(name, ".")
|
||||
if len(parts) != 2 {
|
||||
continue
|
||||
}
|
||||
u, err := strconv.ParseUint(parts[0], 10, 64)
|
||||
if err != nil {
|
||||
return mask, errors.Wrapf(err, "unexpected directory entry breaks listing, %s", parts[0])
|
||||
}
|
||||
if u >= fieldparams.NumberOfColumns {
|
||||
return mask, errIndexOutOfBounds
|
||||
}
|
||||
mask[u] = true
|
||||
}
|
||||
return mask, nil
|
||||
}
|
||||
|
||||
// Clear deletes all files on the filesystem.
|
||||
func (bs *BlobStorage) Clear() error {
|
||||
dirs, err := listDir(bs.fs, ".")
|
||||
@@ -318,6 +462,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)
|
||||
}
|
||||
|
||||
@@ -46,6 +46,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 +57,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",
|
||||
|
||||
@@ -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.Attestation, forkDigest [4]byte) {
|
||||
func (s *Service) internalBroadcastAttestation(
|
||||
ctx context.Context,
|
||||
subnet uint64,
|
||||
att *ethpb.Attestation,
|
||||
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.New("attempted to broadcast nil data column sidecar")
|
||||
}
|
||||
|
||||
// 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 blob 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)
|
||||
}
|
||||
|
||||
@@ -40,15 +40,21 @@ const (
|
||||
udp6
|
||||
)
|
||||
|
||||
type quicProtocol uint16
|
||||
type (
|
||||
quicProtocol uint16
|
||||
custodySubnetCount uint64
|
||||
)
|
||||
|
||||
// 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
|
||||
@@ -58,6 +64,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()
|
||||
@@ -244,6 +254,11 @@ func (s *Service) createLocalNode(
|
||||
localNode.Set(quicEntry)
|
||||
}
|
||||
|
||||
if features.Get().EnablePeerDAS {
|
||||
custodySubnetEntry := custodySubnetCount(params.BeaconConfig().CustodyRequirement)
|
||||
localNode.Set(custodySubnetEntry)
|
||||
}
|
||||
|
||||
localNode.SetFallbackIP(ipAddr)
|
||||
localNode.SetFallbackUDP(udpPort)
|
||||
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -22,6 +22,7 @@ var gossipTopicMappings = map[string]proto.Message{
|
||||
SyncCommitteeSubnetTopicFormat: ðpb.SyncCommitteeMessage{},
|
||||
BlsToExecutionChangeSubnetTopicFormat: ðpb.SignedBLSToExecutionChange{},
|
||||
BlobSubnetTopicFormat: ðpb.BlobSidecar{},
|
||||
DataColumnSubnetTopicFormat: ðpb.DataColumnSidecar{},
|
||||
}
|
||||
|
||||
// GossipTopicMappings is a function to return the assigned data type
|
||||
|
||||
@@ -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"
|
||||
@@ -30,12 +31,18 @@ type P2P interface {
|
||||
MetadataProvider
|
||||
}
|
||||
|
||||
type Acceser interface {
|
||||
Broadcaster
|
||||
PeerManager
|
||||
}
|
||||
|
||||
// Broadcaster broadcasts messages to peers over the p2p pubsub protocol.
|
||||
type Broadcaster interface {
|
||||
Broadcast(context.Context, proto.Message) error
|
||||
BroadcastAttestation(ctx context.Context, subnet uint64, att *ethpb.Attestation) 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 +88,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)
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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 */)
|
||||
}
|
||||
|
||||
|
||||
@@ -43,6 +43,9 @@ 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"
|
||||
|
||||
const (
|
||||
// V1 RPC Topics
|
||||
// RPCStatusTopicV1 defines the v1 topic for the status rpc method.
|
||||
@@ -65,6 +68,9 @@ 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
|
||||
|
||||
// V2 RPC Topics
|
||||
// RPCBlocksByRangeTopicV2 defines v2 the topic for the blocks by range rpc method.
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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:
|
||||
@@ -202,6 +210,25 @@ 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
|
||||
}
|
||||
subsMap, err := peerdas.CustodyColumnSubnets(id, params.BeaconConfig().CustodyRequirement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
subs := make([]uint64, 0, len(subsMap))
|
||||
for sub := range subsMap {
|
||||
subs = append(subs, sub)
|
||||
}
|
||||
|
||||
cache.ColumnSubnetIDs.AddColumnSubnets(subs)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Spec pseudocode definition:
|
||||
//
|
||||
// def compute_subscribed_subnets(node_id: NodeID, epoch: Epoch) -> Sequence[SubnetID]:
|
||||
@@ -351,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()
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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"
|
||||
)
|
||||
|
||||
@@ -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")
|
||||
)
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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"
|
||||
@@ -18,9 +19,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"
|
||||
@@ -251,7 +255,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()
|
||||
|
||||
@@ -264,14 +276,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 = handleUnblidedBlock(block, req, isPeerDASEnabled)
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "%s: %v", "handle unblided block", err)
|
||||
}
|
||||
}
|
||||
|
||||
root, err := block.Block().HashTreeRoot()
|
||||
@@ -292,8 +306,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()
|
||||
@@ -305,47 +325,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 handleUnblidedBlock(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")
|
||||
@@ -361,7 +417,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 {
|
||||
for i, sc := range sidecars {
|
||||
if err := vs.P2P.BroadcastBlob(ctx, uint64(i), sc); err != nil {
|
||||
return errors.Wrap(err, "broadcast blob failed")
|
||||
@@ -371,10 +427,12 @@ func (vs *Server) broadcastAndReceiveBlobs(ctx context.Context, sidecars []*ethp
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "ROBlob creation failed")
|
||||
}
|
||||
|
||||
verifiedBlob := blocks.NewVerifiedROBlob(readOnlySc)
|
||||
if err := vs.BlobReceiver.ReceiveBlob(ctx, verifiedBlob); err != nil {
|
||||
return errors.Wrap(err, "receive blob failed")
|
||||
}
|
||||
|
||||
vs.OperationNotifier.OperationFeed().Send(&feed.Event{
|
||||
Type: operation.BlobSidecarReceived,
|
||||
Data: &operation.BlobSidecarReceivedData{Blob: &verifiedBlob},
|
||||
@@ -383,6 +441,31 @@ func (vs *Server) broadcastAndReceiveBlobs(ctx context.Context, sidecars []*ethp
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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 {
|
||||
for i, sidecar := range sidecars {
|
||||
if err := vs.P2P.BroadcastDataColumn(ctx, uint64(i), 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
|
||||
}
|
||||
|
||||
// PrepareBeaconProposer caches and updates the fee recipient for the given proposer.
|
||||
func (vs *Server) PrepareBeaconProposer(
|
||||
_ context.Context, request *ethpb.PrepareBeaconProposerRequest,
|
||||
|
||||
@@ -58,6 +58,13 @@ func (vs *Server) getLocalPayload(ctx context.Context, blk interfaces.ReadOnlyBe
|
||||
slot := blk.Slot()
|
||||
vIdx := blk.ProposerIndex()
|
||||
headRoot := blk.ParentRoot()
|
||||
var err error
|
||||
if headRoot == [32]byte{} {
|
||||
headRoot, err = vs.BeaconDB.GenesisBlockRoot(ctx)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
}
|
||||
logFields := logrus.Fields{
|
||||
"validatorIndex": vIdx,
|
||||
"slot": slot,
|
||||
@@ -71,7 +78,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
|
||||
|
||||
@@ -778,7 +778,7 @@ func TestProposer_ProposeBlock_OK(t *testing.T) {
|
||||
return ðpb.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",
|
||||
},
|
||||
}
|
||||
|
||||
|
||||
@@ -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 depositcache.PendingDepositsFetcher
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -25,6 +25,7 @@ go_library(
|
||||
"rpc_blob_sidecars_by_range.go",
|
||||
"rpc_blob_sidecars_by_root.go",
|
||||
"rpc_chunked_response.go",
|
||||
"rpc_data_column_sidecars_by_root.go",
|
||||
"rpc_goodbye.go",
|
||||
"rpc_metadata.go",
|
||||
"rpc_ping.go",
|
||||
@@ -37,6 +38,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 +49,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 +74,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",
|
||||
|
||||
@@ -39,6 +39,8 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
|
||||
topic = p2p.GossipTypeMapping[reflect.TypeOf(ðpb.SyncCommitteeMessage{})]
|
||||
case strings.Contains(topic, p2p.GossipBlobSidecarMessage):
|
||||
topic = p2p.GossipTypeMapping[reflect.TypeOf(ðpb.BlobSidecar{})]
|
||||
case strings.Contains(topic, p2p.GossipDataColumnSidecarMessage):
|
||||
topic = p2p.GossipTypeMapping[reflect.TypeOf(ðpb.DataColumnSidecar{})]
|
||||
}
|
||||
|
||||
base := p2p.GossipTopicMappings(topic, 0)
|
||||
|
||||
@@ -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,13 @@ func (s *Service) registerRPCHandlersAltair() {
|
||||
}
|
||||
|
||||
func (s *Service) registerRPCHandlersDeneb() {
|
||||
if features.Get().EnablePeerDAS {
|
||||
s.registerRPC(
|
||||
p2p.RPCDataColumnSidecarsByRootTopicV1,
|
||||
s.dataColumnSidecarByRootRPCHandler,
|
||||
)
|
||||
return
|
||||
}
|
||||
s.registerRPC(
|
||||
p2p.RPCBlobSidecarsByRangeTopicV1,
|
||||
s.blobSidecarsByRangeRPCHandler,
|
||||
|
||||
@@ -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"
|
||||
)
|
||||
@@ -175,3 +176,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
|
||||
}
|
||||
|
||||
149
beacon-chain/sync/rpc_data_column_sidecars_by_root.go
Normal file
149
beacon-chain/sync/rpc_data_column_sidecars_by_root.go
Normal file
@@ -0,0 +1,149 @@
|
||||
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/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")
|
||||
}
|
||||
|
||||
columnIdents := *ref
|
||||
if err := validateDataColummnsByRootRequest(columnIdents); err != nil {
|
||||
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
|
||||
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
|
||||
return err
|
||||
}
|
||||
// Sort the identifiers so that requests for the same blob root will be adjacent, minimizing db lookups.
|
||||
sort.Sort(columnIdents)
|
||||
|
||||
// TODO: Customize data column batches too
|
||||
batchSize := flags.Get().BlobBatchLimit
|
||||
var ticker *time.Ticker
|
||||
if len(columnIdents) > 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 columns.
|
||||
custodiedColumns, err := peerdas.CustodyColumns(s.cfg.p2p.NodeID(), params.BeaconConfig().CustodyRequirement)
|
||||
if err != nil {
|
||||
log.WithError(err).Errorf("unexpected error retrieving the node id")
|
||||
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
|
||||
return err
|
||||
}
|
||||
|
||||
for i := range columnIdents {
|
||||
if err := ctx.Err(); err != nil {
|
||||
closeStream(stream, log)
|
||||
return err
|
||||
}
|
||||
|
||||
// Throttle request processing to no more than batchSize/sec.
|
||||
if ticker != nil && i != 0 && i%batchSize == 0 {
|
||||
<-ticker.C
|
||||
}
|
||||
s.rateLimiter.add(stream, 1)
|
||||
root, idx := bytesutil.ToBytes32(columnIdents[i].BlockRoot), columnIdents[i].Index
|
||||
|
||||
isCustodied := custodiedColumns[idx]
|
||||
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
|
||||
sc, err := s.cfg.blobStorage.GetColumn(root, idx)
|
||||
if err != nil {
|
||||
if db.IsNotFound(err) {
|
||||
log.WithError(err).WithFields(logrus.Fields{
|
||||
"root": fmt.Sprintf("%#x", root),
|
||||
"index": idx,
|
||||
}).Debugf("Peer requested data column sidecar by root not found in db")
|
||||
continue
|
||||
}
|
||||
log.WithError(err).Errorf("unexpected db error retrieving data column, root=%x, index=%d", root, idx)
|
||||
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
|
||||
return err
|
||||
}
|
||||
|
||||
// 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 sc.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", columnIdents[i].BlockRoot)
|
||||
return types.ErrDataColumnLTMinRequest
|
||||
}
|
||||
|
||||
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, 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)
|
||||
}
|
||||
@@ -208,6 +208,29 @@ 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.ROBlob, error) {
|
||||
if uint64(len(*req)) > params.BeaconConfig().MaxRequestDataColumnSidecars {
|
||||
return nil, errors.Wrapf(p2ptypes.ErrMaxDataColumnReqExceeded, "length=%d", len(*req))
|
||||
}
|
||||
|
||||
topic, err := p2p.TopicFromMessage(p2p.DataColumnSidecarsByRootName, slots.ToEpoch(tor.CurrentSlot()))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
log.WithField("topic", topic).Debug("Sending data column sidecar request")
|
||||
stream, err := p2pApi.Send(ctx, req, topic, pid)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer closeStream(stream, log)
|
||||
|
||||
maxCol := params.BeaconConfig().MaxRequestDataColumnSidecars
|
||||
return readChunkEncodedBlobs(stream, p2pApi.Encoding(), ctxMap, blobValidatorFromRootReq(req), maxCol)
|
||||
}
|
||||
|
||||
// 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
|
||||
|
||||
@@ -51,7 +51,8 @@ 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 seenBlobSize = seenBlockSize * 6 // Each block can have max 6 blobs.
|
||||
const seenDataColumnSize = seenBlockSize * 128 // Each block can have max 128 data columns.
|
||||
const seenUnaggregatedAttSize = 20000
|
||||
const seenAggregatedAttSize = 16384
|
||||
const seenSyncMsgSize = 1000 // Maximum of 512 sync committee members, 1000 is a safe amount.
|
||||
@@ -102,6 +103,7 @@ type config struct {
|
||||
type blockchainService interface {
|
||||
blockchain.BlockReceiver
|
||||
blockchain.BlobReceiver
|
||||
blockchain.DataColumnReceiver
|
||||
blockchain.HeadFetcher
|
||||
blockchain.FinalizationFetcher
|
||||
blockchain.ForkFetcher
|
||||
@@ -133,6 +135,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
|
||||
@@ -276,6 +280,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)
|
||||
|
||||
@@ -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(ðpb.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(ðpb.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 {
|
||||
|
||||
@@ -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)
|
||||
|
||||
33
beacon-chain/sync/subscriber_data_column_sidecar.go
Normal file
33
beacon-chain/sync/subscriber_data_column_sidecar.go
Normal file
@@ -0,0 +1,33 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"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)
|
||||
|
||||
if err := s.cfg.chain.ReceiveDataColumn(ctx, dc); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
|
||||
Type: opfeed.DataColumnSidecarReceived,
|
||||
Data: &opfeed.DataColumnSidecarReceivedData{
|
||||
DataColumn: &dc,
|
||||
},
|
||||
})
|
||||
|
||||
return nil
|
||||
}
|
||||
160
beacon-chain/sync/validate_data_column.go
Normal file
160
beacon-chain/sync/validate_data_column.go
Normal file
@@ -0,0 +1,160 @@
|
||||
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/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
|
||||
}
|
||||
// TODO Verify KZG inclusion proof of data column sidecar
|
||||
|
||||
// TODO Verify KZG proofs of column sidecar
|
||||
|
||||
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,
|
||||
}).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
|
||||
}
|
||||
@@ -73,6 +73,8 @@ 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
|
||||
|
||||
SaveInvalidBlock bool // SaveInvalidBlock saves invalid block to temp.
|
||||
SaveInvalidBlob bool // SaveInvalidBlob saves invalid blob to temp.
|
||||
@@ -259,6 +261,10 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
|
||||
logEnabled(EnableQUIC)
|
||||
cfg.EnableQUIC = true
|
||||
}
|
||||
if ctx.IsSet(EnablePeerDAS.Name) {
|
||||
logEnabled(EnablePeerDAS)
|
||||
cfg.EnablePeerDAS = true
|
||||
}
|
||||
|
||||
cfg.AggregateIntervals = [3]time.Duration{aggregateFirstInterval.Value, aggregateSecondInterval.Value, aggregateThirdInterval.Value}
|
||||
Init(cfg)
|
||||
|
||||
@@ -170,6 +170,10 @@ var (
|
||||
Name: "enable-quic",
|
||||
Usage: "Enables connection using the QUIC protocol for peers which support it.",
|
||||
}
|
||||
EnablePeerDAS = &cli.BoolFlag{
|
||||
Name: "peer-das",
|
||||
Usage: "Enables Prysm to run with the experimental peer data availability sampling scheme.",
|
||||
}
|
||||
)
|
||||
|
||||
// devModeFlags holds list of flags that are set when development mode is on.
|
||||
@@ -228,6 +232,7 @@ var BeaconChainFlags = append(deprecatedBeaconFlags, append(deprecatedFlags, []c
|
||||
EnableLightClient,
|
||||
BlobSaveFsync,
|
||||
EnableQUIC,
|
||||
EnablePeerDAS,
|
||||
}...)...)
|
||||
|
||||
// E2EBeaconChainFlags contains a list of the beacon chain feature flags to be tested in E2E.
|
||||
|
||||
@@ -33,4 +33,5 @@ const (
|
||||
BlobSize = 131072 // defined to match blob.size in bazel ssz codegen
|
||||
KzgCommitmentInclusionProofDepth = 17 // Merkle proof depth for blob_kzg_commitments list item
|
||||
NextSyncCommitteeBranchDepth = 5 // NextSyncCommitteeBranchDepth defines the depth of the next sync committee branch.
|
||||
NumberOfColumns = 128 // NumberOfColumns refers to the specified number of data columns that can exist in a network.
|
||||
)
|
||||
|
||||
@@ -33,4 +33,5 @@ const (
|
||||
BlobSize = 131072 // defined to match blob.size in bazel ssz codegen
|
||||
KzgCommitmentInclusionProofDepth = 17 // Merkle proof depth for blob_kzg_commitments list item
|
||||
NextSyncCommitteeBranchDepth = 5 // NextSyncCommitteeBranchDepth defines the depth of the next sync committee branch.
|
||||
NumberOfColumns = 128 // NumberOfColumns refers to the specified number of data columns that can exist in a network.
|
||||
)
|
||||
|
||||
@@ -235,6 +235,11 @@ type BeaconChainConfig struct {
|
||||
MinPerEpochChurnLimitElectra uint64 `yaml:"MIN_PER_EPOCH_CHURN_LIMIT_ELECTRA" spec:"true"` // MinPerEpochChurnLimitElectra is the minimum amount of churn allotted for validator rotations for electra.
|
||||
MaxRequestDataColumnSidecars uint64 `yaml:"MAX_REQUEST_DATA_COLUMN_SIDECARS" spec:"true"` // MaxRequestDataColumnSidecars is the maximum number of data column sidecars in a single request
|
||||
|
||||
// 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.
|
||||
|
||||
// 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.
|
||||
|
||||
@@ -276,6 +276,11 @@ var mainnetBeaconConfig = &BeaconChainConfig{
|
||||
MinPerEpochChurnLimitElectra: 128000000000,
|
||||
MaxPerEpochActivationExitChurnLimit: 256000000000,
|
||||
|
||||
// PeerDAS
|
||||
SamplesPerSlot: 8,
|
||||
CustodyRequirement: 1,
|
||||
MinEpochsForDataColumnSidecarsRequest: 4096,
|
||||
|
||||
// Values related to networking parameters.
|
||||
GossipMaxSize: 10 * 1 << 20, // 10 MiB
|
||||
MaxChunkSize: 10 * 1 << 20, // 10 MiB
|
||||
|
||||
@@ -10,6 +10,7 @@ go_library(
|
||||
"proto.go",
|
||||
"roblob.go",
|
||||
"roblock.go",
|
||||
"rodatacolumn.go",
|
||||
"setters.go",
|
||||
"types.go",
|
||||
],
|
||||
|
||||
@@ -80,6 +80,32 @@ 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")
|
||||
}
|
||||
|
||||
return proof, nil
|
||||
}
|
||||
|
||||
// leavesFromCommitments hashes each commitment to construct a slice of roots
|
||||
func leavesFromCommitments(commitments [][]byte) [][]byte {
|
||||
leaves := make([][]byte, len(commitments))
|
||||
|
||||
122
consensus-types/blocks/rodatacolumn.go
Normal file
122
consensus-types/blocks/rodatacolumn.go
Normal 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}
|
||||
}
|
||||
@@ -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)
|
||||
|
||||
4
deps.bzl
4
deps.bzl
@@ -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:ffWmm0RUR2+VqJsCkf94HqgEwZi2fgbm2iq+O/GdJNI=",
|
||||
version = "v1.0.1-0.20240422190800-13be436f5927",
|
||||
)
|
||||
go_repository(
|
||||
name = "com_github_ethereum_go_ethereum",
|
||||
|
||||
2
go.mod
2
go.mod
@@ -17,6 +17,7 @@ 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.1-0.20240422190800-13be436f5927
|
||||
github.com/ethereum/go-ethereum v1.13.5
|
||||
github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5
|
||||
github.com/fsnotify/fsnotify v1.6.0
|
||||
@@ -133,7 +134,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
|
||||
|
||||
4
go.sum
4
go.sum
@@ -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.1-0.20240422190800-13be436f5927 h1:ffWmm0RUR2+VqJsCkf94HqgEwZi2fgbm2iq+O/GdJNI=
|
||||
github.com/ethereum/c-kzg-4844 v1.0.1-0.20240422190800-13be436f5927/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=
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
// Code generated by fastssz. DO NOT EDIT.
|
||||
// Hash: 5890b3492dbdff08d332879e83ae45e7bd9f94da0716b1b0517f1766028a8d67
|
||||
// Hash: 13c946aa898cca1afa84687b619bc5a10fc79a46340e98dcfb07dde835d39a0c
|
||||
package v1
|
||||
|
||||
import (
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
// Code generated by fastssz. DO NOT EDIT.
|
||||
// Hash: 2ed480e3c144fb091e0aa2757a79e78da573f90b18d0d8acd35fa9705f6c1b08
|
||||
// Hash: b6cc6e65f0679b3152fb6379d539ceeea78acabb1e413b6eacfac2d847537b1f
|
||||
package eth
|
||||
|
||||
import (
|
||||
|
||||
@@ -131,6 +131,7 @@ ssz_gen_marshal(
|
||||
"BlobSidecars",
|
||||
"BlobIdentifier",
|
||||
"DepositSnapshot",
|
||||
"DataColumnSidecar",
|
||||
],
|
||||
)
|
||||
|
||||
@@ -229,6 +230,7 @@ ssz_proto_files(
|
||||
"beacon_block.proto",
|
||||
"beacon_state.proto",
|
||||
"blobs.proto",
|
||||
"data_columns.proto",
|
||||
"sync_committee.proto",
|
||||
"withdrawals.proto",
|
||||
],
|
||||
|
||||
225
proto/prysm/v1alpha1/data_columns.pb.go
generated
Executable file
225
proto/prysm/v1alpha1/data_columns.pb.go
generated
Executable file
@@ -0,0 +1,225 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.31.0
|
||||
// protoc v4.25.1
|
||||
// source: proto/prysm/v1alpha1/data_columns.proto
|
||||
|
||||
package eth
|
||||
|
||||
import (
|
||||
reflect "reflect"
|
||||
sync "sync"
|
||||
|
||||
_ "github.com/prysmaticlabs/prysm/v5/proto/eth/ext"
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
)
|
||||
|
||||
const (
|
||||
// Verify that this generated code is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
|
||||
// Verify that runtime/protoimpl is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
||||
)
|
||||
|
||||
type DataColumnSidecar struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
ColumnIndex uint64 `protobuf:"varint,1,opt,name=column_index,json=columnIndex,proto3" json:"column_index,omitempty"`
|
||||
DataColumn [][]byte `protobuf:"bytes,2,rep,name=data_column,json=dataColumn,proto3" json:"data_column,omitempty" ssz-max:"4096" ssz-size:"?,2048"`
|
||||
KzgCommitments [][]byte `protobuf:"bytes,3,rep,name=kzg_commitments,json=kzgCommitments,proto3" json:"kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"`
|
||||
KzgProof [][]byte `protobuf:"bytes,4,rep,name=kzg_proof,json=kzgProof,proto3" json:"kzg_proof,omitempty" ssz-max:"4096" ssz-size:"?,48"`
|
||||
SignedBlockHeader *SignedBeaconBlockHeader `protobuf:"bytes,5,opt,name=signed_block_header,json=signedBlockHeader,proto3" json:"signed_block_header,omitempty"`
|
||||
KzgCommitmentsInclusionProof [][]byte `protobuf:"bytes,6,rep,name=kzg_commitments_inclusion_proof,json=kzgCommitmentsInclusionProof,proto3" json:"kzg_commitments_inclusion_proof,omitempty" ssz-size:"4,32"`
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) Reset() {
|
||||
*x = DataColumnSidecar{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*DataColumnSidecar) ProtoMessage() {}
|
||||
|
||||
func (x *DataColumnSidecar) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use DataColumnSidecar.ProtoReflect.Descriptor instead.
|
||||
func (*DataColumnSidecar) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP(), []int{0}
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) GetColumnIndex() uint64 {
|
||||
if x != nil {
|
||||
return x.ColumnIndex
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) GetDataColumn() [][]byte {
|
||||
if x != nil {
|
||||
return x.DataColumn
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) GetKzgCommitments() [][]byte {
|
||||
if x != nil {
|
||||
return x.KzgCommitments
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) GetKzgProof() [][]byte {
|
||||
if x != nil {
|
||||
return x.KzgProof
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) GetSignedBlockHeader() *SignedBeaconBlockHeader {
|
||||
if x != nil {
|
||||
return x.SignedBlockHeader
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *DataColumnSidecar) GetKzgCommitmentsInclusionProof() [][]byte {
|
||||
if x != nil {
|
||||
return x.KzgCommitmentsInclusionProof
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
var File_proto_prysm_v1alpha1_data_columns_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_proto_prysm_v1alpha1_data_columns_proto_rawDesc = []byte{
|
||||
0x0a, 0x27, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31,
|
||||
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6c, 0x75,
|
||||
0x6d, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x65, 0x74, 0x68, 0x65, 0x72,
|
||||
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
|
||||
0x1a, 0x1b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x65, 0x78, 0x74, 0x2f,
|
||||
0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70,
|
||||
0x68, 0x61, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x86, 0x03, 0x0a, 0x11, 0x44, 0x61, 0x74, 0x61, 0x43,
|
||||
0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12,
|
||||
0x33, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x02,
|
||||
0x20, 0x03, 0x28, 0x0c, 0x42, 0x12, 0x8a, 0xb5, 0x18, 0x06, 0x3f, 0x2c, 0x32, 0x30, 0x34, 0x38,
|
||||
0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x6f,
|
||||
0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x39, 0x0a, 0x0f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d,
|
||||
0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a,
|
||||
0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52,
|
||||
0x0e, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12,
|
||||
0x2d, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x04, 0x20, 0x03,
|
||||
0x28, 0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04,
|
||||
0x34, 0x30, 0x39, 0x36, 0x52, 0x08, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x5e,
|
||||
0x0a, 0x13, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68,
|
||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74,
|
||||
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70,
|
||||
0x68, 0x61, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e,
|
||||
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x11, 0x73, 0x69, 0x67,
|
||||
0x6e, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4f,
|
||||
0x0a, 0x1f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74,
|
||||
0x73, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x6f,
|
||||
0x66, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x08, 0x8a, 0xb5, 0x18, 0x04, 0x34, 0x2c, 0x33,
|
||||
0x32, 0x52, 0x1c, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74,
|
||||
0x73, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x42,
|
||||
0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
|
||||
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, 0x44,
|
||||
0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50,
|
||||
0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72,
|
||||
0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73,
|
||||
0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
|
||||
0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15,
|
||||
0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
|
||||
0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
|
||||
0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_rawDescOnce sync.Once
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_rawDescData = file_proto_prysm_v1alpha1_data_columns_proto_rawDesc
|
||||
)
|
||||
|
||||
func file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP() []byte {
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_rawDescOnce.Do(func() {
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_prysm_v1alpha1_data_columns_proto_rawDescData)
|
||||
})
|
||||
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_proto_prysm_v1alpha1_data_columns_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
|
||||
var file_proto_prysm_v1alpha1_data_columns_proto_goTypes = []interface{}{
|
||||
(*DataColumnSidecar)(nil), // 0: ethereum.eth.v1alpha1.DataColumnSidecar
|
||||
(*SignedBeaconBlockHeader)(nil), // 1: ethereum.eth.v1alpha1.SignedBeaconBlockHeader
|
||||
}
|
||||
var file_proto_prysm_v1alpha1_data_columns_proto_depIdxs = []int32{
|
||||
1, // 0: ethereum.eth.v1alpha1.DataColumnSidecar.signed_block_header:type_name -> ethereum.eth.v1alpha1.SignedBeaconBlockHeader
|
||||
1, // [1:1] is the sub-list for method output_type
|
||||
1, // [1:1] is the sub-list for method input_type
|
||||
1, // [1:1] is the sub-list for extension type_name
|
||||
1, // [1:1] is the sub-list for extension extendee
|
||||
0, // [0:1] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_proto_prysm_v1alpha1_data_columns_proto_init() }
|
||||
func file_proto_prysm_v1alpha1_data_columns_proto_init() {
|
||||
if File_proto_prysm_v1alpha1_data_columns_proto != nil {
|
||||
return
|
||||
}
|
||||
file_proto_prysm_v1alpha1_beacon_block_proto_init()
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*DataColumnSidecar); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_proto_prysm_v1alpha1_data_columns_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 1,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
GoTypes: file_proto_prysm_v1alpha1_data_columns_proto_goTypes,
|
||||
DependencyIndexes: file_proto_prysm_v1alpha1_data_columns_proto_depIdxs,
|
||||
MessageInfos: file_proto_prysm_v1alpha1_data_columns_proto_msgTypes,
|
||||
}.Build()
|
||||
File_proto_prysm_v1alpha1_data_columns_proto = out.File
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_rawDesc = nil
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_goTypes = nil
|
||||
file_proto_prysm_v1alpha1_data_columns_proto_depIdxs = nil
|
||||
}
|
||||
36
proto/prysm/v1alpha1/data_columns.proto
Normal file
36
proto/prysm/v1alpha1/data_columns.proto
Normal file
@@ -0,0 +1,36 @@
|
||||
// Copyright 2024 Offchain Labs.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
syntax = "proto3";
|
||||
|
||||
package ethereum.eth.v1alpha1;
|
||||
|
||||
import "proto/eth/ext/options.proto";
|
||||
import "proto/prysm/v1alpha1/beacon_block.proto";
|
||||
|
||||
option csharp_namespace = "Ethereum.Eth.v1alpha1";
|
||||
option go_package = "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1;eth";
|
||||
option java_multiple_files = true;
|
||||
option java_outer_classname = "DataColumnsProto";
|
||||
option java_package = "org.ethereum.eth.v1alpha1";
|
||||
option php_namespace = "Ethereum\\Eth\\v1alpha1";
|
||||
|
||||
|
||||
message DataColumnSidecar {
|
||||
uint64 column_index = 1;
|
||||
repeated bytes data_column = 2 [(ethereum.eth.ext.ssz_size) = "?,bytes_per_cell.size", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"];
|
||||
repeated bytes kzg_commitments = 3 [(ethereum.eth.ext.ssz_size) = "?,48", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"];
|
||||
repeated bytes kzg_proof = 4 [(ethereum.eth.ext.ssz_size) = "?,48", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"];
|
||||
SignedBeaconBlockHeader signed_block_header = 5;
|
||||
repeated bytes kzg_commitments_inclusion_proof = 6 [(ethereum.eth.ext.ssz_size) = "kzg_commitments_inclusion_proof_depth.size,32"];
|
||||
}
|
||||
@@ -1,5 +1,5 @@
|
||||
// Code generated by fastssz. DO NOT EDIT.
|
||||
// Hash: eec7f00cb63ce6e76f0c38f25c23206f9700f1c9ba9f295908168d59f9af3f60
|
||||
// Hash: 5325c06f5c92e96fffd12287f98fb86ab31bae235aad33db7e507802d618cb28
|
||||
package eth
|
||||
|
||||
import (
|
||||
@@ -15122,6 +15122,326 @@ func (b *BlobIdentifier) HashTreeRootWith(hh *ssz.Hasher) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
// MarshalSSZ ssz marshals the DataColumnSidecar object
|
||||
func (d *DataColumnSidecar) MarshalSSZ() ([]byte, error) {
|
||||
return ssz.MarshalSSZ(d)
|
||||
}
|
||||
|
||||
// MarshalSSZTo ssz marshals the DataColumnSidecar object to a target array
|
||||
func (d *DataColumnSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) {
|
||||
dst = buf
|
||||
offset := int(356)
|
||||
|
||||
// Field (0) 'ColumnIndex'
|
||||
dst = ssz.MarshalUint64(dst, d.ColumnIndex)
|
||||
|
||||
// Offset (1) 'DataColumn'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
offset += len(d.DataColumn) * 2048
|
||||
|
||||
// Offset (2) 'KzgCommitments'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
offset += len(d.KzgCommitments) * 48
|
||||
|
||||
// Offset (3) 'KzgProof'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
offset += len(d.KzgProof) * 48
|
||||
|
||||
// Field (4) 'SignedBlockHeader'
|
||||
if d.SignedBlockHeader == nil {
|
||||
d.SignedBlockHeader = new(SignedBeaconBlockHeader)
|
||||
}
|
||||
if dst, err = d.SignedBlockHeader.MarshalSSZTo(dst); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (5) 'KzgCommitmentsInclusionProof'
|
||||
if size := len(d.KzgCommitmentsInclusionProof); size != 4 {
|
||||
err = ssz.ErrVectorLengthFn("--.KzgCommitmentsInclusionProof", size, 4)
|
||||
return
|
||||
}
|
||||
for ii := 0; ii < 4; ii++ {
|
||||
if size := len(d.KzgCommitmentsInclusionProof[ii]); size != 32 {
|
||||
err = ssz.ErrBytesLengthFn("--.KzgCommitmentsInclusionProof[ii]", size, 32)
|
||||
return
|
||||
}
|
||||
dst = append(dst, d.KzgCommitmentsInclusionProof[ii]...)
|
||||
}
|
||||
|
||||
// Field (1) 'DataColumn'
|
||||
if size := len(d.DataColumn); size > 4096 {
|
||||
err = ssz.ErrListTooBigFn("--.DataColumn", size, 4096)
|
||||
return
|
||||
}
|
||||
for ii := 0; ii < len(d.DataColumn); ii++ {
|
||||
if size := len(d.DataColumn[ii]); size != 2048 {
|
||||
err = ssz.ErrBytesLengthFn("--.DataColumn[ii]", size, 2048)
|
||||
return
|
||||
}
|
||||
dst = append(dst, d.DataColumn[ii]...)
|
||||
}
|
||||
|
||||
// Field (2) 'KzgCommitments'
|
||||
if size := len(d.KzgCommitments); size > 4096 {
|
||||
err = ssz.ErrListTooBigFn("--.KzgCommitments", size, 4096)
|
||||
return
|
||||
}
|
||||
for ii := 0; ii < len(d.KzgCommitments); ii++ {
|
||||
if size := len(d.KzgCommitments[ii]); size != 48 {
|
||||
err = ssz.ErrBytesLengthFn("--.KzgCommitments[ii]", size, 48)
|
||||
return
|
||||
}
|
||||
dst = append(dst, d.KzgCommitments[ii]...)
|
||||
}
|
||||
|
||||
// Field (3) 'KzgProof'
|
||||
if size := len(d.KzgProof); size > 4096 {
|
||||
err = ssz.ErrListTooBigFn("--.KzgProof", size, 4096)
|
||||
return
|
||||
}
|
||||
for ii := 0; ii < len(d.KzgProof); ii++ {
|
||||
if size := len(d.KzgProof[ii]); size != 48 {
|
||||
err = ssz.ErrBytesLengthFn("--.KzgProof[ii]", size, 48)
|
||||
return
|
||||
}
|
||||
dst = append(dst, d.KzgProof[ii]...)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// UnmarshalSSZ ssz unmarshals the DataColumnSidecar object
|
||||
func (d *DataColumnSidecar) UnmarshalSSZ(buf []byte) error {
|
||||
var err error
|
||||
size := uint64(len(buf))
|
||||
if size < 356 {
|
||||
return ssz.ErrSize
|
||||
}
|
||||
|
||||
tail := buf
|
||||
var o1, o2, o3 uint64
|
||||
|
||||
// Field (0) 'ColumnIndex'
|
||||
d.ColumnIndex = ssz.UnmarshallUint64(buf[0:8])
|
||||
|
||||
// Offset (1) 'DataColumn'
|
||||
if o1 = ssz.ReadOffset(buf[8:12]); o1 > size {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
if o1 < 356 {
|
||||
return ssz.ErrInvalidVariableOffset
|
||||
}
|
||||
|
||||
// Offset (2) 'KzgCommitments'
|
||||
if o2 = ssz.ReadOffset(buf[12:16]); o2 > size || o1 > o2 {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
// Offset (3) 'KzgProof'
|
||||
if o3 = ssz.ReadOffset(buf[16:20]); o3 > size || o2 > o3 {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
// Field (4) 'SignedBlockHeader'
|
||||
if d.SignedBlockHeader == nil {
|
||||
d.SignedBlockHeader = new(SignedBeaconBlockHeader)
|
||||
}
|
||||
if err = d.SignedBlockHeader.UnmarshalSSZ(buf[20:228]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Field (5) 'KzgCommitmentsInclusionProof'
|
||||
d.KzgCommitmentsInclusionProof = make([][]byte, 4)
|
||||
for ii := 0; ii < 4; ii++ {
|
||||
if cap(d.KzgCommitmentsInclusionProof[ii]) == 0 {
|
||||
d.KzgCommitmentsInclusionProof[ii] = make([]byte, 0, len(buf[228:356][ii*32:(ii+1)*32]))
|
||||
}
|
||||
d.KzgCommitmentsInclusionProof[ii] = append(d.KzgCommitmentsInclusionProof[ii], buf[228:356][ii*32:(ii+1)*32]...)
|
||||
}
|
||||
|
||||
// Field (1) 'DataColumn'
|
||||
{
|
||||
buf = tail[o1:o2]
|
||||
num, err := ssz.DivideInt2(len(buf), 2048, 4096)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
d.DataColumn = make([][]byte, num)
|
||||
for ii := 0; ii < num; ii++ {
|
||||
if cap(d.DataColumn[ii]) == 0 {
|
||||
d.DataColumn[ii] = make([]byte, 0, len(buf[ii*2048:(ii+1)*2048]))
|
||||
}
|
||||
d.DataColumn[ii] = append(d.DataColumn[ii], buf[ii*2048:(ii+1)*2048]...)
|
||||
}
|
||||
}
|
||||
|
||||
// Field (2) 'KzgCommitments'
|
||||
{
|
||||
buf = tail[o2:o3]
|
||||
num, err := ssz.DivideInt2(len(buf), 48, 4096)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
d.KzgCommitments = make([][]byte, num)
|
||||
for ii := 0; ii < num; ii++ {
|
||||
if cap(d.KzgCommitments[ii]) == 0 {
|
||||
d.KzgCommitments[ii] = make([]byte, 0, len(buf[ii*48:(ii+1)*48]))
|
||||
}
|
||||
d.KzgCommitments[ii] = append(d.KzgCommitments[ii], buf[ii*48:(ii+1)*48]...)
|
||||
}
|
||||
}
|
||||
|
||||
// Field (3) 'KzgProof'
|
||||
{
|
||||
buf = tail[o3:]
|
||||
num, err := ssz.DivideInt2(len(buf), 48, 4096)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
d.KzgProof = make([][]byte, num)
|
||||
for ii := 0; ii < num; ii++ {
|
||||
if cap(d.KzgProof[ii]) == 0 {
|
||||
d.KzgProof[ii] = make([]byte, 0, len(buf[ii*48:(ii+1)*48]))
|
||||
}
|
||||
d.KzgProof[ii] = append(d.KzgProof[ii], buf[ii*48:(ii+1)*48]...)
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// SizeSSZ returns the ssz encoded size in bytes for the DataColumnSidecar object
|
||||
func (d *DataColumnSidecar) SizeSSZ() (size int) {
|
||||
size = 356
|
||||
|
||||
// Field (1) 'DataColumn'
|
||||
size += len(d.DataColumn) * 2048
|
||||
|
||||
// Field (2) 'KzgCommitments'
|
||||
size += len(d.KzgCommitments) * 48
|
||||
|
||||
// Field (3) 'KzgProof'
|
||||
size += len(d.KzgProof) * 48
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// HashTreeRoot ssz hashes the DataColumnSidecar object
|
||||
func (d *DataColumnSidecar) HashTreeRoot() ([32]byte, error) {
|
||||
return ssz.HashWithDefaultHasher(d)
|
||||
}
|
||||
|
||||
// HashTreeRootWith ssz hashes the DataColumnSidecar object with a hasher
|
||||
func (d *DataColumnSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
|
||||
indx := hh.Index()
|
||||
|
||||
// Field (0) 'ColumnIndex'
|
||||
hh.PutUint64(d.ColumnIndex)
|
||||
|
||||
// Field (1) 'DataColumn'
|
||||
{
|
||||
if size := len(d.DataColumn); size > 4096 {
|
||||
err = ssz.ErrListTooBigFn("--.DataColumn", size, 4096)
|
||||
return
|
||||
}
|
||||
subIndx := hh.Index()
|
||||
for _, i := range d.DataColumn {
|
||||
if len(i) != 2048 {
|
||||
err = ssz.ErrBytesLength
|
||||
return
|
||||
}
|
||||
hh.PutBytes(i)
|
||||
}
|
||||
|
||||
numItems := uint64(len(d.DataColumn))
|
||||
if ssz.EnableVectorizedHTR {
|
||||
hh.MerkleizeWithMixinVectorizedHTR(subIndx, numItems, 4096)
|
||||
} else {
|
||||
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
|
||||
}
|
||||
}
|
||||
|
||||
// Field (2) 'KzgCommitments'
|
||||
{
|
||||
if size := len(d.KzgCommitments); size > 4096 {
|
||||
err = ssz.ErrListTooBigFn("--.KzgCommitments", size, 4096)
|
||||
return
|
||||
}
|
||||
subIndx := hh.Index()
|
||||
for _, i := range d.KzgCommitments {
|
||||
if len(i) != 48 {
|
||||
err = ssz.ErrBytesLength
|
||||
return
|
||||
}
|
||||
hh.PutBytes(i)
|
||||
}
|
||||
|
||||
numItems := uint64(len(d.KzgCommitments))
|
||||
if ssz.EnableVectorizedHTR {
|
||||
hh.MerkleizeWithMixinVectorizedHTR(subIndx, numItems, 4096)
|
||||
} else {
|
||||
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
|
||||
}
|
||||
}
|
||||
|
||||
// Field (3) 'KzgProof'
|
||||
{
|
||||
if size := len(d.KzgProof); size > 4096 {
|
||||
err = ssz.ErrListTooBigFn("--.KzgProof", size, 4096)
|
||||
return
|
||||
}
|
||||
subIndx := hh.Index()
|
||||
for _, i := range d.KzgProof {
|
||||
if len(i) != 48 {
|
||||
err = ssz.ErrBytesLength
|
||||
return
|
||||
}
|
||||
hh.PutBytes(i)
|
||||
}
|
||||
|
||||
numItems := uint64(len(d.KzgProof))
|
||||
if ssz.EnableVectorizedHTR {
|
||||
hh.MerkleizeWithMixinVectorizedHTR(subIndx, numItems, 4096)
|
||||
} else {
|
||||
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
|
||||
}
|
||||
}
|
||||
|
||||
// Field (4) 'SignedBlockHeader'
|
||||
if err = d.SignedBlockHeader.HashTreeRootWith(hh); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (5) 'KzgCommitmentsInclusionProof'
|
||||
{
|
||||
if size := len(d.KzgCommitmentsInclusionProof); size != 4 {
|
||||
err = ssz.ErrVectorLengthFn("--.KzgCommitmentsInclusionProof", size, 4)
|
||||
return
|
||||
}
|
||||
subIndx := hh.Index()
|
||||
for _, i := range d.KzgCommitmentsInclusionProof {
|
||||
if len(i) != 32 {
|
||||
err = ssz.ErrBytesLength
|
||||
return
|
||||
}
|
||||
hh.Append(i)
|
||||
}
|
||||
|
||||
if ssz.EnableVectorizedHTR {
|
||||
hh.MerkleizeVectorizedHTR(subIndx)
|
||||
} else {
|
||||
hh.Merkleize(subIndx)
|
||||
}
|
||||
}
|
||||
|
||||
if ssz.EnableVectorizedHTR {
|
||||
hh.MerkleizeVectorizedHTR(indx)
|
||||
} else {
|
||||
hh.Merkleize(indx)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// MarshalSSZ ssz marshals the Status object
|
||||
func (s *Status) MarshalSSZ() ([]byte, error) {
|
||||
return ssz.MarshalSSZ(s)
|
||||
|
||||
@@ -26,6 +26,11 @@ mainnet = {
|
||||
"max_blobs_per_block.size": "6",
|
||||
"max_blob_commitments.size": "4096",
|
||||
"kzg_commitment_inclusion_proof_depth.size": "17",
|
||||
"field_elements_per_cell.size": "64",
|
||||
"field_elements_per_ext_blob.size": "8192",
|
||||
"bytes_per_cell.size": "2048", # FIELD_ELEMENTS_PER_CELL * BYTES_PER_FIELD_ELEMENT
|
||||
"cells_per_blob.size": "128",
|
||||
"kzg_commitments_inclusion_proof_depth.size": "4",
|
||||
}
|
||||
|
||||
minimal = {
|
||||
@@ -48,6 +53,11 @@ minimal = {
|
||||
"max_blobs_per_block.size": "6",
|
||||
"max_blob_commitments.size": "16",
|
||||
"kzg_commitment_inclusion_proof_depth.size": "9",
|
||||
"field_elements_per_cell.size": "64",
|
||||
"field_elements_per_ext_blob.size": "8192",
|
||||
"bytes_per_cell.size": "2048", # FIELD_ELEMENTS_PER_CELL * BYTES_PER_FIELD_ELEMENT
|
||||
"cells_per_blob.size": "128",
|
||||
"kzg_commitments_inclusion_proof_depth.size": "4",
|
||||
}
|
||||
|
||||
###### Rules definitions #######
|
||||
|
||||
@@ -111,10 +111,6 @@ func GethCancunTime(genesisTime uint64, cfg *clparams.BeaconChainConfig) *uint64
|
||||
// like in an e2e test. The parameters are minimal but the full value is returned unmarshaled so that it can be
|
||||
// customized as desired.
|
||||
func GethTestnetGenesis(genesisTime uint64, cfg *clparams.BeaconChainConfig) *core.Genesis {
|
||||
ttd, ok := big.NewInt(0).SetString(clparams.BeaconConfig().TerminalTotalDifficulty, 10)
|
||||
if !ok {
|
||||
panic(fmt.Sprintf("unable to parse TerminalTotalDifficulty as an integer = %s", clparams.BeaconConfig().TerminalTotalDifficulty))
|
||||
}
|
||||
|
||||
shanghaiTime := GethShanghaiTime(genesisTime, cfg)
|
||||
cancunTime := GethCancunTime(genesisTime, cfg)
|
||||
@@ -135,14 +131,10 @@ func GethTestnetGenesis(genesisTime uint64, cfg *clparams.BeaconChainConfig) *co
|
||||
ArrowGlacierBlock: bigz,
|
||||
GrayGlacierBlock: bigz,
|
||||
MergeNetsplitBlock: bigz,
|
||||
TerminalTotalDifficulty: ttd,
|
||||
TerminalTotalDifficultyPassed: false,
|
||||
Clique: ¶ms.CliqueConfig{
|
||||
Period: cfg.SecondsPerETH1Block,
|
||||
Epoch: 20000,
|
||||
},
|
||||
ShanghaiTime: shanghaiTime,
|
||||
CancunTime: cancunTime,
|
||||
TerminalTotalDifficultyPassed: true,
|
||||
TerminalTotalDifficulty: bigz,
|
||||
ShanghaiTime: shanghaiTime,
|
||||
CancunTime: cancunTime,
|
||||
}
|
||||
da := defaultDepositContractAllocation(cfg.DepositContractAddress)
|
||||
ma := minerAllocation()
|
||||
|
||||
@@ -153,7 +153,14 @@ func (s *PremineGenesisConfig) empty() (state.BeaconState, error) {
|
||||
return nil, err
|
||||
}
|
||||
case version.Deneb:
|
||||
e, err = state_native.InitializeFromProtoDeneb(ðpb.BeaconStateDeneb{})
|
||||
e, err = state_native.InitializeFromProtoDeneb(ðpb.BeaconStateDeneb{
|
||||
BlockRoots: bRoots,
|
||||
StateRoots: sRoots,
|
||||
RandaoMixes: mixes,
|
||||
Balances: []uint64{},
|
||||
InactivityScores: []uint64{},
|
||||
Validators: []*ethpb.Validator{},
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@@ -21,7 +21,7 @@ func e2eMinimal(t *testing.T, cfg *params.BeaconChainConfig, cfgo ...types.E2ECo
|
||||
|
||||
// Run for 12 epochs if not in long-running to confirm long-running has no issues.
|
||||
var err error
|
||||
epochsToRun := 14
|
||||
epochsToRun := 6
|
||||
epochStr, longRunning := os.LookupEnv("E2E_EPOCHS")
|
||||
if longRunning {
|
||||
epochsToRun, err = strconv.Atoi(epochStr)
|
||||
@@ -36,32 +36,6 @@ func e2eMinimal(t *testing.T, cfg *params.BeaconChainConfig, cfgo ...types.E2ECo
|
||||
tracingPort := e2eParams.TestParams.Ports.JaegerTracingPort
|
||||
tracingEndpoint := fmt.Sprintf("127.0.0.1:%d", tracingPort)
|
||||
evals := []types.Evaluator{
|
||||
ev.PeersConnect,
|
||||
ev.HealthzCheck,
|
||||
ev.MetricsCheck,
|
||||
ev.ValidatorsAreActive,
|
||||
ev.ValidatorsParticipatingAtEpoch(2),
|
||||
ev.FinalizationOccurs(3),
|
||||
ev.VerifyBlockGraffiti,
|
||||
ev.PeersCheck,
|
||||
ev.ProposeVoluntaryExit,
|
||||
ev.ValidatorsHaveExited,
|
||||
ev.SubmitWithdrawal,
|
||||
ev.ValidatorsHaveWithdrawn,
|
||||
ev.ProcessesDepositsInBlocks,
|
||||
ev.ActivatesDepositedValidators,
|
||||
ev.DepositedValidatorsAreActive,
|
||||
ev.ValidatorsVoteWithTheMajority,
|
||||
ev.ColdStateCheckpoint,
|
||||
ev.AltairForkTransition,
|
||||
ev.BellatrixForkTransition,
|
||||
ev.CapellaForkTransition,
|
||||
ev.DenebForkTransition,
|
||||
ev.APIGatewayV1Alpha1VerifyIntegrity,
|
||||
ev.FinishedSyncing,
|
||||
ev.AllNodesHaveSameHead,
|
||||
ev.ValidatorSyncParticipation,
|
||||
ev.FeeRecipientIsPresent,
|
||||
//ev.TransactionsPresent, TODO: Re-enable Transaction evaluator once it tx pool issues are fixed.
|
||||
}
|
||||
testConfig := &types.E2EConfig{
|
||||
|
||||
@@ -53,8 +53,7 @@ var ValidatorsParticipatingAtEpoch = func(epoch primitives.Epoch) types.Evaluato
|
||||
var ValidatorSyncParticipation = types.Evaluator{
|
||||
Name: "validator_sync_participation_%d",
|
||||
Policy: func(e primitives.Epoch) bool {
|
||||
fEpoch := params.BeaconConfig().AltairForkEpoch
|
||||
return policies.OnwardsNthEpoch(fEpoch)(e)
|
||||
return false
|
||||
},
|
||||
Evaluation: validatorsSyncParticipation,
|
||||
}
|
||||
|
||||
@@ -9,6 +9,6 @@ import (
|
||||
)
|
||||
|
||||
func TestEndToEnd_MinimalConfig(t *testing.T) {
|
||||
r := e2eMinimal(t, types.InitForkCfg(version.Phase0, version.Deneb, params.E2ETestConfig()), types.WithCheckpointSync())
|
||||
r := e2eMinimal(t, types.InitForkCfg(version.Deneb, version.Deneb, params.E2ETestConfig()), types.WithCheckpointSync())
|
||||
r.run()
|
||||
}
|
||||
|
||||
@@ -76,6 +76,9 @@ type E2EConfig struct {
|
||||
|
||||
func GenesisFork() int {
|
||||
cfg := params.BeaconConfig()
|
||||
if cfg.DenebForkEpoch == 0 {
|
||||
return version.Deneb
|
||||
}
|
||||
if cfg.CapellaForkEpoch == 0 {
|
||||
return version.Capella
|
||||
}
|
||||
|
||||
4
testing/validator-mock/node_client_mock.go
generated
4
testing/validator-mock/node_client_mock.go
generated
@@ -21,8 +21,8 @@ import (
|
||||
|
||||
// MockNodeClient is a mock of NodeClient interface.
|
||||
type MockNodeClient struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockNodeClientMockRecorder
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockNodeClientMockRecorder
|
||||
healthTracker *beacon.NodeHealthTracker
|
||||
}
|
||||
|
||||
|
||||
@@ -76,4 +76,3 @@ func (mr *MockJsonRestHandlerMockRecorder) Post(ctx, endpoint, headers, data, re
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Post", reflect.TypeOf((*MockJsonRestHandler)(nil).Post), ctx, endpoint, headers, data, resp)
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user