Compare commits

...

14 Commits

Author SHA1 Message Date
Kasey
7ee6985a67 remove unused var 2025-06-27 21:41:19 -05:00
kasey
794e0b6da8 Update encoding/ssz/list.go
Co-authored-by: Bastin <43618253+Inspector-Butters@users.noreply.github.com>
2025-06-27 11:33:43 -05:00
Kasey
1d25da229e generic ssz list encoding 2025-06-27 00:05:57 -05:00
Manu NALEPA
97f416b3a7 dataColumnSidecarByRootRPCHandler: Do not rely any more on map iteration, which does not produce reproducible output order. (#15441) 2025-06-26 13:07:19 +00:00
Manu NALEPA
1c1e0f38bb PeerDAS: Implement beacon API blob sidecar endpoint for Fulu (#15436)
* `TestGetBlob`: Refactor (no functional change).

* `GenerateTestFuluBlockWithSidecars`: Generate commitments consistent with blobs.

This is actually not needed for this PR, but it does not hurt.

* Beacon API: Implement blob sidecars endpoint for Fulu.

* Fix Radek's comment.

* Update beacon-chain/rpc/lookup/blocker.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Fix Radek's comment.

* Update beacon-chain/rpc/lookup/blocker.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

---------

Co-authored-by: Radosław Kapka <rkapka@wp.pl>
2025-06-26 10:05:22 +00:00
Manu NALEPA
121914d0d7 Implement SendDataColumnSidecarsByRangeRequest and SendDataColumnSidecarsByRootRequest. (#15430)
* Implement `SendDataColumnSidecarsByRangeRequest` and `SendDataColumnSidecarsByRootRequest`.

* Fix James's comment.

* Fix James's comment.

* Fix James' comment.

* Fix marshaller.
2025-06-24 21:40:16 +00:00
Manu NALEPA
e8625cd89d Peerdas various (#15423)
* Topic mapping: Groupe `const` and `var`.

Cosmetic change, no functional change.

* `TopicFromMessage`: Do not assume anymore that all Fulu specific topics are V3 only.

* Proto: Remove unused `DataColumnIdentifier` and add new `StatusV2`.

`DataColumnIdentifier` was removed in the spec here: https://github.com/ethereum/consensus-specs/pull/4284. Eventually, we stopped using it in Prysm, but never removed the corresponding proto message.

The new `StatusV2` is introduced in the spec here: https://github.com/ethereum/consensus-specs/pull/4374

* `readChunkedDataColumnSideCar` ==> `readChunkedDataColumnSidecar`.

* `rpc_send_request.go`: Reorganize file (no function change).

* `readChunkedDataColumnSidecar`: Add `validationFunctions` parameter and add tests.
2025-06-23 14:47:02 +00:00
terence
667aaf1564 Remove invalid from logs for blob sidecars missing parent or out of range (#15428) 2025-06-22 21:55:44 +00:00
Bastin
e020907d2a Revert "SSZ support for submitAttestationsV2 (#15422)" (#15426)
This reverts commit 9927cea35a.
2025-06-20 21:21:16 +00:00
Bastin
9927cea35a SSZ support for submitAttestationsV2 (#15422)
* ssz support for submitAttestationsV2

* changelog

* lowercase errors

* save 1 line

* Update beacon-chain/rpc/eth/beacon/handlers_pool.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* add comment

---------

Co-authored-by: Radosław Kapka <rkapka@wp.pl>
2025-06-20 17:21:48 +00:00
Manu NALEPA
d4233471d2 PeerDAS: Implement data columns by range RPC handler. (#15421)
* `dataColumnsRPCMinValidSlot`: Add new test case.

* peerDAS: Implement `dataColumnSidecarByRangeRPCHandler`.

* `rateLimitingAmount`:  Define out of the function.

* Fix James' comment.

* Fix James' comment.
2025-06-20 15:22:50 +00:00
james-prysm
d63ae69920 adding ssz for get block endpoint (#15390)
* adding get ssz

* adding some tests

* gaz

* adding ssz to e2e

* wip ssz

* adding in additional check on header type

* remove unused

* renaming json rest handler, and adding in usage of use ssz debug flag

* fixing unit tests

* fixing tests

* gaz

* radek feedback

* Update config/features/config.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Update config/features/flags.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Update config/features/flags.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Update validator/client/beacon-api/get_beacon_block.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Update validator/client/beacon-api/get_beacon_block.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Update validator/client/beacon-api/get_beacon_block.go

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* addressing feedback

* missing import

* another missing import

* fixing tests

* gaz

* removing unused

* gaz

* more radek feedback

* fixing context

* adding in check for non accepted conent type

* reverting to not create more edgecases

---------

Co-authored-by: Radosław Kapka <rkapka@wp.pl>
2025-06-20 14:27:09 +00:00
terence
b9fd32dfff Remove deposit count from sync new block log (#15420)
* Remove deposit count from sync new block log

* Fix tests
2025-06-19 23:49:17 +00:00
Manu NALEPA
559d02bf4d peerDAS: Implement dataColumnSidecarByRootRPCHandler. (#15405)
* `CreateTestVerifiedRoDataColumnSidecars`: Use consistent block root.

* peerDAS: Implement `dataColumnSidecarByRootRPCHandler`.

* Fix James' comment.

* Fix James' comment.
2025-06-19 12:12:45 +00:00
94 changed files with 5273 additions and 1452 deletions

View File

@@ -26,9 +26,6 @@ func logStateTransitionData(b interfaces.ReadOnlyBeaconBlock) error {
if len(b.Body().Attestations()) > 0 {
log = log.WithField("attestations", len(b.Body().Attestations()))
}
if len(b.Body().Deposits()) > 0 {
log = log.WithField("deposits", len(b.Body().Deposits()))
}
if len(b.Body().AttesterSlashings()) > 0 {
log = log.WithField("attesterSlashings", len(b.Body().AttesterSlashings()))
}
@@ -111,7 +108,6 @@ func logBlockSyncStatus(block interfaces.ReadOnlyBeaconBlock, blockRoot [32]byte
"sinceSlotStartTime": prysmTime.Now().Sub(startTime),
"chainServiceProcessedTime": prysmTime.Now().Sub(receivedTime) - daWaitedTime,
"dataAvailabilityWaitedTime": daWaitedTime,
"deposits": len(block.Body().Deposits()),
}
log.WithFields(lf).Debug("Synced new block")
} else {
@@ -159,7 +155,9 @@ func logPayload(block interfaces.ReadOnlyBeaconBlock) error {
if err != nil {
return errors.Wrap(err, "could not get BLSToExecutionChanges")
}
fields["blsToExecutionChanges"] = len(changes)
if len(changes) > 0 {
fields["blsToExecutionChanges"] = len(changes)
}
}
log.WithFields(fields).Debug("Synced new payload")
return nil

View File

@@ -53,7 +53,7 @@ func Test_logStateTransitionData(t *testing.T) {
require.NoError(t, err)
return wb
},
want: "\"Finished applying state transition\" attestations=1 deposits=1 prefix=blockchain slot=0",
want: "\"Finished applying state transition\" attestations=1 prefix=blockchain slot=0",
},
{name: "has attester slashing",
b: func() interfaces.ReadOnlyBeaconBlock {
@@ -93,7 +93,7 @@ func Test_logStateTransitionData(t *testing.T) {
require.NoError(t, err)
return wb
},
want: "\"Finished applying state transition\" attestations=1 attesterSlashings=1 deposits=1 prefix=blockchain proposerSlashings=1 slot=0 voluntaryExits=1",
want: "\"Finished applying state transition\" attestations=1 attesterSlashings=1 prefix=blockchain proposerSlashings=1 slot=0 voluntaryExits=1",
},
{name: "has payload",
b: func() interfaces.ReadOnlyBeaconBlock { return wrappedPayloadBlk },

View File

@@ -258,3 +258,12 @@ func WithLightClientStore(lcs *lightclient.Store) Option {
return nil
}
}
// WithStartWaitingDataColumnSidecars sets a channel that the `areDataColumnsAvailable` function will fill
// in when starting to wait for additional data columns.
func WithStartWaitingDataColumnSidecars(c chan bool) Option {
return func(s *Service) error {
s.startWaitingDataColumnSidecars = c
return nil
}
}

View File

@@ -637,7 +637,11 @@ func missingDataColumnIndices(bs *filesystem.DataColumnStorage, root [fieldparam
// 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 sidecar notifier 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, signedBlock interfaces.ReadOnlySignedBeaconBlock) error {
func (s *Service) isDataAvailable(
ctx context.Context,
root [fieldparams.RootLength]byte,
signedBlock interfaces.ReadOnlySignedBeaconBlock,
) error {
block := signedBlock.Block()
if block == nil {
return errors.New("invalid nil beacon block")
@@ -657,7 +661,11 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signedBloc
// areDataColumnsAvailable blocks until all data columns 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.
func (s *Service) areDataColumnsAvailable(ctx context.Context, root [fieldparams.RootLength]byte, block interfaces.ReadOnlyBeaconBlock) error {
func (s *Service) areDataColumnsAvailable(
ctx context.Context,
root [fieldparams.RootLength]byte,
block interfaces.ReadOnlyBeaconBlock,
) error {
// We are only required to check within MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS
blockSlot, currentSlot := block.Slot(), s.CurrentSlot()
blockEpoch, currentEpoch := slots.ToEpoch(blockSlot), slots.ToEpoch(currentSlot)
@@ -724,6 +732,10 @@ func (s *Service) areDataColumnsAvailable(ctx context.Context, root [fieldparams
return nil
}
if s.startWaitingDataColumnSidecars != nil {
s.startWaitingDataColumnSidecars <- true
}
// Log for DA checks that cross over into the next slot; helpful for debugging.
nextSlot := slots.BeginsAt(block.Slot()+1, s.genesisTime)

View File

@@ -3332,17 +3332,27 @@ func testIsAvailableSetup(t *testing.T, params testIsAvailableParams) (context.C
signedBeaconBlock, err := util.GenerateFullBlockFulu(genesisState, secretKeys, conf, 10 /*block slot*/)
require.NoError(t, err)
root, err := signedBeaconBlock.Block.HashTreeRoot()
block := signedBeaconBlock.Block
bodyRoot, err := block.Body.HashTreeRoot()
require.NoError(t, err)
dataColumnsParams := make([]util.DataColumnParams, 0, len(params.columnsToSave))
root, err := block.HashTreeRoot()
require.NoError(t, err)
dataColumnsParams := make([]util.DataColumnParam, 0, len(params.columnsToSave))
for _, i := range params.columnsToSave {
dataColumnParam := util.DataColumnParams{ColumnIndex: i}
dataColumnParam := util.DataColumnParam{
Index: i,
Slot: block.Slot,
ProposerIndex: block.ProposerIndex,
ParentRoot: block.ParentRoot,
StateRoot: block.StateRoot,
BodyRoot: bodyRoot[:],
}
dataColumnsParams = append(dataColumnsParams, dataColumnParam)
}
dataColumnParamsByBlockRoot := util.DataColumnsParamsByRoot{root: dataColumnsParams}
_, verifiedRODataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
_, verifiedRODataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnsParams)
err = dataColumnStorage.Save(verifiedRODataColumns)
require.NoError(t, err)
@@ -3402,38 +3412,47 @@ func TestIsDataAvailable(t *testing.T) {
})
t.Run("Fulu - some initially missing data columns (no reconstruction)", func(t *testing.T) {
startWaiting := make(chan bool)
testParams := testIsAvailableParams{
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{})},
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{}), WithStartWaitingDataColumnSidecars(startWaiting)},
columnsToSave: []uint64{1, 17, 19, 75, 102, 117, 119}, // 119 is not needed, 42 and 87 are missing
blobKzgCommitmentsCount: 3,
}
ctx, _, service, root, signed := testIsAvailableSetup(t, testParams)
block := signed.Block()
slot := block.Slot()
proposerIndex := block.ProposerIndex()
parentRoot := block.ParentRoot()
stateRoot := block.StateRoot()
bodyRoot, err := block.Body().HashTreeRoot()
require.NoError(t, err)
var wrongRoot [fieldparams.RootLength]byte
copy(wrongRoot[:], root[:])
wrongRoot[0]++ // change the root to simulate a wrong root
_, verifiedSidecarsWrongRoot := util.CreateTestVerifiedRoDataColumnSidecars(
t,
[]util.DataColumnParam{
{Index: 42, Slot: slot + 1}, // Needed index, but not for this slot.
})
_, verifiedSidecarsWrongRoot := util.CreateTestVerifiedRoDataColumnSidecars(t, util.DataColumnsParamsByRoot{wrongRoot: {
{ColumnIndex: 42}, // needed
}})
_, verifiedSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{
{Index: 87, Slot: slot, ProposerIndex: proposerIndex, ParentRoot: parentRoot[:], StateRoot: stateRoot[:], BodyRoot: bodyRoot[:]}, // Needed index
{Index: 1, Slot: slot, ProposerIndex: proposerIndex, ParentRoot: parentRoot[:], StateRoot: stateRoot[:], BodyRoot: bodyRoot[:]}, // Not needed index
{Index: 42, Slot: slot, ProposerIndex: proposerIndex, ParentRoot: parentRoot[:], StateRoot: stateRoot[:], BodyRoot: bodyRoot[:]}, // Needed index
})
_, verifiedSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, util.DataColumnsParamsByRoot{root: {
{ColumnIndex: 87}, // needed
{ColumnIndex: 1}, // not needed
{ColumnIndex: 42}, // needed
}})
go func() {
<-startWaiting
time.AfterFunc(10*time.Millisecond, func() {
err := service.dataColumnStorage.Save(verifiedSidecarsWrongRoot)
require.NoError(t, err)
err = service.dataColumnStorage.Save(verifiedSidecars)
require.NoError(t, err)
})
}()
err := service.isDataAvailable(ctx, root, signed)
err = service.isDataAvailable(ctx, root, signed)
require.NoError(t, err)
})
@@ -3442,6 +3461,9 @@ func TestIsDataAvailable(t *testing.T) {
missingColumns = uint64(2)
cgc = 128
)
startWaiting := make(chan bool)
var custodyInfo peerdas.CustodyInfo
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(cgc)
custodyInfo.ToAdvertiseGroupCount.Set(cgc)
@@ -3454,41 +3476,61 @@ func TestIsDataAvailable(t *testing.T) {
}
testParams := testIsAvailableParams{
options: []Option{WithCustodyInfo(&custodyInfo)},
options: []Option{WithCustodyInfo(&custodyInfo), WithStartWaitingDataColumnSidecars(startWaiting)},
columnsToSave: indices,
blobKzgCommitmentsCount: 3,
}
ctx, _, service, root, signed := testIsAvailableSetup(t, testParams)
block := signed.Block()
slot := block.Slot()
proposerIndex := block.ProposerIndex()
parentRoot := block.ParentRoot()
stateRoot := block.StateRoot()
bodyRoot, err := block.Body().HashTreeRoot()
require.NoError(t, err)
dataColumnParams := make([]util.DataColumnParams, 0, missingColumns)
dataColumnParams := make([]util.DataColumnParam, 0, missingColumns)
for i := minimumColumnsCountToReconstruct - missingColumns; i < minimumColumnsCountToReconstruct; i++ {
dataColumnParam := util.DataColumnParams{ColumnIndex: i}
dataColumnParam := util.DataColumnParam{
Index: i,
Slot: slot,
ProposerIndex: proposerIndex,
ParentRoot: parentRoot[:],
StateRoot: stateRoot[:],
BodyRoot: bodyRoot[:],
}
dataColumnParams = append(dataColumnParams, dataColumnParam)
}
_, verifiedSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, util.DataColumnsParamsByRoot{root: dataColumnParams})
_, verifiedSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParams)
go func() {
<-startWaiting
time.AfterFunc(10*time.Millisecond, func() {
err := service.dataColumnStorage.Save(verifiedSidecars)
require.NoError(t, err)
})
}()
err := service.isDataAvailable(ctx, root, signed)
err = service.isDataAvailable(ctx, root, signed)
require.NoError(t, err)
})
t.Run("Fulu - some columns are definitively missing", func(t *testing.T) {
startWaiting := make(chan bool)
params := testIsAvailableParams{
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{})},
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{}), WithStartWaitingDataColumnSidecars(startWaiting)},
blobKzgCommitmentsCount: 3,
}
ctx, cancel, service, root, signed := testIsAvailableSetup(t, params)
time.AfterFunc(10*time.Millisecond, func() {
go func() {
<-startWaiting
cancel()
})
}()
err := service.isDataAvailable(ctx, root, signed)
require.NotNil(t, err)

View File

@@ -47,27 +47,28 @@ import (
// Service represents a service that handles the internal
// logic of managing the full PoS beacon chain.
type Service struct {
cfg *config
ctx context.Context
cancel context.CancelFunc
genesisTime time.Time
head *head
headLock sync.RWMutex
originBlockRoot [32]byte // genesis root, or weak subjectivity checkpoint root, depending on how the node is initialized
boundaryRoots [][32]byte
checkpointStateCache *cache.CheckpointStateCache
initSyncBlocks map[[32]byte]interfaces.ReadOnlySignedBeaconBlock
initSyncBlocksLock sync.RWMutex
wsVerifier *WeakSubjectivityVerifier
clockSetter startup.ClockSetter
clockWaiter startup.ClockWaiter
syncComplete chan struct{}
blobNotifiers *blobNotifierMap
blockBeingSynced *currentlySyncingBlock
blobStorage *filesystem.BlobStorage
dataColumnStorage *filesystem.DataColumnStorage
slasherEnabled bool
lcStore *lightClient.Store
cfg *config
ctx context.Context
cancel context.CancelFunc
genesisTime time.Time
head *head
headLock sync.RWMutex
originBlockRoot [32]byte // genesis root, or weak subjectivity checkpoint root, depending on how the node is initialized
boundaryRoots [][32]byte
checkpointStateCache *cache.CheckpointStateCache
initSyncBlocks map[[32]byte]interfaces.ReadOnlySignedBeaconBlock
initSyncBlocksLock sync.RWMutex
wsVerifier *WeakSubjectivityVerifier
clockSetter startup.ClockSetter
clockWaiter startup.ClockWaiter
syncComplete chan struct{}
blobNotifiers *blobNotifierMap
blockBeingSynced *currentlySyncingBlock
blobStorage *filesystem.BlobStorage
dataColumnStorage *filesystem.DataColumnStorage
slasherEnabled bool
lcStore *lightClient.Store
startWaitingDataColumnSidecars chan bool // for testing purposes only
}
// config options for the service.

View File

@@ -38,9 +38,9 @@ func TestPersist(t *testing.T) {
t.Run("mixed roots", func(t *testing.T) {
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
dataColumnParamsByBlockRoot := map[[fieldparams.RootLength]byte][]util.DataColumnParams{
{1}: {{ColumnIndex: 1}},
{2}: {{ColumnIndex: 2}},
dataColumnParamsByBlockRoot := []util.DataColumnParam{
{Slot: 1, Index: 1},
{Slot: 2, Index: 2},
}
roSidecars, _ := roSidecarsFromDataColumnParamsByBlockRoot(t, dataColumnParamsByBlockRoot)
@@ -54,8 +54,8 @@ func TestPersist(t *testing.T) {
t.Run("outside DA period", func(t *testing.T) {
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
dataColumnParamsByBlockRoot := map[[fieldparams.RootLength]byte][]util.DataColumnParams{
{1}: {{ColumnIndex: 1}},
dataColumnParamsByBlockRoot := []util.DataColumnParam{
{Slot: 1, Index: 1},
}
roSidecars, _ := roSidecarsFromDataColumnParamsByBlockRoot(t, dataColumnParamsByBlockRoot)
@@ -67,21 +67,24 @@ func TestPersist(t *testing.T) {
})
t.Run("nominal", func(t *testing.T) {
const slot = 42
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
dataColumnParamsByBlockRoot := map[[fieldparams.RootLength]byte][]util.DataColumnParams{
{}: {{ColumnIndex: 1}, {ColumnIndex: 5}},
dataColumnParamsByBlockRoot := []util.DataColumnParam{
{Slot: slot, Index: 1},
{Slot: slot, Index: 5},
}
roSidecars, roDataColumns := roSidecarsFromDataColumnParamsByBlockRoot(t, dataColumnParamsByBlockRoot)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, &peerdas.CustodyInfo{})
err := lazilyPersistentStoreColumns.Persist(0, roSidecars...)
err := lazilyPersistentStoreColumns.Persist(slot, roSidecars...)
require.NoError(t, err)
require.Equal(t, 1, len(lazilyPersistentStoreColumns.cache.entries))
key := cacheKey{slot: 0, root: [fieldparams.RootLength]byte{}}
entry := lazilyPersistentStoreColumns.cache.entries[key]
key := cacheKey{slot: slot, root: roDataColumns[0].BlockRoot()}
entry, ok := lazilyPersistentStoreColumns.cache.entries[key]
require.Equal(t, true, ok)
// A call to Persist does NOT save the sidecars to disk.
require.Equal(t, uint64(0), entry.diskSummary.Count())
@@ -121,24 +124,37 @@ func TestIsDataAvailable(t *testing.T) {
signedBeaconBlockFulu := util.NewBeaconBlockFulu()
signedBeaconBlockFulu.Block.Body.BlobKzgCommitments = commitments
signedRoBlock := newSignedRoBlock(t, signedBeaconBlockFulu)
block := signedRoBlock.Block()
slot := block.Slot()
proposerIndex := block.ProposerIndex()
parentRoot := block.ParentRoot()
stateRoot := block.StateRoot()
bodyRoot, err := block.Body().HashTreeRoot()
require.NoError(t, err)
root := signedRoBlock.Root()
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, newDataColumnsVerifier, &peerdas.CustodyInfo{})
indices := [...]uint64{1, 17, 87, 102}
dataColumnsParams := make([]util.DataColumnParams, 0, len(indices))
dataColumnsParams := make([]util.DataColumnParam, 0, len(indices))
for _, index := range indices {
dataColumnParams := util.DataColumnParams{
ColumnIndex: index,
dataColumnParams := util.DataColumnParam{
Index: index,
KzgCommitments: commitments,
Slot: slot,
ProposerIndex: proposerIndex,
ParentRoot: parentRoot[:],
StateRoot: stateRoot[:],
BodyRoot: bodyRoot[:],
}
dataColumnsParams = append(dataColumnsParams, dataColumnParams)
}
dataColumnsParamsByBlockRoot := util.DataColumnsParamsByRoot{root: dataColumnsParams}
_, verifiedRoDataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnsParamsByBlockRoot)
_, verifiedRoDataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnsParams)
key := cacheKey{root: root}
entry := lazilyPersistentStoreColumns.cache.ensure(key)
@@ -149,7 +165,7 @@ func TestIsDataAvailable(t *testing.T) {
require.NoError(t, err)
}
err := lazilyPersistentStoreColumns.IsDataAvailable(ctx, 0 /*current slot*/, signedRoBlock)
err = lazilyPersistentStoreColumns.IsDataAvailable(ctx, slot, signedRoBlock)
require.NoError(t, err)
actual, err := dataColumnStorage.Get(root, indices[:])
@@ -224,8 +240,8 @@ func TestFullCommitmentsToCheck(t *testing.T) {
}
}
func roSidecarsFromDataColumnParamsByBlockRoot(t *testing.T, dataColumnParamsByBlockRoot util.DataColumnsParamsByRoot) ([]blocks.ROSidecar, []blocks.RODataColumn) {
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
func roSidecarsFromDataColumnParamsByBlockRoot(t *testing.T, parameters []util.DataColumnParam) ([]blocks.ROSidecar, []blocks.RODataColumn) {
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, parameters)
roSidecars := make([]blocks.ROSidecar, 0, len(roDataColumns))
for _, roDataColumn := range roDataColumns {

View File

@@ -28,8 +28,7 @@ func TestEnsureDeleteSetDiskSummary(t *testing.T) {
func TestStash(t *testing.T) {
t.Run("Index too high", func(t *testing.T) {
dataColumnParamsByBlockRoot := util.DataColumnsParamsByRoot{{1}: {{ColumnIndex: 10_000}}}
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 10_000}})
var entry dataColumnCacheEntry
err := entry.stash(&roDataColumns[0])
@@ -37,8 +36,7 @@ func TestStash(t *testing.T) {
})
t.Run("Nominal and already existing", func(t *testing.T) {
dataColumnParamsByBlockRoot := util.DataColumnsParamsByRoot{{1}: {{ColumnIndex: 1}}}
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 1}})
var entry dataColumnCacheEntry
err := entry.stash(&roDataColumns[0])
@@ -76,36 +74,30 @@ func TestFilterDataColumns(t *testing.T) {
})
t.Run("Commitments not equal", func(t *testing.T) {
root := [fieldparams.RootLength]byte{}
commitmentsArray := safeCommitmentsArray{nil, [][]byte{[]byte{1}}}
dataColumnParamsByBlockRoot := util.DataColumnsParamsByRoot{root: {{ColumnIndex: 1}}}
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
roDataColumns, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 1}})
var scs [fieldparams.NumberOfColumns]*blocks.RODataColumn
scs[1] = &roDataColumns[0]
dataColumnCacheEntry := dataColumnCacheEntry{scs: scs}
_, err := dataColumnCacheEntry.filter(root, &commitmentsArray)
_, err := dataColumnCacheEntry.filter(roDataColumns[0].BlockRoot(), &commitmentsArray)
require.NotNil(t, err)
})
t.Run("Nominal", func(t *testing.T) {
root := [fieldparams.RootLength]byte{}
commitmentsArray := safeCommitmentsArray{nil, [][]byte{[]byte{1}}, nil, [][]byte{[]byte{3}}}
diskSummary := filesystem.NewDataColumnStorageSummary(42, [fieldparams.NumberOfColumns]bool{false, true})
dataColumnParamsByBlockRoot := util.DataColumnsParamsByRoot{root: {{ColumnIndex: 3, KzgCommitments: [][]byte{[]byte{3}}}}}
expected, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, dataColumnParamsByBlockRoot)
expected, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: 3, KzgCommitments: [][]byte{[]byte{3}}}})
var scs [fieldparams.NumberOfColumns]*blocks.RODataColumn
scs[3] = &expected[0]
dataColumnCacheEntry := dataColumnCacheEntry{scs: scs, diskSummary: diskSummary}
actual, err := dataColumnCacheEntry.filter(root, &commitmentsArray)
actual, err := dataColumnCacheEntry.filter(expected[0].BlockRoot(), &commitmentsArray)
require.NoError(t, err)
require.DeepEqual(t, expected, actual)

View File

@@ -59,6 +59,7 @@ go_test(
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",

View File

@@ -25,7 +25,7 @@ func directoryPermissions() os.FileMode {
var (
errIndexOutOfBounds = errors.New("blob index in file name >= MAX_BLOBS_PER_BLOCK")
errSidecarEmptySSZData = errors.New("sidecar marshalled to an empty ssz byte slice")
errNoBasePath = errors.New("BlobStorage base path not specified in init")
errNoBlobBasePath = errors.New("BlobStorage base path not specified in init")
)
// BlobStorageOption is a functional option for configuring a BlobStorage.
@@ -85,7 +85,7 @@ func NewBlobStorage(opts ...BlobStorageOption) (*BlobStorage, error) {
// Allow tests to set up a different fs using WithFs.
if b.fs == nil {
if b.base == "" {
return nil, errNoBasePath
return nil, errNoBlobBasePath
}
b.base = path.Clean(b.base)
if err := file.MkdirAll(b.base); err != nil {

View File

@@ -160,7 +160,7 @@ func writeFakeSSZ(t *testing.T, fs afero.Fs, root [32]byte, slot primitives.Slot
func TestNewBlobStorage(t *testing.T) {
_, err := NewBlobStorage()
require.ErrorIs(t, err, errNoBasePath)
require.ErrorIs(t, err, errNoBlobBasePath)
_, err = NewBlobStorage(WithBasePath(path.Join(t.TempDir(), "good")))
require.NoError(t, err)
}

View File

@@ -50,6 +50,7 @@ var (
errTooManyDataColumns = errors.New("too many data columns")
errWrongSszEncodedDataColumnSidecarSize = errors.New("wrong SSZ encoded data column sidecar size")
errDataColumnSidecarsFromDifferentSlots = errors.New("data column sidecars from different slots")
errNoDataColumnBasePath = errors.New("DataColumnStorage base path not specified in init")
)
type (
@@ -142,7 +143,7 @@ func NewDataColumnStorage(ctx context.Context, opts ...DataColumnStorageOption)
// Allow tests to set up a different fs using WithFs.
if storage.fs == nil {
if storage.base == "" {
return nil, errNoBasePath
return nil, errNoDataColumnBasePath
}
storage.base = path.Clean(storage.base)

View File

@@ -7,6 +7,7 @@ import (
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
@@ -18,7 +19,7 @@ func TestNewDataColumnStorage(t *testing.T) {
t.Run("No base path", func(t *testing.T) {
_, err := NewDataColumnStorage(ctx)
require.ErrorIs(t, err, errNoBasePath)
require.ErrorIs(t, err, errNoDataColumnBasePath)
})
t.Run("Nominal", func(t *testing.T) {
@@ -40,32 +41,18 @@ func TestWarmCache(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{0}: {
{Slot: 33, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 1
{Slot: 33, ColumnIndex: 4, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 1
},
{1}: {
{Slot: 128_002, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 4000
{Slot: 128_002, ColumnIndex: 4, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 4000
},
{2}: {
{Slot: 128_003, ColumnIndex: 1, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 4000
{Slot: 128_003, ColumnIndex: 3, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 4000
},
{3}: {
{Slot: 128_034, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 4001
{Slot: 128_034, ColumnIndex: 4, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 4001
},
{4}: {
{Slot: 131_138, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4098
},
{5}: {
{Slot: 131_138, ColumnIndex: 1, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4098
},
{6}: {
{Slot: 131_168, ColumnIndex: 0, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4099
},
[]util.DataColumnParam{
{Slot: 33, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 1
{Slot: 33, Index: 4, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 1
{Slot: 128_002, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 4000
{Slot: 128_002, Index: 4, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 4000
{Slot: 128_003, Index: 1, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 4000
{Slot: 128_003, Index: 3, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 4000
{Slot: 128_034, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 4001
{Slot: 128_034, Index: 4, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 4001
{Slot: 131_138, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4098
{Slot: 131_138, Index: 1, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4098
{Slot: 131_168, Index: 0, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4099
},
)
@@ -76,29 +63,25 @@ func TestWarmCache(t *testing.T) {
storage.WarmCache()
require.Equal(t, primitives.Epoch(4_000), storage.cache.lowestCachedEpoch)
require.Equal(t, 6, len(storage.cache.cache))
require.Equal(t, 5, len(storage.cache.cache))
summary, ok := storage.cache.get([fieldparams.RootLength]byte{1})
summary, ok := storage.cache.get(verifiedRoDataColumnSidecars[2].BlockRoot())
require.Equal(t, true, ok)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_000, mask: [fieldparams.NumberOfColumns]bool{false, false, true, false, true}}, summary)
summary, ok = storage.cache.get([fieldparams.RootLength]byte{2})
summary, ok = storage.cache.get(verifiedRoDataColumnSidecars[4].BlockRoot())
require.Equal(t, true, ok)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_000, mask: [fieldparams.NumberOfColumns]bool{false, true, false, true}}, summary)
summary, ok = storage.cache.get([fieldparams.RootLength]byte{3})
summary, ok = storage.cache.get(verifiedRoDataColumnSidecars[6].BlockRoot())
require.Equal(t, true, ok)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_001, mask: [fieldparams.NumberOfColumns]bool{false, false, true, false, true}}, summary)
summary, ok = storage.cache.get([fieldparams.RootLength]byte{4})
summary, ok = storage.cache.get(verifiedRoDataColumnSidecars[8].BlockRoot())
require.Equal(t, true, ok)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_098, mask: [fieldparams.NumberOfColumns]bool{false, false, true}}, summary)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_098, mask: [fieldparams.NumberOfColumns]bool{false, true, true}}, summary)
summary, ok = storage.cache.get([fieldparams.RootLength]byte{5})
require.Equal(t, true, ok)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_098, mask: [fieldparams.NumberOfColumns]bool{false, true}}, summary)
summary, ok = storage.cache.get([fieldparams.RootLength]byte{6})
summary, ok = storage.cache.get(verifiedRoDataColumnSidecars[10].BlockRoot())
require.Equal(t, true, ok)
require.DeepEqual(t, DataColumnStorageSummary{epoch: 4_099, mask: [fieldparams.NumberOfColumns]bool{true}}, summary)
}
@@ -112,9 +95,7 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{}: {{ColumnIndex: 12}, {ColumnIndex: 1_000_000}, {ColumnIndex: 48}},
},
[]util.DataColumnParam{{Index: 12}, {Index: 1_000_000}, {Index: 48}},
)
_, dataColumnStorage := NewEphemeralDataColumnStorageAndFs(t)
@@ -125,7 +106,7 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
t.Run("one of the column index is too large", func(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{{}: {{ColumnIndex: 12}, {ColumnIndex: 1_000_000}, {ColumnIndex: 48}}},
[]util.DataColumnParam{{Index: 12}, {Index: 1_000_000}, {Index: 48}},
)
_, dataColumnStorage := NewEphemeralDataColumnStorageAndFs(t)
@@ -136,23 +117,34 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
t.Run("different slots", func(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{}: {
{Slot: 1, ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
{Slot: 2, ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 2, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
},
)
// Create a sidecar with a different slot but the same root.
alteredVerifiedRoDataColumnSidecars := make([]blocks.VerifiedRODataColumn, 0, 2)
alteredVerifiedRoDataColumnSidecars = append(alteredVerifiedRoDataColumnSidecars, verifiedRoDataColumnSidecars[0])
altered, err := blocks.NewRODataColumnWithRoot(
verifiedRoDataColumnSidecars[1].RODataColumn.DataColumnSidecar,
verifiedRoDataColumnSidecars[0].BlockRoot(),
)
require.NoError(t, err)
verifiedAltered := blocks.NewVerifiedRODataColumn(altered)
alteredVerifiedRoDataColumnSidecars = append(alteredVerifiedRoDataColumnSidecars, verifiedAltered)
_, dataColumnStorage := NewEphemeralDataColumnStorageAndFs(t)
err := dataColumnStorage.Save(verifiedRoDataColumnSidecars)
err = dataColumnStorage.Save(alteredVerifiedRoDataColumnSidecars)
require.ErrorIs(t, err, errDataColumnSidecarsFromDifferentSlots)
})
t.Run("new file - no data columns to save", func(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{{}: {}},
[]util.DataColumnParam{},
)
_, dataColumnStorage := NewEphemeralDataColumnStorageAndFs(t)
@@ -163,11 +155,9 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
t.Run("new file - different data column size", func(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{}: {
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
{ColumnIndex: 11, DataColumn: []byte{1, 2, 3, 4}},
},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 1, Index: 13, Column: [][]byte{{1}, {2}, {3}, {4}}},
},
)
@@ -179,7 +169,9 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
t.Run("existing file - wrong incoming SSZ encoded size", func(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{{1}: {{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}}}},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
},
)
// Save data columns into a file.
@@ -191,7 +183,9 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
// column index and an different SSZ encoded size.
_, verifiedRoDataColumnSidecars = util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{{1}: {{ColumnIndex: 13, DataColumn: []byte{1, 2, 3, 4}}}},
[]util.DataColumnParam{
{Slot: 1, Index: 13, Column: [][]byte{{1}, {2}, {3}, {4}}},
},
)
// Try to rewrite the file.
@@ -202,17 +196,13 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
t.Run("nominal", func(t *testing.T) {
_, inputVerifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
{ColumnIndex: 11, DataColumn: []byte{3, 4, 5}},
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}}, // OK if duplicate
{ColumnIndex: 13, DataColumn: []byte{6, 7, 8}},
},
{2}: {
{ColumnIndex: 12, DataColumn: []byte{3, 4, 5}},
{ColumnIndex: 13, DataColumn: []byte{6, 7, 8}},
},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 1, Index: 11, Column: [][]byte{{3}, {4}, {5}}},
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}}, // OK if duplicate
{Slot: 1, Index: 13, Column: [][]byte{{6}, {7}, {8}}},
{Slot: 2, Index: 12, Column: [][]byte{{3}, {4}, {5}}},
{Slot: 2, Index: 13, Column: [][]byte{{6}, {7}, {8}}},
},
)
@@ -222,16 +212,12 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
_, inputVerifiedRoDataColumnSidecars = util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}}, // OK if duplicate
{ColumnIndex: 15, DataColumn: []byte{2, 3, 4}},
{ColumnIndex: 1, DataColumn: []byte{2, 3, 4}},
},
{3}: {
{ColumnIndex: 6, DataColumn: []byte{3, 4, 5}},
{ColumnIndex: 2, DataColumn: []byte{6, 7, 8}},
},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}}, // OK if duplicate
{Slot: 1, Index: 15, Column: [][]byte{{2}, {3}, {4}}},
{Slot: 1, Index: 1, Column: [][]byte{{2}, {3}, {4}}},
{Slot: 3, Index: 6, Column: [][]byte{{3}, {4}, {5}}},
{Slot: 3, Index: 2, Column: [][]byte{{6}, {7}, {8}}},
},
)
@@ -240,51 +226,47 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
type fixture struct {
fileName string
blockRoot [fieldparams.RootLength]byte
expectedIndices [mandatoryNumberOfColumns]byte
dataColumnParams []util.DataColumnParams
dataColumnParams []util.DataColumnParam
}
fixtures := []fixture{
{
fileName: "0/0/0x0100000000000000000000000000000000000000000000000000000000000000.sszs",
blockRoot: [fieldparams.RootLength]byte{1},
fileName: "0/0/0x8bb2f09de48c102635622dc27e6de03ae2b22639df7c33edbc8222b2ec423746.sszs",
expectedIndices: [mandatoryNumberOfColumns]byte{
0, nonZeroOffset + 4, 0, 0, 0, 0, 0, 0,
0, 0, 0, nonZeroOffset + 1, nonZeroOffset, nonZeroOffset + 2, 0, nonZeroOffset + 3,
// The rest is filled with zeroes.
},
dataColumnParams: []util.DataColumnParams{
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
{ColumnIndex: 11, DataColumn: []byte{3, 4, 5}},
{ColumnIndex: 13, DataColumn: []byte{6, 7, 8}},
{ColumnIndex: 15, DataColumn: []byte{2, 3, 4}},
{ColumnIndex: 1, DataColumn: []byte{2, 3, 4}},
dataColumnParams: []util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 1, Index: 11, Column: [][]byte{{3}, {4}, {5}}},
{Slot: 1, Index: 13, Column: [][]byte{{6}, {7}, {8}}},
{Slot: 1, Index: 15, Column: [][]byte{{2}, {3}, {4}}},
{Slot: 1, Index: 1, Column: [][]byte{{2}, {3}, {4}}},
},
},
{
fileName: "0/0/0x0200000000000000000000000000000000000000000000000000000000000000.sszs",
blockRoot: [fieldparams.RootLength]byte{2},
fileName: "0/0/0x221f88cae2219050d4e9d8c2d0d83cb4c8ce4c84ab1bb3e0b89f3dec36077c4f.sszs",
expectedIndices: [mandatoryNumberOfColumns]byte{
0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, nonZeroOffset, nonZeroOffset + 1, 0, 0,
// The rest is filled with zeroes.
},
dataColumnParams: []util.DataColumnParams{
{ColumnIndex: 12, DataColumn: []byte{3, 4, 5}},
{ColumnIndex: 13, DataColumn: []byte{6, 7, 8}},
dataColumnParams: []util.DataColumnParam{
{Slot: 2, Index: 12, Column: [][]byte{{3}, {4}, {5}}},
{Slot: 2, Index: 13, Column: [][]byte{{6}, {7}, {8}}},
},
},
{
fileName: "0/0/0x0300000000000000000000000000000000000000000000000000000000000000.sszs",
blockRoot: [fieldparams.RootLength]byte{3},
fileName: "0/0/0x7b163bd57e1c4c8b5048c5389698098f4c957d62d7ce86f4ffa9bdc75c16a18b.sszs",
expectedIndices: [mandatoryNumberOfColumns]byte{
0, 0, nonZeroOffset + 1, 0, 0, 0, nonZeroOffset, 0,
// The rest is filled with zeroes.
},
dataColumnParams: []util.DataColumnParams{
{ColumnIndex: 6, DataColumn: []byte{3, 4, 5}},
{ColumnIndex: 2, DataColumn: []byte{6, 7, 8}},
dataColumnParams: []util.DataColumnParam{
{Slot: 3, Index: 6, Column: [][]byte{{3}, {4}, {5}}},
{Slot: 3, Index: 2, Column: [][]byte{{6}, {7}, {8}}},
},
},
}
@@ -293,7 +275,7 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
// Build expected data column sidecars.
_, expectedDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{fixture.blockRoot: fixture.dataColumnParams},
fixture.dataColumnParams,
)
// Build expected bytes.
@@ -320,6 +302,8 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
expectedBytes = append(expectedBytes, fixture.expectedIndices[:]...)
expectedBytes = append(expectedBytes, sszEncodedDataColumnSidecars...)
blockRoot := expectedDataColumnSidecars[0].BlockRoot()
// Check the actual content of the file.
actualBytes, err := afero.ReadFile(dataColumnStorage.fs, fixture.fileName)
require.NoError(t, err)
@@ -328,18 +312,18 @@ func TestSaveDataColumnsSidecars(t *testing.T) {
// Check the summary.
indices := map[uint64]bool{}
for _, dataColumnParam := range fixture.dataColumnParams {
indices[dataColumnParam.ColumnIndex] = true
indices[dataColumnParam.Index] = true
}
summary := dataColumnStorage.Summary(fixture.blockRoot)
summary := dataColumnStorage.Summary(blockRoot)
for index := range uint64(mandatoryNumberOfColumns) {
require.Equal(t, indices[index], summary.HasIndex(index))
}
err = dataColumnStorage.Remove(fixture.blockRoot)
err = dataColumnStorage.Remove(blockRoot)
require.NoError(t, err)
summary = dataColumnStorage.Summary(fixture.blockRoot)
summary = dataColumnStorage.Summary(blockRoot)
for index := range uint64(mandatoryNumberOfColumns) {
require.Equal(t, false, summary.HasIndex(index))
}
@@ -362,11 +346,9 @@ func TestGetDataColumnSidecars(t *testing.T) {
t.Run("indices not found", func(t *testing.T) {
_, savedVerifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
{ColumnIndex: 14, DataColumn: []byte{2, 3, 4}},
},
[]util.DataColumnParam{
{Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Index: 14, Column: [][]byte{{2}, {3}, {4}}},
},
)
@@ -374,7 +356,7 @@ func TestGetDataColumnSidecars(t *testing.T) {
err := dataColumnStorage.Save(savedVerifiedRoDataColumnSidecars)
require.NoError(t, err)
verifiedRODataColumnSidecars, err := dataColumnStorage.Get([fieldparams.RootLength]byte{1}, []uint64{3, 1, 2})
verifiedRODataColumnSidecars, err := dataColumnStorage.Get(savedVerifiedRoDataColumnSidecars[0].BlockRoot(), []uint64{3, 1, 2})
require.NoError(t, err)
require.Equal(t, 0, len(verifiedRODataColumnSidecars))
})
@@ -382,11 +364,9 @@ func TestGetDataColumnSidecars(t *testing.T) {
t.Run("nominal", func(t *testing.T) {
_, expectedVerifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {
{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}},
{ColumnIndex: 14, DataColumn: []byte{2, 3, 4}},
},
[]util.DataColumnParam{
{Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Index: 14, Column: [][]byte{{2}, {3}, {4}}},
},
)
@@ -394,11 +374,13 @@ func TestGetDataColumnSidecars(t *testing.T) {
err := dataColumnStorage.Save(expectedVerifiedRoDataColumnSidecars)
require.NoError(t, err)
verifiedRODataColumnSidecars, err := dataColumnStorage.Get([fieldparams.RootLength]byte{1}, nil)
root := expectedVerifiedRoDataColumnSidecars[0].BlockRoot()
verifiedRODataColumnSidecars, err := dataColumnStorage.Get(root, nil)
require.NoError(t, err)
require.DeepSSZEqual(t, expectedVerifiedRoDataColumnSidecars, verifiedRODataColumnSidecars)
verifiedRODataColumnSidecars, err = dataColumnStorage.Get([fieldparams.RootLength]byte{1}, []uint64{12, 13, 14})
verifiedRODataColumnSidecars, err = dataColumnStorage.Get(root, []uint64{12, 13, 14})
require.NoError(t, err)
require.DeepSSZEqual(t, expectedVerifiedRoDataColumnSidecars, verifiedRODataColumnSidecars)
})
@@ -414,15 +396,11 @@ func TestRemove(t *testing.T) {
t.Run("nominal", func(t *testing.T) {
_, inputVerifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {
{Slot: 32, ColumnIndex: 10, DataColumn: []byte{1, 2, 3}},
{Slot: 32, ColumnIndex: 11, DataColumn: []byte{2, 3, 4}},
},
{2}: {
{Slot: 33, ColumnIndex: 10, DataColumn: []byte{1, 2, 3}},
{Slot: 33, ColumnIndex: 11, DataColumn: []byte{2, 3, 4}},
},
[]util.DataColumnParam{
{Slot: 32, Index: 10, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 32, Index: 11, Column: [][]byte{{2}, {3}, {4}}},
{Slot: 33, Index: 10, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 33, Index: 11, Column: [][]byte{{2}, {3}, {4}}},
},
)
@@ -430,22 +408,22 @@ func TestRemove(t *testing.T) {
err := dataColumnStorage.Save(inputVerifiedRoDataColumnSidecars)
require.NoError(t, err)
err = dataColumnStorage.Remove([fieldparams.RootLength]byte{1})
err = dataColumnStorage.Remove(inputVerifiedRoDataColumnSidecars[0].BlockRoot())
require.NoError(t, err)
summary := dataColumnStorage.Summary([fieldparams.RootLength]byte{1})
summary := dataColumnStorage.Summary(inputVerifiedRoDataColumnSidecars[0].BlockRoot())
require.Equal(t, primitives.Epoch(0), summary.epoch)
require.Equal(t, uint64(0), summary.Count())
summary = dataColumnStorage.Summary([fieldparams.RootLength]byte{2})
summary = dataColumnStorage.Summary(inputVerifiedRoDataColumnSidecars[3].BlockRoot())
require.Equal(t, primitives.Epoch(1), summary.epoch)
require.Equal(t, uint64(2), summary.Count())
actual, err := dataColumnStorage.Get([fieldparams.RootLength]byte{1}, nil)
actual, err := dataColumnStorage.Get(inputVerifiedRoDataColumnSidecars[0].BlockRoot(), nil)
require.NoError(t, err)
require.Equal(t, 0, len(actual))
actual, err = dataColumnStorage.Get([fieldparams.RootLength]byte{2}, nil)
actual, err = dataColumnStorage.Get(inputVerifiedRoDataColumnSidecars[3].BlockRoot(), nil)
require.NoError(t, err)
require.Equal(t, 2, len(actual))
})
@@ -454,9 +432,9 @@ func TestRemove(t *testing.T) {
func TestClear(t *testing.T) {
_, inputVerifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}}},
{2}: {{ColumnIndex: 13, DataColumn: []byte{6, 7, 8}}},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
{Slot: 2, Index: 13, Column: [][]byte{{6}, {7}, {8}}},
},
)
@@ -465,8 +443,8 @@ func TestClear(t *testing.T) {
require.NoError(t, err)
filePaths := []string{
"0/0/0x0100000000000000000000000000000000000000000000000000000000000000.sszs",
"0/0/0x0200000000000000000000000000000000000000000000000000000000000000.sszs",
"0/0/0x8bb2f09de48c102635622dc27e6de03ae2b22639df7c33edbc8222b2ec423746.sszs",
"0/0/0x221f88cae2219050d4e9d8c2d0d83cb4c8ce4c84ab1bb3e0b89f3dec36077c4f.sszs",
}
for _, filePath := range filePaths {
@@ -492,8 +470,8 @@ func TestMetadata(t *testing.T) {
t.Run("wrong version", func(t *testing.T) {
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{1}: {{ColumnIndex: 12, DataColumn: []byte{1, 2, 3}}},
[]util.DataColumnParam{
{Slot: 1, Index: 12, Column: [][]byte{{1}, {2}, {3}}},
},
)
@@ -503,7 +481,7 @@ func TestMetadata(t *testing.T) {
require.NoError(t, err)
// Alter the version.
const filePath = "0/0/0x0100000000000000000000000000000000000000000000000000000000000000.sszs"
const filePath = "0/0/0x8bb2f09de48c102635622dc27e6de03ae2b22639df7c33edbc8222b2ec423746.sszs"
file, err := dataColumnStorage.fs.OpenFile(filePath, os.O_WRONLY, os.FileMode(0600))
require.NoError(t, err)
@@ -643,31 +621,19 @@ func TestPrune(t *testing.T) {
}
_, verifiedRoDataColumnSidecars := util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{0}: {
{Slot: 33, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 1
{Slot: 33, ColumnIndex: 4, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 1
},
{1}: {
{Slot: 128_002, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 4000
{Slot: 128_002, ColumnIndex: 4, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 4000
},
{2}: {
{Slot: 128_003, ColumnIndex: 1, DataColumn: []byte{1, 2, 3}}, // Period 0 - Epoch 4000
{Slot: 128_003, ColumnIndex: 3, DataColumn: []byte{2, 3, 4}}, // Period 0 - Epoch 4000
},
{3}: {
{Slot: 131_138, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4098
{Slot: 131_138, ColumnIndex: 3, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4098
},
{4}: {
{Slot: 131_169, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4099
{Slot: 131_169, ColumnIndex: 3, DataColumn: []byte{1, 2, 3}}, // Period 1 - Epoch 4099
},
{5}: {
{Slot: 262_144, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}, // Period 2 - Epoch 8192
{Slot: 262_144, ColumnIndex: 3, DataColumn: []byte{1, 2, 3}}, // Period 2 - Epoch 8292
},
[]util.DataColumnParam{
{Slot: 33, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 1
{Slot: 33, Index: 4, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 1
{Slot: 128_002, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 4000
{Slot: 128_002, Index: 4, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 4000
{Slot: 128_003, Index: 1, Column: [][]byte{{1}, {2}, {3}}}, // Period 0 - Epoch 4000
{Slot: 128_003, Index: 3, Column: [][]byte{{2}, {3}, {4}}}, // Period 0 - Epoch 4000
{Slot: 131_138, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4098
{Slot: 131_138, Index: 3, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4098
{Slot: 131_169, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4099
{Slot: 131_169, Index: 3, Column: [][]byte{{1}, {2}, {3}}}, // Period 1 - Epoch 4099
{Slot: 262_144, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 2 - Epoch 8192
{Slot: 262_144, Index: 3, Column: [][]byte{{1}, {2}, {3}}}, // Period 2 - Epoch 8292
},
)
@@ -696,31 +662,31 @@ func TestPrune(t *testing.T) {
dirs, err = listDir(dataColumnStorage.fs, "0/1")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0000000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0x775283f428813c949b7e8af07f01fef9790137f021b3597ad2d0d81e8be8f0f0.sszs"}, dirs))
dirs, err = listDir(dataColumnStorage.fs, "0/4000")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{
"0x0200000000000000000000000000000000000000000000000000000000000000.sszs",
"0x0100000000000000000000000000000000000000000000000000000000000000.sszs",
"0x9977031132157ebb9c81bce952003ce07a4f54e921ca63b7693d1562483fdf9f.sszs",
"0xb2b14d9d858fa99b70f0405e4e39f38e51e36dd9a70343c109e24eeb5f77e369.sszs",
}, dirs))
dirs, err = listDir(dataColumnStorage.fs, "1/4098")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0300000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0x5106745cdd6b1aa3602ef4d000ef373af672019264c167fa4bd641a1094aa5c5.sszs"}, dirs))
dirs, err = listDir(dataColumnStorage.fs, "1/4099")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0400000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0x4e5f2bd5bb84bf0422af8edd1cc5a52cc6cea85baf3d66d172fe41831ac1239c.sszs"}, dirs))
dirs, err = listDir(dataColumnStorage.fs, "2/8192")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0500000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0xa8adba7446eb56a01a9dd6d55e9c3990b10c91d43afb77847b4a21ac4ee62527.sszs"}, dirs))
_, verifiedRoDataColumnSidecars = util.CreateTestVerifiedRoDataColumnSidecars(
t,
util.DataColumnsParamsByRoot{
{6}: {{Slot: 451_141, ColumnIndex: 2, DataColumn: []byte{1, 2, 3}}}, // Period 3 - Epoch 14_098
[]util.DataColumnParam{
{Slot: 451_141, Index: 2, Column: [][]byte{{1}, {2}, {3}}}, // Period 3 - Epoch 14_098
},
)
@@ -748,14 +714,14 @@ func TestPrune(t *testing.T) {
dirs, err = listDir(dataColumnStorage.fs, "1/4099")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0400000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0x4e5f2bd5bb84bf0422af8edd1cc5a52cc6cea85baf3d66d172fe41831ac1239c.sszs"}, dirs))
dirs, err = listDir(dataColumnStorage.fs, "2/8192")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0500000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0xa8adba7446eb56a01a9dd6d55e9c3990b10c91d43afb77847b4a21ac4ee62527.sszs"}, dirs))
dirs, err = listDir(dataColumnStorage.fs, "3/14098")
require.NoError(t, err)
require.Equal(t, true, compareSlices([]string{"0x0600000000000000000000000000000000000000000000000000000000000000.sszs"}, dirs))
require.Equal(t, true, compareSlices([]string{"0x0de28a18cae63cbc6f0b20dc1afb0b1df38da40824a5f09f92d485ade04de97f.sszs"}, dirs))
})
}

View File

@@ -87,45 +87,47 @@ type serviceFlagOpts struct {
// full PoS node. It handles the lifecycle of the entire system and registers
// services to a service registry.
type BeaconNode struct {
cliCtx *cli.Context
ctx context.Context
cancel context.CancelFunc
services *runtime.ServiceRegistry
lock sync.RWMutex
stop chan struct{} // Channel to wait for termination notifications.
db db.Database
slasherDB db.SlasherDatabase
attestationCache *cache.AttestationCache
attestationPool attestations.Pool
exitPool voluntaryexits.PoolManager
slashingsPool slashings.PoolManager
syncCommitteePool synccommittee.Pool
blsToExecPool blstoexec.PoolManager
depositCache cache.DepositCache
trackedValidatorsCache *cache.TrackedValidatorsCache
payloadIDCache *cache.PayloadIDCache
stateFeed *event.Feed
blockFeed *event.Feed
opFeed *event.Feed
stateGen *stategen.State
collector *bcnodeCollector
slasherBlockHeadersFeed *event.Feed
slasherAttestationsFeed *event.Feed
finalizedStateAtStartUp state.BeaconState
serviceFlagOpts *serviceFlagOpts
GenesisInitializer genesis.Initializer
CheckpointInitializer checkpoint.Initializer
forkChoicer forkchoice.ForkChoicer
clockWaiter startup.ClockWaiter
BackfillOpts []backfill.ServiceOption
initialSyncComplete chan struct{}
BlobStorage *filesystem.BlobStorage
BlobStorageOptions []filesystem.BlobStorageOption
custodyInfo *peerdas.CustodyInfo
verifyInitWaiter *verification.InitializerWaiter
syncChecker *initialsync.SyncChecker
slasherEnabled bool
lcStore *lightclient.Store
cliCtx *cli.Context
ctx context.Context
cancel context.CancelFunc
services *runtime.ServiceRegistry
lock sync.RWMutex
stop chan struct{} // Channel to wait for termination notifications.
db db.Database
slasherDB db.SlasherDatabase
attestationCache *cache.AttestationCache
attestationPool attestations.Pool
exitPool voluntaryexits.PoolManager
slashingsPool slashings.PoolManager
syncCommitteePool synccommittee.Pool
blsToExecPool blstoexec.PoolManager
depositCache cache.DepositCache
trackedValidatorsCache *cache.TrackedValidatorsCache
payloadIDCache *cache.PayloadIDCache
stateFeed *event.Feed
blockFeed *event.Feed
opFeed *event.Feed
stateGen *stategen.State
collector *bcnodeCollector
slasherBlockHeadersFeed *event.Feed
slasherAttestationsFeed *event.Feed
finalizedStateAtStartUp state.BeaconState
serviceFlagOpts *serviceFlagOpts
GenesisInitializer genesis.Initializer
CheckpointInitializer checkpoint.Initializer
forkChoicer forkchoice.ForkChoicer
clockWaiter startup.ClockWaiter
BackfillOpts []backfill.ServiceOption
initialSyncComplete chan struct{}
BlobStorage *filesystem.BlobStorage
BlobStorageOptions []filesystem.BlobStorageOption
DataColumnStorage *filesystem.DataColumnStorage
DataColumnStorageOptions []filesystem.DataColumnStorageOption
custodyInfo *peerdas.CustodyInfo
verifyInitWaiter *verification.InitializerWaiter
syncChecker *initialsync.SyncChecker
slasherEnabled bool
lcStore *lightclient.Store
}
// New creates a new node instance, sets up configuration options, and registers
@@ -193,6 +195,15 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
beacon.BlobStorage = blobs
}
if beacon.DataColumnStorage == nil {
dataColumnStorage, err := filesystem.NewDataColumnStorage(cliCtx.Context, beacon.DataColumnStorageOptions...)
if err != nil {
return nil, errors.Wrap(err, "new data column storage")
}
beacon.DataColumnStorage = dataColumnStorage
}
bfs, err := startBaseServices(cliCtx, beacon, depositAddress)
if err != nil {
return nil, errors.Wrap(err, "could not start modules")
@@ -780,6 +791,7 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
blockchain.WithClockSynchronizer(gs),
blockchain.WithSyncComplete(syncComplete),
blockchain.WithBlobStorage(b.BlobStorage),
blockchain.WithDataColumnStorage(b.DataColumnStorage),
blockchain.WithTrackedValidatorsCache(b.trackedValidatorsCache),
blockchain.WithPayloadIDCache(b.payloadIDCache),
blockchain.WithSyncChecker(b.syncChecker),
@@ -868,6 +880,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithInitialSyncComplete(initialSyncComplete),
regularsync.WithStateNotifier(b),
regularsync.WithBlobStorage(b.BlobStorage),
regularsync.WithDataColumnStorage(b.DataColumnStorage),
regularsync.WithVerifierWaiter(b.verifyInitWaiter),
regularsync.WithAvailableBlocker(bFillStore),
regularsync.WithSlasherEnabled(b.slasherEnabled),

View File

@@ -54,7 +54,12 @@ func TestNodeClose_OK(t *testing.T) {
cmd.ValidatorMonitorIndicesFlag.Value.SetInt(1)
ctx, cancel := newCliContextWithCancel(&app, set)
node, err := New(ctx, cancel, WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)))
options := []Option{
WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)),
WithDataColumnStorage(filesystem.NewEphemeralDataColumnStorage(t)),
}
node, err := New(ctx, cancel, options...)
require.NoError(t, err)
node.Close()
@@ -72,10 +77,16 @@ func TestNodeStart_Ok(t *testing.T) {
require.NoError(t, set.Set("suggested-fee-recipient", "0x6e35733c5af9B61374A128e6F85f553aF09ff89A"))
ctx, cancel := newCliContextWithCancel(&app, set)
node, err := New(ctx, cancel, WithBlockchainFlagOptions([]blockchain.Option{}),
options := []Option{
WithBlockchainFlagOptions([]blockchain.Option{}),
WithBuilderFlagOptions([]builder.Option{}),
WithExecutionChainOptions([]execution.Option{}),
WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)))
WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)),
WithDataColumnStorage(filesystem.NewEphemeralDataColumnStorage(t)),
}
node, err := New(ctx, cancel, options...)
require.NoError(t, err)
node.services = &runtime.ServiceRegistry{}
go func() {
@@ -96,10 +107,16 @@ func TestNodeStart_SyncChecker(t *testing.T) {
require.NoError(t, set.Set("suggested-fee-recipient", "0x6e35733c5af9B61374A128e6F85f553aF09ff89A"))
ctx, cancel := newCliContextWithCancel(&app, set)
node, err := New(ctx, cancel, WithBlockchainFlagOptions([]blockchain.Option{}),
options := []Option{
WithBlockchainFlagOptions([]blockchain.Option{}),
WithBuilderFlagOptions([]builder.Option{}),
WithExecutionChainOptions([]execution.Option{}),
WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)))
WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)),
WithDataColumnStorage(filesystem.NewEphemeralDataColumnStorage(t)),
}
node, err := New(ctx, cancel, options...)
require.NoError(t, err)
go func() {
node.Start()
@@ -128,10 +145,13 @@ func TestClearDB(t *testing.T) {
set.String("suggested-fee-recipient", "0x6e35733c5af9B61374A128e6F85f553aF09ff89A", "fee recipient")
require.NoError(t, set.Set("suggested-fee-recipient", "0x6e35733c5af9B61374A128e6F85f553aF09ff89A"))
context, cancel := newCliContextWithCancel(&app, set)
options := []Option{
WithExecutionChainOptions([]execution.Option{execution.WithHttpEndpoint(endpoint)}),
WithBlobStorage(filesystem.NewEphemeralBlobStorage(t)),
WithDataColumnStorage(filesystem.NewEphemeralDataColumnStorage(t)),
}
_, err = New(context, cancel, options...)
require.NoError(t, err)
require.LogsContain(t, hook, "Removing database")

View File

@@ -50,3 +50,20 @@ func WithBlobStorageOptions(opt ...filesystem.BlobStorageOption) Option {
return nil
}
}
// WithDataColumnStorage sets the DataColumnStorage backend for the BeaconNode
func WithDataColumnStorage(bs *filesystem.DataColumnStorage) Option {
return func(bn *BeaconNode) error {
bn.DataColumnStorage = bs
return nil
}
}
// WithDataColumnStorageOptions appends 1 or more filesystem.DataColumnStorageOption on the beacon node,
// to be used when initializing data column storage.
func WithDataColumnStorageOptions(opt ...filesystem.DataColumnStorageOption) Option {
return func(bn *BeaconNode) error {
bn.DataColumnStorageOptions = append(bn.DataColumnStorageOptions, opt...)
return nil
}
}

View File

@@ -712,7 +712,7 @@ func TestService_BroadcastDataColumn(t *testing.T) {
subnet := peerdas.ComputeSubnetForDataColumnSidecar(columnIndex)
topic := fmt.Sprintf(topicFormat, digest, subnet)
roSidecars, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, util.DataColumnsParamsByRoot{{}: {{ColumnIndex: columnIndex}}})
roSidecars, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: columnIndex}})
sidecar := roSidecars[0].DataColumnSidecar
// Async listen for the pubsub, must be before the broadcast.

View File

@@ -22,44 +22,52 @@ const (
SchemaVersionV3 = "/3"
)
// Specifies the protocol prefix for all our Req/Resp topics.
const protocolPrefix = "/eth2/beacon_chain/req"
const (
// Specifies the protocol prefix for all our Req/Resp topics.
protocolPrefix = "/eth2/beacon_chain/req"
// StatusMessageName specifies the name for the status message topic.
const StatusMessageName = "/status"
// StatusMessageName specifies the name for the status message topic.
StatusMessageName = "/status"
// GoodbyeMessageName specifies the name for the goodbye message topic.
const GoodbyeMessageName = "/goodbye"
// GoodbyeMessageName specifies the name for the goodbye message topic.
GoodbyeMessageName = "/goodbye"
// BeaconBlocksByRangeMessageName specifies the name for the beacon blocks by range message topic.
const BeaconBlocksByRangeMessageName = "/beacon_blocks_by_range"
// BeaconBlocksByRangeMessageName specifies the name for the beacon blocks by range message topic.
BeaconBlocksByRangeMessageName = "/beacon_blocks_by_range"
// BeaconBlocksByRootsMessageName specifies the name for the beacon blocks by root message topic.
const BeaconBlocksByRootsMessageName = "/beacon_blocks_by_root"
// BeaconBlocksByRootsMessageName specifies the name for the beacon blocks by root message topic.
BeaconBlocksByRootsMessageName = "/beacon_blocks_by_root"
// PingMessageName Specifies the name for the ping message topic.
const PingMessageName = "/ping"
// PingMessageName Specifies the name for the ping message topic.
PingMessageName = "/ping"
// MetadataMessageName specifies the name for the metadata message topic.
const MetadataMessageName = "/metadata"
// MetadataMessageName specifies the name for the metadata message topic.
MetadataMessageName = "/metadata"
// BlobSidecarsByRangeName is the name for the BlobSidecarsByRange v1 message topic.
const BlobSidecarsByRangeName = "/blob_sidecars_by_range"
// BlobSidecarsByRangeName is the name for the BlobSidecarsByRange v1 message topic.
BlobSidecarsByRangeName = "/blob_sidecars_by_range"
// BlobSidecarsByRootName is the name for the BlobSidecarsByRoot v1 message topic.
const BlobSidecarsByRootName = "/blob_sidecars_by_root"
// BlobSidecarsByRootName is the name for the BlobSidecarsByRoot v1 message topic.
BlobSidecarsByRootName = "/blob_sidecars_by_root"
// LightClientBootstrapName is the name for the LightClientBootstrap message topic,
const LightClientBootstrapName = "/light_client_bootstrap"
// LightClientBootstrapName is the name for the LightClientBootstrap message topic,
LightClientBootstrapName = "/light_client_bootstrap"
// LightClientUpdatesByRangeName is the name for the LightClientUpdatesByRange topic.
const LightClientUpdatesByRangeName = "/light_client_updates_by_range"
// LightClientUpdatesByRangeName is the name for the LightClientUpdatesByRange topic.
LightClientUpdatesByRangeName = "/light_client_updates_by_range"
// LightClientFinalityUpdateName is the name for the LightClientFinalityUpdate topic.
const LightClientFinalityUpdateName = "/light_client_finality_update"
// LightClientFinalityUpdateName is the name for the LightClientFinalityUpdate topic.
LightClientFinalityUpdateName = "/light_client_finality_update"
// LightClientOptimisticUpdateName is the name for the LightClientOptimisticUpdate topic.
const LightClientOptimisticUpdateName = "/light_client_optimistic_update"
// LightClientOptimisticUpdateName is the name for the LightClientOptimisticUpdate topic.
LightClientOptimisticUpdateName = "/light_client_optimistic_update"
// DataColumnSidecarsByRootName is the name for the DataColumnSidecarsByRoot v1 message topic.
DataColumnSidecarsByRootName = "/data_column_sidecars_by_root"
// DataColumnSidecarsByRangeName is the name for the DataColumnSidecarsByRange v1 message topic.
DataColumnSidecarsByRangeName = "/data_column_sidecars_by_range"
)
const (
// V1 RPC Topics
@@ -92,6 +100,12 @@ const (
RPCLightClientFinalityUpdateTopicV1 = protocolPrefix + LightClientFinalityUpdateName + SchemaVersionV1
// RPCLightClientOptimisticUpdateTopicV1 is a topic for requesting a light client Optimistic update.
RPCLightClientOptimisticUpdateTopicV1 = protocolPrefix + LightClientOptimisticUpdateName + SchemaVersionV1
// RPCDataColumnSidecarsByRootTopicV1 is a topic for requesting data column sidecars by their block root.
// /eth2/beacon_chain/req/data_column_sidecars_by_root/1 - New in Fulu.
RPCDataColumnSidecarsByRootTopicV1 = protocolPrefix + DataColumnSidecarsByRootName + SchemaVersionV1
// RPCDataColumnSidecarsByRangeTopicV1 is a topic for requesting data column sidecars by their slot.
// /eth2/beacon_chain/req/data_column_sidecars_by_range/1 - New in Fulu.
RPCDataColumnSidecarsByRangeTopicV1 = protocolPrefix + DataColumnSidecarsByRangeName + SchemaVersionV1
// V2 RPC Topics
// RPCBlocksByRangeTopicV2 defines v2 the topic for the blocks by range rpc method.
@@ -112,87 +126,103 @@ const (
)
// RPCTopicMappings map the base message type to the rpc request.
var RPCTopicMappings = map[string]interface{}{
// RPC Status Message
RPCStatusTopicV1: new(pb.Status),
// RPC Goodbye Message
RPCGoodByeTopicV1: new(primitives.SSZUint64),
// RPC Block By Range Message
RPCBlocksByRangeTopicV1: new(pb.BeaconBlocksByRangeRequest),
RPCBlocksByRangeTopicV2: new(pb.BeaconBlocksByRangeRequest),
// RPC Block By Root Message
RPCBlocksByRootTopicV1: new(p2ptypes.BeaconBlockByRootsReq),
RPCBlocksByRootTopicV2: new(p2ptypes.BeaconBlockByRootsReq),
// RPC Ping Message
RPCPingTopicV1: new(primitives.SSZUint64),
// RPC Metadata Message
RPCMetaDataTopicV1: new(interface{}),
RPCMetaDataTopicV2: new(interface{}),
RPCMetaDataTopicV3: new(interface{}),
// BlobSidecarsByRange v1 Message
RPCBlobSidecarsByRangeTopicV1: new(pb.BlobSidecarsByRangeRequest),
// BlobSidecarsByRoot v1 Message
RPCBlobSidecarsByRootTopicV1: new(p2ptypes.BlobSidecarsByRootReq),
var (
RPCTopicMappings = map[string]interface{}{
// RPC Status Message
RPCStatusTopicV1: new(pb.Status),
// Light client
RPCLightClientBootstrapTopicV1: new([fieldparams.RootLength]byte),
RPCLightClientUpdatesByRangeTopicV1: new(pb.LightClientUpdatesByRangeRequest),
RPCLightClientFinalityUpdateTopicV1: new(interface{}),
RPCLightClientOptimisticUpdateTopicV1: new(interface{}),
}
// RPC Goodbye Message
RPCGoodByeTopicV1: new(primitives.SSZUint64),
// Maps all registered protocol prefixes.
var protocolMapping = map[string]bool{
protocolPrefix: true,
}
// RPC Block By Range Message
RPCBlocksByRangeTopicV1: new(pb.BeaconBlocksByRangeRequest),
RPCBlocksByRangeTopicV2: new(pb.BeaconBlocksByRangeRequest),
// Maps all the protocol message names for the different rpc
// topics.
var messageMapping = map[string]bool{
StatusMessageName: true,
GoodbyeMessageName: true,
BeaconBlocksByRangeMessageName: true,
BeaconBlocksByRootsMessageName: true,
PingMessageName: true,
MetadataMessageName: true,
BlobSidecarsByRangeName: true,
BlobSidecarsByRootName: true,
LightClientBootstrapName: true,
LightClientUpdatesByRangeName: true,
LightClientFinalityUpdateName: true,
LightClientOptimisticUpdateName: true,
}
// RPC Block By Root Message
RPCBlocksByRootTopicV1: new(p2ptypes.BeaconBlockByRootsReq),
RPCBlocksByRootTopicV2: new(p2ptypes.BeaconBlockByRootsReq),
// Maps all the RPC messages which are to updated in altair.
var altairMapping = map[string]bool{
BeaconBlocksByRangeMessageName: true,
BeaconBlocksByRootsMessageName: true,
MetadataMessageName: true,
}
// RPC Ping Message
RPCPingTopicV1: new(primitives.SSZUint64),
// Maps all the RPC messages which are to updated in fulu.
var fuluMapping = map[string]bool{
MetadataMessageName: true,
}
// RPC Metadata Message
RPCMetaDataTopicV1: new(interface{}),
RPCMetaDataTopicV2: new(interface{}),
RPCMetaDataTopicV3: new(interface{}),
var versionMapping = map[string]bool{
SchemaVersionV1: true,
SchemaVersionV2: true,
SchemaVersionV3: true,
}
// BlobSidecarsByRange v1 Message
RPCBlobSidecarsByRangeTopicV1: new(pb.BlobSidecarsByRangeRequest),
// OmitContextBytesV1 keeps track of which RPC methods do not write context bytes in their v1 incarnations.
// Phase0 did not have the notion of context bytes, which prefix wire-encoded values with a [4]byte identifier
// to convey the schema for the receiver to use. These RPCs had a version bump to V2 when the context byte encoding
// was introduced. For other RPC methods, context bytes are always required.
var OmitContextBytesV1 = map[string]bool{
StatusMessageName: true,
GoodbyeMessageName: true,
BeaconBlocksByRangeMessageName: true,
BeaconBlocksByRootsMessageName: true,
PingMessageName: true,
MetadataMessageName: true,
}
// BlobSidecarsByRoot v1 Message
RPCBlobSidecarsByRootTopicV1: new(p2ptypes.BlobSidecarsByRootReq),
// Light client
RPCLightClientBootstrapTopicV1: new([fieldparams.RootLength]byte),
RPCLightClientUpdatesByRangeTopicV1: new(pb.LightClientUpdatesByRangeRequest),
RPCLightClientFinalityUpdateTopicV1: new(interface{}),
RPCLightClientOptimisticUpdateTopicV1: new(interface{}),
// DataColumnSidecarsByRange v1 Message
RPCDataColumnSidecarsByRangeTopicV1: new(pb.DataColumnSidecarsByRangeRequest),
// DataColumnSidecarsByRoot v1 Message
RPCDataColumnSidecarsByRootTopicV1: new(p2ptypes.DataColumnsByRootIdentifiers),
}
// Maps all registered protocol prefixes.
protocolMapping = map[string]bool{
protocolPrefix: true,
}
// Maps all the protocol message names for the different rpc topics.
messageMapping = map[string]bool{
StatusMessageName: true,
GoodbyeMessageName: true,
BeaconBlocksByRangeMessageName: true,
BeaconBlocksByRootsMessageName: true,
PingMessageName: true,
MetadataMessageName: true,
BlobSidecarsByRangeName: true,
BlobSidecarsByRootName: true,
LightClientBootstrapName: true,
LightClientUpdatesByRangeName: true,
LightClientFinalityUpdateName: true,
LightClientOptimisticUpdateName: true,
DataColumnSidecarsByRootName: true,
DataColumnSidecarsByRangeName: true,
}
// Maps all the RPC messages which are to updated in altair.
altairMapping = map[string]string{
BeaconBlocksByRangeMessageName: SchemaVersionV2,
BeaconBlocksByRootsMessageName: SchemaVersionV2,
MetadataMessageName: SchemaVersionV2,
}
// Maps all the RPC messages which are to updated in fulu.
fuluMapping = map[string]string{
MetadataMessageName: SchemaVersionV3,
}
versionMapping = map[string]bool{
SchemaVersionV1: true,
SchemaVersionV2: true,
SchemaVersionV3: true,
}
// OmitContextBytesV1 keeps track of which RPC methods do not write context bytes in their v1 incarnations.
// Phase0 did not have the notion of context bytes, which prefix wire-encoded values with a [4]byte identifier
// to convey the schema for the receiver to use. These RPCs had a version bump to V2 when the context byte encoding
// was introduced. For other RPC methods, context bytes are always required.
OmitContextBytesV1 = map[string]bool{
StatusMessageName: true,
GoodbyeMessageName: true,
BeaconBlocksByRangeMessageName: true,
BeaconBlocksByRootsMessageName: true,
PingMessageName: true,
MetadataMessageName: true,
}
)
// VerifyTopicMapping verifies that the topic and its accompanying
// message type is correct.
@@ -314,13 +344,17 @@ func TopicFromMessage(msg string, epoch primitives.Epoch) (string, error) {
beaconConfig := params.BeaconConfig()
// Check if the message is to be updated in fulu.
if epoch >= beaconConfig.FuluForkEpoch && fuluMapping[msg] {
return protocolPrefix + msg + SchemaVersionV3, nil
if epoch >= beaconConfig.FuluForkEpoch {
if version, ok := fuluMapping[msg]; ok {
return protocolPrefix + msg + version, nil
}
}
// Check if the message is to be updated in altair.
if epoch >= beaconConfig.AltairForkEpoch && altairMapping[msg] {
return protocolPrefix + msg + SchemaVersionV2, nil
if epoch >= beaconConfig.AltairForkEpoch {
if version, ok := altairMapping[msg]; ok {
return protocolPrefix + msg + version, nil
}
}
return protocolPrefix + msg + SchemaVersionV1, nil

View File

@@ -119,50 +119,31 @@ func TestTopicFromMessage_CorrectType(t *testing.T) {
})
t.Run("after altair fork but before fulu fork", func(t *testing.T) {
for m := range messageMapping {
topic, err := TopicFromMessage(m, altairForkEpoch)
require.NoError(t, err)
// Not modified in altair fork.
topic, err := TopicFromMessage(GoodbyeMessageName, altairForkEpoch)
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/goodbye/1", topic)
if altairMapping[m] {
require.Equal(t, true, strings.Contains(topic, SchemaVersionV2))
_, _, version, err := TopicDeconstructor(topic)
require.NoError(t, err)
require.Equal(t, SchemaVersionV2, version)
continue
}
require.Equal(t, true, strings.Contains(topic, SchemaVersionV1))
_, _, version, err := TopicDeconstructor(topic)
require.NoError(t, err)
require.Equal(t, SchemaVersionV1, version)
}
// Modified in altair fork.
topic, err = TopicFromMessage(MetadataMessageName, altairForkEpoch)
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/metadata/2", topic)
})
t.Run("after fulu fork", func(t *testing.T) {
for m := range messageMapping {
topic, err := TopicFromMessage(m, fuluForkEpoch)
require.NoError(t, err)
// Not modified in any fork.
topic, err := TopicFromMessage(GoodbyeMessageName, fuluForkEpoch)
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/goodbye/1", topic)
if fuluMapping[m] {
require.Equal(t, true, strings.Contains(topic, SchemaVersionV3))
_, _, version, err := TopicDeconstructor(topic)
require.NoError(t, err)
require.Equal(t, SchemaVersionV3, version)
continue
}
// Modified in altair fork.
topic, err = TopicFromMessage(BeaconBlocksByRangeMessageName, fuluForkEpoch)
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/beacon_blocks_by_range/2", topic)
if altairMapping[m] {
require.Equal(t, true, strings.Contains(topic, SchemaVersionV2))
_, _, version, err := TopicDeconstructor(topic)
require.NoError(t, err)
require.Equal(t, SchemaVersionV2, version)
continue
}
require.Equal(t, true, strings.Contains(topic, SchemaVersionV1))
_, _, version, err := TopicDeconstructor(topic)
require.NoError(t, err)
require.Equal(t, SchemaVersionV1, version)
}
// Modified both in altair and fulu fork.
topic, err = TopicFromMessage(MetadataMessageName, fuluForkEpoch)
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/metadata/3", topic)
})
}

View File

@@ -25,6 +25,7 @@ go_library(
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",
"//encoding/bytesutil:go_default_library",
"//encoding/ssz:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/metadata:go_default_library",
@@ -45,10 +46,10 @@ go_test(
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//encoding/bytesutil:go_default_library",
"//encoding/ssz:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"//testing/assert:go_default_library",
"//testing/require:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
],
)

View File

@@ -9,10 +9,13 @@ 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 epoch < 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")
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")
)

View File

@@ -5,19 +5,18 @@ package types
import (
"bytes"
"encoding/binary"
"sort"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/encoding/ssz"
eth "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
fastssz "github.com/prysmaticlabs/fastssz"
)
const (
maxErrorLength = 256
bytesPerLengthOffset = 4
maxErrorLength = 256
)
// SSZBytes is a bytes slice that satisfies the fast-ssz interface.
@@ -25,11 +24,11 @@ type SSZBytes []byte
// HashTreeRoot hashes the uint64 object following the SSZ standard.
func (b *SSZBytes) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(b)
return fastssz.HashWithDefaultHasher(b)
}
// HashTreeRootWith hashes the uint64 object with the given hasher.
func (b *SSZBytes) HashTreeRootWith(hh *ssz.Hasher) error {
func (b *SSZBytes) HashTreeRootWith(hh *fastssz.Hasher) error {
indx := hh.Index()
hh.PutBytes(*b)
hh.Merkleize(indx)
@@ -74,7 +73,7 @@ func (r *BeaconBlockByRootsReq) UnmarshalSSZ(buf []byte) error {
return errors.Errorf("expected buffer with length of up to %d but received length %d", maxLength, bufLen)
}
if bufLen%fieldparams.RootLength != 0 {
return ssz.ErrIncorrectByteSize
return fastssz.ErrIncorrectByteSize
}
numOfRoots := bufLen / fieldparams.RootLength
roots := make([][fieldparams.RootLength]byte, 0, numOfRoots)
@@ -131,14 +130,6 @@ func (m *ErrorMessage) UnmarshalSSZ(buf []byte) error {
// BlobSidecarsByRootReq is used to specify a list of blob targets (root+index) in a BlobSidecarsByRoot RPC request.
type BlobSidecarsByRootReq []*eth.BlobIdentifier
// BlobIdentifier is a fixed size value, so we can compute its fixed size at start time (see init below)
var blobIdSize int
// SizeSSZ returns the size of the serialized representation.
func (b *BlobSidecarsByRootReq) SizeSSZ() int {
return len(*b) * blobIdSize
}
// MarshalSSZTo appends the serialized BlobSidecarsByRootReq value to the provided byte slice.
func (b *BlobSidecarsByRootReq) MarshalSSZTo(dst []byte) ([]byte, error) {
// A List without an enclosing container is marshaled exactly like a vector, no length offset required.
@@ -151,38 +142,22 @@ func (b *BlobSidecarsByRootReq) MarshalSSZTo(dst []byte) ([]byte, error) {
// MarshalSSZ serializes the BlobSidecarsByRootReq value to a byte slice.
func (b *BlobSidecarsByRootReq) MarshalSSZ() ([]byte, error) {
buf := make([]byte, len(*b)*blobIdSize)
for i, id := range *b {
by, err := id.MarshalSSZ()
if err != nil {
return nil, err
}
copy(buf[i*blobIdSize:(i+1)*blobIdSize], by)
}
return buf, nil
return ssz.MarshalListFixedElement[*eth.BlobIdentifier](*b)
}
func newBSBR() *eth.BlobIdentifier { return &eth.BlobIdentifier{} }
// UnmarshalSSZ unmarshals the provided bytes buffer into the
// BlobSidecarsByRootReq value.
func (b *BlobSidecarsByRootReq) UnmarshalSSZ(buf []byte) error {
bufLen := len(buf)
maxLength := int(params.BeaconConfig().MaxRequestBlobSidecarsElectra) * blobIdSize
if bufLen > maxLength {
return errors.Wrapf(ssz.ErrIncorrectListSize, "expected buffer with length of up to %d but received length %d", maxLength, bufLen)
v, err := ssz.UnmarshalListFixedElement[*eth.BlobIdentifier](buf, newBSBR)
if err != nil {
return errors.Wrapf(err, "failed to unmarshal BlobSidecarsByRootReq")
}
if bufLen%blobIdSize != 0 {
return errors.Wrapf(ssz.ErrIncorrectByteSize, "size=%d", bufLen)
}
count := bufLen / blobIdSize
*b = make([]*eth.BlobIdentifier, count)
for i := 0; i < count; i++ {
id := &eth.BlobIdentifier{}
err := id.UnmarshalSSZ(buf[i*blobIdSize : (i+1)*blobIdSize])
if err != nil {
return err
}
(*b)[i] = id
if len(v) > int(params.BeaconConfig().MaxRequestBlobSidecarsElectra) {
return ErrMaxBlobReqExceeded
}
*b = v
return nil
}
@@ -213,106 +188,29 @@ func (s BlobSidecarsByRootReq) Len() int {
// ====================================
// DataColumnsByRootIdentifiers section
// ====================================
var _ ssz.Marshaler = (*DataColumnsByRootIdentifiers)(nil)
var _ ssz.Unmarshaler = (*DataColumnsByRootIdentifiers)(nil)
var _ fastssz.Marshaler = &DataColumnsByRootIdentifiers{}
var _ fastssz.Unmarshaler = &DataColumnsByRootIdentifiers{}
// DataColumnsByRootIdentifiers is used to specify a list of data column targets (root+index) in a DataColumnSidecarsByRoot RPC request.
type DataColumnsByRootIdentifiers []*eth.DataColumnsByRootIdentifier
// DataColumnIdentifier is a fixed size value, so we can compute its fixed size at start time (see init below)
var dataColumnIdSize int
func newDCRI() *eth.DataColumnsByRootIdentifier { return &eth.DataColumnsByRootIdentifier{} }
// UnmarshalSSZ implements ssz.Unmarshaler. It unmarshals the provided bytes buffer into the DataColumnSidecarsByRootReq value.
func (d *DataColumnsByRootIdentifiers) UnmarshalSSZ(buf []byte) error {
// Exit early if the buffer is too small.
if len(buf) < bytesPerLengthOffset {
return nil
v, err := ssz.UnmarshalListVariableElement[*eth.DataColumnsByRootIdentifier](buf, newDCRI)
if err != nil {
return errors.Wrapf(err, "failed to unmarshal DataColumnsByRootIdentifiers")
}
// Get the size of the offsets.
offsetEnd := binary.LittleEndian.Uint32(buf[:bytesPerLengthOffset])
if offsetEnd%bytesPerLengthOffset != 0 {
return errors.Errorf("expected offsets size to be a multiple of %d but got %d", bytesPerLengthOffset, offsetEnd)
}
count := offsetEnd / bytesPerLengthOffset
if count < 1 {
return nil
}
maxSize := params.BeaconConfig().MaxRequestBlocksDeneb
if uint64(count) > maxSize {
return errors.Errorf("data column identifiers list exceeds max size: %d > %d", count, maxSize)
}
if offsetEnd > uint32(len(buf)) {
return errors.Errorf("offsets value %d larger than buffer %d", offsetEnd, len(buf))
}
valueStart := offsetEnd
// Decode the identifers.
*d = make([]*eth.DataColumnsByRootIdentifier, count)
var start uint32
end := uint32(len(buf))
for i := count; i > 0; i-- {
offsetEnd -= bytesPerLengthOffset
start = binary.LittleEndian.Uint32(buf[offsetEnd : offsetEnd+bytesPerLengthOffset])
if start > end {
return errors.Errorf("expected offset[%d] %d to be less than %d", i-1, start, end)
}
if start < valueStart {
return errors.Errorf("offset[%d] %d indexes before value section %d", i-1, start, valueStart)
}
// Decode the identifier.
ident := &eth.DataColumnsByRootIdentifier{}
if err := ident.UnmarshalSSZ(buf[start:end]); err != nil {
return err
}
(*d)[i-1] = ident
end = start
}
*d = v
return nil
}
func (d *DataColumnsByRootIdentifiers) MarshalSSZ() ([]byte, error) {
var err error
count := len(*d)
maxSize := params.BeaconConfig().MaxRequestBlocksDeneb
if uint64(count) > maxSize {
return nil, errors.Errorf("data column identifiers list exceeds max size: %d > %d", count, maxSize)
}
if len(*d) == 0 {
return []byte{}, nil
}
sizes := make([]uint32, count)
valTotal := uint32(0)
for i, elem := range *d {
if elem == nil {
return nil, errors.New("nil item in DataColumnsByRootIdentifiers list")
}
sizes[i] = uint32(elem.SizeSSZ())
valTotal += sizes[i]
}
offSize := uint32(4 * len(*d))
out := make([]byte, offSize, offSize+valTotal)
for i := range sizes {
binary.LittleEndian.PutUint32(out[i*4:i*4+4], offSize)
offSize += sizes[i]
}
for _, elem := range *d {
out, err = elem.MarshalSSZTo(out)
if err != nil {
return nil, err
}
}
return out, nil
return ssz.MarshalListVariableElement[*eth.DataColumnsByRootIdentifier](*d)
}
// MarshalSSZTo implements ssz.Marshaler. It appends the serialized DataColumnSidecarsByRootReq value to the provided byte slice.
func (d *DataColumnsByRootIdentifiers) MarshalSSZTo(dst []byte) ([]byte, error) {
func (d DataColumnsByRootIdentifiers) MarshalSSZTo(dst []byte) ([]byte, error) {
obj, err := d.MarshalSSZ()
if err != nil {
return nil, err
@@ -321,19 +219,11 @@ func (d *DataColumnsByRootIdentifiers) MarshalSSZTo(dst []byte) ([]byte, error)
}
// SizeSSZ implements ssz.Marshaler. It returns the size of the serialized representation.
func (d *DataColumnsByRootIdentifiers) SizeSSZ() int {
func (d DataColumnsByRootIdentifiers) SizeSSZ() int {
size := 0
for i := 0; i < len(*d); i++ {
for i := 0; i < len(d); i++ {
size += 4
size += (*d)[i].SizeSSZ()
size += (d)[i].SizeSSZ()
}
return size
}
func init() {
blobSizer := &eth.BlobIdentifier{}
blobIdSize = blobSizer.SizeSSZ()
dataColumnSizer := &eth.DataColumnSidecarsByRangeRequest{}
dataColumnIdSize = dataColumnSizer.SizeSSZ()
}

View File

@@ -8,10 +8,10 @@ import (
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/encoding/ssz"
eth "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/testing/assert"
"github.com/OffchainLabs/prysm/v6/testing/require"
ssz "github.com/prysmaticlabs/fastssz"
)
func generateBlobIdentifiers(n int) []*eth.BlobIdentifier {
@@ -51,7 +51,7 @@ func TestBlobSidecarsByRootReq_MarshalSSZ(t *testing.T) {
{
name: "beyond max list",
ids: generateBlobIdentifiers(int(params.BeaconConfig().MaxRequestBlobSidecarsElectra) + 1),
unmarshalErr: ssz.ErrIncorrectListSize,
unmarshalErr: ErrMaxBlobReqExceeded,
},
{
name: "wonky unmarshal size",
@@ -60,7 +60,7 @@ func TestBlobSidecarsByRootReq_MarshalSSZ(t *testing.T) {
in = append(in, byte(0))
return in
},
unmarshalErr: ssz.ErrIncorrectByteSize,
unmarshalErr: ssz.ErrInvalidFixedEncodingLen,
},
}
@@ -305,7 +305,8 @@ func TestDataColumnSidecarsByRootReq_MarshalUnmarshal(t *testing.T) {
name: "size too big",
ids: generateDataColumnIdentifiers(1),
unmarshalMod: func(in []byte) []byte {
maxLen := params.BeaconConfig().MaxRequestDataColumnSidecars * uint64(dataColumnIdSize)
sizer := &eth.DataColumnSidecarsByRangeRequest{}
maxLen := params.BeaconConfig().MaxRequestDataColumnSidecars * uint64(sizer.SizeSSZ())
add := make([]byte, maxLen)
in = append(in, add...)
return in

View File

@@ -10,11 +10,13 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/rpc/core:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
@@ -36,7 +38,9 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/blockchain/kzg:go_default_library",
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/rpc/core:go_default_library",
@@ -45,6 +49,7 @@ go_test(
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/state/stategen/mock:go_default_library",
"//beacon-chain/verification:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/primitives:go_default_library",

View File

@@ -3,12 +3,15 @@ package lookup
import (
"context"
"fmt"
"math"
"strconv"
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/rpc/core"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
@@ -49,6 +52,7 @@ type BeaconDbBlocker struct {
ChainInfoFetcher blockchain.ChainInfoFetcher
GenesisTimeFetcher blockchain.TimeFetcher
BlobStorage *filesystem.BlobStorage
DataColumnStorage *filesystem.DataColumnStorage
}
// Block returns the beacon block for a given identifier. The identifier can be one of:
@@ -212,64 +216,190 @@ func (p *BeaconDbBlocker) Blobs(ctx context.Context, id string, indices []int) (
root := bytesutil.ToBytes32(rootSlice)
b, err := p.BeaconDB.Block(ctx, root)
roSignedBlock, err := p.BeaconDB.Block(ctx, root)
if err != nil {
return nil, &core.RpcError{Err: errors.Wrapf(err, "failed to retrieve block %#x from db", rootSlice), Reason: core.Internal}
}
if b == nil {
if roSignedBlock == nil {
return nil, &core.RpcError{Err: fmt.Errorf("block %#x not found in db", rootSlice), Reason: core.NotFound}
}
// if block is not in the retention window, return 200 w/ empty list
if !p.BlobStorage.WithinRetentionPeriod(slots.ToEpoch(b.Block().Slot()), slots.ToEpoch(p.GenesisTimeFetcher.CurrentSlot())) {
// If block is not in the retention window, return 200 w/ empty list
if !p.BlobStorage.WithinRetentionPeriod(slots.ToEpoch(roSignedBlock.Block().Slot()), slots.ToEpoch(p.GenesisTimeFetcher.CurrentSlot())) {
return make([]*blocks.VerifiedROBlob, 0), nil
}
commitments, err := b.Block().Body().BlobKzgCommitments()
roBlock := roSignedBlock.Block()
commitments, err := roBlock.Body().BlobKzgCommitments()
if err != nil {
return nil, &core.RpcError{Err: errors.Wrapf(err, "failed to retrieve kzg commitments from block %#x", rootSlice), Reason: core.Internal}
}
// if there are no commitments return 200 w/ empty list
// If there are no commitments return 200 w/ empty list
if len(commitments) == 0 {
return make([]*blocks.VerifiedROBlob, 0), nil
}
sum := p.BlobStorage.Summary(root)
// Compute the first Fulu slot.
fuluForkEpoch := params.BeaconConfig().FuluForkEpoch
fuluForkSlot := primitives.Slot(math.MaxUint64)
if fuluForkEpoch != primitives.Epoch(math.MaxUint64) {
fuluForkSlot, err = slots.EpochStart(fuluForkEpoch)
if err != nil {
return nil, &core.RpcError{Err: errors.Wrap(err, "could not calculate peerDAS start slot"), Reason: core.Internal}
}
}
if len(indices) == 0 {
for i := range commitments {
if sum.HasIndex(uint64(i)) {
indices = append(indices, i)
if roBlock.Slot() >= fuluForkSlot {
roBlock, err := blocks.NewROBlockWithRoot(roSignedBlock, root)
if err != nil {
return nil, &core.RpcError{Err: errors.Wrapf(err, "failed to create roBlock with root %#x", root), Reason: core.Internal}
}
return p.blobsFromStoredDataColumns(roBlock, indices)
}
return p.blobsFromStoredBlobs(commitments, root, indices)
}
// blobsFromStoredBlobs retrieves blob sidercars corresponding to `indices` and `root` from the store.
// This function expects blob sidecars to be stored (aka. no data column sidecars).
func (p *BeaconDbBlocker) blobsFromStoredBlobs(commitments [][]byte, root [fieldparams.RootLength]byte, indices []int) ([]*blocks.VerifiedROBlob, *core.RpcError) {
summary := p.BlobStorage.Summary(root)
maxBlobCount := summary.MaxBlobsForEpoch()
for _, index := range indices {
if uint64(index) >= maxBlobCount {
return nil, &core.RpcError{
Err: fmt.Errorf("requested index %d is bigger than the maximum possible blob count %d", index, maxBlobCount),
Reason: core.BadRequest,
}
}
} else {
for _, ix := range indices {
if uint64(ix) >= sum.MaxBlobsForEpoch() {
return nil, &core.RpcError{
Err: fmt.Errorf("requested index %d is bigger than the maximum possible blob count %d", ix, sum.MaxBlobsForEpoch()),
Reason: core.BadRequest,
}
}
if !sum.HasIndex(uint64(ix)) {
return nil, &core.RpcError{
Err: fmt.Errorf("requested index %d not found", ix),
Reason: core.NotFound,
}
if !summary.HasIndex(uint64(index)) {
return nil, &core.RpcError{
Err: fmt.Errorf("requested index %d not found", index),
Reason: core.NotFound,
}
}
}
blobs := make([]*blocks.VerifiedROBlob, len(indices))
for i, index := range indices {
vblob, err := p.BlobStorage.Get(root, uint64(index))
// If no indices are provided, use all indices that are available in the summary.
if len(indices) == 0 {
for index := range commitments {
if summary.HasIndex(uint64(index)) {
indices = append(indices, index)
}
}
}
// Retrieve blob sidecars from the store.
blobs := make([]*blocks.VerifiedROBlob, 0, len(indices))
for _, index := range indices {
blobSidecar, err := p.BlobStorage.Get(root, uint64(index))
if err != nil {
return nil, &core.RpcError{
Err: fmt.Errorf("could not retrieve blob for block root %#x at index %d", rootSlice, index),
Err: fmt.Errorf("could not retrieve blob for block root %#x at index %d", root, index),
Reason: core.Internal,
}
}
blobs[i] = &vblob
blobs = append(blobs, &blobSidecar)
}
return blobs, nil
}
// blobsFromStoredDataColumns retrieves data column sidecars from the store,
// reconstructs the whole matrix if needed, converts the matrix to blobs,
// and then returns converted blobs corresponding to `indices` and `root`.
// This function expects data column sidecars to be stored (aka. no blob sidecars).
// If not enough data column sidecars are available to convert blobs from them
// (either directly or after reconstruction), an error is returned.
func (p *BeaconDbBlocker) blobsFromStoredDataColumns(block blocks.ROBlock, indices []int) ([]*blocks.VerifiedROBlob, *core.RpcError) {
root := block.Root()
// Use all indices if none are provided.
if len(indices) == 0 {
commitments, err := block.Block().Body().BlobKzgCommitments()
if err != nil {
return nil, &core.RpcError{
Err: errors.Wrap(err, "could not retrieve blob commitments"),
Reason: core.Internal,
}
}
for index := range commitments {
indices = append(indices, index)
}
}
// Count how many columns we have in the store.
summary := p.DataColumnStorage.Summary(root)
stored := summary.Stored()
count := uint64(len(stored))
if count < peerdas.MinimumColumnsCountToReconstruct() {
// There is no way to reconstruct the data columns.
return nil, &core.RpcError{
Err: errors.Errorf("the node does not custody enough data columns to reconstruct blobs - please start the beacon node with the `--%s` flag to ensure this call to succeed, or retry later if it is already the case", flags.SubscribeAllDataSubnets.Name),
Reason: core.NotFound,
}
}
// Retrieve from the database needed data columns.
verifiedRoDataColumnSidecars, err := p.neededDataColumnSidecars(root, stored)
if err != nil {
return nil, &core.RpcError{
Err: errors.Wrap(err, "needed data column sidecars"),
Reason: core.Internal,
}
}
// Reconstruct blob sidecars from data column sidecars.
verifiedRoBlobSidecars, err := peerdas.ReconstructBlobs(block, verifiedRoDataColumnSidecars, indices)
if err != nil {
return nil, &core.RpcError{
Err: errors.Wrap(err, "blobs from data columns"),
Reason: core.Internal,
}
}
return verifiedRoBlobSidecars, nil
}
// neededDataColumnSidecars retrieves all data column sidecars corresponding to (non extended) blobs if available,
// else retrieves all data column sidecars from the store.
func (p *BeaconDbBlocker) neededDataColumnSidecars(root [fieldparams.RootLength]byte, stored map[uint64]bool) ([]blocks.VerifiedRODataColumn, error) {
// Check if we have all the non-extended data columns.
cellsPerBlob := fieldparams.CellsPerBlob
blobIndices := make([]uint64, 0, cellsPerBlob)
hasAllBlobColumns := true
for i := range uint64(cellsPerBlob) {
if !stored[i] {
hasAllBlobColumns = false
break
}
blobIndices = append(blobIndices, i)
}
if hasAllBlobColumns {
// Retrieve only the non-extended data columns.
verifiedRoSidecars, err := p.DataColumnStorage.Get(root, blobIndices)
if err != nil {
return nil, errors.Wrap(err, "data columns storage get")
}
return verifiedRoSidecars, nil
}
// Retrieve all the data columns.
verifiedRoSidecars, err := p.DataColumnStorage.Get(root, nil)
if err != nil {
return nil, errors.Wrap(err, "data columns storage get")
}
return verifiedRoSidecars, nil
}

View File

@@ -8,12 +8,15 @@ import (
"testing"
"time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/kzg"
mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
testDB "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/rpc/core"
"github.com/OffchainLabs/prysm/v6/beacon-chain/rpc/testutil"
"github.com/OffchainLabs/prysm/v6/beacon-chain/verification"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
@@ -158,172 +161,335 @@ func TestGetBlock(t *testing.T) {
}
func TestGetBlob(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.DenebForkEpoch = 1
params.OverrideBeaconConfig(cfg)
const (
slot = 123
blobCount = 4
denebForEpoch = 1
fuluForkEpoch = 2
)
setupDeneb := func(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.DenebForkEpoch = denebForEpoch
params.OverrideBeaconConfig(cfg)
}
setupFulu := func(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.DenebForkEpoch = denebForEpoch
cfg.FuluForkEpoch = fuluForkEpoch
params.OverrideBeaconConfig(cfg)
}
ctx := t.Context()
db := testDB.SetupDB(t)
denebBlock, blobs := util.GenerateTestDenebBlockWithSidecar(t, [32]byte{}, 123, 4)
require.NoError(t, db.SaveBlock(t.Context(), denebBlock))
_, bs := filesystem.NewEphemeralBlobStorageAndFs(t)
testSidecars := verification.FakeVerifySliceForTest(t, blobs)
for i := range testSidecars {
require.NoError(t, bs.Save(testSidecars[i]))
// Start the trusted setup.
err := kzg.Start()
require.NoError(t, err)
// Create and save Deneb block and blob sidecars.
_, blobStorage := filesystem.NewEphemeralBlobStorageAndFs(t)
denebBlock, storedBlobSidecars := util.GenerateTestDenebBlockWithSidecar(t, [fieldparams.RootLength]byte{}, slot, blobCount)
denebBlockRoot := denebBlock.Root()
verifiedStoredSidecars := verification.FakeVerifySliceForTest(t, storedBlobSidecars)
for i := range verifiedStoredSidecars {
err := blobStorage.Save(verifiedStoredSidecars[i])
require.NoError(t, err)
}
blockRoot := blobs[0].BlockRoot()
err = db.SaveBlock(t.Context(), denebBlock)
require.NoError(t, err)
// Create Electra block and blob sidecars. (Electra block = Fulu block),
// save the block, convert blob sidecars to data column sidecars and save the block.
fuluForkSlot := fuluForkEpoch * params.BeaconConfig().SlotsPerEpoch
fuluBlock, fuluBlobSidecars := util.GenerateTestElectraBlockWithSidecar(t, [fieldparams.RootLength]byte{}, fuluForkSlot, blobCount)
fuluBlockRoot := fuluBlock.Root()
cellsAndProofsList := make([]kzg.CellsAndProofs, 0, len(fuluBlobSidecars))
for _, blob := range fuluBlobSidecars {
var kzgBlob kzg.Blob
copy(kzgBlob[:], blob.Blob)
cellsAndProogs, err := kzg.ComputeCellsAndKZGProofs(&kzgBlob)
require.NoError(t, err)
cellsAndProofsList = append(cellsAndProofsList, cellsAndProogs)
}
dataColumnSidecarPb, err := peerdas.DataColumnSidecars(fuluBlock, cellsAndProofsList)
require.NoError(t, err)
verifiedRoDataColumnSidecars := make([]blocks.VerifiedRODataColumn, 0, len(dataColumnSidecarPb))
for _, sidecarPb := range dataColumnSidecarPb {
roDataColumn, err := blocks.NewRODataColumnWithRoot(sidecarPb, fuluBlockRoot)
require.NoError(t, err)
verifiedRoDataColumn := blocks.NewVerifiedRODataColumn(roDataColumn)
verifiedRoDataColumnSidecars = append(verifiedRoDataColumnSidecars, verifiedRoDataColumn)
}
err = db.SaveBlock(t.Context(), fuluBlock)
require.NoError(t, err)
t.Run("genesis", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{}
_, rpcErr := blocker.Blobs(ctx, "genesis", nil)
assert.Equal(t, http.StatusBadRequest, core.ErrorReasonToHTTP(rpcErr.Reason))
assert.StringContains(t, "blobs are not supported for Phase 0 fork", rpcErr.Err.Error())
require.Equal(t, http.StatusBadRequest, core.ErrorReasonToHTTP(rpcErr.Reason))
require.StringContains(t, "blobs are not supported for Phase 0 fork", rpcErr.Err.Error())
})
t.Run("head", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{Root: blockRoot[:]},
ChainInfoFetcher: &mockChain.ChainService{Root: denebBlockRoot[:]},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
retrievedVerifiedSidecars, rpcErr := blocker.Blobs(ctx, "head", nil)
require.IsNil(t, rpcErr)
require.Equal(t, blobCount, len(retrievedVerifiedSidecars))
for i := range blobCount {
expected := verifiedStoredSidecars[i]
actual := retrievedVerifiedSidecars[i].BlobSidecar
require.NotNil(t, actual)
require.Equal(t, expected.Index, actual.Index)
require.DeepEqual(t, expected.Blob, actual.Blob)
require.DeepEqual(t, expected.KzgCommitment, actual.KzgCommitment)
require.DeepEqual(t, expected.KzgProof, actual.KzgProof)
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, "head", nil)
assert.Equal(t, rpcErr == nil, true)
require.Equal(t, 4, len(verifiedBlobs))
sidecar := verifiedBlobs[0].BlobSidecar
require.NotNil(t, sidecar)
assert.Equal(t, uint64(0), sidecar.Index)
assert.DeepEqual(t, blobs[0].Blob, sidecar.Blob)
assert.DeepEqual(t, blobs[0].KzgCommitment, sidecar.KzgCommitment)
assert.DeepEqual(t, blobs[0].KzgProof, sidecar.KzgProof)
sidecar = verifiedBlobs[1].BlobSidecar
require.NotNil(t, sidecar)
assert.Equal(t, uint64(1), sidecar.Index)
assert.DeepEqual(t, blobs[1].Blob, sidecar.Blob)
assert.DeepEqual(t, blobs[1].KzgCommitment, sidecar.KzgCommitment)
assert.DeepEqual(t, blobs[1].KzgProof, sidecar.KzgProof)
sidecar = verifiedBlobs[2].BlobSidecar
require.NotNil(t, sidecar)
assert.Equal(t, uint64(2), sidecar.Index)
assert.DeepEqual(t, blobs[2].Blob, sidecar.Blob)
assert.DeepEqual(t, blobs[2].KzgCommitment, sidecar.KzgCommitment)
assert.DeepEqual(t, blobs[2].KzgProof, sidecar.KzgProof)
sidecar = verifiedBlobs[3].BlobSidecar
require.NotNil(t, sidecar)
assert.Equal(t, uint64(3), sidecar.Index)
assert.DeepEqual(t, blobs[3].Blob, sidecar.Blob)
assert.DeepEqual(t, blobs[3].KzgCommitment, sidecar.KzgCommitment)
assert.DeepEqual(t, blobs[3].KzgProof, sidecar.KzgProof)
})
t.Run("finalized", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: blockRoot[:]}},
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: denebBlockRoot[:]}},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, "finalized", nil)
assert.Equal(t, rpcErr == nil, true)
require.Equal(t, 4, len(verifiedBlobs))
verifiedSidecars, rpcErr := blocker.Blobs(ctx, "finalized", nil)
require.IsNil(t, rpcErr)
require.Equal(t, blobCount, len(verifiedSidecars))
})
t.Run("justified", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{CurrentJustifiedCheckPoint: &ethpb.Checkpoint{Root: blockRoot[:]}},
ChainInfoFetcher: &mockChain.ChainService{CurrentJustifiedCheckPoint: &ethpb.Checkpoint{Root: denebBlockRoot[:]}},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, "justified", nil)
assert.Equal(t, rpcErr == nil, true)
require.Equal(t, 4, len(verifiedBlobs))
verifiedSidecars, rpcErr := blocker.Blobs(ctx, "justified", nil)
require.IsNil(t, rpcErr)
require.Equal(t, blobCount, len(verifiedSidecars))
})
t.Run("root", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, hexutil.Encode(blockRoot[:]), nil)
assert.Equal(t, rpcErr == nil, true)
require.Equal(t, 4, len(verifiedBlobs))
verifiedBlobs, rpcErr := blocker.Blobs(ctx, hexutil.Encode(denebBlockRoot[:]), nil)
require.IsNil(t, rpcErr)
require.Equal(t, blobCount, len(verifiedBlobs))
})
t.Run("slot", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, "123", nil)
assert.Equal(t, rpcErr == nil, true)
require.Equal(t, 4, len(verifiedBlobs))
require.IsNil(t, rpcErr)
require.Equal(t, blobCount, len(verifiedBlobs))
})
t.Run("one blob only", func(t *testing.T) {
const index = 2
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: blockRoot[:]}},
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: denebBlockRoot[:]}},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, "123", []int{2})
assert.Equal(t, rpcErr == nil, true)
require.Equal(t, 1, len(verifiedBlobs))
sidecar := verifiedBlobs[0].BlobSidecar
require.NotNil(t, sidecar)
assert.Equal(t, uint64(2), sidecar.Index)
assert.DeepEqual(t, blobs[2].Blob, sidecar.Blob)
assert.DeepEqual(t, blobs[2].KzgCommitment, sidecar.KzgCommitment)
assert.DeepEqual(t, blobs[2].KzgProof, sidecar.KzgProof)
retrievedVerifiedSidecars, rpcErr := blocker.Blobs(ctx, "123", []int{index})
require.IsNil(t, rpcErr)
require.Equal(t, 1, len(retrievedVerifiedSidecars))
expected := verifiedStoredSidecars[index]
actual := retrievedVerifiedSidecars[0].BlobSidecar
require.NotNil(t, actual)
require.Equal(t, uint64(index), actual.Index)
require.DeepEqual(t, expected.Blob, actual.Blob)
require.DeepEqual(t, expected.KzgCommitment, actual.KzgCommitment)
require.DeepEqual(t, expected.KzgProof, actual.KzgProof)
})
t.Run("no blobs returns an empty array", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: blockRoot[:]}},
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: denebBlockRoot[:]}},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: filesystem.NewEphemeralBlobStorage(t),
}
verifiedBlobs, rpcErr := blocker.Blobs(ctx, "123", nil)
assert.Equal(t, rpcErr == nil, true)
require.IsNil(t, rpcErr)
require.Equal(t, 0, len(verifiedBlobs))
})
t.Run("no blob at index", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: blockRoot[:]}},
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: denebBlockRoot[:]}},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
noBlobIndex := len(blobs) + 1
noBlobIndex := len(storedBlobSidecars) + 1
_, rpcErr := blocker.Blobs(ctx, "123", []int{0, noBlobIndex})
require.NotNil(t, rpcErr)
assert.Equal(t, core.ErrorReason(core.NotFound), rpcErr.Reason)
require.Equal(t, core.ErrorReason(core.NotFound), rpcErr.Reason)
})
t.Run("index too big", func(t *testing.T) {
setupDeneb(t)
blocker := &BeaconDbBlocker{
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: blockRoot[:]}},
ChainInfoFetcher: &mockChain.ChainService{FinalizedCheckPoint: &ethpb.Checkpoint{Root: denebBlockRoot[:]}},
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: bs,
BlobStorage: blobStorage,
}
_, rpcErr := blocker.Blobs(ctx, "123", []int{0, math.MaxInt})
require.NotNil(t, rpcErr)
assert.Equal(t, core.ErrorReason(core.BadRequest), rpcErr.Reason)
require.Equal(t, core.ErrorReason(core.BadRequest), rpcErr.Reason)
})
t.Run("not enough stored data column sidecars", func(t *testing.T) {
setupFulu(t)
_, dataColumnStorage := filesystem.NewEphemeralDataColumnStorageAndFs(t)
err = dataColumnStorage.Save(verifiedRoDataColumnSidecars[:fieldparams.CellsPerBlob-1])
require.NoError(t, err)
blocker := &BeaconDbBlocker{
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: blobStorage,
DataColumnStorage: dataColumnStorage,
}
_, rpcErr := blocker.Blobs(ctx, hexutil.Encode(fuluBlockRoot[:]), nil)
require.NotNil(t, rpcErr)
require.Equal(t, core.ErrorReason(core.NotFound), rpcErr.Reason)
})
t.Run("reconstruction needed", func(t *testing.T) {
setupFulu(t)
_, dataColumnStorage := filesystem.NewEphemeralDataColumnStorageAndFs(t)
err = dataColumnStorage.Save(verifiedRoDataColumnSidecars[1 : peerdas.MinimumColumnsCountToReconstruct()+1])
require.NoError(t, err)
blocker := &BeaconDbBlocker{
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: blobStorage,
DataColumnStorage: dataColumnStorage,
}
retrievedVerifiedRoBlobs, rpcErr := blocker.Blobs(ctx, hexutil.Encode(fuluBlockRoot[:]), nil)
require.IsNil(t, rpcErr)
require.Equal(t, len(fuluBlobSidecars), len(retrievedVerifiedRoBlobs))
for i, retrievedVerifiedRoBlob := range retrievedVerifiedRoBlobs {
retrievedBlobSidecarPb := retrievedVerifiedRoBlob.BlobSidecar
initialBlobSidecarPb := fuluBlobSidecars[i].BlobSidecar
require.DeepSSZEqual(t, initialBlobSidecarPb, retrievedBlobSidecarPb)
}
})
t.Run("no reconstruction needed", func(t *testing.T) {
setupFulu(t)
_, dataColumnStorage := filesystem.NewEphemeralDataColumnStorageAndFs(t)
err = dataColumnStorage.Save(verifiedRoDataColumnSidecars)
require.NoError(t, err)
blocker := &BeaconDbBlocker{
GenesisTimeFetcher: &testutil.MockGenesisTimeFetcher{
Genesis: time.Now(),
},
BeaconDB: db,
BlobStorage: blobStorage,
DataColumnStorage: dataColumnStorage,
}
retrievedVerifiedRoBlobs, rpcErr := blocker.Blobs(ctx, hexutil.Encode(fuluBlockRoot[:]), nil)
require.IsNil(t, rpcErr)
require.Equal(t, len(fuluBlobSidecars), len(retrievedVerifiedRoBlobs))
for i, retrievedVerifiedRoBlob := range retrievedVerifiedRoBlobs {
retrievedBlobSidecarPb := retrievedVerifiedRoBlob.BlobSidecar
initialBlobSidecarPb := fuluBlobSidecars[i].BlobSidecar
require.DeepSSZEqual(t, initialBlobSidecarPb, retrievedBlobSidecarPb)
}
})
}

View File

@@ -25,6 +25,8 @@ go_library(
"rpc_blob_sidecars_by_range.go",
"rpc_blob_sidecars_by_root.go",
"rpc_chunked_response.go",
"rpc_data_column_sidecars_by_range.go",
"rpc_data_column_sidecars_by_root.go",
"rpc_goodbye.go",
"rpc_light_client.go",
"rpc_metadata.go",
@@ -170,6 +172,8 @@ go_test(
"rpc_beacon_blocks_by_root_test.go",
"rpc_blob_sidecars_by_range_test.go",
"rpc_blob_sidecars_by_root_test.go",
"rpc_data_column_sidecars_by_range_test.go",
"rpc_data_column_sidecars_by_root_test.go",
"rpc_goodbye_test.go",
"rpc_handler_test.go",
"rpc_light_client_test.go",
@@ -280,6 +284,7 @@ go_test(
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
"@com_github_stretchr_testify//require:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
],
)

View File

@@ -15,13 +15,17 @@ import (
"github.com/sirupsen/logrus"
)
var ErrNoValidDigest = errors.New("no valid digest matched")
var ErrUnrecognizedVersion = errors.New("cannot determine context bytes for unrecognized object")
var (
ErrNoValidDigest = errors.New("no valid digest matched")
ErrUnrecognizedVersion = errors.New("cannot determine context bytes for unrecognized object")
)
var responseCodeSuccess = byte(0x00)
var responseCodeInvalidRequest = byte(0x01)
var responseCodeServerError = byte(0x02)
var responseCodeResourceUnavailable = byte(0x03)
var (
responseCodeSuccess = byte(0x00)
responseCodeInvalidRequest = byte(0x01)
responseCodeServerError = byte(0x02)
responseCodeResourceUnavailable = byte(0x03)
)
func (s *Service) generateErrorResponse(code byte, reason string) ([]byte, error) {
return createErrorResponse(code, reason, s.cfg.p2p)

View File

@@ -89,6 +89,13 @@ var (
Buckets: []float64{5, 10, 50, 100, 150, 250, 500, 1000, 2000},
},
)
rpcDataColumnsByRangeResponseLatency = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "rpc_data_columns_by_range_response_latency_milliseconds",
Help: "Captures total time to respond to rpc DataColumnsByRange requests in a milliseconds distribution",
Buckets: []float64{5, 10, 50, 100, 150, 250, 500, 1000, 2000},
},
)
arrivalBlockPropagationHistogram = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "block_arrival_latency_milliseconds",

View File

@@ -173,6 +173,14 @@ func WithBlobStorage(b *filesystem.BlobStorage) Option {
}
}
// WithDataColumnStorage gives the sync package direct access to DataColumnStorage.
func WithDataColumnStorage(b *filesystem.DataColumnStorage) Option {
return func(s *Service) error {
s.cfg.dataColumnStorage = b
return nil
}
}
// WithVerifierWaiter gives the sync package direct access to the verifier waiter.
func WithVerifierWaiter(v *verification.InitializerWaiter) Option {
return func(s *Service) error {

View File

@@ -47,6 +47,10 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
allowedBlobsPerSecond := float64(flags.Get().BlobBatchLimit)
allowedBlobsBurst := int64(flags.Get().BlobBatchLimitBurstFactor * flags.Get().BlobBatchLimit)
// Initialize data column limits.
allowedDataColumnsPerSecond := float64(flags.Get().DataColumnBatchLimit)
allowedDataColumnsBurst := int64(flags.Get().DataColumnBatchLimitBurstFactor * flags.Get().DataColumnBatchLimit)
// Set topic map for all rpc topics.
topicMap := make(map[string]*leakybucket.Collector, len(p2p.RPCTopicMappings))
// Goodbye Message
@@ -67,6 +71,9 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
// for BlobSidecarsByRoot and BlobSidecarsByRange
blobCollector := leakybucket.NewCollector(allowedBlobsPerSecond, allowedBlobsBurst, blockBucketPeriod, false)
// for DataColumnSidecarsByRoot and DataColumnSidecarsByRange
dataColumnSidecars := leakybucket.NewCollector(allowedDataColumnsPerSecond, allowedDataColumnsBurst, blockBucketPeriod, false)
// BlocksByRoots requests
topicMap[addEncoding(p2p.RPCBlocksByRootTopicV1)] = blockCollector
topicMap[addEncoding(p2p.RPCBlocksByRootTopicV2)] = blockCollectorV2
@@ -86,6 +93,11 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
topicMap[addEncoding(p2p.RPCLightClientOptimisticUpdateTopicV1)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */)
topicMap[addEncoding(p2p.RPCLightClientFinalityUpdateTopicV1)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */)
// DataColumnSidecarsByRootV1
topicMap[addEncoding(p2p.RPCDataColumnSidecarsByRootTopicV1)] = dataColumnSidecars
// DataColumnSidecarsByRangeV1
topicMap[addEncoding(p2p.RPCDataColumnSidecarsByRangeTopicV1)] = dataColumnSidecars
// General topic for all rpc requests.
topicMap[rpcLimiterTopic] = leakybucket.NewCollector(5, defaultBurstLimit*2, leakyBucketPeriod, false /* deleteEmptyBuckets */)

View File

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

View File

@@ -39,6 +39,21 @@ type rpcHandler func(context.Context, interface{}, libp2pcore.Stream) error
// rpcHandlerByTopicFromFork returns the RPC handlers for a given fork index.
func (s *Service) rpcHandlerByTopicFromFork(forkIndex int) (map[string]rpcHandler, error) {
// Fulu: https://github.com/ethereum/consensus-specs/blob/dev/specs/fulu/p2p-interface.md#messages
if forkIndex >= version.Fulu {
return map[string]rpcHandler{
p2p.RPCGoodByeTopicV1: s.goodbyeRPCHandler,
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler,
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler,
p2p.RPCPingTopicV1: s.pingHandler,
p2p.RPCMetaDataTopicV3: s.metaDataHandler, // Modified in Fulu
p2p.RPCBlobSidecarsByRootTopicV1: s.blobSidecarByRootRPCHandler,
p2p.RPCBlobSidecarsByRangeTopicV1: s.blobSidecarsByRangeRPCHandler,
p2p.RPCDataColumnSidecarsByRootTopicV1: s.dataColumnSidecarByRootRPCHandler, // Added in Fulu
p2p.RPCDataColumnSidecarsByRangeTopicV1: s.dataColumnSidecarsByRangeRPCHandler, // Added in Fulu
}, nil
}
// Electra: https://github.com/ethereum/consensus-specs/blob/dev/specs/electra/p2p-interface.md#messages
if forkIndex >= version.Electra {
return map[string]rpcHandler{
@@ -258,9 +273,15 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
// since some requests do not have any data in the payload, we
// do not decode anything.
if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 ||
baseTopic == p2p.RPCLightClientOptimisticUpdateTopicV1 ||
baseTopic == p2p.RPCLightClientFinalityUpdateTopicV1 {
topics := map[string]bool{
p2p.RPCMetaDataTopicV1: true,
p2p.RPCMetaDataTopicV2: true,
p2p.RPCMetaDataTopicV3: true,
p2p.RPCLightClientOptimisticUpdateTopicV1: true,
p2p.RPCLightClientFinalityUpdateTopicV1: true,
}
if topics[baseTopic] {
if err := handle(ctx, base, stream); err != nil {
messageFailedProcessingCounter.WithLabelValues(topic).Inc()
if !errors.Is(err, p2ptypes.ErrWrongForkDigestVersion) {

View File

@@ -9,6 +9,7 @@ import (
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/network/forks"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/time/slots"
libp2pcore "github.com/libp2p/go-libp2p/core"
@@ -238,3 +239,30 @@ func WriteLightClientFinalityUpdateChunk(stream libp2pcore.Stream, tor blockchai
_, err = encoding.EncodeWithMaxLength(stream, update)
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 {
// Success response code.
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil {
return errors.Wrap(err, "stream write")
}
// Fork digest.
genesisValidatorsRoot := tor.GenesisValidatorsRoot()
ctxBytes, err := forks.ForkDigestFromEpoch(slots.ToEpoch(sidecar.SignedBlockHeader.Header.Slot), genesisValidatorsRoot[:])
if err != nil {
return errors.Wrap(err, "fork digest from epoch")
}
if err := writeContextToStream(ctxBytes[:], stream); err != nil {
return errors.Wrap(err, "write context to stream")
}
// Sidecar.
if _, err = encoding.EncodeWithMaxLength(stream, sidecar); err != nil {
return errors.Wrap(err, "encode with max length")
}
return nil
}

View File

@@ -0,0 +1,218 @@
package sync
import (
"context"
"slices"
"time"
p2ptypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
// We count a single request as a single rate limiting amount, regardless of the number of columns requested.
const rateLimitingAmount = 1
var notDataColumnsByRangeIdentifiersError = errors.New("not data columns by range identifiers")
// dataColumnSidecarsByRangeRPCHandler looks up the request data columns from the database from a given start slot index
func (s *Service) dataColumnSidecarsByRangeRPCHandler(ctx context.Context, msg interface{}, stream libp2pcore.Stream) error {
ctx, span := trace.StartSpan(ctx, "sync.DataColumnSidecarsByRangeHandler")
defer span.End()
// Check if the message type is the one expected.
request, ok := msg.(*pb.DataColumnSidecarsByRangeRequest)
if !ok {
return notDataColumnsByRangeIdentifiersError
}
ctx, cancel := context.WithTimeout(ctx, respTimeout)
defer cancel()
SetRPCStreamDeadlines(stream)
beaconConfig := params.BeaconConfig()
maxRequestDataColumnSidecars := beaconConfig.MaxRequestDataColumnSidecars
remotePeer := stream.Conn().RemotePeer()
requestedColumns := request.Columns
// Format log fields.
var requestedColumnsLog interface{} = "all"
if uint64(len(requestedColumns)) != beaconConfig.NumberOfColumns {
requestedColumnsLog = requestedColumns
}
log := log.WithFields(logrus.Fields{
"remotePeer": remotePeer,
"requestedColumns": requestedColumnsLog,
"startSlot": request.StartSlot,
"count": request.Count,
})
// Validate the request regarding rate limiting.
if err := s.rateLimiter.validateRequest(stream, rateLimitingAmount); err != nil {
return errors.Wrap(err, "rate limiter validate request")
}
// Validate the request regarding its parameters.
rangeParameters, err := validateDataColumnsByRange(request, s.cfg.chain.CurrentSlot())
if err != nil {
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(remotePeer)
tracing.AnnotateError(span, err)
return errors.Wrap(err, "validate data columns by range")
}
if rangeParameters == nil {
log.Debug("No data columns by range to serve")
return nil
}
log.Debug("Serving data columns by range request")
// Ticker to stagger out large requests.
ticker := time.NewTicker(time.Second)
batcher, err := newBlockRangeBatcher(*rangeParameters, s.cfg.beaconDB, s.rateLimiter, s.cfg.chain.IsCanonical, ticker)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)
return errors.Wrap(err, "new block range batcher")
}
// Derive the wanted columns for the request.
wantedColumns := make([]uint64, len(request.Columns))
copy(wantedColumns, request.Columns)
// Sort the wanted columns.
slices.Sort(wantedColumns)
var batch blockBatch
for batch, ok = batcher.next(ctx, stream); ok; batch, ok = batcher.next(ctx, stream) {
batchStart := time.Now()
maxRequestDataColumnSidecars, err = s.streamDataColumnBatch(ctx, batch, maxRequestDataColumnSidecars, wantedColumns, stream)
rpcDataColumnsByRangeResponseLatency.Observe(float64(time.Since(batchStart).Milliseconds()))
if err != nil {
return err
}
// Once the quota is reached, we're done serving the request.
if maxRequestDataColumnSidecars == 0 {
log.WithField("initialQuota", beaconConfig.MaxRequestDataColumnSidecars).Debug("Reached quota for data column sidecars by range request")
break
}
}
if err := batch.error(); err != nil {
log.WithError(err).Debug("error in DataColumnSidecarsByRange batch")
// If we hit a rate limit, the error response has already been written, and the stream is already closed.
if !errors.Is(err, p2ptypes.ErrRateLimited) {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
}
tracing.AnnotateError(span, err)
return err
}
closeStream(stream, log)
return nil
}
func (s *Service) streamDataColumnBatch(ctx context.Context, batch blockBatch, quota uint64, wantedDataColumnIndices []uint64, stream libp2pcore.Stream) (uint64, error) {
_, span := trace.StartSpan(ctx, "sync.streamDataColumnBatch")
defer span.End()
// Defensive check to guard against underflow.
if quota == 0 {
return 0, nil
}
// Loop over the blocks in the batch.
for _, block := range batch.canonical() {
// Get the block blockRoot.
blockRoot := block.Root()
// Retrieve the data column sidecars from the store.
verifiedRODataColumns, err := s.cfg.dataColumnStorage.Get(blockRoot, wantedDataColumnIndices)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
return quota, errors.Wrapf(err, "get data column sidecars: block root %#x", blockRoot)
}
// Write the retrieved sidecars to the stream.
for _, verifiedRODataColumn := range verifiedRODataColumns {
sidecar := verifiedRODataColumn.DataColumnSidecar
SetStreamWriteDeadline(stream, defaultWriteDuration)
if err := WriteDataColumnSidecarChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), sidecar); err != nil {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, err)
return quota, errors.Wrap(err, "write data column sidecar chunk")
}
s.rateLimiter.add(stream, rateLimitingAmount)
quota -= 1
// Stop streaming results once the quota of writes for the request is consumed.
if quota == 0 {
return 0, nil
}
}
}
return quota, nil
}
func validateDataColumnsByRange(request *pb.DataColumnSidecarsByRangeRequest, currentSlot primitives.Slot) (*rangeParams, error) {
startSlot, count := request.StartSlot, request.Count
if count == 0 {
return nil, errors.Wrap(p2ptypes.ErrInvalidRequest, "invalid request count parameter")
}
endSlot, err := request.StartSlot.SafeAdd(count - 1)
if err != nil {
return nil, errors.Wrap(p2ptypes.ErrInvalidRequest, "overflow start + count -1")
}
// Peers may overshoot the current slot when in initial sync,
// so we don't want to penalize them by treating the request as an error.
if startSlot > currentSlot {
return nil, nil
}
// Compute the oldest slot we'll allow a peer to request, based on the current slot.
minStartSlot, err := dataColumnsRPCMinValidSlot(currentSlot)
if err != nil {
return nil, errors.Wrap(p2ptypes.ErrInvalidRequest, "data columns RPC min valid slot")
}
// Return early if there is nothing to serve.
if endSlot < minStartSlot {
return nil, nil
}
// Do not serve sidecars for slots before the minimum valid slot or after the current slot.
startSlot = max(startSlot, minStartSlot)
endSlot = min(endSlot, currentSlot)
sizeMinusOne, err := endSlot.SafeSub(uint64(startSlot))
if err != nil {
return nil, errors.Errorf("overflow end - start: %d - %d - should never happen", endSlot, startSlot)
}
size, err := sizeMinusOne.SafeAdd(1)
if err != nil {
return nil, errors.Wrap(p2ptypes.ErrInvalidRequest, "overflow end - start + 1")
}
rangeParameters := &rangeParams{start: startSlot, end: endSlot, size: uint64(size)}
return rangeParameters, nil
}

View File

@@ -0,0 +1,301 @@
package sync
import (
"context"
"fmt"
"io"
"math"
"sync"
"testing"
"time"
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/protocol"
"github.com/pkg/errors"
"github.com/stretchr/testify/require"
chainMock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
testDB "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
consensusblocks "github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/util"
)
func TestDataColumnSidecarsByRangeRPCHandler(t *testing.T) {
ctx := context.Background()
t.Run("wrong message type", func(t *testing.T) {
service := &Service{}
err := service.dataColumnSidecarsByRangeRPCHandler(ctx, nil, nil)
require.ErrorIs(t, err, notDataColumnsByRangeIdentifiersError)
})
t.Run("invalid request", func(t *testing.T) {
slot := primitives.Slot(400)
localP2P, remoteP2P := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
service := &Service{
cfg: &config{
p2p: localP2P,
chain: &chainMock.ChainService{
Slot: &slot,
},
},
rateLimiter: newRateLimiter(localP2P),
}
protocolID := protocol.ID(fmt.Sprintf("%s/ssz_snappy", p2p.RPCDataColumnSidecarsByRangeTopicV1))
var wg sync.WaitGroup
wg.Add(1)
remoteP2P.BHost.SetStreamHandler(protocolID, func(stream network.Stream) {
defer wg.Done()
code, _, err := readStatusCodeNoDeadline(stream, localP2P.Encoding())
require.NoError(t, err)
require.Equal(t, responseCodeInvalidRequest, code)
})
localP2P.Connect(remoteP2P)
stream, err := localP2P.BHost.NewStream(ctx, remoteP2P.BHost.ID(), protocolID)
require.NoError(t, err)
msg := &pb.DataColumnSidecarsByRangeRequest{
Count: 0, // Invalid count
}
require.Equal(t, true, localP2P.Peers().Scorers().BadResponsesScorer().Score(remoteP2P.PeerID()) >= 0)
err = service.dataColumnSidecarsByRangeRPCHandler(ctx, msg, stream)
require.NotNil(t, err)
require.Equal(t, true, localP2P.Peers().Scorers().BadResponsesScorer().Score(remoteP2P.PeerID()) < 0)
if util.WaitTimeout(&wg, 1*time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
})
t.Run("nominal", func(t *testing.T) {
params.SetupTestConfigCleanup(t)
beaconConfig := params.BeaconConfig()
beaconConfig.FuluForkEpoch = 0
params.OverrideBeaconConfig(beaconConfig)
slot := primitives.Slot(400)
params := []util.DataColumnParam{
{Slot: 10, Index: 1}, {Slot: 10, Index: 2}, {Slot: 10, Index: 3},
{Slot: 40, Index: 4}, {Slot: 40, Index: 6},
{Slot: 45, Index: 7}, {Slot: 45, Index: 8}, {Slot: 45, Index: 9},
}
_, verifiedRODataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, params)
storage := filesystem.NewEphemeralDataColumnStorage(t)
err := storage.Save(verifiedRODataColumns)
require.NoError(t, err)
localP2P, remoteP2P := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
protocolID := protocol.ID(fmt.Sprintf("%s/ssz_snappy", p2p.RPCDataColumnSidecarsByRangeTopicV1))
roots := [][fieldparams.RootLength]byte{
verifiedRODataColumns[0].BlockRoot(),
verifiedRODataColumns[3].BlockRoot(),
verifiedRODataColumns[5].BlockRoot(),
}
slots := []primitives.Slot{
verifiedRODataColumns[0].Slot(),
verifiedRODataColumns[3].Slot(),
verifiedRODataColumns[5].Slot(),
}
beaconDB := testDB.SetupDB(t)
roBlocks := make([]blocks.ROBlock, 0, len(roots))
for i := range 3 {
signedBeaconBlockPb := util.NewBeaconBlock()
signedBeaconBlockPb.Block.Slot = slots[i]
if i != 0 {
signedBeaconBlockPb.Block.ParentRoot = roots[i-1][:]
}
signedBeaconBlock, err := consensusblocks.NewSignedBeaconBlock(signedBeaconBlockPb)
require.NoError(t, err)
// There is a discrepancy between the root of the beacon block and the rodata column root,
// but for the sake of this test, we actually don't care.
roblock, err := consensusblocks.NewROBlockWithRoot(signedBeaconBlock, roots[i])
require.NoError(t, err)
roBlocks = append(roBlocks, roblock)
}
err = beaconDB.SaveROBlocks(ctx, roBlocks, false /*cache*/)
require.NoError(t, err)
service := &Service{
cfg: &config{
p2p: localP2P,
beaconDB: beaconDB,
chain: &chainMock.ChainService{
Slot: &slot,
},
dataColumnStorage: storage,
},
rateLimiter: newRateLimiter(localP2P),
}
ctxMap := ContextByteVersions{
[4]byte{245, 165, 253, 66}: version.Fulu,
}
root0 := verifiedRODataColumns[0].BlockRoot()
root3 := verifiedRODataColumns[3].BlockRoot()
root5 := verifiedRODataColumns[5].BlockRoot()
var wg sync.WaitGroup
wg.Add(1)
remoteP2P.BHost.SetStreamHandler(protocolID, func(stream network.Stream) {
defer wg.Done()
sidecars := make([]*blocks.RODataColumn, 0, 5)
for i := uint64(0); ; /* no stop condition */ i++ {
sidecar, err := readChunkedDataColumnSidecar(stream, remoteP2P, ctxMap)
if errors.Is(err, io.EOF) {
// End of stream.
break
}
require.NoError(t, err)
sidecars = append(sidecars, sidecar)
}
require.Equal(t, 8, len(sidecars))
require.Equal(t, root0, sidecars[0].BlockRoot())
require.Equal(t, root0, sidecars[1].BlockRoot())
require.Equal(t, root0, sidecars[2].BlockRoot())
require.Equal(t, root3, sidecars[3].BlockRoot())
require.Equal(t, root3, sidecars[4].BlockRoot())
require.Equal(t, root5, sidecars[5].BlockRoot())
require.Equal(t, root5, sidecars[6].BlockRoot())
require.Equal(t, root5, sidecars[7].BlockRoot())
require.Equal(t, uint64(1), sidecars[0].Index)
require.Equal(t, uint64(2), sidecars[1].Index)
require.Equal(t, uint64(3), sidecars[2].Index)
require.Equal(t, uint64(4), sidecars[3].Index)
require.Equal(t, uint64(6), sidecars[4].Index)
require.Equal(t, uint64(7), sidecars[5].Index)
require.Equal(t, uint64(8), sidecars[6].Index)
require.Equal(t, uint64(9), sidecars[7].Index)
})
localP2P.Connect(remoteP2P)
stream, err := localP2P.BHost.NewStream(ctx, remoteP2P.BHost.ID(), protocolID)
require.NoError(t, err)
msg := &pb.DataColumnSidecarsByRangeRequest{
StartSlot: 5,
Count: 50,
Columns: []uint64{1, 2, 3, 4, 6, 7, 8, 9, 10},
}
err = service.dataColumnSidecarsByRangeRPCHandler(ctx, msg, stream)
require.NoError(t, err)
})
}
func TestValidateDataColumnsByRange(t *testing.T) {
maxUint := primitives.Slot(math.MaxUint64)
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig()
config.FuluForkEpoch = 10
config.MinEpochsForDataColumnSidecarsRequest = 4096
params.OverrideBeaconConfig(config)
tests := []struct {
name string
startSlot primitives.Slot
count uint64
currentSlot primitives.Slot
expected *rangeParams
expectErr bool
errContains string
}{
{
name: "zero count returns error",
count: 0,
expectErr: true,
errContains: "invalid request count parameter",
},
{
name: "overflow in addition returns error",
startSlot: maxUint - 5,
count: 10,
currentSlot: maxUint,
expectErr: true,
errContains: "overflow start + count -1",
},
{
name: "start greater than current returns nil",
startSlot: 150,
count: 10,
currentSlot: 100,
expected: nil,
expectErr: false,
},
{
name: "end slot greater than min start slot returns nil",
startSlot: 150,
count: 100,
currentSlot: 300,
expected: nil,
expectErr: false,
},
{
name: "range within limits",
startSlot: 350,
count: 10,
currentSlot: 400,
expected: &rangeParams{start: 350, end: 359, size: 10},
expectErr: false,
},
{
name: "range exceeds limits",
startSlot: 0,
count: 10_000,
currentSlot: 400,
expected: &rangeParams{start: 320, end: 400, size: 81},
expectErr: false,
},
}
for _, tc := range tests {
t.Run(tc.name, func(t *testing.T) {
request := &pb.DataColumnSidecarsByRangeRequest{
StartSlot: tc.startSlot,
Count: tc.count,
}
rangeParameters, err := validateDataColumnsByRange(request, tc.currentSlot)
if tc.expectErr {
require.ErrorContains(t, err, tc.errContains)
return
}
require.NoError(t, err)
require.Equal(t, tc.expected, rangeParameters)
})
}
}

View File

@@ -0,0 +1,183 @@
package sync
import (
"context"
"fmt"
"math"
"slices"
"time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
"github.com/OffchainLabs/prysm/v6/time/slots"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
var (
notDataColumnsByRootIdentifiersError = errors.New("not data columns by root identifiers")
tickerDelay = time.Second
)
// dataColumnSidecarByRootRPCHandler handles the data column sidecars by root RPC request.
// https://github.com/ethereum/consensus-specs/blob/dev/specs/fulu/p2p-interface.md#datacolumnsidecarsbyroot-v1
func (s *Service) dataColumnSidecarByRootRPCHandler(ctx context.Context, msg interface{}, stream libp2pcore.Stream) error {
ctx, span := trace.StartSpan(ctx, "sync.dataColumnSidecarByRootRPCHandler")
defer span.End()
batchSize := flags.Get().DataColumnBatchLimit
numberOfColumns := params.BeaconConfig().NumberOfColumns
// Check if the message type is the one expected.
ref, ok := msg.(*types.DataColumnsByRootIdentifiers)
if !ok {
return notDataColumnsByRootIdentifiersError
}
requestedColumnIdents := *ref
remotePeerId := stream.Conn().RemotePeer()
ctx, cancel := context.WithTimeout(ctx, ttfbTimeout)
defer cancel()
SetRPCStreamDeadlines(stream)
// Penalize peers that send invalid requests.
if err := validateDataColumnsByRootRequest(requestedColumnIdents); err != nil {
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(remotePeerId)
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
return errors.Wrap(err, "validate data columns by root request")
}
requestedColumnsByRoot := make(map[[fieldparams.RootLength]byte][]uint64)
for _, columnIdent := range requestedColumnIdents {
var root [fieldparams.RootLength]byte
copy(root[:], columnIdent.BlockRoot)
requestedColumnsByRoot[root] = append(requestedColumnsByRoot[root], columnIdent.Columns...)
}
// Sort by column index for each root.
for _, columns := range requestedColumnsByRoot {
slices.Sort(columns)
}
// Format nice logs.
requestedColumnsByRootLog := make(map[string]interface{})
for root, columns := range requestedColumnsByRoot {
rootStr := fmt.Sprintf("%#x", root)
requestedColumnsByRootLog[rootStr] = "all"
if uint64(len(columns)) != numberOfColumns {
requestedColumnsByRootLog[rootStr] = columns
}
}
// Compute the oldest slot we'll allow a peer to request, based on the current slot.
minReqSlot, err := dataColumnsRPCMinValidSlot(s.cfg.clock.CurrentSlot())
if err != nil {
return errors.Wrapf(err, "data columns RPC min valid slot")
}
log := log.WithFields(logrus.Fields{
"peer": remotePeerId,
"columns": requestedColumnsByRootLog,
})
defer closeStream(stream, log)
var ticker *time.Ticker
if len(requestedColumnIdents) > batchSize {
ticker = time.NewTicker(tickerDelay)
}
log.Debug("Serving data column sidecar by root request")
count := 0
for _, ident := range requestedColumnIdents {
if err := ctx.Err(); err != nil {
closeStream(stream, log)
return errors.Wrap(err, "context error")
}
root := bytesutil.ToBytes32(ident.BlockRoot)
columns := ident.Columns
// Throttle request processing to no more than batchSize/sec.
for range columns {
if ticker != nil && count != 0 && count%batchSize == 0 {
<-ticker.C
}
count++
}
s.rateLimiter.add(stream, int64(len(columns)))
// Retrieve the requested sidecars from the store.
verifiedRODataColumns, err := s.cfg.dataColumnStorage.Get(root, columns)
if err != nil {
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
return errors.Wrap(err, "get data column sidecars")
}
for _, verifiedRODataColumn := range verifiedRODataColumns {
// Filter out data column sidecars that are too old.
if verifiedRODataColumn.SignedBlockHeader.Header.Slot < minReqSlot {
continue
}
SetStreamWriteDeadline(stream, defaultWriteDuration)
if chunkErr := WriteDataColumnSidecarChunk(stream, s.cfg.chain, s.cfg.p2p.Encoding(), verifiedRODataColumn.DataColumnSidecar); chunkErr != nil {
s.writeErrorResponseToStream(responseCodeServerError, types.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, chunkErr)
return chunkErr
}
}
}
return nil
}
// validateDataColumnsByRootRequest checks if the request for data column sidecars is valid.
func validateDataColumnsByRootRequest(colIdents types.DataColumnsByRootIdentifiers) error {
total := uint64(0)
for _, id := range colIdents {
total += uint64(len(id.Columns))
}
if total > params.BeaconConfig().MaxRequestDataColumnSidecars {
return types.ErrMaxDataColumnReqExceeded
}
return nil
}
// dataColumnsRPCMinValidSlot returns the minimum slot that a peer can request data column sidecars for.
func dataColumnsRPCMinValidSlot(currentSlot primitives.Slot) (primitives.Slot, error) {
// Avoid overflow if we're running on a config where fulu is set to far future epoch.
if !params.FuluEnabled() {
return primitives.Slot(math.MaxUint64), nil
}
beaconConfig := params.BeaconConfig()
minReqEpochs := beaconConfig.MinEpochsForDataColumnSidecarsRequest
minStartEpoch := beaconConfig.FuluForkEpoch
currEpoch := slots.ToEpoch(currentSlot)
if currEpoch > minReqEpochs && currEpoch-minReqEpochs > minStartEpoch {
minStartEpoch = currEpoch - minReqEpochs
}
epochStart, err := slots.EpochStart(minStartEpoch)
if err != nil {
return 0, errors.Wrapf(err, "epoch start for epoch %d", minStartEpoch)
}
return epochStart, nil
}

View File

@@ -0,0 +1,321 @@
package sync
import (
"context"
"io"
"math"
"sync"
"testing"
"time"
chainMock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/protocol"
"github.com/pkg/errors"
)
func TestDataColumnSidecarsByRootRPCHandler(t *testing.T) {
ctx := context.Background()
t.Run("wrong message type", func(t *testing.T) {
service := &Service{}
err := service.dataColumnSidecarByRootRPCHandler(ctx, nil, nil)
require.ErrorIs(t, err, notDataColumnsByRootIdentifiersError)
})
t.Run("invalid request", func(t *testing.T) {
params.SetupTestConfigCleanup(t)
beaconConfig := params.BeaconConfig()
beaconConfig.MaxRequestDataColumnSidecars = 1
params.OverrideBeaconConfig(beaconConfig)
localP2P := p2ptest.NewTestP2P(t)
service := &Service{cfg: &config{p2p: localP2P}}
protocolID := protocol.ID(p2p.RPCDataColumnSidecarsByRootTopicV1)
remoteP2P := p2ptest.NewTestP2P(t)
var wg sync.WaitGroup
wg.Add(1)
remoteP2P.BHost.SetStreamHandler(protocolID, func(stream network.Stream) {
defer wg.Done()
code, errMsg, err := readStatusCodeNoDeadline(stream, localP2P.Encoding())
require.NoError(t, err)
require.Equal(t, responseCodeInvalidRequest, code)
require.Equal(t, types.ErrMaxDataColumnReqExceeded.Error(), errMsg)
})
localP2P.Connect(remoteP2P)
stream, err := localP2P.BHost.NewStream(ctx, remoteP2P.BHost.ID(), protocolID)
require.NoError(t, err)
msg := &types.DataColumnsByRootIdentifiers{{Columns: []uint64{1, 2, 3}}}
require.Equal(t, true, localP2P.Peers().Scorers().BadResponsesScorer().Score(remoteP2P.PeerID()) >= 0)
err = service.dataColumnSidecarByRootRPCHandler(ctx, msg, stream)
require.NotNil(t, err)
require.Equal(t, true, localP2P.Peers().Scorers().BadResponsesScorer().Score(remoteP2P.PeerID()) < 0)
if util.WaitTimeout(&wg, 1*time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
})
t.Run("nominal", func(t *testing.T) {
resetFlags := flags.Get()
gFlags := new(flags.GlobalFlags)
gFlags.DataColumnBatchLimit = 2
flags.Init(gFlags)
defer flags.Init(resetFlags)
// Setting the ticker to 0 will cause the ticker to panic.
// Setting it to the minimum value instead.
refTickerDelay := tickerDelay
tickerDelay = time.Nanosecond
defer func() {
tickerDelay = refTickerDelay
}()
params.SetupTestConfigCleanup(t)
beaconConfig := params.BeaconConfig()
beaconConfig.FuluForkEpoch = 1
params.OverrideBeaconConfig(beaconConfig)
localP2P := p2ptest.NewTestP2P(t)
clock := startup.NewClock(time.Now(), [fieldparams.RootLength]byte{})
params := []util.DataColumnParam{
{Slot: 10, Index: 1}, {Slot: 10, Index: 2}, {Slot: 10, Index: 3},
{Slot: 40, Index: 4}, {Slot: 40, Index: 6},
{Slot: 45, Index: 7}, {Slot: 45, Index: 8}, {Slot: 45, Index: 9},
}
_, verifiedRODataColumns := util.CreateTestVerifiedRoDataColumnSidecars(t, params)
storage := filesystem.NewEphemeralDataColumnStorage(t)
err := storage.Save(verifiedRODataColumns)
require.NoError(t, err)
service := &Service{
cfg: &config{
p2p: localP2P,
clock: clock,
dataColumnStorage: storage,
chain: &chainMock.ChainService{},
},
rateLimiter: newRateLimiter(localP2P),
}
protocolID := protocol.ID(p2p.RPCDataColumnSidecarsByRootTopicV1)
remoteP2P := p2ptest.NewTestP2P(t)
var wg sync.WaitGroup
wg.Add(1)
ctxMap := ContextByteVersions{
[4]byte{245, 165, 253, 66}: version.Fulu,
}
root0 := verifiedRODataColumns[0].BlockRoot()
root3 := verifiedRODataColumns[3].BlockRoot()
root5 := verifiedRODataColumns[5].BlockRoot()
remoteP2P.BHost.SetStreamHandler(protocolID, func(stream network.Stream) {
defer wg.Done()
sidecars := make([]*blocks.RODataColumn, 0, 5)
for i := uint64(0); ; /* no stop condition */ i++ {
sidecar, err := readChunkedDataColumnSidecar(stream, remoteP2P, ctxMap)
if errors.Is(err, io.EOF) {
// End of stream.
break
}
require.NoError(t, err)
sidecars = append(sidecars, sidecar)
}
require.Equal(t, 5, len(sidecars))
require.Equal(t, root3, sidecars[0].BlockRoot())
require.Equal(t, root3, sidecars[1].BlockRoot())
require.Equal(t, root5, sidecars[2].BlockRoot())
require.Equal(t, root5, sidecars[3].BlockRoot())
require.Equal(t, root5, sidecars[4].BlockRoot())
require.Equal(t, uint64(4), sidecars[0].Index)
require.Equal(t, uint64(6), sidecars[1].Index)
require.Equal(t, uint64(7), sidecars[2].Index)
require.Equal(t, uint64(8), sidecars[3].Index)
require.Equal(t, uint64(9), sidecars[4].Index)
})
localP2P.Connect(remoteP2P)
stream, err := localP2P.BHost.NewStream(ctx, remoteP2P.BHost.ID(), protocolID)
require.NoError(t, err)
msg := &types.DataColumnsByRootIdentifiers{
{
BlockRoot: root0[:],
Columns: []uint64{1, 2, 3},
},
{
BlockRoot: root3[:],
Columns: []uint64{4, 5, 6},
},
{
BlockRoot: root5[:],
Columns: []uint64{7, 8, 9},
},
}
err = service.dataColumnSidecarByRootRPCHandler(ctx, msg, stream)
require.NoError(t, err)
require.Equal(t, true, localP2P.Peers().Scorers().BadResponsesScorer().Score(remoteP2P.PeerID()) >= 0)
if util.WaitTimeout(&wg, 1*time.Minute) {
t.Fatal("Did not receive stream within 1 sec")
}
})
}
func TestValidateDataColumnsByRootRequest(t *testing.T) {
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig()
maxCols := uint64(10) // Set a small value for testing
config.MaxRequestDataColumnSidecars = maxCols
params.OverrideBeaconConfig(config)
tests := []struct {
name string
colIdents types.DataColumnsByRootIdentifiers
expectedErr error
}{
{
name: "Invalid request - multiple identifiers exceed max",
colIdents: types.DataColumnsByRootIdentifiers{
{
BlockRoot: make([]byte, fieldparams.RootLength),
Columns: make([]uint64, maxCols/2+1),
},
{
BlockRoot: make([]byte, fieldparams.RootLength),
Columns: make([]uint64, maxCols/2+1),
},
},
expectedErr: types.ErrMaxDataColumnReqExceeded,
},
{
name: "Valid request - less than max",
colIdents: types.DataColumnsByRootIdentifiers{
{
BlockRoot: make([]byte, fieldparams.RootLength),
Columns: make([]uint64, maxCols-1),
},
},
expectedErr: nil,
},
{
name: "Valid request - multiple identifiers sum to max",
colIdents: types.DataColumnsByRootIdentifiers{
{
BlockRoot: make([]byte, fieldparams.RootLength),
Columns: make([]uint64, maxCols/2),
},
{
BlockRoot: make([]byte, fieldparams.RootLength),
Columns: make([]uint64, maxCols/2),
},
},
expectedErr: nil,
},
}
// Run tests
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
err := validateDataColumnsByRootRequest(tt.colIdents)
if tt.expectedErr == nil {
require.NoError(t, err)
} else {
require.ErrorIs(t, err, tt.expectedErr)
}
})
}
}
func TestDataColumnsRPCMinValidSlot(t *testing.T) {
type testCase struct {
name string
fuluForkEpoch primitives.Epoch
minReqEpochs primitives.Epoch
currentSlot primitives.Slot
expected primitives.Slot
}
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
testCases := []testCase{
{
name: "Fulu not enabled",
fuluForkEpoch: math.MaxUint64, // Disable Fulu
minReqEpochs: 5,
currentSlot: 0,
expected: primitives.Slot(math.MaxUint64),
},
{
name: "Current epoch is before fulu fork epoch",
fuluForkEpoch: 10,
minReqEpochs: 5,
currentSlot: primitives.Slot(8 * slotsPerEpoch),
expected: primitives.Slot(10 * slotsPerEpoch),
},
{
name: "Current epoch is fulu fork epoch",
fuluForkEpoch: 10,
minReqEpochs: 5,
currentSlot: primitives.Slot(10 * slotsPerEpoch),
expected: primitives.Slot(10 * slotsPerEpoch),
},
{
name: "Current epoch between fulu fork epoch and minReqEpochs",
fuluForkEpoch: 10,
minReqEpochs: 20,
currentSlot: primitives.Slot(15 * slotsPerEpoch),
expected: primitives.Slot(10 * slotsPerEpoch),
},
{
name: "Current epoch after fulu fork epoch + minReqEpochs",
fuluForkEpoch: 10,
minReqEpochs: 5,
currentSlot: primitives.Slot(20 * slotsPerEpoch),
expected: primitives.Slot(15 * slotsPerEpoch),
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig()
config.FuluForkEpoch = tc.fuluForkEpoch
config.MinEpochsForDataColumnSidecarsRequest = tc.minReqEpochs
params.OverrideBeaconConfig(config)
actual, err := dataColumnsRPCMinValidSlot(tc.currentSlot)
require.NoError(t, err)
require.Equal(t, tc.expected, actual)
})
}
}

View File

@@ -20,8 +20,8 @@ type rpcHandlerTest struct {
s *Service
}
func (rt *rpcHandlerTest) testHandler(nh network.StreamHandler, rh rpcHandler, rhi interface{}) {
ctx, cancel := context.WithTimeout(rt.t.Context(), rt.timeout)
func (rt *rpcHandlerTest) testHandler(streamHandler network.StreamHandler, rpcHandler rpcHandler, message interface{}) {
ctx, cancel := context.WithTimeout(context.Background(), rt.timeout)
defer func() {
cancel()
}()
@@ -36,16 +36,18 @@ func (rt *rpcHandlerTest) testHandler(nh network.StreamHandler, rh rpcHandler, r
defer func() {
require.NoError(rt.t, client.Disconnect(server.PeerID()))
}()
require.Equal(rt.t, 1, len(client.BHost.Network().Peers()), "Expected peers to be connected")
h := func(stream network.Stream) {
handler := func(stream network.Stream) {
defer w.Done()
nh(stream)
streamHandler(stream)
}
server.BHost.SetStreamHandler(rt.topic, h)
server.BHost.SetStreamHandler(rt.topic, handler)
stream, err := client.BHost.NewStream(ctx, server.BHost.ID(), rt.topic)
require.NoError(rt.t, err)
err = rh(ctx, rhi, stream)
err = rpcHandler(ctx, message, stream)
if rt.err == nil {
require.NoError(rt.t, err)
} else {

View File

@@ -4,12 +4,14 @@ import (
"context"
"fmt"
"io"
"slices"
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/encoder"
p2ptypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/beacon-chain/verification"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
@@ -30,17 +32,24 @@ var errBlobUnmarshal = errors.New("Could not unmarshal chunk-encoded blob")
// Any error from the following declaration block should result in peer downscoring.
var (
// ErrInvalidFetchedData is used to signal that an error occurred which should result in peer downscoring.
ErrInvalidFetchedData = errors.New("invalid data returned from peer")
errBlobIndexOutOfBounds = errors.Wrap(verification.ErrBlobInvalid, "blob index out of range")
errMaxRequestBlobSidecarsExceeded = errors.Wrap(verification.ErrBlobInvalid, "peer exceeded req blob chunk tx limit")
errChunkResponseSlotNotAsc = errors.Wrap(verification.ErrBlobInvalid, "blob slot not higher than previous block root")
errChunkResponseIndexNotAsc = errors.Wrap(verification.ErrBlobInvalid, "blob indices for a block must start at 0 and increase by 1")
errUnrequested = errors.Wrap(verification.ErrBlobInvalid, "received BlobSidecar in response that was not requested")
errBlobResponseOutOfBounds = errors.Wrap(verification.ErrBlobInvalid, "received BlobSidecar with slot outside BlobSidecarsByRangeRequest bounds")
errChunkResponseBlockMismatch = errors.Wrap(verification.ErrBlobInvalid, "blob block details do not match")
errChunkResponseParentMismatch = errors.Wrap(verification.ErrBlobInvalid, "parent root for response element doesn't match previous element root")
ErrInvalidFetchedData = errors.New("invalid data returned from peer")
errBlobIndexOutOfBounds = errors.Wrap(verification.ErrBlobInvalid, "blob index out of range")
errMaxRequestBlobSidecarsExceeded = errors.Wrap(verification.ErrBlobInvalid, "peer exceeded req blob chunk tx limit")
errChunkResponseSlotNotAsc = errors.Wrap(verification.ErrBlobInvalid, "blob slot not higher than previous block root")
errChunkResponseIndexNotAsc = errors.Wrap(verification.ErrBlobInvalid, "blob indices for a block must start at 0 and increase by 1")
errUnrequested = errors.Wrap(verification.ErrBlobInvalid, "received BlobSidecar in response that was not requested")
errBlobResponseOutOfBounds = errors.Wrap(verification.ErrBlobInvalid, "received BlobSidecar with slot outside BlobSidecarsByRangeRequest bounds")
errChunkResponseBlockMismatch = errors.Wrap(verification.ErrBlobInvalid, "blob block details do not match")
errChunkResponseParentMismatch = errors.Wrap(verification.ErrBlobInvalid, "parent root for response element doesn't match previous element root")
errDataColumnChunkedReadFailure = errors.New("failed to read stream of chunk-encoded data columns")
errMaxRequestDataColumnSidecarsExceeded = errors.New("count of requested data column sidecars exceeds MAX_REQUEST_DATA_COLUMN_SIDECARS")
errMaxResponseDataColumnSidecarsExceeded = errors.New("peer returned more data column sidecars than requested")
)
// ------
// Blocks
// ------
// BeaconBlockProcessor defines a block processing function, which allows to start utilizing
// blocks even before all blocks are ready.
type BeaconBlockProcessor func(block interfaces.ReadOnlySignedBeaconBlock) error
@@ -154,6 +163,14 @@ func SendBeaconBlocksByRootRequest(
return blocks, nil
}
// -------------
// Blob sidecars
// -------------
// BlobResponseValidation represents a function that can validate aspects of a single unmarshaled blob sidecar
// that was received from a peer in response to an rpc request.
type BlobResponseValidation func(blocks.ROBlob) error
func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle, p2pApi p2p.SenderEncoder, pid peer.ID, ctxMap ContextByteVersions, req *ethpb.BlobSidecarsByRangeRequest, bvs ...BlobResponseValidation) ([]blocks.ROBlob, error) {
topic, err := p2p.TopicFromMessage(p2p.BlobSidecarsByRangeName, slots.ToEpoch(tor.CurrentSlot()))
if err != nil {
@@ -215,10 +232,6 @@ func SendBlobSidecarByRoot(
return readChunkEncodedBlobs(stream, p2pApi.Encoding(), ctxMap, blobValidatorFromRootReq(req), max)
}
// 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
func composeBlobValidations(vf ...BlobResponseValidation) BlobResponseValidation {
return func(blob blocks.ROBlob) error {
for i := range vf {
@@ -383,3 +396,308 @@ func readChunkedBlobSidecar(stream network.Stream, encoding encoder.NetworkEncod
return rob, nil
}
// --------------------
// Data column sidecars
// --------------------
// SendDataColumnSidecarsByRangeRequest sends a request for data column sidecars by range
// and returns the fetched data column sidecars.
func SendDataColumnSidecarsByRangeRequest(
ctx context.Context,
tor blockchain.TemporalOracle,
p2pApi p2p.P2P,
pid peer.ID,
ctxMap ContextByteVersions,
request *ethpb.DataColumnSidecarsByRangeRequest,
) ([]blocks.RODataColumn, error) {
// Return early if nothing to request.
if request == nil || request.Count == 0 || len(request.Columns) == 0 {
return nil, nil
}
beaconConfig := params.BeaconConfig()
numberOfColumns := beaconConfig.NumberOfColumns
maxRequestDataColumnSidecars := params.BeaconConfig().MaxRequestDataColumnSidecars
// Check if we do not request too many sidecars.
columnsCount := uint64(len(request.Columns))
totalCount := request.Count * columnsCount
if totalCount > maxRequestDataColumnSidecars {
return nil, errors.Wrapf(errMaxRequestDataColumnSidecarsExceeded, "requestedCount=%d, allowedCount=%d", totalCount, maxRequestDataColumnSidecars)
}
// Build the topic.
currentSlot := tor.CurrentSlot()
currentEpoch := slots.ToEpoch(currentSlot)
topic, err := p2p.TopicFromMessage(p2p.DataColumnSidecarsByRangeName, currentEpoch)
if err != nil {
return nil, errors.Wrap(err, "topic from message")
}
// Build the logs.
var columnsLog interface{} = "all"
if columnsCount < numberOfColumns {
columns := request.Columns
slices.Sort(columns)
columnsLog = columns
}
log := log.WithFields(logrus.Fields{
"peer": pid,
"topic": topic,
"startSlot": request.StartSlot,
"count": request.Count,
"columns": columnsLog,
"totalCount": totalCount,
})
// Send the request.
stream, err := p2pApi.Send(ctx, request, topic, pid)
if err != nil {
return nil, errors.Wrap(err, "p2p send")
}
defer closeStream(stream, log)
// Read the data column sidecars from the stream.
roDataColumns := make([]blocks.RODataColumn, 0, totalCount)
for range totalCount {
// Avoid reading extra chunks if the context is done.
if err := ctx.Err(); err != nil {
return nil, err
}
validatorSlotWithinBounds, err := isSidecarSlotWithinBounds(request)
if err != nil {
return nil, errors.Wrap(err, "is sidecar slot within bounds")
}
roDataColumn, err := readChunkedDataColumnSidecar(
stream, p2pApi, ctxMap,
validatorSlotWithinBounds,
isSidecarIndexRequested(request),
)
if errors.Is(err, io.EOF) {
return roDataColumns, nil
}
if err != nil {
return nil, errors.Wrap(err, "read chunked data column sidecar")
}
if roDataColumn == nil {
return nil, errors.New("nil data column sidecar, should never happen")
}
roDataColumns = append(roDataColumns, *roDataColumn)
}
// All requested sidecars were delivered by the peer. Expecting EOF.
if _, err := readChunkedDataColumnSidecar(stream, p2pApi, ctxMap); !errors.Is(err, io.EOF) {
return nil, errors.Wrapf(errMaxResponseDataColumnSidecarsExceeded, "requestedCount=%d", totalCount)
}
return roDataColumns, nil
}
// isSidecarSlotWithinBounds verifies that the slot of the data column sidecar is within the bounds of the request.
func isSidecarSlotWithinBounds(request *ethpb.DataColumnSidecarsByRangeRequest) (DataColumnResponseValidation, error) {
// endSlot is exclusive (while request.StartSlot is inclusive).
endSlot, err := request.StartSlot.SafeAdd(request.Count)
if err != nil {
return nil, errors.Wrap(err, "calculate end slot")
}
validator := func(sidecar blocks.RODataColumn) error {
slot := sidecar.Slot()
if !(request.StartSlot <= slot && slot < endSlot) {
return errors.Errorf("data column sidecar slot %d out of range [%d, %d[", slot, request.StartSlot, endSlot)
}
return nil
}
return validator, nil
}
// isSidecarIndexRequested verifies that the index of the data column sidecar is found in the requested indices.
func isSidecarIndexRequested(request *ethpb.DataColumnSidecarsByRangeRequest) DataColumnResponseValidation {
requestedIndices := make(map[uint64]bool)
for _, col := range request.Columns {
requestedIndices[col] = true
}
return func(sidecar blocks.RODataColumn) error {
columnIndex := sidecar.Index
if !requestedIndices[columnIndex] {
return errors.Errorf("data column sidecar index %d not found in requested indices", columnIndex)
}
return nil
}
}
// SendDataColumnSidecarsByRootRequest sends a request for data column sidecars by root
// and returns the fetched data column sidecars.
func SendDataColumnSidecarsByRootRequest(
ctx context.Context,
tor blockchain.TemporalOracle,
p2pApi p2p.P2P,
pid peer.ID,
ctxMap ContextByteVersions,
request p2ptypes.DataColumnsByRootIdentifiers,
) ([]blocks.RODataColumn, error) {
// Return early if the request is nil.
if request == nil {
return nil, nil
}
// Compute how many sidecars are requested.
count := uint64(0)
for _, identifier := range request {
count += uint64(len(identifier.Columns))
}
// Return early if nothing to request.
if count == 0 {
return nil, nil
}
// Verify that the request count is within the maximum allowed.
maxRequestDataColumnSidecars := params.BeaconConfig().MaxRequestDataColumnSidecars
if count > maxRequestDataColumnSidecars {
return nil, errors.Wrapf(errMaxRequestDataColumnSidecarsExceeded, "current: %d, max: %d", count, maxRequestDataColumnSidecars)
}
// Get the topic for the request.
topic, err := p2p.TopicFromMessage(p2p.DataColumnSidecarsByRootName, slots.ToEpoch(tor.CurrentSlot()))
if err != nil {
return nil, errors.Wrap(err, "topic from message")
}
// Send the request to the peer.
stream, err := p2pApi.Send(ctx, request, topic, pid)
if err != nil {
return nil, errors.Wrap(err, "p2p api send")
}
defer closeStream(stream, log)
// Read the data column sidecars from the stream.
roDataColumns := make([]blocks.RODataColumn, 0, count)
// Read the data column sidecars from the stream.
for range count {
roDataColumn, err := readChunkedDataColumnSidecar(stream, p2pApi, ctxMap, isSidecarIndexRootRequested(request))
if errors.Is(err, io.EOF) {
return roDataColumns, nil
}
if err != nil {
return nil, errors.Wrap(err, "read chunked data column sidecar")
}
if roDataColumn == nil {
return nil, errors.Wrap(err, "nil data column sidecar, should never happen")
}
roDataColumns = append(roDataColumns, *roDataColumn)
}
// All requested sidecars were delivered by the peer. Expecting EOF.
if _, err := readChunkedDataColumnSidecar(stream, p2pApi, ctxMap); !errors.Is(err, io.EOF) {
return nil, errors.Wrapf(errMaxResponseDataColumnSidecarsExceeded, "requestedCount=%d", count)
}
return roDataColumns, nil
}
func isSidecarIndexRootRequested(request p2ptypes.DataColumnsByRootIdentifiers) DataColumnResponseValidation {
columnsIndexFromRoot := make(map[[fieldparams.RootLength]byte]map[uint64]bool)
for _, sidecar := range request {
blockRoot := bytesutil.ToBytes32(sidecar.BlockRoot)
if columnsIndexFromRoot[blockRoot] == nil {
columnsIndexFromRoot[blockRoot] = make(map[uint64]bool)
}
for _, column := range sidecar.Columns {
columnsIndexFromRoot[blockRoot][column] = true
}
}
return func(sidecar blocks.RODataColumn) error {
root, index := sidecar.BlockRoot(), sidecar.Index
indices, ok := columnsIndexFromRoot[root]
if !ok {
return errors.Errorf("root #%x returned by peer but not requested", root)
}
if !indices[index] {
return errors.Errorf("index %d for root #%x returned by peer but not requested", index, root)
}
return nil
}
}
// DataColumnResponseValidation represents a function that can validate aspects of a single unmarshaled data column sidecar
// that was received from a peer in response to an rpc request.
type DataColumnResponseValidation func(column blocks.RODataColumn) error
func readChunkedDataColumnSidecar(
stream network.Stream,
p2pApi p2p.P2P,
ctxMap ContextByteVersions,
validationFunctions ...DataColumnResponseValidation,
) (*blocks.RODataColumn, error) {
// Read the status code from the stream.
statusCode, errMessage, err := ReadStatusCode(stream, p2pApi.Encoding())
if err != nil {
return nil, errors.Wrap(err, "read status code")
}
if statusCode != 0 {
return nil, errors.Wrap(errDataColumnChunkedReadFailure, errMessage)
}
// Retrieve the fork digest.
ctxBytes, err := readContextFromStream(stream)
if err != nil {
return nil, errors.Wrap(err, "read context from stream")
}
// Check if the fork digest is recognized.
msgVersion, ok := ctxMap[bytesutil.ToBytes4(ctxBytes)]
if !ok {
return nil, errors.Errorf("unrecognized fork digest %#x", ctxBytes)
}
// Check if we are on Fulu.
if msgVersion < version.Fulu {
return nil, errors.Errorf(
"unexpected context bytes for DataColumnSidecar, ctx=%#x, msgVersion=%v, minimalSupportedVersion=%v",
ctxBytes, version.String(msgVersion), version.String(version.Fulu),
)
}
// Decode the data column sidecar from the stream.
dataColumnSidecar := new(ethpb.DataColumnSidecar)
if err := p2pApi.Encoding().DecodeWithMaxLength(stream, dataColumnSidecar); err != nil {
return nil, errors.Wrap(err, "failed to decode the protobuf-encoded BlobSidecar message from RPC chunk stream")
}
// Create a read-only data column from the data column sidecar.
roDataColumn, err := blocks.NewRODataColumn(dataColumnSidecar)
if err != nil {
return nil, errors.Wrap(err, "new read only data column")
}
// Run validation functions.
for _, validationFunction := range validationFunctions {
if err := validationFunction(roDataColumn); err != nil {
return nil, errors.Wrap(err, "validation function")
}
}
return &roDataColumn, nil
}

View File

@@ -5,12 +5,15 @@ import (
"errors"
"fmt"
"io"
"sync"
"testing"
"time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
p2pTypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
p2ptypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v6/beacon-chain/verification"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
@@ -20,6 +23,7 @@ import (
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/assert"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
@@ -882,3 +886,745 @@ func TestSendBlobsByRangeRequest(t *testing.T) {
func TestErrInvalidFetchedDataDistinction(t *testing.T) {
require.Equal(t, false, errors.Is(ErrInvalidFetchedData, verification.ErrBlobInvalid))
}
func TestSendDataColumnSidecarsByRangeRequest(t *testing.T) {
nilTestCases := []struct {
name string
request *ethpb.DataColumnSidecarsByRangeRequest
}{
{
name: "nil request",
request: nil,
},
{
name: "count is 0",
request: &ethpb.DataColumnSidecarsByRangeRequest{},
},
{
name: "columns is nil",
request: &ethpb.DataColumnSidecarsByRangeRequest{Count: 1},
},
}
for _, tc := range nilTestCases {
t.Run(tc.name, func(t *testing.T) {
actual, err := SendDataColumnSidecarsByRangeRequest(t.Context(), nil, nil, "aRandomPID", nil, tc.request)
require.NoError(t, err)
require.IsNil(t, actual)
})
}
t.Run("too many columns in request", func(t *testing.T) {
params.SetupTestConfigCleanup(t)
beaconConfig := params.BeaconConfig()
beaconConfig.MaxRequestDataColumnSidecars = 0
params.OverrideBeaconConfig(beaconConfig)
request := &ethpb.DataColumnSidecarsByRangeRequest{Count: 1, Columns: []uint64{1, 2, 3}}
_, err := SendDataColumnSidecarsByRangeRequest(t.Context(), nil, nil, "aRandomPID", nil, request)
require.ErrorContains(t, errMaxRequestDataColumnSidecarsExceeded.Error(), err)
})
type slotIndex struct {
Slot primitives.Slot
Index uint64
}
createSidecar := func(slotIndex slotIndex) *ethpb.DataColumnSidecar {
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
return &ethpb.DataColumnSidecar{
Index: slotIndex.Index,
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: slotIndex.Slot,
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
}
testCases := []struct {
name string
slotIndices []slotIndex
expectedError error
}{
{
name: "too many responses",
slotIndices: []slotIndex{
{Slot: 0, Index: 1},
{Slot: 0, Index: 2},
{Slot: 0, Index: 3},
{Slot: 1, Index: 1},
{Slot: 1, Index: 2},
{Slot: 1, Index: 3},
{Slot: 0, Index: 3}, // Duplicate
},
expectedError: errMaxResponseDataColumnSidecarsExceeded,
},
{
name: "perfect match",
slotIndices: []slotIndex{
{Slot: 0, Index: 1},
{Slot: 0, Index: 2},
{Slot: 0, Index: 3},
{Slot: 1, Index: 1},
{Slot: 1, Index: 2},
{Slot: 1, Index: 3},
},
},
{
name: "few responses than maximum possible",
slotIndices: []slotIndex{
{Slot: 0, Index: 1},
{Slot: 0, Index: 2},
{Slot: 0, Index: 3},
{Slot: 1, Index: 1},
{Slot: 1, Index: 2},
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
protocol := fmt.Sprintf("%s/ssz_snappy", p2p.RPCDataColumnSidecarsByRangeTopicV1)
clock := startup.NewClock(time.Now(), [fieldparams.RootLength]byte{})
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
p1.Connect(p2)
expected := make([]*ethpb.DataColumnSidecar, 0, len(tc.slotIndices))
for _, slotIndex := range tc.slotIndices {
sidecar := createSidecar(slotIndex)
expected = append(expected, sidecar)
}
requestSent := &ethpb.DataColumnSidecarsByRangeRequest{
StartSlot: 0,
Count: 2,
Columns: []uint64{1, 3, 2},
}
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(protocol, func(stream network.Stream) {
wg.Done()
requestReceived := new(ethpb.DataColumnSidecarsByRangeRequest)
err := p2.Encoding().DecodeWithMaxLength(stream, requestReceived)
assert.NoError(t, err)
assert.DeepSSZEqual(t, requestSent, requestReceived)
for _, sidecar := range expected {
err := WriteDataColumnSidecarChunk(stream, clock, p2.Encoding(), sidecar)
assert.NoError(t, err)
}
err = stream.CloseWrite()
assert.NoError(t, err)
})
ctx := t.Context()
ctxMap := ContextByteVersions{[4]byte{245, 165, 253, 66}: version.Fulu}
actual, err := SendDataColumnSidecarsByRangeRequest(ctx, clock, p1, p2.PeerID(), ctxMap, requestSent)
if tc.expectedError != nil {
require.ErrorContains(t, tc.expectedError.Error(), err)
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
return
}
require.Equal(t, len(expected), len(actual))
for i := range expected {
require.DeepSSZEqual(t, expected[i], actual[i].DataColumnSidecar)
}
})
}
}
func TestIsSidecarSlotWithinBounds(t *testing.T) {
request := &ethpb.DataColumnSidecarsByRangeRequest{
StartSlot: 10,
Count: 10,
}
validator, err := isSidecarSlotWithinBounds(request)
require.NoError(t, err)
testCases := []struct {
name string
slot primitives.Slot
isErrorExpected bool
}{
{
name: "too soon",
slot: 9,
isErrorExpected: true,
},
{
name: "too late",
slot: 20,
isErrorExpected: true,
},
{
name: "within bounds",
slot: 15,
isErrorExpected: false,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
sidecarPb := &ethpb.DataColumnSidecar{
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: tc.slot,
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
sidecar, err := blocks.NewRODataColumn(sidecarPb)
require.NoError(t, err)
err = validator(sidecar)
if tc.isErrorExpected {
require.NotNil(t, err)
return
}
require.NoError(t, err)
})
}
}
func TestIsSidecarIndexRequested(t *testing.T) {
request := &ethpb.DataColumnSidecarsByRangeRequest{
Columns: []uint64{2, 9, 4},
}
validator := isSidecarIndexRequested(request)
testCases := []struct {
name string
index uint64
isErrorExpected bool
}{
{
name: "not requested",
index: 1,
isErrorExpected: true,
},
{
name: "requested",
index: 9,
isErrorExpected: false,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
sidecarPb := &ethpb.DataColumnSidecar{
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: 0,
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
Index: tc.index,
}
sidecar, err := blocks.NewRODataColumn(sidecarPb)
require.NoError(t, err)
err = validator(sidecar)
if tc.isErrorExpected {
require.NotNil(t, err)
return
}
require.NoError(t, err)
})
}
}
func TestSendDataColumnSidecarsByRootRequest(t *testing.T) {
nilTestCases := []struct {
name string
request p2ptypes.DataColumnsByRootIdentifiers
}{
{
name: "nil request",
request: nil,
},
{
name: "count is 0",
request: p2ptypes.DataColumnsByRootIdentifiers{{}, {}},
},
}
for _, tc := range nilTestCases {
t.Run(tc.name, func(t *testing.T) {
actual, err := SendDataColumnSidecarsByRootRequest(t.Context(), nil, nil, "aRandomPID", nil, tc.request)
require.NoError(t, err)
require.IsNil(t, actual)
})
}
t.Run("too many columns in request", func(t *testing.T) {
params.SetupTestConfigCleanup(t)
beaconConfig := params.BeaconConfig()
beaconConfig.MaxRequestDataColumnSidecars = 4
params.OverrideBeaconConfig(beaconConfig)
request := p2ptypes.DataColumnsByRootIdentifiers{
{Columns: []uint64{1, 2, 3}},
{Columns: []uint64{4, 5, 6}},
}
_, err := SendDataColumnSidecarsByRootRequest(t.Context(), nil, nil, "aRandomPID", nil, request)
require.ErrorContains(t, errMaxRequestDataColumnSidecarsExceeded.Error(), err)
})
type slotIndex struct {
Slot primitives.Slot
Index uint64
}
createSidecar := func(rootIndex slotIndex) blocks.RODataColumn {
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
sidecarPb := &ethpb.DataColumnSidecar{
Index: rootIndex.Index,
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
roSidecar, err := blocks.NewRODataColumn(sidecarPb)
require.NoError(t, err)
return roSidecar
}
testCases := []struct {
name string
slotIndices []slotIndex
expectedError error
}{
{
name: "too many responses",
slotIndices: []slotIndex{
{Slot: 1, Index: 1},
{Slot: 1, Index: 2},
{Slot: 1, Index: 3},
{Slot: 2, Index: 1},
{Slot: 2, Index: 2},
{Slot: 2, Index: 3},
{Slot: 1, Index: 3}, // Duplicate
},
expectedError: errMaxResponseDataColumnSidecarsExceeded,
},
{
name: "perfect match",
slotIndices: []slotIndex{
{Slot: 1, Index: 1},
{Slot: 1, Index: 2},
{Slot: 1, Index: 3},
{Slot: 2, Index: 1},
{Slot: 2, Index: 2},
{Slot: 2, Index: 3},
},
},
{
name: "few responses than maximum possible",
slotIndices: []slotIndex{
{Slot: 1, Index: 1},
{Slot: 1, Index: 2},
{Slot: 1, Index: 3},
{Slot: 2, Index: 1},
{Slot: 2, Index: 2},
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
protocol := fmt.Sprintf("%s/ssz_snappy", p2p.RPCDataColumnSidecarsByRootTopicV1)
clock := startup.NewClock(time.Now(), [fieldparams.RootLength]byte{})
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
p1.Connect(p2)
expected := make([]blocks.RODataColumn, 0, len(tc.slotIndices))
for _, slotIndex := range tc.slotIndices {
roSidecar := createSidecar(slotIndex)
expected = append(expected, roSidecar)
}
blockRoot1, blockRoot2 := expected[0].BlockRoot(), expected[3].BlockRoot()
sentRequest := p2ptypes.DataColumnsByRootIdentifiers{
{BlockRoot: blockRoot1[:], Columns: []uint64{1, 2, 3}},
{BlockRoot: blockRoot2[:], Columns: []uint64{1, 2, 3}},
}
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(protocol, func(stream network.Stream) {
wg.Done()
requestReceived := new(p2ptypes.DataColumnsByRootIdentifiers)
err := p2.Encoding().DecodeWithMaxLength(stream, requestReceived)
assert.NoError(t, err)
require.Equal(t, len(sentRequest), len(*requestReceived))
for i := range sentRequest {
require.DeepSSZEqual(t, (sentRequest)[i], (*requestReceived)[i])
}
for _, sidecar := range expected {
err := WriteDataColumnSidecarChunk(stream, clock, p2.Encoding(), sidecar.DataColumnSidecar)
assert.NoError(t, err)
}
err = stream.CloseWrite()
assert.NoError(t, err)
})
ctx := t.Context()
ctxMap := ContextByteVersions{[4]byte{245, 165, 253, 66}: version.Fulu}
actual, err := SendDataColumnSidecarsByRootRequest(ctx, clock, p1, p2.PeerID(), ctxMap, sentRequest)
if tc.expectedError != nil {
require.ErrorContains(t, tc.expectedError.Error(), err)
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
return
}
require.Equal(t, len(expected), len(actual))
for i := range expected {
require.DeepSSZEqual(t, expected[i], actual[i])
}
})
}
}
func TestIsSidecarIndexRootRequested(t *testing.T) {
testCases := []struct {
name string
root [fieldparams.RootLength]byte
index uint64
isErrorExpected bool
}{
{
name: "non requested root",
root: [fieldparams.RootLength]byte{2},
isErrorExpected: true,
},
{
name: "non requested index",
root: [fieldparams.RootLength]byte{1},
index: 3,
isErrorExpected: true,
},
{
name: "nominal",
root: [fieldparams.RootLength]byte{1},
index: 2,
isErrorExpected: false,
},
}
request := types.DataColumnsByRootIdentifiers{
{BlockRoot: []byte{1}, Columns: []uint64{1, 2}},
}
validator := isSidecarIndexRootRequested(request)
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
sidecarPb := &ethpb.DataColumnSidecar{
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
Index: tc.index,
}
// There is a discrepancy between `tc.root` and the real root,
// but we don't care about it here.
sidecar, err := blocks.NewRODataColumnWithRoot(sidecarPb, tc.root)
require.NoError(t, err)
err = validator(sidecar)
if tc.isErrorExpected {
require.NotNil(t, err)
return
}
require.NoError(t, err)
})
}
}
func TestReadChunkedDataColumnSidecar(t *testing.T) {
t.Run("non nil status code", func(t *testing.T) {
const reason = "a dummy reason"
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(p2p.RPCDataColumnSidecarsByRootTopicV1, func(stream network.Stream) {
defer wg.Done()
_, err := readChunkedDataColumnSidecar(stream, p2, nil)
require.ErrorContains(t, reason, err)
})
p1.Connect(p2)
stream, err := p1.BHost.NewStream(t.Context(), p2.PeerID(), p2p.RPCDataColumnSidecarsByRootTopicV1)
require.NoError(t, err)
writeErrorResponseToStream(responseCodeInvalidRequest, reason, stream, p1)
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
})
t.Run("unrecognized fork digest", func(t *testing.T) {
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(p2p.RPCDataColumnSidecarsByRootTopicV1, func(stream network.Stream) {
defer wg.Done()
_, err := readChunkedDataColumnSidecar(stream, p2, ContextByteVersions{})
require.ErrorContains(t, "unrecognized fork digest", err)
})
p1.Connect(p2)
stream, err := p1.BHost.NewStream(t.Context(), p2.PeerID(), p2p.RPCDataColumnSidecarsByRootTopicV1)
require.NoError(t, err)
_, err = stream.Write([]byte{responseCodeSuccess})
require.NoError(t, err)
err = writeContextToStream([]byte{42, 42, 42, 42}, stream)
require.NoError(t, err)
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
})
t.Run("before fulu", func(t *testing.T) {
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(p2p.RPCDataColumnSidecarsByRootTopicV1, func(stream network.Stream) {
defer wg.Done()
_, err := readChunkedDataColumnSidecar(stream, p2, ContextByteVersions{[4]byte{1, 2, 3, 4}: version.Phase0})
require.ErrorContains(t, "unexpected context bytes", err)
})
p1.Connect(p2)
stream, err := p1.BHost.NewStream(t.Context(), p2.PeerID(), p2p.RPCDataColumnSidecarsByRootTopicV1)
require.NoError(t, err)
_, err = stream.Write([]byte{responseCodeSuccess})
require.NoError(t, err)
err = writeContextToStream([]byte{1, 2, 3, 4}, stream)
require.NoError(t, err)
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
})
t.Run("one validation failed", func(t *testing.T) {
const reason = "a dummy reason"
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(p2p.RPCDataColumnSidecarsByRootTopicV1, func(stream network.Stream) {
defer wg.Done()
validationOne := func(column blocks.RODataColumn) error {
return nil
}
validationTwo := func(column blocks.RODataColumn) error {
return errors.New(reason)
}
_, err := readChunkedDataColumnSidecar(
stream,
p2,
ContextByteVersions{[4]byte{1, 2, 3, 4}: version.Fulu},
validationOne, // OK
validationTwo, // Fail
)
require.ErrorContains(t, reason, err)
})
p1.Connect(p2)
stream, err := p1.BHost.NewStream(t.Context(), p2.PeerID(), p2p.RPCDataColumnSidecarsByRootTopicV1)
require.NoError(t, err)
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
// Success response code.
_, err = stream.Write([]byte{responseCodeSuccess})
require.NoError(t, err)
// Fork digest.
err = writeContextToStream([]byte{1, 2, 3, 4}, stream)
require.NoError(t, err)
// Sidecar.
_, err = p1.Encoding().EncodeWithMaxLength(stream, &ethpb.DataColumnSidecar{
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
})
require.NoError(t, err)
if util.WaitTimeout(&wg, time.Minute) {
t.Fatal("Did not receive stream within 1 sec")
}
})
t.Run("nominal", func(t *testing.T) {
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
const count = 4
kzgCommitmentsInclusionProof := make([][]byte, 0, count)
for range count {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, 32))
}
expected := &ethpb.DataColumnSidecar{
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
var wg sync.WaitGroup
wg.Add(1)
p2.SetStreamHandler(p2p.RPCDataColumnSidecarsByRootTopicV1, func(stream network.Stream) {
defer wg.Done()
actual, err := readChunkedDataColumnSidecar(stream, p2, ContextByteVersions{[4]byte{1, 2, 3, 4}: version.Fulu})
require.NoError(t, err)
require.DeepSSZEqual(t, expected, actual.DataColumnSidecar)
})
p1.Connect(p2)
stream, err := p1.BHost.NewStream(t.Context(), p2.PeerID(), p2p.RPCDataColumnSidecarsByRootTopicV1)
require.NoError(t, err)
// Success response code.
_, err = stream.Write([]byte{responseCodeSuccess})
require.NoError(t, err)
// Fork digest.
err = writeContextToStream([]byte{1, 2, 3, 4}, stream)
require.NoError(t, err)
// Sidecar.
_, err = p1.Encoding().EncodeWithMaxLength(stream, expected)
require.NoError(t, err)
if util.WaitTimeout(&wg, time.Minute) {
t.Fatal("Did not receive stream within 1 sec")
}
})
}

View File

@@ -102,6 +102,7 @@ type config struct {
clock *startup.Clock
stateNotifier statefeed.Notifier
blobStorage *filesystem.BlobStorage
dataColumnStorage *filesystem.DataColumnStorage
}
// This defines the interface for interacting with block chain service

View File

@@ -15,10 +15,12 @@ func TestMain(m *testing.M) {
resetFlags := flags.Get()
flags.Init(&flags.GlobalFlags{
BlockBatchLimit: 64,
BlockBatchLimitBurstFactor: 10,
BlobBatchLimit: 32,
BlobBatchLimitBurstFactor: 2,
BlockBatchLimit: 64,
BlockBatchLimitBurstFactor: 10,
BlobBatchLimit: 32,
BlobBatchLimitBurstFactor: 2,
DataColumnBatchLimit: 4096,
DataColumnBatchLimitBurstFactor: 2,
})
defer func() {
flags.Init(resetFlags)

View File

@@ -30,16 +30,16 @@ var (
ErrBlobIndexInvalid = errors.Join(ErrBlobInvalid, errors.New("incorrect blob sidecar index"))
// errFromFutureSlot means RequireSlotNotTooEarly failed.
errFromFutureSlot = errors.Join(ErrBlobInvalid, errors.New("slot is too far in the future"))
errFromFutureSlot = errors.New("slot is too far in the future")
// errSlotNotAfterFinalized means RequireSlotAboveFinalized failed.
errSlotNotAfterFinalized = errors.Join(ErrBlobInvalid, errors.New("slot <= finalized checkpoint"))
errSlotNotAfterFinalized = errors.New("slot <= finalized checkpoint")
// ErrInvalidProposerSignature means RequireValidProposerSignature failed.
ErrInvalidProposerSignature = errors.Join(ErrBlobInvalid, errors.New("proposer signature could not be verified"))
// errSidecarParentNotSeen means RequireSidecarParentSeen failed.
errSidecarParentNotSeen = errors.Join(ErrBlobInvalid, errors.New("parent root has not been seen"))
errSidecarParentNotSeen = errors.New("parent root has not been seen")
// errSidecarParentInvalid means RequireSidecarParentValid failed.
errSidecarParentInvalid = errors.Join(ErrBlobInvalid, errors.New("parent block is not valid"))

View File

@@ -0,0 +1,7 @@
### Added
- New ssz-only flag for validator client to enable calling rest apis in SSZ, starting with get block endpoint.
### Changed
- when REST api is enabled the get Block api defaults to requesting and receiving SSZ instead of JSON, JSON is the fallback.

View File

@@ -0,0 +1,2 @@
## Added
- Methods to generically encode/decode independent lists of ssz values.

View File

@@ -0,0 +1,2 @@
### Added
- Implement beacon API blob sidecar enpoint for Fulu.

View File

@@ -0,0 +1,2 @@
### Added
- Implement `dataColumnSidecarsByRangeRPCHandler`.

View File

@@ -0,0 +1,2 @@
### Added
- Implement `dataColumnSidecarByRootRPCHandler`.

View File

@@ -0,0 +1,2 @@
### Fixed
- Non deterministic output order of `dataColumnSidecarByRootRPCHandler`.

View File

@@ -0,0 +1,3 @@
### Added
- Implement `SendDataColumnSidecarsByRangeRequest`.
- Implement `SendDataColumnSidecarsByRootRequest`.

View File

@@ -0,0 +1,9 @@
### Changed
- In `TopicFromMessage`: Do not assume anymore that all Fulu specific topic are V3 only.
- `readChunkedDataColumnSidecar`: Add `validationFunctions` parameter and add tests.
### Added
- New `StatusV2` proto message.
### Removed
- Unused `DataColumnIdentifier` proto message.

3
changelog/tt_milk.md Normal file
View File

@@ -0,0 +1,3 @@
### Removed
- Remove deposit count from sync new block log

3
changelog/tt_steak.md Normal file
View File

@@ -0,0 +1,3 @@
### Changed
- Remove "invalid" from logs for incoming blob sidecar that is missing parent or out of range slot

View File

@@ -212,6 +212,18 @@ var (
Usage: "The factor by which blob batch limit may increase on burst.",
Value: 3,
}
// DataColumnBatchLimit specifies the requested data column batch size.
DataColumnBatchLimit = &cli.IntFlag{
Name: "data-column-batch-limit",
Usage: "The amount of data columns the local peer is bounded to request and respond to in a batch.",
Value: 4096,
}
// DataColumnBatchLimitBurstFactor specifies the factor by which data column batch size may increase.
DataColumnBatchLimitBurstFactor = &cli.IntFlag{
Name: "data-column-batch-limit-burst-factor",
Usage: "The factor by which data column batch limit may increase on burst.",
Value: 2,
}
// DisableDebugRPCEndpoints disables the debug Beacon API namespace.
DisableDebugRPCEndpoints = &cli.BoolFlag{
Name: "disable-debug-rpc-endpoints",

View File

@@ -8,15 +8,17 @@ import (
// GlobalFlags specifies all the global flags for the
// beacon node.
type GlobalFlags struct {
SubscribeToAllSubnets bool
SubscribeAllDataSubnets bool
MinimumSyncPeers int
MinimumPeersPerSubnet int
MaxConcurrentDials int
BlockBatchLimit int
BlockBatchLimitBurstFactor int
BlobBatchLimit int
BlobBatchLimitBurstFactor int
SubscribeToAllSubnets bool
SubscribeAllDataSubnets bool
MinimumSyncPeers int
MinimumPeersPerSubnet int
MaxConcurrentDials int
BlockBatchLimit int
BlockBatchLimitBurstFactor int
BlobBatchLimit int
BlobBatchLimitBurstFactor int
DataColumnBatchLimit int
DataColumnBatchLimitBurstFactor int
}
var globalConfig *GlobalFlags
@@ -53,8 +55,11 @@ func ConfigureGlobalFlags(ctx *cli.Context) {
cfg.BlockBatchLimitBurstFactor = ctx.Int(BlockBatchLimitBurstFactor.Name)
cfg.BlobBatchLimit = ctx.Int(BlobBatchLimit.Name)
cfg.BlobBatchLimitBurstFactor = ctx.Int(BlobBatchLimitBurstFactor.Name)
cfg.DataColumnBatchLimit = ctx.Int(DataColumnBatchLimit.Name)
cfg.DataColumnBatchLimitBurstFactor = ctx.Int(DataColumnBatchLimitBurstFactor.Name)
cfg.MinimumPeersPerSubnet = ctx.Int(MinPeersPerSubnet.Name)
cfg.MaxConcurrentDials = ctx.Int(MaxConcurrentDials.Name)
configureMinimumPeers(ctx, cfg)
Init(cfg)

View File

@@ -14,7 +14,6 @@ import (
)
var (
// BlobStoragePathFlag defines a flag to start the beacon chain from a give genesis state file.
BlobStoragePathFlag = &cli.PathFlag{
Name: "blob-path",
Usage: "Location for blob storage. Default location will be a 'blobs' directory next to the beacon db.",
@@ -30,6 +29,10 @@ var (
Usage: layoutFlagUsage(),
Value: filesystem.LayoutNameFlat,
}
DataColumnStoragePathFlag = &cli.PathFlag{
Name: "data-column-path",
Usage: "Location for data column storage. Default location will be a 'data-columns' directory next to the beacon db.",
}
)
func layoutOptions() string {
@@ -54,15 +57,23 @@ func validateLayoutFlag(_ *cli.Context, v string) error {
// create a cancellable context. If we switch to using App.RunContext, we can set up this cancellation in the cmd
// package instead, and allow the functional options to tap into context cancellation.
func BeaconNodeOptions(c *cli.Context) ([]node.Option, error) {
e, err := blobRetentionEpoch(c)
blobRetentionEpoch, err := blobRetentionEpoch(c)
if err != nil {
return nil, err
return nil, errors.Wrap(err, "blob retention epoch")
}
opts := []node.Option{node.WithBlobStorageOptions(
filesystem.WithBlobRetentionEpochs(e),
blobStorageOptions := node.WithBlobStorageOptions(
filesystem.WithBlobRetentionEpochs(blobRetentionEpoch),
filesystem.WithBasePath(blobStoragePath(c)),
filesystem.WithLayout(c.String(BlobStorageLayout.Name)), // This is validated in the Action func for BlobStorageLayout.
)}
)
dataColumnStorageOption := node.WithDataColumnStorageOptions(
filesystem.WithDataColumnRetentionEpochs(blobRetentionEpoch),
filesystem.WithDataColumnBasePath(dataColumnStoragePath(c)),
)
opts := []node.Option{blobStorageOptions, dataColumnStorageOption}
return opts, nil
}
@@ -75,6 +86,16 @@ func blobStoragePath(c *cli.Context) string {
return blobsPath
}
func dataColumnStoragePath(c *cli.Context) string {
dataColumnsPath := c.Path(DataColumnStoragePathFlag.Name)
if dataColumnsPath == "" {
// append a "data-columns" subdir to the end of the data dir path
dataColumnsPath = path.Join(c.String(cmd.DataDirFlag.Name), "data-columns")
}
return dataColumnsPath
}
var errInvalidBlobRetentionEpochs = errors.New("value is smaller than spec minimum")
// blobRetentionEpoch returns the spec default MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUEST

View File

@@ -52,6 +52,7 @@ type Flags struct {
EnableExperimentalAttestationPool bool // EnableExperimentalAttestationPool enables an experimental attestation pool design.
EnableDutiesV2 bool // EnableDutiesV2 sets validator client to use the get Duties V2 endpoint
EnableWeb bool // EnableWeb enables the webui on the validator client
SSZOnly bool // SSZOnly forces the validator client to use SSZ for communication with the beacon node when REST mode is enabled (useful for debugging)
// Logging related toggles.
DisableGRPCConnectionLogs bool // Disables logging when a new grpc client has connected.
EnableFullSSZDataLogging bool // Enables logging for full ssz data on rejected gossip messages
@@ -344,6 +345,11 @@ func ConfigureValidator(ctx *cli.Context) error {
logEnabled(EnableWebFlag)
cfg.EnableWeb = true
}
if ctx.Bool(SSZOnly.Name) {
logEnabled(SSZOnly)
cfg.SSZOnly = true
}
cfg.KeystoreImportDebounceInterval = ctx.Duration(dynamicKeyReloadDebounceInterval.Name)
Init(cfg)
return nil

View File

@@ -201,6 +201,12 @@ var (
Usage: "(Work in progress): Enables the web portal for the validator client.",
Value: false,
}
// SSZOnly forces the validator client to use SSZ for communication with the beacon node when REST mode is enabled
SSZOnly = &cli.BoolFlag{
Name: "ssz-only",
Usage: "(debug): Forces the validator client to use SSZ for communication with the beacon node when REST mode is enabled",
}
)
// devModeFlags holds list of flags that are set when development mode is on.
@@ -223,6 +229,7 @@ var ValidatorFlags = append(deprecatedFlags, []cli.Flag{
EnableBeaconRESTApi,
EnableDutiesV2,
EnableWebFlag,
SSZOnly,
}...)
// E2EValidatorFlags contains a list of the validator feature flags to be tested in E2E.

View File

@@ -3,9 +3,11 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"errors.go",
"hashers.go",
"helpers.go",
"htrutils.go",
"list.go",
"merkleize.go",
"slice_root.go",
],

17
encoding/ssz/errors.go Normal file
View File

@@ -0,0 +1,17 @@
package ssz
import "github.com/pkg/errors"
var (
ErrInvalidEncodingLength = errors.New("invalid encoded length")
ErrInvalidFixedEncodingLen = errors.Wrap(ErrInvalidEncodingLength, "not multiple of fixed size")
ErrEncodingSmallerThanOffset = errors.Wrap(ErrInvalidEncodingLength, "smaller than a single offset")
ErrInvalidOffset = errors.New("invalid offset")
ErrOffsetIntoFixed = errors.Wrap(ErrInvalidOffset, "does not point past fixed section of encoding")
ErrOffsetExceedsBuffer = errors.Wrap(ErrInvalidOffset, "exceeds buffer length")
ErrNegativeRelativeOffset = errors.Wrap(ErrInvalidOffset, "less than previous offset")
ErrOffsetInsufficient = errors.Wrap(ErrInvalidOffset, "insufficient difference relative to previous")
ErrOffsetSectionMisaligned = errors.Wrap(ErrInvalidOffset, "offset bytes are not a multiple of offset size")
ErrOffsetDecodedMismatch = errors.New("unmarshaled size does not relative offsets")
)

210
encoding/ssz/list.go Normal file
View File

@@ -0,0 +1,210 @@
package ssz
import (
"encoding/binary"
"math"
"github.com/pkg/errors"
)
const offsetLen = 4 // each list element offset is a 4-byte uint32
type Marshalable interface {
MarshalSSZTo(buf []byte) ([]byte, error)
SizeSSZ() int
}
type Unmarshalable interface {
UnmarshalSSZ(buf []byte) error
SizeSSZ() int
}
// MarshalListFixedElement encodes a slice of fixed sized elements as an ssz list.
// A list of fixed-size elements is marshaled by concatenating the marshaled bytes
// of each element in the list.
//
// MarshalListVariableElement should be used for variable-sized elements.
// SSZ Lists have different encoding rules depending whether their elements are fixed- or variable-sized,
// and we can't differentiate them by the ssz interface, so it is the caller's responsibility to
// pick the correct method.
func MarshalListFixedElement[T Marshalable](elems []T) ([]byte, error) {
if len(elems) == 0 {
return nil, nil
}
size := elems[0].SizeSSZ()
buf := make([]byte, 0, len(elems)*size)
for _, elem := range elems {
if elem.SizeSSZ() != size {
return nil, ErrInvalidFixedEncodingLen
}
var err error
buf, err = elem.MarshalSSZTo(buf)
if err != nil {
return nil, errors.Wrap(err, "marshal ssz")
}
}
return buf, nil
}
// MarshalListVariableElement marshals a list of variable-sized elements.
// A list of variable-sized elements is marshaled by first writing the offsets of each element to the
// beginning of the byte sequence (the fixed size section of the variable sized list container), followed
// by the encoded values of each element at the indicated offset relative to the beginning of the byte sequence.
//
// MarshalListFixedElement should be used for fixed-size elements.
// SSZ Lists have different encoding rules depending whether their elements are fixed- or variable-sized,
// and we can't differentiate them by the ssz interface, so it is the caller's responsibility to
// pick the correct method.
func MarshalListVariableElement[T Marshalable](elems []T) ([]byte, error) {
var err error
var total uint32
nElems := len(elems)
if nElems == 0 {
return nil, nil
}
sizes := make([]uint32, nElems)
for i, e := range elems {
sizes[i], err = safeUint32(e.SizeSSZ())
if err != nil {
return nil, err
}
total += sizes[i]
}
nextOffset, err := safeUint32(nElems * offsetLen)
if err != nil {
return nil, err
}
buf := make([]byte, 0, total+nextOffset)
for _, size := range sizes {
buf = binary.LittleEndian.AppendUint32(buf, nextOffset)
nextOffset += size
}
for _, elem := range elems {
buf, err = elem.MarshalSSZTo(buf)
if err != nil {
return nil, err
}
}
return buf, nil
}
// UnmarshalListVariableElement unmarshals a ssz-encoded list of variable-sized elements.
// Because this generic method is parameterized by a [T Unmarshalable] interface type,
// it is unable to initialize elements of the list internally. That is why the caller must
// provide the `newt` function that returns a new instance of the type [T] to be unmarshaled.
// This func will be called for each element in the list to create a new instance of [T].
//
// UnmarshalListFixedElement should be used for fixed-size elements.
// SSZ Lists have different encoding rules depending whether their elements are fixed- or variable-sized,
// and we can't differentiate them by the ssz interface, so it is the caller's responsibility to
// pick the correct method.
func UnmarshalListVariableElement[T Unmarshalable](buf []byte, newt func() T) ([]T, error) {
bufLen := len(buf)
if bufLen == 0 {
return nil, nil
}
if bufLen < offsetLen {
return nil, ErrEncodingSmallerThanOffset
}
fixedSize := uint32(newt().SizeSSZ())
bufLen32 := uint32(bufLen)
first := binary.LittleEndian.Uint32(buf)
// Rather than just return a zero element list in this case,
// we want to explicitly reject this input as invalid
if first < offsetLen {
return nil, ErrOffsetIntoFixed
}
if first%offsetLen != 0 {
return nil, ErrOffsetSectionMisaligned
}
if first > bufLen32 {
return nil, ErrOffsetExceedsBuffer
}
nElems := int(first) / offsetLen // lint:ignore uintcast -- int has higher precision than uint32 on 64 bit systems, so this is 100% safe
if nElems == 0 {
return nil, nil
}
buf = buf[offsetLen:]
sizes := make([]uint32, nElems)
// We've already looked at the offset of the first element (to perform validation on it)
// so we just need to iterate over the remaining offsets, aka nElems-1 times.
// The size of each element is computed relative to the next offset, so this loop is effectively
// looking ahead +1 (starting with a `buf` that has already had the first offset sliced off),
// with the final element handled as a special case outside the loop (using the size of the entire buffer
// as the ending bound).
previous := first
for i := 0; i < nElems-1; i++ {
next := binary.LittleEndian.Uint32(buf)
if next > bufLen32 {
return nil, ErrOffsetExceedsBuffer
}
if next < previous {
return nil, ErrNegativeRelativeOffset
}
sizes[i] = next - previous
if sizes[i] < fixedSize {
return nil, ErrOffsetInsufficient
}
buf = buf[offsetLen:]
previous = next
}
sizes[len(sizes)-1] = bufLen32 - previous
elements := make([]T, nElems)
for i, size := range sizes {
elem := newt()
if err := elem.UnmarshalSSZ(buf[:size]); err != nil {
return nil, errors.Wrap(err, "unmarshal ssz")
}
szi := int(size) // lint:ignore uintcast -- int has higher precision than uint32 on 64 bit systems, so this is 100% safe
if elem.SizeSSZ() != szi {
return nil, ErrOffsetDecodedMismatch
}
elements[i] = elem
buf = buf[size:]
}
return elements, nil
}
// UnmarshalListFixedElement unmarshals a ssz-encoded list of variable-sized elements.
// A List of fixed-size elements is encoded as a concatenation of the marshaled bytes of each
// element, so after performing some safety checks on the alignment and size of the buffer,
// we simply iterate over the buffer in chunks of the fixed size and unmarshal each element.
// Because this generic method is parameterized by a [T Unmarshalable] interface type,
// it is unable to initialize elements of the list internally. That is why the caller must
// provide the `newt` function that returns a new instance of the type [T] to be unmarshaled.
// This func will be called for each element in the list to create a new instance of [T].
//
// UnmarshalListFixedElement should be used for fixed-size elements.
// SSZ Lists have different encoding rules depending whether their elements are fixed- or variable-sized,
// and we can't differentiate them by the ssz interface, so it is the caller's responsibility to
// pick the correct method.
func UnmarshalListFixedElement[T Unmarshalable](buf []byte, newt func() T) ([]T, error) {
bufLen := len(buf)
if bufLen == 0 {
return nil, nil
}
fixedSize := newt().SizeSSZ()
if bufLen%fixedSize != 0 {
return nil, ErrInvalidFixedEncodingLen
}
nElems := bufLen / fixedSize
elements := make([]T, nElems)
for i := 0; i < nElems; i++ {
elem := newt()
if err := elem.UnmarshalSSZ(buf[i*fixedSize : (i+1)*fixedSize]); err != nil {
return nil, errors.Wrap(err, "unmarshal ssz")
}
elements[i] = elem
}
return elements, nil
}
func safeUint32(val int) (uint32, error) {
if val < 0 || val > math.MaxUint32 {
return 0, errors.New("value exceeds uint32 range")
}
return uint32(val), nil // lint:ignore uintcast -- integer value explicitly checked to prevent truncation
}

View File

@@ -2,14 +2,13 @@
# Common
##############################################################################
load("@rules_proto//proto:defs.bzl", "proto_library")
##############################################################################
# Go
##############################################################################
# gazelle:ignore
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("@rules_proto//proto:defs.bzl", "proto_library")
load("//proto:ssz_proto_library.bzl", "ssz_proto_files")
load("//tools:ssz.bzl", "SSZ_DEPS", "ssz_gen_marshal")
@@ -189,6 +188,7 @@ ssz_fulu_objs = [
"DataColumnIdentifier",
"DataColumnsByRootIdentifier",
"DataColumnSidecar",
"StatusV2",
"SignedBeaconBlockContentsFulu",
"SignedBeaconBlockFulu",
"SignedBlindedBeaconBlockFulu",
@@ -359,15 +359,17 @@ go_library(
importpath = "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1",
visibility = ["//visibility:public"],
deps = SSZ_DEPS + [
"//consensus-types/primitives:go_default_library",
"//encoding/bytesutil:go_default_library",
"//math:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/eth/ext:go_default_library",
"//runtime/version:go_default_library",
"//consensus-types/primitives:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_golang_protobuf//proto:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library", # keep
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
"@googleapis//google/api:annotations_go_proto",
"@io_bazel_rules_go//proto/wkt:descriptor_go_proto",
"@io_bazel_rules_go//proto/wkt:empty_go_proto",
@@ -382,8 +384,6 @@ go_library(
"@org_golang_google_protobuf//runtime/protoimpl:go_default_library",
"@org_golang_google_protobuf//types/descriptorpb:go_default_library",
"@org_golang_google_protobuf//types/known/emptypb:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
],
)

View File

@@ -5,6 +5,12 @@ import (
enginev1 "github.com/OffchainLabs/prysm/v6/proto/engine/v1"
)
// GenericConverter defines any struct that can be converted to a generic beacon block.
// We assume all your versioned block structs implement this method.
type GenericConverter interface {
ToGeneric() (*GenericBeaconBlock, error)
}
// ----------------------------------------------------------------------------
// Phase 0
// ----------------------------------------------------------------------------

View File

@@ -109,61 +109,6 @@ func (x *DataColumnSidecar) GetKzgCommitmentsInclusionProof() [][]byte {
return nil
}
type DataColumnIdentifier struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
BlockRoot []byte `protobuf:"bytes,1,opt,name=block_root,json=blockRoot,proto3" json:"block_root,omitempty" ssz-size:"32"`
Index uint64 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"`
}
func (x *DataColumnIdentifier) Reset() {
*x = DataColumnIdentifier{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *DataColumnIdentifier) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*DataColumnIdentifier) ProtoMessage() {}
func (x *DataColumnIdentifier) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[1]
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 DataColumnIdentifier.ProtoReflect.Descriptor instead.
func (*DataColumnIdentifier) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP(), []int{1}
}
func (x *DataColumnIdentifier) GetBlockRoot() []byte {
if x != nil {
return x.BlockRoot
}
return nil
}
func (x *DataColumnIdentifier) GetIndex() uint64 {
if x != nil {
return x.Index
}
return 0
}
type DataColumnsByRootIdentifier struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -176,7 +121,7 @@ type DataColumnsByRootIdentifier struct {
func (x *DataColumnsByRootIdentifier) Reset() {
*x = DataColumnsByRootIdentifier{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[2]
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -189,7 +134,7 @@ func (x *DataColumnsByRootIdentifier) String() string {
func (*DataColumnsByRootIdentifier) ProtoMessage() {}
func (x *DataColumnsByRootIdentifier) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[2]
mi := &file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[1]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -202,7 +147,7 @@ func (x *DataColumnsByRootIdentifier) ProtoReflect() protoreflect.Message {
// Deprecated: Use DataColumnsByRootIdentifier.ProtoReflect.Descriptor instead.
func (*DataColumnsByRootIdentifier) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP(), []int{2}
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP(), []int{1}
}
func (x *DataColumnsByRootIdentifier) GetBlockRoot() []byte {
@@ -253,29 +198,24 @@ var file_proto_prysm_v1alpha1_data_columns_proto_rawDesc = []byte{
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, 0x22, 0x53, 0x0a, 0x14, 0x44,
0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66,
0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f,
0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x22, 0x67, 0x0a, 0x1b, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x42,
0x79, 0x52, 0x6f, 0x6f, 0x74, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12,
0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x21, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x04, 0x42, 0x07, 0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38,
0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x42, 0x9a, 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, 0x39, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e,
0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 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,
0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, 0x67, 0x0a, 0x1b, 0x44,
0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x42, 0x79, 0x52, 0x6f, 0x6f, 0x74,
0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f,
0x74, 0x12, 0x21, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03,
0x28, 0x04, 0x42, 0x07, 0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x07, 0x63, 0x6f, 0x6c,
0x75, 0x6d, 0x6e, 0x73, 0x42, 0x9a, 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, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f,
0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 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 (
@@ -290,15 +230,14 @@ func file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP() []byte {
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescData
}
var file_proto_prysm_v1alpha1_data_columns_proto_msgTypes = make([]protoimpl.MessageInfo, 3)
var file_proto_prysm_v1alpha1_data_columns_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
var file_proto_prysm_v1alpha1_data_columns_proto_goTypes = []interface{}{
(*DataColumnSidecar)(nil), // 0: ethereum.eth.v1alpha1.DataColumnSidecar
(*DataColumnIdentifier)(nil), // 1: ethereum.eth.v1alpha1.DataColumnIdentifier
(*DataColumnsByRootIdentifier)(nil), // 2: ethereum.eth.v1alpha1.DataColumnsByRootIdentifier
(*SignedBeaconBlockHeader)(nil), // 3: ethereum.eth.v1alpha1.SignedBeaconBlockHeader
(*DataColumnsByRootIdentifier)(nil), // 1: ethereum.eth.v1alpha1.DataColumnsByRootIdentifier
(*SignedBeaconBlockHeader)(nil), // 2: ethereum.eth.v1alpha1.SignedBeaconBlockHeader
}
var file_proto_prysm_v1alpha1_data_columns_proto_depIdxs = []int32{
3, // 0: ethereum.eth.v1alpha1.DataColumnSidecar.signed_block_header:type_name -> ethereum.eth.v1alpha1.SignedBeaconBlockHeader
2, // 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
@@ -326,18 +265,6 @@ func file_proto_prysm_v1alpha1_data_columns_proto_init() {
}
}
file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DataColumnIdentifier); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_prysm_v1alpha1_data_columns_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DataColumnsByRootIdentifier); i {
case 0:
return &v.state
@@ -356,7 +283,7 @@ func file_proto_prysm_v1alpha1_data_columns_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_prysm_v1alpha1_data_columns_proto_rawDesc,
NumEnums: 0,
NumMessages: 3,
NumMessages: 2,
NumExtensions: 0,
NumServices: 0,
},

View File

@@ -45,11 +45,6 @@ message DataColumnSidecar {
"kzg_commitments_inclusion_proof_depth.size,32" ];
}
message DataColumnIdentifier {
bytes block_root = 1 [ (ethereum.eth.ext.ssz_size) = "32" ];
uint64 index = 2;
}
message DataColumnsByRootIdentifier {
bytes block_root = 1 [ (ethereum.eth.ext.ssz_size) = "32" ];
repeated uint64 columns = 2 [ (ethereum.eth.ext.ssz_max) = "128" ];

View File

@@ -2246,77 +2246,6 @@ func (d *DataColumnSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
return
}
// MarshalSSZ ssz marshals the DataColumnIdentifier object
func (d *DataColumnIdentifier) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(d)
}
// MarshalSSZTo ssz marshals the DataColumnIdentifier object to a target array
func (d *DataColumnIdentifier) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
// Field (0) 'BlockRoot'
if size := len(d.BlockRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.BlockRoot", size, 32)
return
}
dst = append(dst, d.BlockRoot...)
// Field (1) 'Index'
dst = ssz.MarshalUint64(dst, d.Index)
return
}
// UnmarshalSSZ ssz unmarshals the DataColumnIdentifier object
func (d *DataColumnIdentifier) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size != 40 {
return ssz.ErrSize
}
// Field (0) 'BlockRoot'
if cap(d.BlockRoot) == 0 {
d.BlockRoot = make([]byte, 0, len(buf[0:32]))
}
d.BlockRoot = append(d.BlockRoot, buf[0:32]...)
// Field (1) 'Index'
d.Index = ssz.UnmarshallUint64(buf[32:40])
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the DataColumnIdentifier object
func (d *DataColumnIdentifier) SizeSSZ() (size int) {
size = 40
return
}
// HashTreeRoot ssz hashes the DataColumnIdentifier object
func (d *DataColumnIdentifier) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(d)
}
// HashTreeRootWith ssz hashes the DataColumnIdentifier object with a hasher
func (d *DataColumnIdentifier) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'BlockRoot'
if size := len(d.BlockRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.BlockRoot", size, 32)
return
}
hh.PutBytes(d.BlockRoot)
// Field (1) 'Index'
hh.PutUint64(d.Index)
hh.Merkleize(indx)
return
}
// MarshalSSZ ssz marshals the DataColumnsByRootIdentifier object
func (d *DataColumnsByRootIdentifier) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(d)
@@ -2436,3 +2365,132 @@ func (d *DataColumnsByRootIdentifier) HashTreeRootWith(hh *ssz.Hasher) (err erro
hh.Merkleize(indx)
return
}
// MarshalSSZ ssz marshals the StatusV2 object
func (s *StatusV2) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(s)
}
// MarshalSSZTo ssz marshals the StatusV2 object to a target array
func (s *StatusV2) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
// Field (0) 'ForkDigest'
if size := len(s.ForkDigest); size != 4 {
err = ssz.ErrBytesLengthFn("--.ForkDigest", size, 4)
return
}
dst = append(dst, s.ForkDigest...)
// Field (1) 'FinalizedRoot'
if size := len(s.FinalizedRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.FinalizedRoot", size, 32)
return
}
dst = append(dst, s.FinalizedRoot...)
// Field (2) 'FinalizedEpoch'
dst = ssz.MarshalUint64(dst, uint64(s.FinalizedEpoch))
// Field (3) 'HeadRoot'
if size := len(s.HeadRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.HeadRoot", size, 32)
return
}
dst = append(dst, s.HeadRoot...)
// Field (4) 'HeadSlot'
dst = ssz.MarshalUint64(dst, uint64(s.HeadSlot))
// Field (5) 'EarliestAvailableSlot'
dst = ssz.MarshalUint64(dst, uint64(s.EarliestAvailableSlot))
return
}
// UnmarshalSSZ ssz unmarshals the StatusV2 object
func (s *StatusV2) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size != 92 {
return ssz.ErrSize
}
// Field (0) 'ForkDigest'
if cap(s.ForkDigest) == 0 {
s.ForkDigest = make([]byte, 0, len(buf[0:4]))
}
s.ForkDigest = append(s.ForkDigest, buf[0:4]...)
// Field (1) 'FinalizedRoot'
if cap(s.FinalizedRoot) == 0 {
s.FinalizedRoot = make([]byte, 0, len(buf[4:36]))
}
s.FinalizedRoot = append(s.FinalizedRoot, buf[4:36]...)
// Field (2) 'FinalizedEpoch'
s.FinalizedEpoch = github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Epoch(ssz.UnmarshallUint64(buf[36:44]))
// Field (3) 'HeadRoot'
if cap(s.HeadRoot) == 0 {
s.HeadRoot = make([]byte, 0, len(buf[44:76]))
}
s.HeadRoot = append(s.HeadRoot, buf[44:76]...)
// Field (4) 'HeadSlot'
s.HeadSlot = github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[76:84]))
// Field (5) 'EarliestAvailableSlot'
s.EarliestAvailableSlot = github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[84:92]))
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the StatusV2 object
func (s *StatusV2) SizeSSZ() (size int) {
size = 92
return
}
// HashTreeRoot ssz hashes the StatusV2 object
func (s *StatusV2) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(s)
}
// HashTreeRootWith ssz hashes the StatusV2 object with a hasher
func (s *StatusV2) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'ForkDigest'
if size := len(s.ForkDigest); size != 4 {
err = ssz.ErrBytesLengthFn("--.ForkDigest", size, 4)
return
}
hh.PutBytes(s.ForkDigest)
// Field (1) 'FinalizedRoot'
if size := len(s.FinalizedRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.FinalizedRoot", size, 32)
return
}
hh.PutBytes(s.FinalizedRoot)
// Field (2) 'FinalizedEpoch'
hh.PutUint64(uint64(s.FinalizedEpoch))
// Field (3) 'HeadRoot'
if size := len(s.HeadRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.HeadRoot", size, 32)
return
}
hh.PutBytes(s.HeadRoot)
// Field (4) 'HeadSlot'
hh.PutUint64(uint64(s.HeadSlot))
// Field (5) 'EarliestAvailableSlot'
hh.PutUint64(uint64(s.EarliestAvailableSlot))
hh.Merkleize(indx)
return
}

View File

@@ -104,6 +104,93 @@ func (x *Status) GetHeadSlot() github_com_OffchainLabs_prysm_v6_consensus_types_
return github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot(0)
}
type StatusV2 struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ForkDigest []byte `protobuf:"bytes,1,opt,name=fork_digest,json=forkDigest,proto3" json:"fork_digest,omitempty" ssz-size:"4"`
FinalizedRoot []byte `protobuf:"bytes,2,opt,name=finalized_root,json=finalizedRoot,proto3" json:"finalized_root,omitempty" ssz-size:"32"`
FinalizedEpoch github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Epoch `protobuf:"varint,3,opt,name=finalized_epoch,json=finalizedEpoch,proto3" json:"finalized_epoch,omitempty" cast-type:"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Epoch"`
HeadRoot []byte `protobuf:"bytes,4,opt,name=head_root,json=headRoot,proto3" json:"head_root,omitempty" ssz-size:"32"`
HeadSlot github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot `protobuf:"varint,5,opt,name=head_slot,json=headSlot,proto3" json:"head_slot,omitempty" cast-type:"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"`
EarliestAvailableSlot github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot `protobuf:"varint,6,opt,name=earliest_available_slot,json=earliestAvailableSlot,proto3" json:"earliest_available_slot,omitempty" cast-type:"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"`
}
func (x *StatusV2) Reset() {
*x = StatusV2{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *StatusV2) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*StatusV2) ProtoMessage() {}
func (x *StatusV2) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[1]
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 StatusV2.ProtoReflect.Descriptor instead.
func (*StatusV2) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{1}
}
func (x *StatusV2) GetForkDigest() []byte {
if x != nil {
return x.ForkDigest
}
return nil
}
func (x *StatusV2) GetFinalizedRoot() []byte {
if x != nil {
return x.FinalizedRoot
}
return nil
}
func (x *StatusV2) GetFinalizedEpoch() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Epoch {
if x != nil {
return x.FinalizedEpoch
}
return github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Epoch(0)
}
func (x *StatusV2) GetHeadRoot() []byte {
if x != nil {
return x.HeadRoot
}
return nil
}
func (x *StatusV2) GetHeadSlot() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot {
if x != nil {
return x.HeadSlot
}
return github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot(0)
}
func (x *StatusV2) GetEarliestAvailableSlot() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot {
if x != nil {
return x.EarliestAvailableSlot
}
return github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot(0)
}
type BeaconBlocksByRangeRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -117,7 +204,7 @@ type BeaconBlocksByRangeRequest struct {
func (x *BeaconBlocksByRangeRequest) Reset() {
*x = BeaconBlocksByRangeRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[1]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[2]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -130,7 +217,7 @@ func (x *BeaconBlocksByRangeRequest) String() string {
func (*BeaconBlocksByRangeRequest) ProtoMessage() {}
func (x *BeaconBlocksByRangeRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[1]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[2]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -143,7 +230,7 @@ func (x *BeaconBlocksByRangeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use BeaconBlocksByRangeRequest.ProtoReflect.Descriptor instead.
func (*BeaconBlocksByRangeRequest) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{1}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{2}
}
func (x *BeaconBlocksByRangeRequest) GetStartSlot() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot {
@@ -180,7 +267,7 @@ type ENRForkID struct {
func (x *ENRForkID) Reset() {
*x = ENRForkID{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[2]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[3]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -193,7 +280,7 @@ func (x *ENRForkID) String() string {
func (*ENRForkID) ProtoMessage() {}
func (x *ENRForkID) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[2]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[3]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -206,7 +293,7 @@ func (x *ENRForkID) ProtoReflect() protoreflect.Message {
// Deprecated: Use ENRForkID.ProtoReflect.Descriptor instead.
func (*ENRForkID) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{2}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{3}
}
func (x *ENRForkID) GetCurrentForkDigest() []byte {
@@ -242,7 +329,7 @@ type MetaDataV0 struct {
func (x *MetaDataV0) Reset() {
*x = MetaDataV0{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[3]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[4]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -255,7 +342,7 @@ func (x *MetaDataV0) String() string {
func (*MetaDataV0) ProtoMessage() {}
func (x *MetaDataV0) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[3]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[4]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -268,7 +355,7 @@ func (x *MetaDataV0) ProtoReflect() protoreflect.Message {
// Deprecated: Use MetaDataV0.ProtoReflect.Descriptor instead.
func (*MetaDataV0) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{3}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{4}
}
func (x *MetaDataV0) GetSeqNumber() uint64 {
@@ -298,7 +385,7 @@ type MetaDataV1 struct {
func (x *MetaDataV1) Reset() {
*x = MetaDataV1{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[4]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -311,7 +398,7 @@ func (x *MetaDataV1) String() string {
func (*MetaDataV1) ProtoMessage() {}
func (x *MetaDataV1) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[4]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -324,7 +411,7 @@ func (x *MetaDataV1) ProtoReflect() protoreflect.Message {
// Deprecated: Use MetaDataV1.ProtoReflect.Descriptor instead.
func (*MetaDataV1) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{4}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{5}
}
func (x *MetaDataV1) GetSeqNumber() uint64 {
@@ -362,7 +449,7 @@ type MetaDataV2 struct {
func (x *MetaDataV2) Reset() {
*x = MetaDataV2{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -375,7 +462,7 @@ func (x *MetaDataV2) String() string {
func (*MetaDataV2) ProtoMessage() {}
func (x *MetaDataV2) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -388,7 +475,7 @@ func (x *MetaDataV2) ProtoReflect() protoreflect.Message {
// Deprecated: Use MetaDataV2.ProtoReflect.Descriptor instead.
func (*MetaDataV2) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{5}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{6}
}
func (x *MetaDataV2) GetSeqNumber() uint64 {
@@ -431,7 +518,7 @@ type BlobSidecarsByRangeRequest struct {
func (x *BlobSidecarsByRangeRequest) Reset() {
*x = BlobSidecarsByRangeRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -444,7 +531,7 @@ func (x *BlobSidecarsByRangeRequest) String() string {
func (*BlobSidecarsByRangeRequest) ProtoMessage() {}
func (x *BlobSidecarsByRangeRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -457,7 +544,7 @@ func (x *BlobSidecarsByRangeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use BlobSidecarsByRangeRequest.ProtoReflect.Descriptor instead.
func (*BlobSidecarsByRangeRequest) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{6}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{7}
}
func (x *BlobSidecarsByRangeRequest) GetStartSlot() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot {
@@ -487,7 +574,7 @@ type DataColumnSidecarsByRangeRequest struct {
func (x *DataColumnSidecarsByRangeRequest) Reset() {
*x = DataColumnSidecarsByRangeRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[8]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -500,7 +587,7 @@ func (x *DataColumnSidecarsByRangeRequest) String() string {
func (*DataColumnSidecarsByRangeRequest) ProtoMessage() {}
func (x *DataColumnSidecarsByRangeRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[8]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -513,7 +600,7 @@ func (x *DataColumnSidecarsByRangeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use DataColumnSidecarsByRangeRequest.ProtoReflect.Descriptor instead.
func (*DataColumnSidecarsByRangeRequest) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{7}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{8}
}
func (x *DataColumnSidecarsByRangeRequest) GetStartSlot() github_com_OffchainLabs_prysm_v6_consensus_types_primitives.Slot {
@@ -549,7 +636,7 @@ type LightClientUpdatesByRangeRequest struct {
func (x *LightClientUpdatesByRangeRequest) Reset() {
*x = LightClientUpdatesByRangeRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[8]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[9]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -562,7 +649,7 @@ func (x *LightClientUpdatesByRangeRequest) String() string {
func (*LightClientUpdatesByRangeRequest) ProtoMessage() {}
func (x *LightClientUpdatesByRangeRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[8]
mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[9]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -575,7 +662,7 @@ func (x *LightClientUpdatesByRangeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use LightClientUpdatesByRangeRequest.ProtoReflect.Descriptor instead.
func (*LightClientUpdatesByRangeRequest) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{8}
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{9}
}
func (x *LightClientUpdatesByRangeRequest) GetStartPeriod() uint64 {
@@ -624,109 +711,138 @@ var file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc = []byte{
0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f,
0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f,
0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52,
0x08, 0x68, 0x65, 0x61, 0x64, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xab, 0x01, 0x0a, 0x1a, 0x42, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72,
0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5,
0x18, 0x40, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66,
0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f,
0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70,
0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c,
0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a,
0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f,
0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28,
0x04, 0x52, 0x04, 0x73, 0x74, 0x65, 0x70, 0x22, 0xe4, 0x01, 0x0a, 0x09, 0x45, 0x4e, 0x52, 0x46,
0x6f, 0x72, 0x6b, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74,
0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01,
0x28, 0x0c, 0x42, 0x05, 0x8a, 0xb5, 0x18, 0x01, 0x34, 0x52, 0x11, 0x63, 0x75, 0x72, 0x72, 0x65,
0x6e, 0x74, 0x46, 0x6f, 0x72, 0x6b, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x31, 0x0a, 0x11,
0x6e, 0x65, 0x78, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f,
0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x05, 0x8a, 0xb5, 0x18, 0x01, 0x34, 0x52, 0x0f,
0x6e, 0x65, 0x78, 0x74, 0x46, 0x6f, 0x72, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12,
0x6d, 0x0a, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x65, 0x70, 0x6f,
0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69,
0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63,
0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70,
0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52,
0x0d, 0x6e, 0x65, 0x78, 0x74, 0x46, 0x6f, 0x72, 0x6b, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x80,
0x01, 0x0a, 0x0a, 0x4d, 0x65, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x30, 0x12, 0x1d, 0x0a,
0x0a, 0x73, 0x65, 0x71, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
0x04, 0x52, 0x09, 0x73, 0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x07,
0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x39, 0x82,
0xb5, 0x18, 0x30, 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, 0x67, 0x6f, 0x2d, 0x62,
0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f,
0x72, 0x36, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x38, 0x52, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74,
0x73, 0x22, 0xd6, 0x01, 0x0a, 0x0a, 0x4d, 0x65, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x31,
0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x71, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01,
0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x73, 0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12,
0x53, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c,
0x42, 0x39, 0x82, 0xb5, 0x18, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
0x08, 0x68, 0x65, 0x61, 0x64, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xd7, 0x03, 0x0a, 0x08, 0x53, 0x74,
0x61, 0x74, 0x75, 0x73, 0x56, 0x32, 0x12, 0x26, 0x0a, 0x0b, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x64,
0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x05, 0x8a, 0xb5, 0x18,
0x01, 0x34, 0x52, 0x0a, 0x66, 0x6f, 0x72, 0x6b, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x2d,
0x0a, 0x0e, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x6f, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0d,
0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x6e, 0x0a,
0x0f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68,
0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x0e, 0x66,
0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x23, 0x0a,
0x09, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c,
0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x08, 0x68, 0x65, 0x61, 0x64, 0x52, 0x6f,
0x6f, 0x74, 0x12, 0x61, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18,
0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68, 0x75,
0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61,
0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73,
0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d,
0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x08, 0x68, 0x65, 0x61,
0x64, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x7c, 0x0a, 0x17, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73,
0x74, 0x5f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x15, 0x65, 0x61,
0x72, 0x6c, 0x69, 0x65, 0x73, 0x74, 0x41, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x53,
0x6c, 0x6f, 0x74, 0x22, 0xab, 0x01, 0x0a, 0x1a, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74,
0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a,
0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x74, 0x65,
0x70, 0x22, 0xe4, 0x01, 0x0a, 0x09, 0x45, 0x4e, 0x52, 0x46, 0x6f, 0x72, 0x6b, 0x49, 0x44, 0x12,
0x35, 0x0a, 0x13, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f,
0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x05, 0x8a, 0xb5,
0x18, 0x01, 0x34, 0x52, 0x11, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x46, 0x6f, 0x72, 0x6b,
0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x31, 0x0a, 0x11, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x66,
0x6f, 0x72, 0x6b, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x05, 0x8a, 0xb5, 0x18, 0x01, 0x34, 0x52, 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x46, 0x6f,
0x72, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x6d, 0x0a, 0x0f, 0x6e, 0x65, 0x78,
0x74, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01,
0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f,
0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73,
0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69,
0x76, 0x65, 0x73, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x46,
0x6f, 0x72, 0x6b, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x80, 0x01, 0x0a, 0x0a, 0x4d, 0x65, 0x74,
0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x30, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x71, 0x5f, 0x6e,
0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x73, 0x65, 0x71,
0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74,
0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x39, 0x82, 0xb5, 0x18, 0x30, 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, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c,
0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x36, 0x34, 0x8a, 0xb5, 0x18,
0x01, 0x38, 0x52, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x22, 0xd6, 0x01, 0x0a, 0x0a,
0x4d, 0x65, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x31, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65,
0x71, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09,
0x73, 0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x74, 0x74,
0x6e, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x39, 0x82, 0xb5, 0x18, 0x30,
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, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66,
0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x36, 0x34,
0x8a, 0xb5, 0x18, 0x01, 0x38, 0x52, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x12, 0x54,
0x0a, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c,
0x42, 0x38, 0x82, 0xb5, 0x18, 0x2f, 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, 0x67,
0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65,
0x63, 0x74, 0x6f, 0x72, 0x36, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x38, 0x52, 0x07, 0x61, 0x74, 0x74,
0x6e, 0x65, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x38, 0x82, 0xb5, 0x18, 0x2f, 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, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64,
0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x31,
0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x4d,
0x65, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x32, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x71,
0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x73,
0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x6e,
0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x39, 0x82, 0xb5, 0x18, 0x30, 0x67,
0x63, 0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x31, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63,
0x6e, 0x65, 0x74, 0x73, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x4d, 0x65, 0x74, 0x61, 0x44, 0x61, 0x74,
0x61, 0x56, 0x32, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x71, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65,
0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x73, 0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62,
0x65, 0x72, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x39, 0x82, 0xb5, 0x18, 0x30, 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, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69,
0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x36, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x38, 0x52, 0x07,
0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e,
0x65, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x38, 0x82, 0xb5, 0x18, 0x2f, 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, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69,
0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x36, 0x34, 0x8a,
0xb5, 0x18, 0x01, 0x38, 0x52, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x12, 0x54, 0x0a,
0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x38, 0x82, 0xb5, 0x18, 0x2f, 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, 0x67, 0x6f,
0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63,
0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x31, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e,
0x65, 0x74, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x5f, 0x67,
0x72, 0x6f, 0x75, 0x70, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04,
0x52, 0x11, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f,
0x75, 0x6e, 0x74, 0x22, 0x97, 0x01, 0x0a, 0x1a, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65,
0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74,
0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xc0, 0x01,
0x0a, 0x20, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65,
0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74,
0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, 0x0a,
0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x07,
0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
0x22, 0x5b, 0x0a, 0x20, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x55,
0x70, 0x64, 0x61, 0x74, 0x65, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x65,
0x72, 0x69, 0x6f, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72,
0x74, 0x50, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x9a, 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, 0x50, 0x32, 0x50,
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a,
0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63,
0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76,
0x36, 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, 0x56, 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,
0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5,
0x18, 0x01, 0x31, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x12, 0x2e, 0x0a,
0x13, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x63,
0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x11, 0x63, 0x75, 0x73, 0x74,
0x6f, 0x64, 0x79, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x97, 0x01,
0x0a, 0x1a, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x42, 0x79,
0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a,
0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04,
0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73,
0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65,
0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f,
0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04,
0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xc0, 0x01, 0x0a, 0x20, 0x44, 0x61, 0x74, 0x61,
0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x42, 0x79,
0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a,
0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04,
0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73,
0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65,
0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f,
0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04,
0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d,
0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x07, 0x92, 0xb5, 0x18, 0x03, 0x31, 0x32,
0x38, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x5b, 0x0a, 0x20, 0x4c, 0x69,
0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x73,
0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21,
0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x18, 0x01,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x50, 0x65, 0x72, 0x69, 0x6f,
0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04,
0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x42, 0x9a, 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, 0x50, 0x32, 0x50, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75,
0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61,
0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 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, 0x56, 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 (
@@ -741,17 +857,18 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP() []byte {
return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescData
}
var file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 9)
var file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 10)
var file_proto_prysm_v1alpha1_p2p_messages_proto_goTypes = []interface{}{
(*Status)(nil), // 0: ethereum.eth.v1alpha1.Status
(*BeaconBlocksByRangeRequest)(nil), // 1: ethereum.eth.v1alpha1.BeaconBlocksByRangeRequest
(*ENRForkID)(nil), // 2: ethereum.eth.v1alpha1.ENRForkID
(*MetaDataV0)(nil), // 3: ethereum.eth.v1alpha1.MetaDataV0
(*MetaDataV1)(nil), // 4: ethereum.eth.v1alpha1.MetaDataV1
(*MetaDataV2)(nil), // 5: ethereum.eth.v1alpha1.MetaDataV2
(*BlobSidecarsByRangeRequest)(nil), // 6: ethereum.eth.v1alpha1.BlobSidecarsByRangeRequest
(*DataColumnSidecarsByRangeRequest)(nil), // 7: ethereum.eth.v1alpha1.DataColumnSidecarsByRangeRequest
(*LightClientUpdatesByRangeRequest)(nil), // 8: ethereum.eth.v1alpha1.LightClientUpdatesByRangeRequest
(*StatusV2)(nil), // 1: ethereum.eth.v1alpha1.StatusV2
(*BeaconBlocksByRangeRequest)(nil), // 2: ethereum.eth.v1alpha1.BeaconBlocksByRangeRequest
(*ENRForkID)(nil), // 3: ethereum.eth.v1alpha1.ENRForkID
(*MetaDataV0)(nil), // 4: ethereum.eth.v1alpha1.MetaDataV0
(*MetaDataV1)(nil), // 5: ethereum.eth.v1alpha1.MetaDataV1
(*MetaDataV2)(nil), // 6: ethereum.eth.v1alpha1.MetaDataV2
(*BlobSidecarsByRangeRequest)(nil), // 7: ethereum.eth.v1alpha1.BlobSidecarsByRangeRequest
(*DataColumnSidecarsByRangeRequest)(nil), // 8: ethereum.eth.v1alpha1.DataColumnSidecarsByRangeRequest
(*LightClientUpdatesByRangeRequest)(nil), // 9: ethereum.eth.v1alpha1.LightClientUpdatesByRangeRequest
}
var file_proto_prysm_v1alpha1_p2p_messages_proto_depIdxs = []int32{
0, // [0:0] is the sub-list for method output_type
@@ -780,7 +897,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BeaconBlocksByRangeRequest); i {
switch v := v.(*StatusV2); i {
case 0:
return &v.state
case 1:
@@ -792,7 +909,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ENRForkID); i {
switch v := v.(*BeaconBlocksByRangeRequest); i {
case 0:
return &v.state
case 1:
@@ -804,7 +921,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*MetaDataV0); i {
switch v := v.(*ENRForkID); i {
case 0:
return &v.state
case 1:
@@ -816,7 +933,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*MetaDataV1); i {
switch v := v.(*MetaDataV0); i {
case 0:
return &v.state
case 1:
@@ -828,7 +945,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*MetaDataV2); i {
switch v := v.(*MetaDataV1); i {
case 0:
return &v.state
case 1:
@@ -840,7 +957,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlobSidecarsByRangeRequest); i {
switch v := v.(*MetaDataV2); i {
case 0:
return &v.state
case 1:
@@ -852,7 +969,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DataColumnSidecarsByRangeRequest); i {
switch v := v.(*BlobSidecarsByRangeRequest); i {
case 0:
return &v.state
case 1:
@@ -864,6 +981,18 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DataColumnSidecarsByRangeRequest); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*LightClientUpdatesByRangeRequest); i {
case 0:
return &v.state
@@ -882,7 +1011,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc,
NumEnums: 0,
NumMessages: 9,
NumMessages: 10,
NumExtensions: 0,
NumServices: 0,
},

View File

@@ -26,6 +26,24 @@ message Status {
];
}
message StatusV2 {
bytes fork_digest = 1 [(ethereum.eth.ext.ssz_size) = "4"];
bytes finalized_root = 2 [(ethereum.eth.ext.ssz_size) = "32"];
uint64 finalized_epoch = 3 [
(ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Epoch"
];
bytes head_root = 4 [(ethereum.eth.ext.ssz_size) = "32"];
uint64 head_slot = 5 [
(ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"
];
uint64 earliest_available_slot = 6 [
(ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"
];
}
message BeaconBlocksByRangeRequest {
uint64 start_slot = 1 [
(ethereum.eth.ext.cast_type) =

View File

@@ -248,6 +248,9 @@ func (v *ValidatorNode) Start(ctx context.Context) error {
args = append(args,
fmt.Sprintf("--%s=http://localhost:%d", flags.BeaconRESTApiProviderFlag.Name, beaconRestApiPort),
fmt.Sprintf("--%s", features.EnableBeaconRESTApi.Name))
if v.config.UseSSZOnly {
args = append(args, fmt.Sprintf("--%s", features.SSZOnly.Name))
}
}
// Only apply e2e flags to the current branch. New flags may not exist in previous release.

View File

@@ -29,6 +29,10 @@ func TestEndToEnd_MinimalConfig_ValidatorRESTApi(t *testing.T) {
e2eMinimal(t, types.InitForkCfg(version.Bellatrix, version.Electra, params.E2ETestConfig()), types.WithCheckpointSync(), types.WithValidatorRESTApi()).run()
}
func TestEndToEnd_MinimalConfig_ValidatorRESTApi_SSZ(t *testing.T) {
e2eMinimal(t, types.InitForkCfg(version.Bellatrix, version.Electra, params.E2ETestConfig()), types.WithCheckpointSync(), types.WithValidatorRESTApi(), types.WithSSZOnly()).run()
}
func TestEndToEnd_ScenarioRun_EEOffline(t *testing.T) {
t.Skip("TODO(#10242) Prysm is current unable to handle an offline e2e")
cfg := types.InitForkCfg(version.Bellatrix, version.Deneb, params.E2ETestConfig())

View File

@@ -51,6 +51,12 @@ func WithValidatorRESTApi() E2EConfigOpt {
}
}
func WithSSZOnly() E2EConfigOpt {
return func(cfg *E2EConfig) {
cfg.UseSSZOnly = true
}
}
func WithBuilder() E2EConfigOpt {
return func(cfg *E2EConfig) {
cfg.UseBuilder = true
@@ -70,6 +76,7 @@ type E2EConfig struct {
UseFixedPeerIDs bool
UseValidatorCrossClient bool
UseBeaconRestApi bool
UseSSZOnly bool
UseBuilder bool
EpochsToRun uint64
Seed int64

View File

@@ -70,7 +70,6 @@ go_library(
"//testing/require:go_default_library",
"//time/slots:go_default_library",
"@com_github_crate_crypto_go_kzg_4844//:go_default_library",
"@com_github_ethereum_c_kzg_4844//bindings/go:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",

View File

@@ -3,80 +3,91 @@ package util
import (
"testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/kzg"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
ckzg4844 "github.com/ethereum/c-kzg-4844/v2/bindings/go"
)
type (
DataColumnParams struct {
Slot primitives.Slot
ColumnIndex uint64
KzgCommitments [][]byte
DataColumn []byte // A whole data cell will be filled with the content of one item of this slice.
}
// DataColumnParam is a struct that holds parameters for creating test RODataColumn and VerifiedRODataColumn sidecars.
DataColumnParam struct {
Index uint64
Column [][]byte
KzgCommitments [][]byte
KzgProofs [][]byte
KzgCommitmentsInclusionProof [][]byte
DataColumnsParamsByRoot map[[fieldparams.RootLength]byte][]DataColumnParams
// Part of the beacon block header.
Slot primitives.Slot
ProposerIndex primitives.ValidatorIndex
ParentRoot []byte
StateRoot []byte
BodyRoot []byte
}
)
func CreateTestVerifiedRoDataColumnSidecars(t *testing.T, dataColumnParamsByBlockRoot DataColumnsParamsByRoot) ([]blocks.RODataColumn, []blocks.VerifiedRODataColumn) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.FuluForkEpoch = 0
params.OverrideBeaconConfig(cfg)
count := 0
for _, indices := range dataColumnParamsByBlockRoot {
count += len(indices)
}
// CreateTestVerifiedRoDataColumnSidecars creates test RODataColumn and VerifiedRODataColumn sidecars for testing purposes.
func CreateTestVerifiedRoDataColumnSidecars(t *testing.T, params []DataColumnParam) ([]blocks.RODataColumn, []blocks.VerifiedRODataColumn) {
const (
kzgCommitmentsInclusionProofSize = 4
proofSize = 32
)
count := len(params)
verifiedRoDataColumnSidecars := make([]blocks.VerifiedRODataColumn, 0, count)
rodataColumnSidecars := make([]blocks.RODataColumn, 0, count)
for blockRoot, params := range dataColumnParamsByBlockRoot {
for _, param := range params {
dataColumn := make([][]byte, 0, len(param.DataColumn))
for _, value := range param.DataColumn {
cell := make([]byte, ckzg4844.BytesPerCell)
for i := range ckzg4844.BytesPerCell {
cell[i] = value
}
dataColumn = append(dataColumn, cell)
}
kzgCommitmentsInclusionProof := make([][]byte, 4)
for i := range kzgCommitmentsInclusionProof {
kzgCommitmentsInclusionProof[i] = make([]byte, 32)
}
for _, param := range params {
var parentRoot, stateRoot, bodyRoot [fieldparams.RootLength]byte
copy(parentRoot[:], param.ParentRoot)
copy(stateRoot[:], param.StateRoot)
copy(bodyRoot[:], param.BodyRoot)
dataColumnSidecar := &ethpb.DataColumnSidecar{
Index: param.ColumnIndex,
KzgCommitments: param.KzgCommitments,
Column: dataColumn,
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: param.Slot,
ParentRoot: make([]byte, fieldparams.RootLength),
StateRoot: make([]byte, fieldparams.RootLength),
BodyRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
}
roDataColumnSidecar, err := blocks.NewRODataColumnWithRoot(dataColumnSidecar, blockRoot)
if err != nil {
t.Fatal(err)
}
rodataColumnSidecars = append(rodataColumnSidecars, roDataColumnSidecar)
verifiedRoDataColumnSidecar := blocks.NewVerifiedRODataColumn(roDataColumnSidecar)
verifiedRoDataColumnSidecars = append(verifiedRoDataColumnSidecars, verifiedRoDataColumnSidecar)
column := make([][]byte, 0, len(param.Column))
for _, cell := range param.Column {
var completeCell [kzg.BytesPerCell]byte
copy(completeCell[:], cell)
column = append(column, completeCell[:])
}
kzgCommitmentsInclusionProof := make([][]byte, 0, kzgCommitmentsInclusionProofSize)
for range kzgCommitmentsInclusionProofSize {
kzgCommitmentsInclusionProof = append(kzgCommitmentsInclusionProof, make([]byte, proofSize))
}
for i, proof := range param.KzgCommitmentsInclusionProof {
copy(kzgCommitmentsInclusionProof[i], proof)
}
dataColumnSidecar := &ethpb.DataColumnSidecar{
Index: param.Index,
Column: column,
KzgCommitments: param.KzgCommitments,
KzgProofs: param.KzgProofs,
SignedBlockHeader: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: param.Slot,
ProposerIndex: param.ProposerIndex,
ParentRoot: parentRoot[:],
StateRoot: stateRoot[:],
BodyRoot: bodyRoot[:],
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
KzgCommitmentsInclusionProof: kzgCommitmentsInclusionProof,
}
roDataColumnSidecar, err := blocks.NewRODataColumn(dataColumnSidecar)
if err != nil {
t.Fatal(err)
}
rodataColumnSidecars = append(rodataColumnSidecars, roDataColumnSidecar)
verifiedRoDataColumnSidecar := blocks.NewVerifiedRODataColumn(roDataColumnSidecar)
verifiedRoDataColumnSidecars = append(verifiedRoDataColumnSidecars, verifiedRoDataColumnSidecar)
}
return rodataColumnSidecars, verifiedRoDataColumnSidecars

View File

@@ -1,7 +1,6 @@
package util
import (
"encoding/binary"
"math/big"
"testing"
@@ -110,14 +109,21 @@ func GenerateTestFuluBlockWithSidecars(t *testing.T, blobCount int, options ...F
block.Block.ParentRoot = generator.parent[:]
block.Block.ProposerIndex = generator.proposer
block.Block.Body.BlobKzgCommitments = make([][]byte, blobCount)
for i := range blobCount {
var commitment [fieldparams.KzgCommitmentSize]byte
binary.LittleEndian.PutUint16(commitment[:16], uint16(i))
binary.LittleEndian.PutUint16(commitment[16:32], uint16(generator.slot))
block.Block.Body.BlobKzgCommitments[i] = commitment[:]
blobs := make([]kzg.Blob, 0, generator.blobCount)
commitments := make([][]byte, 0, generator.blobCount)
for i := range generator.blobCount {
blob := kzg.Blob{uint8(i)}
commitment, err := kzg.BlobToKZGCommitment(&blob)
require.NoError(t, err)
blobs = append(blobs, blob)
commitments = append(commitments, commitment[:])
}
block.Block.Body.BlobKzgCommitments = commitments
body, err := blocks.NewBeaconBlockBody(block.Block.Body)
require.NoError(t, err)
@@ -149,39 +155,30 @@ func GenerateTestFuluBlockWithSidecars(t *testing.T, blobCount int, options ...F
root, err := block.Block.HashTreeRoot()
require.NoError(t, err)
sbb, err := blocks.NewSignedBeaconBlock(block)
signedBeaconBlock, err := blocks.NewSignedBeaconBlock(block)
require.NoError(t, err)
sh, err := sbb.Header()
require.NoError(t, err)
blobs := make([]kzg.Blob, blobCount)
for i, commitment := range block.Block.Body.BlobKzgCommitments {
roSidecars := GenerateTestDenebBlobSidecar(t, root, sh, i, commitment, inclusion[i])
blobs[i] = kzg.Blob(roSidecars.Blob)
}
cellsAndProofs := GenerateCellsAndProofs(t, blobs)
dataColumns, err := peerdas.DataColumnSidecars(sbb, cellsAndProofs)
sidecars, err := peerdas.DataColumnSidecars(signedBeaconBlock, cellsAndProofs)
require.NoError(t, err)
roSidecars := make([]blocks.RODataColumn, 0, len(dataColumns))
roVerifiedSidecars := make([]blocks.VerifiedRODataColumn, 0, len(dataColumns))
for _, dataColumn := range dataColumns {
roSidecar, err := blocks.NewRODataColumnWithRoot(dataColumn, root)
roSidecars := make([]blocks.RODataColumn, 0, len(sidecars))
verifiedRoSidecars := make([]blocks.VerifiedRODataColumn, 0, len(sidecars))
for _, sidecar := range sidecars {
roSidecar, err := blocks.NewRODataColumnWithRoot(sidecar, root)
require.NoError(t, err)
roVerifiedSidecar := blocks.NewVerifiedRODataColumn(roSidecar)
roSidecars = append(roSidecars, roSidecar)
roVerifiedSidecars = append(roVerifiedSidecars, roVerifiedSidecar)
verifiedRoSidecars = append(verifiedRoSidecars, roVerifiedSidecar)
}
rob, err := blocks.NewROBlockWithRoot(sbb, root)
roBlock, err := blocks.NewROBlockWithRoot(signedBeaconBlock, root)
require.NoError(t, err)
return rob, roSidecars, roVerifiedSidecars
return roBlock, roSidecars, verifiedRoSidecars
}
func GenerateCellsAndProofs(t testing.TB, blobs []kzg.Blob) []kzg.CellsAndProofs {

View File

@@ -87,7 +87,7 @@ func (acm *CLIManager) prepareBeaconClients(ctx context.Context) (*iface.Validat
acm.beaconApiTimeout,
)
restHandler := beaconApi.NewBeaconApiJsonRestHandler(
restHandler := beaconApi.NewBeaconApiRestHandler(
http.Client{Timeout: acm.beaconApiTimeout},
acm.beaconApiEndpoint,
)

View File

@@ -18,7 +18,6 @@ go_library(
"genesis.go",
"get_beacon_block.go",
"index.go",
"json_rest_handler.go",
"log.go",
"metrics.go",
"prepare_beacon_proposer.go",
@@ -27,6 +26,7 @@ go_library(
"propose_exit.go",
"prysm_beacon_chain_client.go",
"registration.go",
"rest_handler_client.go",
"state_validators.go",
"status.go",
"stream_blocks.go",
@@ -47,6 +47,7 @@ go_library(
"//api/server/structs:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
@@ -91,7 +92,6 @@ go_test(
"genesis_test.go",
"get_beacon_block_test.go",
"index_test.go",
"json_rest_handler_test.go",
"prepare_beacon_proposer_test.go",
"propose_attestation_test.go",
"propose_beacon_block_altair_test.go",
@@ -110,6 +110,7 @@ go_test(
"propose_exit_test.go",
"prysm_beacon_chain_client_test.go",
"registration_test.go",
"rest_handler_client_test.go",
"state_validators_test.go",
"status_test.go",
"stream_blocks_test.go",
@@ -128,6 +129,7 @@ go_test(
"//api/server/structs:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/rpc/eth/shared/testing:go_default_library",
"//config/features:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/validator:go_default_library",
@@ -145,6 +147,8 @@ go_test(
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_golang_protobuf//ptypes/empty",
"@com_github_pkg_errors//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
"@org_golang_google_protobuf//types/known/emptypb:go_default_library",
"@org_golang_google_protobuf//types/known/timestamppb:go_default_library",
"@org_uber_go_mock//gomock:go_default_library",

View File

@@ -17,7 +17,7 @@ import (
type beaconApiChainClient struct {
fallbackClient iface.ChainClient
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
stateValidatorsProvider StateValidatorsProvider
}
@@ -333,7 +333,7 @@ func (c beaconApiChainClient) ValidatorParticipation(ctx context.Context, in *et
return nil, errors.New("beaconApiChainClient.ValidatorParticipation is not implemented. To use a fallback client, pass a fallback client as the last argument of NewBeaconApiChainClientWithFallback.")
}
func NewBeaconApiChainClientWithFallback(jsonRestHandler JsonRestHandler, fallbackClient iface.ChainClient) iface.ChainClient {
func NewBeaconApiChainClientWithFallback(jsonRestHandler RestHandler, fallbackClient iface.ChainClient) iface.ChainClient {
return &beaconApiChainClient{
jsonRestHandler: jsonRestHandler,
fallbackClient: fallbackClient,

View File

@@ -20,7 +20,7 @@ var (
type beaconApiNodeClient struct {
fallbackClient iface.NodeClient
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
genesisProvider GenesisProvider
healthTracker health.Tracker
}
@@ -111,7 +111,7 @@ func (c *beaconApiNodeClient) HealthTracker() health.Tracker {
return c.healthTracker
}
func NewNodeClientWithFallback(jsonRestHandler JsonRestHandler, fallbackClient iface.NodeClient) iface.NodeClient {
func NewNodeClientWithFallback(jsonRestHandler RestHandler, fallbackClient iface.NodeClient) iface.NodeClient {
b := &beaconApiNodeClient{
jsonRestHandler: jsonRestHandler,
fallbackClient: fallbackClient,

View File

@@ -22,13 +22,13 @@ type beaconApiValidatorClient struct {
genesisProvider GenesisProvider
dutiesProvider dutiesProvider
stateValidatorsProvider StateValidatorsProvider
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
beaconBlockConverter BeaconBlockConverter
prysmChainClient iface.PrysmChainClient
isEventStreamRunning bool
}
func NewBeaconApiValidatorClient(jsonRestHandler JsonRestHandler, opts ...ValidatorClientOpt) iface.ValidatorClient {
func NewBeaconApiValidatorClient(jsonRestHandler RestHandler, opts ...ValidatorClientOpt) iface.ValidatorClient {
c := &beaconApiValidatorClient{
genesisProvider: &beaconApiGenesisProvider{jsonRestHandler: jsonRestHandler},
dutiesProvider: beaconApiDutiesProvider{jsonRestHandler: jsonRestHandler},

View File

@@ -27,7 +27,7 @@ type dutiesProvider interface {
}
type beaconApiDutiesProvider struct {
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
}
type attesterDuty struct {

View File

@@ -20,7 +20,7 @@ type GenesisProvider interface {
}
type beaconApiGenesisProvider struct {
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
genesis *structs.Genesis
once sync.Once
}

View File

@@ -6,7 +6,10 @@ import (
"encoding/json"
"fmt"
neturl "net/url"
"strconv"
"strings"
"github.com/OffchainLabs/prysm/v6/api"
"github.com/OffchainLabs/prysm/v6/api/apiutil"
"github.com/OffchainLabs/prysm/v6/api/server/structs"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
@@ -22,160 +25,224 @@ func (c *beaconApiValidatorClient) beaconBlock(ctx context.Context, slot primiti
if len(graffiti) > 0 {
queryParams.Add("graffiti", hexutil.Encode(graffiti))
}
queryUrl := apiutil.BuildURL(fmt.Sprintf("/eth/v3/validator/blocks/%d", slot), queryParams)
produceBlockV3ResponseJson := structs.ProduceBlockV3Response{}
err := c.jsonRestHandler.Get(ctx, queryUrl, &produceBlockV3ResponseJson)
data, header, err := c.jsonRestHandler.GetSSZ(ctx, queryUrl)
if err != nil {
return nil, err
}
return processBlockResponse(
produceBlockV3ResponseJson.Version,
produceBlockV3ResponseJson.ExecutionPayloadBlinded,
json.NewDecoder(bytes.NewReader(produceBlockV3ResponseJson.Data)),
)
if strings.Contains(header.Get("Content-Type"), api.OctetStreamMediaType) {
ver, err := version.FromString(header.Get(api.VersionHeader))
if err != nil {
return nil, errors.Wrap(err, fmt.Sprintf("unsupported header version %s", header.Get(api.VersionHeader)))
}
isBlindedRaw := header.Get(api.ExecutionPayloadBlindedHeader)
isBlinded, err := strconv.ParseBool(isBlindedRaw)
if err != nil {
return nil, err
}
return processBlockSSZResponse(ver, data, isBlinded)
} else {
decoder := json.NewDecoder(bytes.NewBuffer(data))
produceBlockV3ResponseJson := structs.ProduceBlockV3Response{}
if err = decoder.Decode(&produceBlockV3ResponseJson); err != nil {
return nil, errors.Wrapf(err, "failed to decode response body into json for %s", queryUrl)
}
return processBlockJSONResponse(
produceBlockV3ResponseJson.Version,
produceBlockV3ResponseJson.ExecutionPayloadBlinded,
json.NewDecoder(bytes.NewReader(produceBlockV3ResponseJson.Data)),
)
}
}
// nolint: gocognit
func processBlockResponse(ver string, isBlinded bool, decoder *json.Decoder) (*ethpb.GenericBeaconBlock, error) {
var response *ethpb.GenericBeaconBlock
func processBlockSSZResponse(ver int, data []byte, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if ver >= version.Fulu {
return processBlockSSZResponseFulu(data, isBlinded)
}
if ver >= version.Electra {
return processBlockSSZResponseElectra(data, isBlinded)
}
if ver >= version.Deneb {
return processBlockSSZResponseDeneb(data, isBlinded)
}
if ver >= version.Capella {
return processBlockSSZResponseCapella(data, isBlinded)
}
if ver >= version.Bellatrix {
return processBlockSSZResponseBellatrix(data, isBlinded)
}
if ver >= version.Altair {
block := &ethpb.BeaconBlockAltair{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Altair{Altair: block}}, nil
}
if ver >= version.Phase0 {
block := &ethpb.BeaconBlock{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Phase0{Phase0: block}}, nil
}
return nil, fmt.Errorf("unsupported block version %s", version.String(ver))
}
func processBlockSSZResponseFulu(data []byte, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
blindedBlock := &ethpb.BlindedBeaconBlockFulu{}
if err := blindedBlock.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedFulu{BlindedFulu: blindedBlock}, IsBlinded: true}, nil
}
block := &ethpb.BeaconBlockContentsFulu{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Fulu{Fulu: block}}, nil
}
func processBlockSSZResponseElectra(data []byte, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
blindedBlock := &ethpb.BlindedBeaconBlockElectra{}
if err := blindedBlock.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedElectra{BlindedElectra: blindedBlock}, IsBlinded: true}, nil
}
block := &ethpb.BeaconBlockContentsElectra{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Electra{Electra: block}}, nil
}
func processBlockSSZResponseDeneb(data []byte, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
blindedBlock := &ethpb.BlindedBeaconBlockDeneb{}
if err := blindedBlock.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedDeneb{BlindedDeneb: blindedBlock}, IsBlinded: true}, nil
}
block := &ethpb.BeaconBlockContentsDeneb{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Deneb{Deneb: block}}, nil
}
func processBlockSSZResponseCapella(data []byte, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
blindedBlock := &ethpb.BlindedBeaconBlockCapella{}
if err := blindedBlock.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedCapella{BlindedCapella: blindedBlock}, IsBlinded: true}, nil
}
block := &ethpb.BeaconBlockCapella{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Capella{Capella: block}}, nil
}
func processBlockSSZResponseBellatrix(data []byte, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
blindedBlock := &ethpb.BlindedBeaconBlockBellatrix{}
if err := blindedBlock.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedBellatrix{BlindedBellatrix: blindedBlock}, IsBlinded: true}, nil
}
block := &ethpb.BeaconBlockBellatrix{}
if err := block.UnmarshalSSZ(data); err != nil {
return nil, err
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Bellatrix{Bellatrix: block}}, nil
}
func convertBlockToGeneric(decoder *json.Decoder, dest ethpb.GenericConverter, version string, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
typeName := version
if isBlinded {
typeName = "blinded " + typeName
}
if err := decoder.Decode(dest); err != nil {
return nil, errors.Wrapf(err, "failed to decode %s block response json", typeName)
}
genericBlock, err := dest.ToGeneric()
if err != nil {
return nil, errors.Wrapf(err, "failed to convert %s block", typeName)
}
return genericBlock, nil
}
func processBlockJSONResponse(ver string, isBlinded bool, decoder *json.Decoder) (*ethpb.GenericBeaconBlock, error) {
if decoder == nil {
return nil, errors.New("no produce block json decoder found")
}
switch ver {
case version.String(version.Phase0):
jsonPhase0Block := structs.BeaconBlock{}
if err := decoder.Decode(&jsonPhase0Block); err != nil {
return nil, errors.Wrap(err, "failed to decode phase0 block response json")
}
genericBlock, err := jsonPhase0Block.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get phase0 block")
}
response = genericBlock
return convertBlockToGeneric(decoder, &structs.BeaconBlock{}, version.String(version.Phase0), false)
case version.String(version.Altair):
jsonAltairBlock := structs.BeaconBlockAltair{}
if err := decoder.Decode(&jsonAltairBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode altair block response json")
}
genericBlock, err := jsonAltairBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get altair block")
}
response = genericBlock
return convertBlockToGeneric(decoder, &structs.BeaconBlockAltair{}, "altair", false)
case version.String(version.Bellatrix):
if isBlinded {
jsonBellatrixBlock := structs.BlindedBeaconBlockBellatrix{}
if err := decoder.Decode(&jsonBellatrixBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode blinded bellatrix block response json")
}
genericBlock, err := jsonBellatrixBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get blinded bellatrix block")
}
response = genericBlock
} else {
jsonBellatrixBlock := structs.BeaconBlockBellatrix{}
if err := decoder.Decode(&jsonBellatrixBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode bellatrix block response json")
}
genericBlock, err := jsonBellatrixBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get bellatrix block")
}
response = genericBlock
}
return processBellatrixBlock(decoder, isBlinded)
case version.String(version.Capella):
if isBlinded {
jsonCapellaBlock := structs.BlindedBeaconBlockCapella{}
if err := decoder.Decode(&jsonCapellaBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode blinded capella block response json")
}
genericBlock, err := jsonCapellaBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get blinded capella block")
}
response = genericBlock
} else {
jsonCapellaBlock := structs.BeaconBlockCapella{}
if err := decoder.Decode(&jsonCapellaBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode capella block response json")
}
genericBlock, err := jsonCapellaBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get capella block")
}
response = genericBlock
}
return processCapellaBlock(decoder, isBlinded)
case version.String(version.Deneb):
if isBlinded {
jsonDenebBlock := structs.BlindedBeaconBlockDeneb{}
if err := decoder.Decode(&jsonDenebBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode blinded deneb block response json")
}
genericBlock, err := jsonDenebBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get blinded deneb block")
}
response = genericBlock
} else {
jsonDenebBlockContents := structs.BeaconBlockContentsDeneb{}
if err := decoder.Decode(&jsonDenebBlockContents); err != nil {
return nil, errors.Wrap(err, "failed to decode deneb block response json")
}
genericBlock, err := jsonDenebBlockContents.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get deneb block")
}
response = genericBlock
}
return processDenebBlock(decoder, isBlinded)
case version.String(version.Electra):
if isBlinded {
jsonElectraBlock := structs.BlindedBeaconBlockElectra{}
if err := decoder.Decode(&jsonElectraBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode blinded electra block response json")
}
genericBlock, err := jsonElectraBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get blinded electra block")
}
response = genericBlock
} else {
jsonElectraBlockContents := structs.BeaconBlockContentsElectra{}
if err := decoder.Decode(&jsonElectraBlockContents); err != nil {
return nil, errors.Wrap(err, "failed to decode electra block response json")
}
genericBlock, err := jsonElectraBlockContents.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get electra block")
}
response = genericBlock
}
return processElectraBlock(decoder, isBlinded)
case version.String(version.Fulu):
if isBlinded {
jsonFuluBlock := structs.BlindedBeaconBlockFulu{}
if err := decoder.Decode(&jsonFuluBlock); err != nil {
return nil, errors.Wrap(err, "failed to decode blinded fulu block response json")
}
genericBlock, err := jsonFuluBlock.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get blinded fulu block")
}
response = genericBlock
} else {
jsonFuluBlockContents := structs.BeaconBlockContentsFulu{}
if err := decoder.Decode(&jsonFuluBlockContents); err != nil {
return nil, errors.Wrap(err, "failed to decode fulu block response json")
}
genericBlock, err := jsonFuluBlockContents.ToGeneric()
if err != nil {
return nil, errors.Wrap(err, "failed to get fulu block")
}
response = genericBlock
}
return processFuluBlock(decoder, isBlinded)
default:
return nil, errors.Errorf("unsupported consensus version `%s`", ver)
}
return response, nil
}
func processBellatrixBlock(decoder *json.Decoder, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
return convertBlockToGeneric(decoder, &structs.BlindedBeaconBlockBellatrix{}, "bellatrix", true)
}
return convertBlockToGeneric(decoder, &structs.BeaconBlockBellatrix{}, "bellatrix", false)
}
func processCapellaBlock(decoder *json.Decoder, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
return convertBlockToGeneric(decoder, &structs.BlindedBeaconBlockCapella{}, "capella", true)
}
return convertBlockToGeneric(decoder, &structs.BeaconBlockCapella{}, "capella", false)
}
func processDenebBlock(decoder *json.Decoder, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
return convertBlockToGeneric(decoder, &structs.BlindedBeaconBlockDeneb{}, "deneb", true)
}
return convertBlockToGeneric(decoder, &structs.BeaconBlockContentsDeneb{}, "deneb", false)
}
func processElectraBlock(decoder *json.Decoder, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
return convertBlockToGeneric(decoder, &structs.BlindedBeaconBlockElectra{}, "electra", true)
}
return convertBlockToGeneric(decoder, &structs.BeaconBlockContentsElectra{}, "electra", false)
}
func processFuluBlock(decoder *json.Decoder, isBlinded bool) (*ethpb.GenericBeaconBlock, error) {
if isBlinded {
return convertBlockToGeneric(decoder, &structs.BlindedBeaconBlockFulu{}, "fulu", true)
}
return convertBlockToGeneric(decoder, &structs.BeaconBlockContentsFulu{}, "fulu", false)
}

File diff suppressed because it is too large Load Diff

View File

@@ -56,6 +56,23 @@ func (mr *MockJsonRestHandlerMockRecorder) Get(ctx, endpoint, resp any) *gomock.
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockJsonRestHandler)(nil).Get), ctx, endpoint, resp)
}
// GetSSZ mocks base method.
func (m *MockJsonRestHandler) GetSSZ(ctx context.Context, endpoint string) ([]byte, http.Header, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "GetSSZ", ctx, endpoint)
ret0, _ := ret[0].([]byte)
ret1, _ := ret[1].(http.Header)
ret2, _ := ret[2].(error)
return ret0, ret1, ret2
}
// GetSSZ indicates an expected call of GetSSZ.
func (mr *MockJsonRestHandlerMockRecorder) GetSSZ(ctx, endpoint any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSSZ", reflect.TypeOf((*MockJsonRestHandler)(nil).GetSSZ), ctx, endpoint)
}
// Host mocks base method.
func (m *MockJsonRestHandler) Host() string {
m.ctrl.T.Helper()

View File

@@ -18,7 +18,7 @@ import (
)
// NewPrysmChainClient returns implementation of iface.PrysmChainClient.
func NewPrysmChainClient(jsonRestHandler JsonRestHandler, nodeClient iface.NodeClient) iface.PrysmChainClient {
func NewPrysmChainClient(jsonRestHandler RestHandler, nodeClient iface.NodeClient) iface.PrysmChainClient {
return prysmChainClient{
jsonRestHandler: jsonRestHandler,
nodeClient: nodeClient,
@@ -26,7 +26,7 @@ func NewPrysmChainClient(jsonRestHandler JsonRestHandler, nodeClient iface.NodeC
}
type prysmChainClient struct {
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
nodeClient iface.NodeClient
}

View File

@@ -4,49 +4,53 @@ import (
"bytes"
"context"
"encoding/json"
"fmt"
"io"
"net/http"
"strings"
"github.com/OffchainLabs/prysm/v6/api"
"github.com/OffchainLabs/prysm/v6/config/features"
"github.com/OffchainLabs/prysm/v6/network/httputil"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
type JsonRestHandler interface {
type RestHandler interface {
Get(ctx context.Context, endpoint string, resp interface{}) error
GetSSZ(ctx context.Context, endpoint string) ([]byte, http.Header, error)
Post(ctx context.Context, endpoint string, headers map[string]string, data *bytes.Buffer, resp interface{}) error
HttpClient() *http.Client
Host() string
SetHost(host string)
}
type BeaconApiJsonRestHandler struct {
type BeaconApiRestHandler struct {
client http.Client
host string
}
// NewBeaconApiJsonRestHandler returns a JsonRestHandler
func NewBeaconApiJsonRestHandler(client http.Client, host string) JsonRestHandler {
return &BeaconApiJsonRestHandler{
// NewBeaconApiRestHandler returns a RestHandler
func NewBeaconApiRestHandler(client http.Client, host string) RestHandler {
return &BeaconApiRestHandler{
client: client,
host: host,
}
}
// HttpClient returns the underlying HTTP client of the handler
func (c *BeaconApiJsonRestHandler) HttpClient() *http.Client {
func (c *BeaconApiRestHandler) HttpClient() *http.Client {
return &c.client
}
// Host returns the underlying HTTP host
func (c *BeaconApiJsonRestHandler) Host() string {
func (c *BeaconApiRestHandler) Host() string {
return c.host
}
// Get sends a GET request and decodes the response body as a JSON object into the passed in object.
// If an HTTP error is returned, the body is decoded as a DefaultJsonError JSON object and returned as the first return value.
func (c *BeaconApiJsonRestHandler) Get(ctx context.Context, endpoint string, resp interface{}) error {
func (c *BeaconApiRestHandler) Get(ctx context.Context, endpoint string, resp interface{}) error {
url := c.host + endpoint
req, err := http.NewRequestWithContext(ctx, http.MethodGet, url, nil)
if err != nil {
@@ -66,9 +70,61 @@ func (c *BeaconApiJsonRestHandler) Get(ctx context.Context, endpoint string, res
return decodeResp(httpResp, resp)
}
func (c *BeaconApiRestHandler) GetSSZ(ctx context.Context, endpoint string) ([]byte, http.Header, error) {
url := c.host + endpoint
req, err := http.NewRequestWithContext(ctx, http.MethodGet, url, nil)
if err != nil {
return nil, nil, errors.Wrapf(err, "failed to create request for endpoint %s", url)
}
primaryAcceptType := fmt.Sprintf("%s;q=%s", api.OctetStreamMediaType, "0.95")
secondaryAcceptType := fmt.Sprintf("%s;q=%s", api.JsonMediaType, "0.9")
acceptHeaderString := fmt.Sprintf("%s,%s", primaryAcceptType, secondaryAcceptType)
if features.Get().SSZOnly {
acceptHeaderString = api.OctetStreamMediaType
}
req.Header.Set("Accept", acceptHeaderString)
httpResp, err := c.client.Do(req)
if err != nil {
return nil, nil, errors.Wrapf(err, "failed to perform request for endpoint %s", url)
}
defer func() {
if err := httpResp.Body.Close(); err != nil {
return
}
}()
contentType := httpResp.Header.Get("Content-Type")
body, err := io.ReadAll(httpResp.Body)
if err != nil {
return nil, nil, errors.Wrapf(err, "failed to read response body for %s", httpResp.Request.URL)
}
if !strings.Contains(primaryAcceptType, contentType) {
log.WithFields(logrus.Fields{
"primaryAcceptType": primaryAcceptType,
"secondaryAcceptType": secondaryAcceptType,
"receivedAcceptType": contentType,
}).Debug("Server responded with non primary accept type")
}
// non-2XX codes are a failure
if !strings.HasPrefix(httpResp.Status, "2") {
decoder := json.NewDecoder(bytes.NewBuffer(body))
errorJson := &httputil.DefaultJsonError{}
if err = decoder.Decode(errorJson); err != nil {
return nil, nil, errors.Wrapf(err, "failed to decode response body into error json for %s", httpResp.Request.URL)
}
return nil, nil, errorJson
}
if features.Get().SSZOnly && contentType != api.OctetStreamMediaType {
return nil, nil, errors.Errorf("server responded with non primary accept type %s", contentType)
}
return body, httpResp.Header, nil
}
// Post sends a POST request and decodes the response body as a JSON object into the passed in object.
// If an HTTP error is returned, the body is decoded as a DefaultJsonError JSON object and returned as the first return value.
func (c *BeaconApiJsonRestHandler) Post(
func (c *BeaconApiRestHandler) Post(
ctx context.Context,
apiEndpoint string,
headers map[string]string,
@@ -136,6 +192,6 @@ func decodeResp(httpResp *http.Response, resp interface{}) error {
return nil
}
func (c *BeaconApiJsonRestHandler) SetHost(host string) {
func (c *BeaconApiRestHandler) SetHost(host string) {
c.host = host
}

View File

@@ -2,6 +2,7 @@ package beacon_api
import (
"bytes"
"context"
"encoding/json"
"io"
"net/http"
@@ -15,6 +16,8 @@ import (
"github.com/OffchainLabs/prysm/v6/testing/assert"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
"github.com/sirupsen/logrus/hooks/test"
)
func TestGet(t *testing.T) {
@@ -39,7 +42,7 @@ func TestGet(t *testing.T) {
server := httptest.NewServer(mux)
defer server.Close()
jsonRestHandler := BeaconApiJsonRestHandler{
jsonRestHandler := BeaconApiRestHandler{
client: http.Client{Timeout: time.Second * 5},
host: server.URL,
}
@@ -48,6 +51,98 @@ func TestGet(t *testing.T) {
assert.DeepEqual(t, genesisJson, resp)
}
func TestGetSSZ(t *testing.T) {
ctx := context.Background()
const endpoint = "/example/rest/api/ssz"
genesisJson := &structs.GetGenesisResponse{
Data: &structs.Genesis{
GenesisTime: "123",
GenesisValidatorsRoot: "0x456",
GenesisForkVersion: "0x789",
},
}
t.Run("Successful SSZ response", func(t *testing.T) {
expectedBody := []byte{10, 20, 30, 40}
mux := http.NewServeMux()
mux.HandleFunc(endpoint, func(w http.ResponseWriter, r *http.Request) {
assert.StringContains(t, api.OctetStreamMediaType, r.Header.Get("Accept"))
w.Header().Set("Content-Type", api.OctetStreamMediaType)
_, err := w.Write(expectedBody)
require.NoError(t, err)
})
server := httptest.NewServer(mux)
defer server.Close()
jsonRestHandler := BeaconApiRestHandler{
client: http.Client{Timeout: time.Second * 5},
host: server.URL,
}
body, header, err := jsonRestHandler.GetSSZ(ctx, endpoint)
require.NoError(t, err)
assert.DeepEqual(t, expectedBody, body)
require.StringContains(t, api.OctetStreamMediaType, header.Get("Content-Type"))
})
t.Run("Json Content-Type response", func(t *testing.T) {
logrus.SetLevel(logrus.DebugLevel)
defer logrus.SetLevel(logrus.InfoLevel) // reset it afterwards
logHook := test.NewGlobal()
mux := http.NewServeMux()
mux.HandleFunc(endpoint, func(w http.ResponseWriter, r *http.Request) {
assert.StringContains(t, api.OctetStreamMediaType, r.Header.Get("Accept"))
w.Header().Set("Content-Type", api.JsonMediaType)
marshalledJson, err := json.Marshal(genesisJson)
require.NoError(t, err)
_, err = w.Write(marshalledJson)
require.NoError(t, err)
})
server := httptest.NewServer(mux)
defer server.Close()
jsonRestHandler := BeaconApiRestHandler{
client: http.Client{Timeout: time.Second * 5},
host: server.URL,
}
body, header, err := jsonRestHandler.GetSSZ(ctx, endpoint)
require.NoError(t, err)
assert.LogsContain(t, logHook, "Server responded with non primary accept type")
require.Equal(t, api.JsonMediaType, header.Get("Content-Type"))
resp := &structs.GetGenesisResponse{}
require.NoError(t, json.Unmarshal(body, resp))
require.Equal(t, "123", resp.Data.GenesisTime)
})
t.Run("Wrong Content-Type response, doesn't error out and instead handled downstream", func(t *testing.T) {
logrus.SetLevel(logrus.DebugLevel)
defer logrus.SetLevel(logrus.InfoLevel) // reset it afterwards
logHook := test.NewGlobal()
mux := http.NewServeMux()
mux.HandleFunc(endpoint, func(w http.ResponseWriter, r *http.Request) {
assert.StringContains(t, api.OctetStreamMediaType, r.Header.Get("Accept"))
w.Header().Set("Content-Type", "text/plain") // Invalid content type
_, err := w.Write([]byte("some text"))
require.NoError(t, err)
})
server := httptest.NewServer(mux)
defer server.Close()
jsonRestHandler := BeaconApiRestHandler{
client: http.Client{Timeout: time.Second * 5},
host: server.URL,
}
_, _, err := jsonRestHandler.GetSSZ(ctx, endpoint)
require.NoError(t, err)
assert.LogsContain(t, logHook, "Server responded with non primary accept type")
})
}
func TestPost(t *testing.T) {
ctx := t.Context()
const endpoint = "/example/rest/api/endpoint"
@@ -85,7 +180,7 @@ func TestPost(t *testing.T) {
server := httptest.NewServer(mux)
defer server.Close()
jsonRestHandler := BeaconApiJsonRestHandler{
jsonRestHandler := BeaconApiRestHandler{
client: http.Client{Timeout: time.Second * 5},
host: server.URL,
}

View File

@@ -21,7 +21,7 @@ type StateValidatorsProvider interface {
}
type beaconApiStateValidatorsProvider struct {
jsonRestHandler JsonRestHandler
jsonRestHandler RestHandler
}
func (c beaconApiStateValidatorsProvider) StateValidators(

View File

@@ -9,7 +9,7 @@ import (
validatorHelpers "github.com/OffchainLabs/prysm/v6/validator/helpers"
)
func NewChainClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandler beaconApi.JsonRestHandler) iface.ChainClient {
func NewChainClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandler beaconApi.RestHandler) iface.ChainClient {
grpcClient := grpcApi.NewGrpcChainClient(validatorConn.GetGrpcClientConn())
if features.Get().EnableBeaconRESTApi {
return beaconApi.NewBeaconApiChainClientWithFallback(jsonRestHandler, grpcClient)
@@ -18,7 +18,7 @@ func NewChainClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandl
}
}
func NewPrysmChainClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandler beaconApi.JsonRestHandler) iface.PrysmChainClient {
func NewPrysmChainClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandler beaconApi.RestHandler) iface.PrysmChainClient {
if features.Get().EnableBeaconRESTApi {
return beaconApi.NewPrysmChainClient(jsonRestHandler, nodeClientFactory.NewNodeClient(validatorConn, jsonRestHandler))
} else {

View File

@@ -8,7 +8,7 @@ import (
validatorHelpers "github.com/OffchainLabs/prysm/v6/validator/helpers"
)
func NewNodeClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandler beaconApi.JsonRestHandler) iface.NodeClient {
func NewNodeClient(validatorConn validatorHelpers.NodeConnection, jsonRestHandler beaconApi.RestHandler) iface.NodeClient {
grpcClient := grpcApi.NewNodeClient(validatorConn.GetGrpcClientConn())
if features.Get().EnableBeaconRESTApi {
return beaconApi.NewNodeClientWithFallback(jsonRestHandler, grpcClient)

View File

@@ -179,7 +179,7 @@ func (v *ValidatorService) Start() {
return
}
restHandler := beaconApi.NewBeaconApiJsonRestHandler(
restHandler := beaconApi.NewBeaconApiRestHandler(
http.Client{Timeout: v.conn.GetBeaconApiTimeout(), Transport: otelhttp.NewTransport(http.DefaultTransport)},
hosts[0],
)

View File

@@ -10,7 +10,7 @@ import (
func NewValidatorClient(
validatorConn validatorHelpers.NodeConnection,
jsonRestHandler beaconApi.JsonRestHandler,
jsonRestHandler beaconApi.RestHandler,
opt ...beaconApi.ValidatorClientOpt,
) iface.ValidatorClient {
if features.Get().EnableBeaconRESTApi {

View File

@@ -55,7 +55,7 @@ func (s *Server) registerBeaconClient() error {
s.beaconApiTimeout,
)
restHandler := beaconApi.NewBeaconApiJsonRestHandler(
restHandler := beaconApi.NewBeaconApiRestHandler(
http.Client{Timeout: s.beaconApiTimeout, Transport: otelhttp.NewTransport(http.DefaultTransport)},
s.beaconApiEndpoint,
)