Compare commits

...

39 Commits
v5.3.3 ... rlnc

Author SHA1 Message Date
Potuz
ce48bcd0fc Paralellize commitment computation
- Also fix --rlnc-block-chunks flag to pass it on the validator
- Add a feature flag --rlnc-mesh-size to configure at runtime to how
  many peers we send chunks.
2025-05-15 14:28:09 -03:00
potuz
4217fee54c add benches 2025-05-15 11:37:30 -03:00
potuz
b8ccfb6d13 Adjust meshsize as a function of chunk numbers 2025-05-14 14:20:50 -03:00
potuz
c72e3d9d67 add chunks feature flag 2025-05-13 16:22:56 -03:00
Potuz
bb4022fd01 add larger trusted setup 2025-05-13 15:37:07 -03:00
Potuz
740de753fb Fix genesis block proposal (#15084)
* Fix genesis block proposal

* fix test

* fix test 2
2025-04-24 10:07:55 -03:00
Potuz
11fd5cb40a ignore some chunks and blocks earlier 2025-04-15 14:53:44 -03:00
Potuz
f4b5b89508 delay block proposal 2025-04-15 14:29:37 -03:00
Potuz
dcb1923d0b better logs 2025-04-15 14:23:09 -03:00
Potuz
0c69601014 Add feature flag to delay block broadcast 2025-04-15 13:41:36 -03:00
Potuz
82e74cfa64 do not verify block signatures over RPC 2025-01-29 11:19:31 -03:00
Potuz
a90cdd8d7d create new context to process blocks 2025-01-29 09:56:19 -03:00
Potuz
1220e0798a send block feed when received via chunks 2025-01-29 08:48:26 -03:00
nisdas
8a6d74d636 Use SignedBeaconBlock 2025-01-28 22:41:04 +08:00
Potuz
b6f76033f1 Fix chunking 2025-01-28 08:29:44 -03:00
nisdas
cc5681b0d8 Build Reproducible Test 2025-01-28 14:28:13 +08:00
nisdas
f8ddcb8978 Fix Interface Assertion 2025-01-28 13:02:22 +08:00
Potuz
84051f43bb broadcast chunks on gossip 2025-01-27 16:26:01 -03:00
Potuz
e593797a5c add log when decoding a block 2025-01-27 16:10:22 -03:00
Potuz
51b86abd06 Broadcast chunked blocks 2025-01-27 16:05:41 -03:00
Potuz
c2073dfd3f fix block encoding when chunking 2025-01-27 12:45:42 -03:00
nisdas
ca2a20e992 Fix Validator Config 2025-01-27 21:48:16 +08:00
Nishant Das
e4075b6684 Implement RLNC Broadcasting (#14830)
Add a pubsub API to broadcast chunks
2025-01-27 10:18:54 -03:00
Potuz
50e327c7f8 Do not check blob signature 2025-01-24 06:29:28 -03:00
Potuz
2771bfc5f7 Add the signatures over the commitments to the block before processing the state transition 2025-01-24 06:29:28 -03:00
Potuz
186974fc7f Add endpoint to propose blocks
The implementation of Broadcast is blocked as it needs to be changed
2025-01-24 06:29:28 -03:00
Potuz
9ca832b4fb remove signature check 2025-01-24 06:29:28 -03:00
Potuz
7a00a9cc16 Add Ristretto trusted setup
Load the setup from a file and use it across all clients.
2025-01-24 06:29:28 -03:00
Potuz
9a3bcb876c Add chunk broadcasting
Also make `ReadOnlyBeaconBlockChunk` actually read only and copy the
return values.
2025-01-24 06:29:28 -03:00
Potuz
8bb3c388ed Handle the block when it can be recovered
Still TODO

- Handle chunks for pending blocks
- Handle state transition logic
- Handle rebroadcasting
- Handle block production
2025-01-24 06:29:28 -03:00
Potuz
044456a2c3 prune the chunk cache every 10 minutes 2025-01-24 06:29:28 -03:00
Potuz
303ac87d17 Add a cache for incoming chunks
- Finish validation of each incoming chunk including signature
  verification when needed.

Still TODO:
- Handle chunks for pending blocks
- Convert the decoded block to an actual block and send it to the
  blockchain package.
2025-01-24 06:29:28 -03:00
Potuz
d93ab44a37 Handle incoming chunks on the P2P layer
- Add hooks for the new topic "beacon_block_chunk"
- Add protos for the new messages
- Handle the sync package logic of signature validation
- Create native types for the chunks to be handled in the sync package

Still TODO:
- Handle chunks for pending blocks.
- Handle nodes with incoming chunks, avoid verifying signature twice
- Decode the block and send it to the blockchain package
2025-01-24 06:29:28 -03:00
Potuz
8b905d24a0 Add feature flag 2025-01-24 06:29:28 -03:00
Potuz
f46eaa88bf Add decoding and tests. 2025-01-24 06:29:28 -03:00
Potuz
799bd33158 Add node, receive and prepare message functions 2025-01-24 06:29:28 -03:00
Potuz
f51132fcda Add echelon form and addRow 2025-01-24 06:29:28 -03:00
Potuz
a32ed2eb98 Add chunks and message verifying 2025-01-24 06:29:28 -03:00
Potuz
7043dacaf9 add committer 2025-01-24 06:29:28 -03:00
90 changed files with 70440 additions and 1326 deletions

View File

@@ -68,6 +68,7 @@ go_library(
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync/rlnc:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",

View File

@@ -3,10 +3,13 @@ package blockchain
import (
"context"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
consensus_blocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/forkchoice"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// CachedHeadRoot returns the corresponding value from Forkchoice
@@ -100,3 +103,26 @@ func (s *Service) ParentRoot(root [32]byte) ([32]byte, error) {
defer s.cfg.ForkChoiceStore.RUnlock()
return s.cfg.ForkChoiceStore.ParentRoot(root)
}
// hashForGenesisBlock returns the right hash for the genesis block
func (s *Service) hashForGenesisBlock(ctx context.Context, root [32]byte) ([]byte, error) {
genRoot, err := s.cfg.BeaconDB.GenesisBlockRoot(ctx)
if err != nil {
return nil, errors.Wrap(err, "could not get genesis block root")
}
if root != genRoot {
return nil, errNotGenesisRoot
}
st, err := s.cfg.BeaconDB.GenesisState(ctx)
if err != nil {
return nil, errors.Wrap(err, "could not get genesis state")
}
if st.Version() < version.Bellatrix {
return nil, nil
}
header, err := st.LatestExecutionPayloadHeader()
if err != nil {
return nil, errors.Wrap(err, "could not get latest execution payload header")
}
return bytesutil.SafeCopyBytes(header.BlockHash()), nil
}

View File

@@ -612,3 +612,20 @@ func TestService_IsFinalized(t *testing.T) {
require.Equal(t, true, c.IsFinalized(ctx, br))
require.Equal(t, false, c.IsFinalized(ctx, [32]byte{'c'}))
}
func Test_hashForGenesisRoot(t *testing.T) {
beaconDB := testDB.SetupDB(t)
ctx := context.Background()
c := setupBeaconChain(t, beaconDB)
st, _ := util.DeterministicGenesisStateElectra(t, 10)
require.NoError(t, c.cfg.BeaconDB.SaveGenesisData(ctx, st))
root, err := beaconDB.GenesisBlockRoot(ctx)
require.NoError(t, err)
genRoot, err := c.hashForGenesisBlock(ctx, [32]byte{'a'})
require.ErrorIs(t, err, errNotGenesisRoot)
require.IsNil(t, genRoot)
genRoot, err = c.hashForGenesisBlock(ctx, root)
require.NoError(t, err)
require.Equal(t, [32]byte{}, [32]byte(genRoot))
}

View File

@@ -30,6 +30,8 @@ var (
ErrNotCheckpoint = errors.New("not a checkpoint in forkchoice")
// ErrNilHead is returned when no head is present in the blockchain service.
ErrNilHead = errors.New("nil head")
// errNotGenesisRoot is returned when the root is not the genesis block root.
errNotGenesisRoot = errors.New("root is not the genesis block root")
)
var errMaxBlobsExceeded = errors.New("Expected commitments in block exceeds MAX_BLOBS_PER_BLOCK")

View File

@@ -69,6 +69,18 @@ func (s *Service) notifyForkchoiceUpdate(ctx context.Context, arg *fcuConfig) (*
SafeBlockHash: justifiedHash[:],
FinalizedBlockHash: finalizedHash[:],
}
if len(fcs.HeadBlockHash) != 32 || [32]byte(fcs.HeadBlockHash) == [32]byte{} {
// check if we are sending FCU at genesis
hash, err := s.hashForGenesisBlock(ctx, arg.headRoot)
if errors.Is(err, errNotGenesisRoot) {
log.Error("Sending nil head block hash to execution engine")
return nil, nil
}
if err != nil {
return nil, errors.Wrap(err, "could not get head block hash")
}
fcs.HeadBlockHash = hash
}
if arg.attributes == nil {
arg.attributes = payloadattribute.EmptyWithVersion(headBlk.Version())
}

View File

@@ -84,7 +84,7 @@ func Test_NotifyForkchoiceUpdate_GetPayloadAttrErrorCanContinue(t *testing.T) {
service.cfg.PayloadIDCache.Set(1, [32]byte{}, [8]byte{})
got, err := service.notifyForkchoiceUpdate(ctx, arg)
require.NoError(t, err)
require.DeepEqual(t, got, pid) // We still get a payload ID even though the state is bad. This means it returns until the end.
require.IsNil(t, got)
}
func Test_NotifyForkchoiceUpdate(t *testing.T) {
@@ -113,6 +113,7 @@ func Test_NotifyForkchoiceUpdate(t *testing.T) {
state, blkRoot, err = prepareForkchoiceState(ctx, 2, bellatrixBlkRoot, altairBlkRoot, params.BeaconConfig().ZeroHash, ojc, ofc)
require.NoError(t, err)
require.NoError(t, fcs.InsertNode(ctx, state, blkRoot))
badHash := [32]byte{'h'}
tests := []struct {
name string
@@ -210,7 +211,7 @@ func Test_NotifyForkchoiceUpdate(t *testing.T) {
blk: func() interfaces.ReadOnlySignedBeaconBlock {
b, err := consensusblocks.NewSignedBeaconBlock(&ethpb.SignedBeaconBlockBellatrix{Block: &ethpb.BeaconBlockBellatrix{
Body: &ethpb.BeaconBlockBodyBellatrix{
ExecutionPayload: &v1.ExecutionPayload{},
ExecutionPayload: &v1.ExecutionPayload{BlockHash: badHash[:]},
},
}})
require.NoError(t, err)

View File

@@ -16,6 +16,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
@@ -69,6 +70,14 @@ func WithDepositCache(c cache.DepositCache) Option {
}
}
// WithChunkCommitter for chunk committer.
func WithChunkCommitter(c *rlnc.Committer) Option {
return func(s *Service) error {
s.cfg.ChunkCommitter = c
return nil
}
}
// WithPayloadIDCache for payload ID cache.
func WithPayloadIDCache(c *cache.PayloadIDCache) Option {
return func(s *Service) error {

View File

@@ -651,24 +651,6 @@ func TestOnBlock_NilBlock(t *testing.T) {
require.Equal(t, true, IsInvalidBlock(err))
}
func TestOnBlock_InvalidSignature(t *testing.T) {
service, tr := minimalTestService(t)
ctx := tr.ctx
gs, keys := util.DeterministicGenesisState(t, 32)
require.NoError(t, service.saveGenesisData(ctx, gs))
blk, err := util.GenerateFullBlock(gs, keys, util.DefaultBlockGenConfig(), 1)
require.NoError(t, err)
blk.Signature = []byte{'a'} // Mutate the signature.
wsb, err := consensusblocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
preState, err := service.getBlockPreState(ctx, wsb.Block())
require.NoError(t, err)
_, err = service.validateStateTransition(ctx, preState, wsb)
require.Equal(t, true, IsInvalidBlock(err))
}
func TestOnBlock_CallNewPayloadAndForkchoiceUpdated(t *testing.T) {
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig()

View File

@@ -69,6 +69,10 @@ func (s *Service) ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySig
log.WithField("blockRoot", fmt.Sprintf("%#x", blockRoot)).Debug("Ignoring already synced block")
return nil
}
if s.BlockBeingSynced(blockRoot) {
log.WithField("blockRoot", fmt.Sprintf("%#x", blockRoot)).Debug("Ignoring already syncing block")
return nil
}
receivedTime := time.Now()
s.blockBeingSynced.set(blockRoot)
defer s.blockBeingSynced.unset(blockRoot)

View File

@@ -32,6 +32,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
@@ -92,6 +93,7 @@ type config struct {
FinalizedStateAtStartUp state.BeaconState
ExecutionEngineCaller execution.EngineCaller
SyncChecker Checker
ChunkCommitter *rlnc.Committer
}
// Checker is an interface used to determine if a node is in initial sync

View File

@@ -50,6 +50,11 @@ func (mb *mockBroadcaster) Broadcast(_ context.Context, _ proto.Message) error {
return nil
}
func (mb *mockBroadcaster) BroadcastBlockChunks(_ context.Context, _ []*ethpb.BeaconBlockChunk) error {
mb.broadcastCalled = true
return nil
}
func (mb *mockBroadcaster) BroadcastAttestation(_ context.Context, _ uint64, _ ethpb.Att) error {
mb.broadcastCalled = true
return nil

View File

@@ -291,52 +291,3 @@ func TestProcessBlockHeader_OK(t *testing.T) {
}
assert.Equal(t, true, proto.Equal(nsh, expected), "Expected %v, received %v", expected, nsh)
}
func TestBlockSignatureSet_OK(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().MinGenesisActiveValidatorCount)
for i := 0; i < len(validators); i++ {
validators[i] = &ethpb.Validator{
PublicKey: make([]byte, 32),
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
Slashed: true,
}
}
state, err := util.NewBeaconState()
require.NoError(t, err)
require.NoError(t, state.SetValidators(validators))
require.NoError(t, state.SetSlot(10))
require.NoError(t, state.SetLatestBlockHeader(util.HydrateBeaconHeader(&ethpb.BeaconBlockHeader{
Slot: 9,
ProposerIndex: 0,
})))
latestBlockSignedRoot, err := state.LatestBlockHeader().HashTreeRoot()
require.NoError(t, err)
currentEpoch := time.CurrentEpoch(state)
priv, err := bls.RandKey()
require.NoError(t, err)
pID, err := helpers.BeaconProposerIndex(context.Background(), state)
require.NoError(t, err)
block := util.NewBeaconBlock()
block.Block.Slot = 10
block.Block.ProposerIndex = pID
block.Block.Body.RandaoReveal = bytesutil.PadTo([]byte{'A', 'B', 'C'}, 96)
block.Block.ParentRoot = latestBlockSignedRoot[:]
block.Signature, err = signing.ComputeDomainAndSign(state, currentEpoch, block.Block, params.BeaconConfig().DomainBeaconProposer, priv)
require.NoError(t, err)
proposerIdx, err := helpers.BeaconProposerIndex(context.Background(), state)
require.NoError(t, err)
validators[proposerIdx].Slashed = false
validators[proposerIdx].PublicKey = priv.PublicKey().Marshal()
err = state.UpdateValidatorAtIndex(proposerIdx, validators[proposerIdx])
require.NoError(t, err)
set, err := blocks.BlockSignatureBatch(state, block.Block.ProposerIndex, block.Signature, block.Block.HashTreeRoot)
require.NoError(t, err)
verified, err := set.Verify()
require.NoError(t, err)
assert.Equal(t, true, verified, "Block signature set returned a set which was unable to be verified")
}

View File

@@ -120,24 +120,6 @@ func VerifyBlockSignatureUsingCurrentFork(beaconState state.ReadOnlyBeaconState,
})
}
// BlockSignatureBatch retrieves the block signature batch from the provided block and its corresponding state.
func BlockSignatureBatch(beaconState state.ReadOnlyBeaconState,
proposerIndex primitives.ValidatorIndex,
sig []byte,
rootFunc func() ([32]byte, error)) (*bls.SignatureBatch, error) {
currentEpoch := slots.ToEpoch(beaconState.Slot())
domain, err := signing.Domain(beaconState.Fork(), currentEpoch, params.BeaconConfig().DomainBeaconProposer, beaconState.GenesisValidatorsRoot())
if err != nil {
return nil, err
}
proposer, err := beaconState.ValidatorAtIndex(proposerIndex)
if err != nil {
return nil, err
}
proposerPubKey := proposer.PublicKey
return signing.BlockSignatureBatch(proposerPubKey, sig, domain, rootFunc)
}
// RandaoSignatureBatch retrieves the relevant randao specific signature batch object
// from a block and its corresponding state.
func RandaoSignatureBatch(

View File

@@ -0,0 +1,17 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = ["signature.go"],
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/chunks",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/params:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//network/forks:go_default_library",
"//time/slots:go_default_library",
],
)

View File

@@ -0,0 +1,53 @@
package chunks
import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/network/forks"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
// VerifyChunkSignature verifies the proposer signature of a beacon block chunk.
func VerifyChunkSignature(beaconState state.ReadOnlyBeaconState,
proposerIndex primitives.ValidatorIndex,
sig []byte,
rootFunc func() ([32]byte, error)) error {
currentEpoch := slots.ToEpoch(beaconState.Slot())
domain, err := signing.Domain(beaconState.Fork(), currentEpoch, params.BeaconConfig().DomainBeaconProposer, beaconState.GenesisValidatorsRoot())
if err != nil {
return err
}
proposer, err := beaconState.ValidatorAtIndex(proposerIndex)
if err != nil {
return err
}
proposerPubKey := proposer.PublicKey
return signing.VerifyBlockSigningRoot(proposerPubKey, sig, domain, rootFunc)
}
// VerifyChunkSignatureUsingCurrentFork verifies the proposer signature of a beacon block chunk. This differs
// from the above method by not using fork data from the state and instead retrieving it
// via the respective epoch.
func VerifyChunkSignatureUsingCurrentFork(beaconState state.ReadOnlyBeaconState, chunk interfaces.ReadOnlyBeaconBlockChunk) error {
currentEpoch := slots.ToEpoch(chunk.Slot())
fork, err := forks.Fork(currentEpoch)
if err != nil {
return err
}
domain, err := signing.Domain(fork, currentEpoch, params.BeaconConfig().DomainBeaconProposer, beaconState.GenesisValidatorsRoot())
if err != nil {
return err
}
proposer, err := beaconState.ValidatorAtIndex(chunk.ProposerIndex())
if err != nil {
return err
}
proposerPubKey := proposer.PublicKey
sig := chunk.Signature()
return signing.VerifyBlockSigningRoot(proposerPubKey, sig[:], domain, func() ([32]byte, error) {
return chunk.HeaderRoot(), nil
})
}

View File

@@ -180,12 +180,6 @@ func ProcessBlockNoVerifyAnySig(
return nil, nil, err
}
sig := signed.Signature()
bSet, err := b.BlockSignatureBatch(st, blk.ProposerIndex(), sig[:], blk.HashTreeRoot)
if err != nil {
tracing.AnnotateError(span, err)
return nil, nil, errors.Wrap(err, "could not retrieve block signature set")
}
randaoReveal := signed.Block().Body().RandaoReveal()
rSet, err := b.RandaoSignatureBatch(ctx, st, randaoReveal[:])
if err != nil {
@@ -199,7 +193,7 @@ func ProcessBlockNoVerifyAnySig(
// Merge beacon block, randao and attestations signatures into a set.
set := bls.NewSet()
set.Join(bSet).Join(rSet).Join(aSet)
set.Join(rSet).Join(aSet)
if blk.Version() >= version.Capella {
changes, err := signed.Block().Body().BLSToExecutionChanges()

View File

@@ -158,9 +158,8 @@ func TestProcessBlockNoVerify_SigSetContainsDescriptions(t *testing.T) {
set, _, err := transition.ProcessBlockNoVerifyAnySig(context.Background(), beaconState, wsb)
require.NoError(t, err)
assert.Equal(t, len(set.Signatures), len(set.Descriptions), "Signatures and descriptions do not match up")
assert.Equal(t, "block signature", set.Descriptions[0])
assert.Equal(t, "randao signature", set.Descriptions[1])
assert.Equal(t, "attestation signature", set.Descriptions[2])
assert.Equal(t, "randao signature", set.Descriptions[0])
assert.Equal(t, "attestation signature", set.Descriptions[1])
}
func TestProcessOperationsNoVerifyAttsSigs_OK(t *testing.T) {

View File

@@ -49,6 +49,7 @@ go_library(
"//beacon-chain/sync/checkpoint:go_default_library",
"//beacon-chain/sync/genesis:go_default_library",
"//beacon-chain/sync/initial-sync:go_default_library",
"//beacon-chain/sync/rlnc:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cmd:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",

View File

@@ -53,6 +53,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/checkpoint"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/genesis"
initialsync "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/initial-sync"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/cmd"
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
@@ -121,6 +122,7 @@ type BeaconNode struct {
BlobStorageOptions []filesystem.BlobStorageOption
verifyInitWaiter *verification.InitializerWaiter
syncChecker *initialsync.SyncChecker
chunkCommitter *rlnc.Committer
}
// New creates a new node instance, sets up configuration options, and registers
@@ -136,6 +138,11 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
registry := runtime.NewServiceRegistry()
ctx := cliCtx.Context
committer, err := rlnc.LoadTrustedSetup()
if err != nil {
return nil, errors.Wrap(err, "could not load the committer trusted setup")
}
beacon := &BeaconNode{
cliCtx: cliCtx,
ctx: ctx,
@@ -158,6 +165,7 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
serviceFlagOpts: &serviceFlagOpts{},
initialSyncComplete: make(chan struct{}),
syncChecker: &initialsync.SyncChecker{},
chunkCommitter: committer,
}
for _, opt := range opts {
@@ -751,6 +759,7 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
blockchain.WithBlobStorage(b.BlobStorage),
blockchain.WithTrackedValidatorsCache(b.trackedValidatorsCache),
blockchain.WithPayloadIDCache(b.payloadIDCache),
blockchain.WithChunkCommitter(b.chunkCommitter),
blockchain.WithSyncChecker(b.syncChecker),
)
@@ -836,6 +845,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithBlobStorage(b.BlobStorage),
regularsync.WithVerifierWaiter(b.verifyInitWaiter),
regularsync.WithAvailableBlocker(bFillStore),
regularsync.WithChunkCommitter(b.chunkCommitter),
)
return b.services.RegisterService(rs)
}
@@ -982,6 +992,7 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error {
BlobStorage: b.BlobStorage,
TrackedValidatorsCache: b.trackedValidatorsCache,
PayloadIDCache: b.payloadIDCache,
ChunkCommitter: b.chunkCommitter,
})
return b.services.RegisterService(rpcService)

View File

@@ -7,6 +7,7 @@ import (
"reflect"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/altair"
@@ -228,6 +229,36 @@ func (s *Service) BroadcastBlob(ctx context.Context, subnet uint64, blob *ethpb.
return nil
}
// BroadcastBlockChunks sends the passed messages to the pubsub topic
func (s *Service) BroadcastBlockChunks(ctx context.Context, chunks []*ethpb.BeaconBlockChunk) error {
ctx, span := trace.StartSpan(ctx, "p2p.BroadcastBlob")
defer span.End()
forkDigest, err := s.currentForkDigest()
if err != nil {
err := errors.Wrap(err, "could not retrieve fork digest")
tracing.AnnotateError(span, err)
return err
}
topic := RLNCTopicFormat
topic = fmt.Sprintf(topic, forkDigest)
multipleMessages := make([][]byte, len(chunks))
for i, c := range chunks {
buf := new(bytes.Buffer)
if _, err := s.Encoding().EncodeGossip(buf, c); err != nil {
err := errors.Wrap(err, "could not encode message")
tracing.AnnotateError(span, err)
return err
}
multipleMessages[i] = buf.Bytes()
}
if err := s.PublishMultipleToTopic(ctx, topic+s.Encoding().ProtocolSuffix(), multipleMessages, pubsub.WithRandomPublishing()); err != nil {
err := errors.Wrap(err, "could not publish message")
tracing.AnnotateError(span, err)
return err
}
return nil
}
func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blobSidecar *ethpb.BlobSidecar, forkDigest [4]byte) {
_, span := trace.StartSpan(ctx, "p2p.internalBroadcastBlob")
defer span.End()

View File

@@ -12,6 +12,7 @@ go_library(
visibility = [
"//beacon-chain:__subpackages__",
"//cmd:__subpackages__",
"//validator:__subpackages__",
],
deps = [
"//config/params:go_default_library",

View File

@@ -22,6 +22,7 @@ var gossipTopicMappings = map[string]func() proto.Message{
SyncCommitteeSubnetTopicFormat: func() proto.Message { return &ethpb.SyncCommitteeMessage{} },
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return &ethpb.SignedBLSToExecutionChange{} },
BlobSubnetTopicFormat: func() proto.Message { return &ethpb.BlobSidecar{} },
RLNCTopicFormat: func() proto.Message { return &ethpb.BeaconBlockChunk{} },
}
// GossipTopicMappings is a function to return the assigned data type
@@ -112,7 +113,7 @@ func init() {
GossipTypeMapping[reflect.TypeOf(&ethpb.SingleAttestation{})] = AttestationSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.AttesterSlashingElectra{})] = AttesterSlashingSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedAggregateAttestationAndProofElectra{})] = AggregateAndProofSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.BeaconBlockChunk{})] = RLNCTopicFormat
// Specially handle Fulu objects.
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockFulu{})] = BlockSubnetTopicFormat
}

View File

@@ -33,6 +33,7 @@ type P2P interface {
// Broadcaster broadcasts messages to peers over the p2p pubsub protocol.
type Broadcaster interface {
Broadcast(context.Context, proto.Message) error
BroadcastBlockChunks(context.Context, []*ethpb.BeaconBlockChunk) error
BroadcastAttestation(ctx context.Context, subnet uint64, att ethpb.Att) error
BroadcastSyncCommitteeMessage(ctx context.Context, subnet uint64, sMsg *ethpb.SyncCommitteeMessage) error
BroadcastBlob(ctx context.Context, subnet uint64, blob *ethpb.BlobSidecar) error

View File

@@ -98,6 +98,26 @@ func (s *Service) PublishToTopic(ctx context.Context, topic string, data []byte,
}
}
func (s *Service) PublishMultipleToTopic(ctx context.Context, topic string, msgs [][]byte, opts ...pubsub.PubOpt) error {
topicHandle, err := s.JoinTopic(topic)
if err != nil {
return err
}
// Wait for at least 1 peer to be available to receive the published message.
for {
if len(topicHandle.ListPeers()) > 0 || flags.Get().MinimumSyncPeers == 0 {
return topicHandle.PublishMultiple(ctx, msgs, opts...)
}
select {
case <-ctx.Done():
return errors.Wrapf(ctx.Err(), "unable to find requisite number of peers for topic %s, 0 peers found to publish to", topic)
default:
time.Sleep(100 * time.Millisecond)
}
}
}
// SubscribeToTopic joins (if necessary) and subscribes to PubSub topic.
func (s *Service) SubscribeToTopic(topic string, opts ...pubsub.SubOpt) (*pubsub.Subscription, error) {
s.awaitStateInitialized() // Genesis time and genesis validators root are required to subscribe.

View File

@@ -133,6 +133,10 @@ func (*FakeP2P) Broadcast(_ context.Context, _ proto.Message) error {
return nil
}
func (*FakeP2P) BroadcastBlockChunks(_ context.Context, _ []*ethpb.BeaconBlockChunk) error {
return nil
}
// BroadcastAttestation -- fake.
func (*FakeP2P) BroadcastAttestation(_ context.Context, _ uint64, _ ethpb.Att) error {
return nil

View File

@@ -27,6 +27,11 @@ func (m *MockBroadcaster) Broadcast(_ context.Context, msg proto.Message) error
return nil
}
func (m *MockBroadcaster) BroadcastBlockChunks(_ context.Context, _ []*ethpb.BeaconBlockChunk) error {
m.BroadcastCalled.Store(true)
return nil
}
// BroadcastAttestation records a broadcast occurred.
func (m *MockBroadcaster) BroadcastAttestation(_ context.Context, _ uint64, a ethpb.Att) error {
m.BroadcastCalled.Store(true)

View File

@@ -189,6 +189,11 @@ func (p *TestP2P) Broadcast(_ context.Context, _ proto.Message) error {
return nil
}
func (p *TestP2P) BroadcastBlockChunks(_ context.Context, _ []*ethpb.BeaconBlockChunk) error {
p.BroadcastCalled.Store(true)
return nil
}
// BroadcastAttestation broadcasts an attestation.
func (p *TestP2P) BroadcastAttestation(_ context.Context, _ uint64, _ ethpb.Att) error {
p.BroadcastCalled.Store(true)

View File

@@ -30,6 +30,8 @@ const (
GossipBlsToExecutionChangeMessage = "bls_to_execution_change"
// GossipBlobSidecarMessage is the name for the blob sidecar message type.
GossipBlobSidecarMessage = "blob_sidecar"
// GossipBlockChunkMessage is the name for the block chunk message type.
GossipBlockChunkMessage = "beacon_block_chunk"
// Topic Formats
//
// AttestationSubnetTopicFormat is the topic format for the attestation subnet.
@@ -52,4 +54,6 @@ const (
BlsToExecutionChangeSubnetTopicFormat = GossipProtocolAndDigest + GossipBlsToExecutionChangeMessage
// BlobSubnetTopicFormat is the topic format for the blob subnet.
BlobSubnetTopicFormat = GossipProtocolAndDigest + GossipBlobSidecarMessage + "_%d"
// RLNCTopicFormat is the topic format for the RLNC subnet.
RLNCTopicFormat = GossipProtocolAndDigest + GossipBlockChunkMessage
)

View File

@@ -51,6 +51,7 @@ go_library(
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync:go_default_library",
"//beacon-chain/sync/rlnc:go_default_library",
"//config/features:go_default_library",
"//config/params:go_default_library",
"//io/logs:go_default_library",

View File

@@ -17,6 +17,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/api"
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
coreblocks "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
corehelpers "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filters"
@@ -1182,6 +1183,13 @@ func (s *Server) validateConsensus(ctx context.Context, b *eth.GenericSignedBeac
if err != nil {
return errors.Wrap(err, "could not get parent state")
}
blockRoot, err := blk.Block().HashTreeRoot()
if err != nil {
return errors.Wrap(err, "could not hash block")
}
if err := coreblocks.VerifyBlockSignatureUsingCurrentFork(parentState, blk, blockRoot); err != nil {
return errors.Wrap(err, "could not verify block signature")
}
_, err = transition.ExecuteStateTransition(ctx, parentState, blk)
if err != nil {
return errors.Wrap(err, "could not execute state transition")

View File

@@ -63,6 +63,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync:go_default_library",
"//beacon-chain/sync/rlnc:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",

View File

@@ -22,6 +22,8 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/kv"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
@@ -269,6 +271,60 @@ func (vs *Server) BuildBlockParallel(ctx context.Context, sBlk interfaces.Signed
return vs.constructGenericBeaconBlock(sBlk, bundle, winningBid)
}
// ProposeChunkedBlock handles the proposal of chunked beacon blocks
func (vs *Server) ProposeChunkedBlock(ctx context.Context, req *ethpb.ChunkedBeaconBlock) (*ethpb.ProposeResponse, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.ProposeChunkedBlock")
defer span.End()
if req == nil {
return nil, status.Errorf(codes.InvalidArgument, "empty request")
}
block, err := blocks.NewSignedBeaconBlock(req.Block.Block)
if err != nil {
return nil, status.Errorf(codes.InvalidArgument, "%s: %v", "decode block failed", err)
}
var sidecars []*ethpb.BlobSidecar
if block.IsBlinded() {
block, sidecars, err = vs.handleBlindedBlock(ctx, block)
} else if block.Version() >= version.Deneb {
sidecars, err = vs.blobSidecarsFromUnblindedBlock(block, req.Block)
}
if err != nil {
return nil, status.Errorf(codes.Internal, "%s: %v", "handle block failed", err)
}
root, err := block.Block().HashTreeRoot()
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not hash tree root: %v", err)
}
var wg sync.WaitGroup
errChan := make(chan error, 1)
wg.Add(1)
go func() {
defer wg.Done()
if err := vs.broadcastReceiveChunkedBlock(ctx, req, root); err != nil {
errChan <- errors.Wrap(err, "broadcast/receive block failed")
return
}
errChan <- nil
}()
if err := vs.broadcastAndReceiveBlobs(ctx, sidecars, root); err != nil {
return nil, status.Errorf(codes.Internal, "Could not broadcast/receive blobs: %v", err)
}
wg.Wait()
if err := <-errChan; err != nil {
return nil, status.Errorf(codes.Internal, "Could not broadcast/receive block: %v", err)
}
return &ethpb.ProposeResponse{BlockRoot: root[:]}, nil
}
// ProposeBeaconBlock handles the proposal of beacon blocks.
func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.ProposeBeaconBlock")
@@ -362,12 +418,61 @@ func (vs *Server) blobSidecarsFromUnblindedBlock(block interfaces.SignedBeaconBl
return BuildBlobSidecars(block, rawBlobs, proofs)
}
// broadcastReceiveChunkedBlock broadcasts a chunked block and handles its reception.
func (vs *Server) broadcastReceiveChunkedBlock(ctx context.Context, req *ethpb.ChunkedBeaconBlock, root [32]byte) error {
block, err := blocks.NewSignedBeaconBlock(req.Block.Block)
if err != nil {
return errors.Wrap(err, "block construction failed")
}
messages, err := vs.constructChunkMessages(req)
if err != nil {
return errors.Wrap(err, "could not construct messages")
}
if err := slots.WaitUntil(ctx, vs.TimeFetcher.GenesisTime(), block.Block().Slot(), features.Get().DelayBlockBroadcast); err != nil {
return errors.Wrap(err, "could not wait until broadcast time")
}
if err := vs.P2P.BroadcastBlockChunks(ctx, messages); err != nil {
return errors.Wrap(err, "broadcast failed")
}
vs.BlockNotifier.BlockFeed().Send(&feed.Event{
Type: blockfeed.ReceivedBlock,
Data: &blockfeed.ReceivedBlockData{SignedBlock: block},
})
return vs.BlockReceiver.ReceiveBlock(ctx, block, root, nil)
}
func (s *Server) constructChunkMessages(cBlk *ethpb.ChunkedBeaconBlock) ([]*ethpb.BeaconBlockChunk, error) {
node, err := rlnc.NewNodeFromChunkedBlock(s.ChunkCommitter, cBlk)
if err != nil {
return nil, errors.Wrap(err, "could not construct node")
}
meshSize := features.Get().RLNCMeshSize
multipleMessages := make([]*ethpb.BeaconBlockChunk, 0, meshSize)
for i := uint(0); i < meshSize; i++ {
msg, err := node.PrepareMessage()
if err != nil {
return nil, errors.Wrap(err, "could not prepare message")
}
chunk := &ethpb.BeaconBlockChunk{
Data: msg.Data(),
Coefficients: msg.Coefficients(),
Header: cBlk.Header,
Signature: cBlk.Signature,
}
multipleMessages = append(multipleMessages, chunk)
}
return multipleMessages, nil
}
// broadcastReceiveBlock broadcasts a block and handles its reception.
func (vs *Server) broadcastReceiveBlock(ctx context.Context, block interfaces.SignedBeaconBlock, root [32]byte) error {
protoBlock, err := block.Proto()
if err != nil {
return errors.Wrap(err, "protobuf conversion failed")
}
if err := slots.WaitUntil(ctx, vs.TimeFetcher.GenesisTime(), block.Block().Slot(), features.Get().DelayBlockBroadcast); err != nil {
return errors.Wrap(err, "could not wait until broadcast time")
}
if err := vs.P2P.Broadcast(ctx, protoBlock); err != nil {
return errors.Wrap(err, "broadcast failed")
}

View File

@@ -27,6 +27,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/network/forks"
@@ -80,6 +81,7 @@ type Server struct {
ClockWaiter startup.ClockWaiter
CoreService *core.Service
AttestationStateFetcher blockchain.AttestationStateFetcher
ChunkCommitter *rlnc.Committer
}
// WaitForActivation checks if a validator public key exists in the active validator registry of the current

View File

@@ -40,6 +40,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
chainSync "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/io/logs"
@@ -121,6 +122,7 @@ type Config struct {
BlobStorage *filesystem.BlobStorage
TrackedValidatorsCache *cache.TrackedValidatorsCache
PayloadIDCache *cache.PayloadIDCache
ChunkCommitter *rlnc.Committer
}
// NewService instantiates a new RPC service instance that will
@@ -250,6 +252,7 @@ func NewService(ctx context.Context, cfg *Config) *Service {
TrackedValidatorsCache: s.cfg.TrackedValidatorsCache,
PayloadIDCache: s.cfg.PayloadIDCache,
AttestationStateFetcher: s.cfg.AttestationReceiver,
ChunkCommitter: s.cfg.ChunkCommitter,
}
s.validatorServer = validatorServer
nodeServer := &nodev1alpha1.Server{

View File

@@ -35,6 +35,7 @@ go_library(
"subscriber_beacon_aggregate_proof.go",
"subscriber_beacon_attestation.go",
"subscriber_beacon_blocks.go",
"subscriber_beacon_blocks_chunks.go",
"subscriber_blob_sidecar.go",
"subscriber_bls_to_execution_change.go",
"subscriber_handlers.go",
@@ -66,6 +67,7 @@ go_library(
"//beacon-chain/cache:go_default_library",
"//beacon-chain/core/altair:go_default_library",
"//beacon-chain/core/blocks:go_default_library",
"//beacon-chain/core/chunks:go_default_library",
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/block:go_default_library",
"//beacon-chain/core/feed/operation:go_default_library",
@@ -92,6 +94,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync/backfill/coverage:go_default_library",
"//beacon-chain/sync/rlnc:go_default_library",
"//beacon-chain/sync/verify:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cache/lru:go_default_library",
@@ -100,6 +103,7 @@ go_library(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/chunks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",

View File

@@ -18,6 +18,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/backfill/coverage"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
)
@@ -164,6 +165,14 @@ func WithStateNotifier(n statefeed.Notifier) Option {
}
}
// WithChunkCommitter gives the sync package direct access to the Ristretto trusted setup.
func WithChunkCommitter(c *rlnc.Committer) Option {
return func(s *Service) error {
s.cfg.chunkCommitter = c
return nil
}
}
// WithBlobStorage gives the sync package direct access to BlobStorage.
func WithBlobStorage(b *filesystem.BlobStorage) Option {
return func(s *Service) error {

View File

@@ -0,0 +1,47 @@
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"block_chunk_cache.go",
"committer.go",
"errors.go",
"matrix.go",
"message.go",
"node.go",
"trusted_setup.go",
],
embedsrcs = ["trusted_setup.json"],
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc",
visibility = ["//visibility:public"],
deps = [
"//config/features:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/rand:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_gtank_ristretto255//:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = [
"block_chunk_cache_test.go",
"committer_test.go",
"matrix_test.go",
"message_test.go",
"node_test.go",
"trusted_setup_test.go",
],
embed = [":go_default_library"],
deps = [
"//config/features:go_default_library",
"//consensus-types/chunks:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/require:go_default_library",
"@com_github_gtank_ristretto255//:go_default_library",
],
)

View File

@@ -0,0 +1,110 @@
package rlnc
import (
"sync"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
var maxChunkSize = uint(65536) // 2MB for 10 chunks.
type BlockChunkCache struct {
sync.Mutex
committer *Committer
nodes map[primitives.Slot]map[primitives.ValidatorIndex]*Node
}
func NewBlockChunkCache(committer *Committer) *BlockChunkCache {
return &BlockChunkCache{
committer: committer,
nodes: make(map[primitives.Slot]map[primitives.ValidatorIndex]*Node),
}
}
func (b *BlockChunkCache) AddChunk(chunk interfaces.ReadOnlyBeaconBlockChunk) error {
b.Lock()
defer b.Unlock()
m, err := newMessage(chunk)
if err != nil {
return errors.Wrap(err, "failed to create new message")
}
if _, ok := b.nodes[chunk.Slot()]; !ok {
b.nodes[chunk.Slot()] = make(map[primitives.ValidatorIndex]*Node)
} else if n, ok := b.nodes[chunk.Slot()][chunk.ProposerIndex()]; ok {
return n.receive(m)
}
node := NewNode(b.committer, features.Get().RLNCNumChunks)
if err := node.receive(m); err != nil {
return errors.Wrap(err, "failed to receive message")
}
b.nodes[chunk.Slot()][chunk.ProposerIndex()] = node
return ErrSignatureNotVerified
}
// GetBlockData returns the block for the given slot and proposer index if all the chunks are present.
func (b *BlockChunkCache) GetBlockData(slot primitives.Slot, proposerIndex primitives.ValidatorIndex) ([]byte, error) {
b.Lock()
defer b.Unlock()
if _, ok := b.nodes[slot]; !ok {
return nil, ErrNoData
}
if _, ok := b.nodes[slot][proposerIndex]; !ok {
return nil, ErrNoData
}
node := b.nodes[slot][proposerIndex]
return node.decode() // Only error is ErrNoData when the node is not full.
}
// Prune removes all nodes from before the given slot.
func (b *BlockChunkCache) Prune(slot primitives.Slot) {
b.Lock()
defer b.Unlock()
for s := range b.nodes {
if s < slot {
delete(b.nodes, s)
}
}
}
// RemoveNode removes the node that has the given chunk.
func (b *BlockChunkCache) RemoveNode(chunk interfaces.ReadOnlyBeaconBlockChunk) {
b.Lock()
defer b.Unlock()
if _, ok := b.nodes[chunk.Slot()]; !ok {
return
}
delete(b.nodes[chunk.Slot()], chunk.ProposerIndex())
}
// PrepareMessage prepares a message to broadcast after receiving the given chunk.
func (b *BlockChunkCache) PrepareMessage(chunk interfaces.ReadOnlyBeaconBlockChunk) (*ethpb.BeaconBlockChunk, error) {
b.Lock()
defer b.Unlock()
if _, ok := b.nodes[chunk.Slot()]; !ok {
return nil, ErrNoData
}
if _, ok := b.nodes[chunk.Slot()][chunk.ProposerIndex()]; !ok {
return nil, ErrNoData
}
node := b.nodes[chunk.Slot()][chunk.ProposerIndex()]
msg, err := node.PrepareMessage()
if err != nil {
return nil, errors.Wrap(err, "failed to prepare message")
}
signature := chunk.Signature()
return &ethpb.BeaconBlockChunk{
Data: msg.Data(),
Coefficients: msg.Coefficients(),
Header: chunk.Header(),
Signature: signature[:],
}, nil
}

View File

@@ -0,0 +1,77 @@
package rlnc
import (
"crypto/rand"
"testing"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/consensus-types/chunks"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestBlockChunkCache(t *testing.T) {
// Create a new block chunk cache.
committer := newCommitter(10)
cache := NewBlockChunkCache(committer)
require.NotNil(t, cache)
require.Equal(t, 0, len(cache.nodes))
chunkSize := uint(4)
block := make([]byte, features.Get().RLNCNumChunks*chunkSize*31)
_, err := rand.Read(block)
require.NoError(t, err)
node, err := NewSource(committer, features.Get().RLNCNumChunks, block)
require.NoError(t, err)
// Prepare a message
msg, err := node.PrepareMessage()
require.NoError(t, err)
require.NotNil(t, msg)
chunkProto := &ethpb.BeaconBlockChunk{
Data: msg.Data(),
Coefficients: msg.Coefficients(),
Header: &ethpb.BeaconBlockChunkHeader{
Slot: 1,
ProposerIndex: 1,
ParentRoot: make([]byte, 32),
Commitments: msg.Commitments(),
},
Signature: make([]byte, 96),
}
chunk, err := chunks.NewBlockChunk(chunkProto)
require.NoError(t, err)
// Add the chunk to the cache.
require.ErrorIs(t, ErrSignatureNotVerified, cache.AddChunk(chunk))
require.Equal(t, 1, len(cache.nodes))
// Prepare a second message
msg, err = node.PrepareMessage()
require.NoError(t, err)
require.NotNil(t, msg)
chunkProto = &ethpb.BeaconBlockChunk{
Data: msg.Data(),
Coefficients: msg.Coefficients(),
Header: &ethpb.BeaconBlockChunkHeader{
Slot: 1,
ProposerIndex: 1,
ParentRoot: make([]byte, 32),
Commitments: msg.Commitments(),
},
Signature: make([]byte, 96),
}
chunk, err = chunks.NewBlockChunk(chunkProto)
require.NoError(t, err)
// Add the chunk to the cache
require.NoError(t, cache.AddChunk(chunk)) // No error this time as the signature was verified before
require.Equal(t, 1, len(cache.nodes)) // No new node, same block chunk
cachedNode := cache.nodes[1][1]
require.Equal(t, 2, len(cachedNode.chunks))
message, err := cache.PrepareMessage(chunk)
require.NoError(t, err)
require.DeepEqual(t, message.Header.Commitments, chunkProto.Header.Commitments)
}

View File

@@ -0,0 +1,50 @@
package rlnc
import (
"errors"
ristretto "github.com/gtank/ristretto255"
"github.com/prysmaticlabs/prysm/v5/crypto/rand"
)
// Committer is a structure that holds the Ristretto generators.
type Committer struct {
generators []*ristretto.Element
}
// newCommitter creates a new committer with the number of generators.
// TODO: read the generators from the config file.
func newCommitter(n uint) *Committer {
generators := make([]*ristretto.Element, n)
for i := range generators {
generators[i] = randomElement()
if generators[i] == nil {
return nil
}
}
return &Committer{generators}
}
func (c *Committer) commit(scalars []*ristretto.Scalar) (*ristretto.Element, error) {
if len(scalars) > len(c.generators) {
return nil, errors.New("too many scalars")
}
result := &ristretto.Element{}
return result.VarTimeMultiScalarMult(scalars, c.generators[:len(scalars)]), nil
}
func (c *Committer) Num() int {
return len(c.generators)
}
func randomElement() (ret *ristretto.Element) {
buf := make([]byte, 64)
gen := rand.NewGenerator()
_, err := gen.Read(buf)
if err != nil {
return nil
}
ret = &ristretto.Element{}
ret.FromUniformBytes(buf)
return
}

View File

@@ -0,0 +1,35 @@
package rlnc
import (
"crypto/rand"
"testing"
ristretto "github.com/gtank/ristretto255"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestCommit(t *testing.T) {
n := 5
c := newCommitter(uint(n))
require.NotNil(t, c)
require.Equal(t, n, len(c.generators))
scalars := make([]*ristretto.Scalar, n+1)
randomBytes := make([]byte, 64)
for i := range scalars {
_, err := rand.Read(randomBytes)
require.NoError(t, err)
scalars[i] = &ristretto.Scalar{}
scalars[i].FromUniformBytes(randomBytes)
}
msm := &ristretto.Element{}
msm.VarTimeMultiScalarMult(scalars[:n], c.generators)
_, err := c.commit(scalars[:n-1])
require.NoError(t, err)
_, err = c.commit(scalars)
require.NotNil(t, err)
committment, err := c.commit(scalars[:n])
require.NoError(t, err)
require.Equal(t, 1, committment.Equal(msm))
}

View File

@@ -0,0 +1,12 @@
package rlnc
import "errors"
var ErrInvalidSize = errors.New("invalid size")
var ErrNoData = errors.New("no data")
var ErrIncorrectCommitments = errors.New("incorrect commitments")
var ErrInvalidMessage = errors.New("invalid message")
var ErrLinearlyDependentMessage = errors.New("linearly dependent message")
var ErrInvalidScalar = errors.New("invalid scalar encoding")
var ErrInvalidElement = errors.New("invalid element encoding")
var ErrSignatureNotVerified = errors.New("signature not verified") // ErrSignatureNotVerified is returned when the signature of a chunk is not yet verified.

View File

@@ -0,0 +1,202 @@
package rlnc
import (
"errors"
ristretto "github.com/gtank/ristretto255"
)
var zeroVector = ristretto.NewScalar()
func scalarLC(coeffs []*ristretto.Scalar, data [][]*ristretto.Scalar) (ret []*ristretto.Scalar, err error) {
if len(coeffs) != len(data) {
return nil, errors.New("different number of coefficients and vectors")
}
if len(data) == 0 {
return nil, nil
}
prod := ristretto.Scalar{}
ret = make([]*ristretto.Scalar, len(data[0]))
for i := range ret {
ret[i] = ristretto.NewScalar()
for j, c := range coeffs {
ret[i].Add(ret[i], prod.Multiply(c, data[j][i]))
}
}
return
}
// echelon is a struct that holds the echelon form of a matrix of coefficients and the
// corresponding transformation matrix to get it to that form. That is we are guaranteed to have
// transform * coefficients = triangular
type echelon struct {
coefficients [][]*ristretto.Scalar
triangular [][]*ristretto.Scalar
transform [][]*ristretto.Scalar
}
// newEchelon returns a new echelon struct.
func newEchelon(size uint) *echelon {
transform := make([][]*ristretto.Scalar, size)
for i := range transform {
transform[i] = make([]*ristretto.Scalar, size)
for j := range transform[i] {
if j != i {
transform[i][j] = ristretto.NewScalar()
} else {
transform[i][i] = scalarOne()
}
}
}
return &echelon{
coefficients: make([][]*ristretto.Scalar, 0),
triangular: make([][]*ristretto.Scalar, 0),
transform: transform,
}
}
func identityMatrix(size uint) [][]*ristretto.Scalar {
coefficients := make([][]*ristretto.Scalar, size)
for i := range coefficients {
coefficients[i] = make([]*ristretto.Scalar, size)
for j := range coefficients[i] {
if j == i {
coefficients[i][i] = scalarOne()
} else {
coefficients[i][j] = ristretto.NewScalar()
}
}
}
return coefficients
}
// newIdentityEchelon returns a new echelon struct with the identity coefficients.
func newIdentityEchelon(size uint) *echelon {
return &echelon{
coefficients: identityMatrix(size),
triangular: identityMatrix(size),
transform: identityMatrix(size),
}
}
func (e *echelon) isFull() bool {
if len(e.coefficients) == 0 {
return false
}
return len(e.coefficients) == len(e.coefficients[0])
}
// copyVector returns a copy of a vector.
func copyVector(v []*ristretto.Scalar) []*ristretto.Scalar {
ret := make([]*ristretto.Scalar, len(v))
for i, s := range v {
ret[i] = ristretto.NewScalar()
*ret[i] = *s
}
return ret
}
// firstEntry returns the index of the first non-zero entry in a vector. It returns -1 if the vector is all zeros.
func firstEntry(v []*ristretto.Scalar) int {
for i, s := range v {
if s.Equal(zeroVector) == 0 {
return i
}
}
return -1
}
// isZeroVector returns true if the vector is all zeros.
func isZeroVector(v []*ristretto.Scalar) bool {
return firstEntry(v) == -1
}
func (e *echelon) addRow(row []*ristretto.Scalar) bool {
// do not add malformed rows. This assumes transform is never nil.
if len(row) != len(e.transform[0]) {
return false
}
if isZeroVector(row) {
return false
}
// do not add anything if we have the full data.
if e.isFull() {
return false
}
// Add any incoming row if we are empty.
currentSize := len(e.coefficients)
if currentSize == 0 {
e.coefficients = append(e.coefficients, row)
e.triangular = append(e.triangular, row)
return true
}
// currentSize is the index we are about to add.
tr := copyVector(e.transform[currentSize])
newEchelonRow := copyVector(row)
i := 0
for ; i < currentSize; i++ {
j := firstEntry(e.triangular[i])
k := firstEntry(newEchelonRow)
if k == -1 {
return false
}
if j < k {
continue
}
if j > k {
break
}
pivot := *e.triangular[i][j]
f := *newEchelonRow[j]
for l := range newEchelonRow {
newEchelonRow[l].Multiply(newEchelonRow[l], &pivot)
newEchelonRow[l].Subtract(newEchelonRow[l], ristretto.NewScalar().Multiply(&f, e.triangular[i][l]))
tr[l].Multiply(tr[l], &pivot)
tr[l].Subtract(tr[l], ristretto.NewScalar().Multiply(&f, e.transform[i][l]))
}
}
if isZeroVector(newEchelonRow) {
return false
}
e.triangular = append(e.triangular[:i], append([][]*ristretto.Scalar{newEchelonRow}, e.triangular[i:]...)...)
e.coefficients = append(e.coefficients, row)
if i < currentSize {
e.transform = append(e.transform[:currentSize], e.transform[currentSize+1:]...)
e.transform = append(e.transform[:i], append([][]*ristretto.Scalar{tr}, e.transform[i:]...)...)
return true
}
e.transform[i] = tr
return true
}
func (e *echelon) inverse() (ret [][]*ristretto.Scalar, err error) {
if !e.isFull() {
return nil, ErrNoData
}
ret = make([][]*ristretto.Scalar, len(e.transform))
for i := range e.transform {
ret[i] = make([]*ristretto.Scalar, len(e.transform))
for j := range e.transform {
ret[i][j] = &ristretto.Scalar{}
*ret[i][j] = *e.transform[i][j]
}
}
pivot := &ristretto.Scalar{}
diff := &ristretto.Scalar{}
normalizedDiff := &ristretto.Scalar{}
for i := len(e.triangular) - 1; i >= 0; i-- {
pivot = pivot.Invert(e.triangular[i][i])
for j := 0; j < len(ret); j++ {
ret[i][j] = ret[i][j].Multiply(ret[i][j], pivot)
}
for j := i + 1; j < len(e.triangular); j++ {
diff = diff.Multiply(e.triangular[i][j], pivot)
for k := 0; k < len(ret); k++ {
normalizedDiff = normalizedDiff.Multiply(ret[j][k], diff)
ret[i][k] = ret[i][k].Subtract(ret[i][k], normalizedDiff)
}
}
}
return
}

View File

@@ -0,0 +1,141 @@
package rlnc
import (
"testing"
ristretto "github.com/gtank/ristretto255"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestMatrixMul(t *testing.T) {
// chunks have length 3 and there are two chunks
s11 := randomScalar()
s12 := randomScalar()
s13 := randomScalar()
scalars1 := []*ristretto.Scalar{s11, s12, s13}
s21 := randomScalar()
s22 := randomScalar()
s23 := randomScalar()
scalars2 := []*ristretto.Scalar{s21, s22, s23}
data := [][]*ristretto.Scalar{scalars1, scalars2}
coeff1 := randomScalar()
coeff2 := randomScalar()
coeff3 := randomScalar()
badCofficients := []*ristretto.Scalar{coeff1, coeff2, coeff3}
coefficients := badCofficients[:2]
// Bad number of coefficients
lc, err := scalarLC(badCofficients, data)
require.NotNil(t, err)
require.IsNil(t, lc)
lc, err = scalarLC(coefficients, data)
require.NoError(t, err)
require.Equal(t, len(scalars1), len(lc))
require.NotNil(t, lc[0])
require.Equal(t, 1, ristretto.NewScalar().Add(
ristretto.NewScalar().Multiply(coeff1, s11),
ristretto.NewScalar().Multiply(coeff2, s21)).Equal(lc[0]))
require.Equal(t, 1, ristretto.NewScalar().Add(
ristretto.NewScalar().Multiply(coeff1, s12),
ristretto.NewScalar().Multiply(coeff2, s22)).Equal(lc[1]))
require.Equal(t, 1, ristretto.NewScalar().Add(
ristretto.NewScalar().Multiply(coeff1, s13),
ristretto.NewScalar().Multiply(coeff2, s23)).Equal(lc[2]))
}
func Test_isZeroVector(t *testing.T) {
zero := ristretto.NewScalar()
nonZero := randomScalar()
require.Equal(t, true, isZeroVector([]*ristretto.Scalar{zero, zero, zero}))
require.Equal(t, false, isZeroVector([]*ristretto.Scalar{zero, zero, nonZero}))
}
func Test_scalarOne(t *testing.T) {
scalar := scalarOne()
other := randomScalar()
third := ristretto.NewScalar()
third.Multiply(scalar, other)
require.Equal(t, 1, third.Equal(other))
}
func TestAddRow(t *testing.T) {
e := newEchelon(3)
row := []*ristretto.Scalar{randomScalar(), randomScalar()}
require.Equal(t, false, e.addRow(row))
require.Equal(t, 0, len(e.coefficients))
row = append(row, randomScalar())
require.Equal(t, true, e.addRow(row))
require.Equal(t, 1, len(e.coefficients))
require.Equal(t, false, e.addRow(row))
require.Equal(t, 1, len(e.coefficients))
row = []*ristretto.Scalar{randomScalar(), randomScalar(), randomScalar()}
require.Equal(t, true, e.addRow(row))
require.Equal(t, 2, len(e.coefficients))
require.Equal(t, 1, e.triangular[1][0].Equal(ristretto.NewScalar()))
row = []*ristretto.Scalar{randomScalar(), randomScalar(), randomScalar()}
require.Equal(t, true, e.addRow(row))
require.Equal(t, 3, len(e.coefficients))
require.Equal(t, 1, e.triangular[2][0].Equal(ristretto.NewScalar()))
require.Equal(t, 1, e.triangular[2][1].Equal(ristretto.NewScalar()))
row = []*ristretto.Scalar{randomScalar(), randomScalar(), randomScalar()}
require.Equal(t, false, e.addRow(row))
// Check that the transform * coefficients = triangular
for i := 0; i < 3; i++ {
for j := 0; j < 3; j++ {
prod := ristretto.NewScalar()
for k := 0; k < 3; k++ {
prod = prod.Add(prod, ristretto.NewScalar().Multiply(e.transform[i][k], e.coefficients[k][j]))
}
require.Equal(t, 1, prod.Equal(e.triangular[i][j]))
}
}
}
func TestInverse(t *testing.T) {
e := newEchelon(3)
_, err := e.inverse()
require.ErrorIs(t, ErrNoData, err)
row := []*ristretto.Scalar{randomScalar(), randomScalar(), randomScalar()}
require.Equal(t, true, e.addRow(row))
row = []*ristretto.Scalar{randomScalar(), randomScalar(), randomScalar()}
require.Equal(t, true, e.addRow(row))
row = []*ristretto.Scalar{randomScalar(), randomScalar(), randomScalar()}
require.Equal(t, true, e.addRow(row))
inv, err := e.inverse()
require.NoError(t, err)
require.Equal(t, 3, len(inv))
require.Equal(t, 3, len(inv[0]))
require.Equal(t, 3, len(inv[1]))
require.Equal(t, 3, len(inv[2]))
// Check that the inverse * coefficients = identity
for i := 0; i < 3; i++ {
for j := 0; j < 3; j++ {
prod := ristretto.NewScalar()
for k := 0; k < 3; k++ {
prod = prod.Add(prod, ristretto.NewScalar().Multiply(inv[i][k], e.coefficients[k][j]))
}
if j != i {
require.Equal(t, 1, prod.Equal(ristretto.NewScalar()))
} else {
require.Equal(t, 1, prod.Equal(scalarOne()))
}
}
}
}

View File

@@ -0,0 +1,141 @@
package rlnc
import (
ristretto "github.com/gtank/ristretto255"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/crypto/rand"
"github.com/sirupsen/logrus"
)
type chunk struct {
data []*ristretto.Scalar
coefficients []*ristretto.Scalar
}
type Message struct {
chunk chunk
commitments []*ristretto.Element
}
// NewMessage creates a new message from a chunk interface.
func newMessage(c interfaces.ReadOnlyBeaconBlockChunk) (*Message, error) {
data, err := dataToVector(c.Data())
if err != nil {
return nil, err
}
coefficients, err := dataToVector(c.Coefficients())
if err != nil {
return nil, err
}
chunk := chunk{
data: data,
coefficients: coefficients,
}
commitments, err := dataToElements(c.Commitments())
if err != nil {
return nil, err
}
return &Message{
chunk: chunk,
commitments: commitments,
}, nil
}
// Verify verifies that the message is compatible with the signed committmments
func (m *Message) Verify(c *Committer) bool {
// We should get the same number of coefficients as commitments.
if len(m.chunk.coefficients) != len(m.commitments) {
return false
}
msm, err := c.commit(m.chunk.data)
if err != nil {
return false
}
if len(m.chunk.data) > c.Num() {
return false
}
com := ristretto.NewElement().VarTimeMultiScalarMult(m.chunk.coefficients, m.commitments)
return msm.Equal(com) == 1
}
var scalarOneBytes = [32]byte{1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}
func scalarOne() (ret *ristretto.Scalar) {
ret = &ristretto.Scalar{}
if err := ret.Decode(scalarOneBytes[:]); err != nil {
logrus.Error("failed to decode scalar one")
}
return
}
func randomScalar() (ret *ristretto.Scalar) {
buf := make([]byte, 64)
gen := rand.NewGenerator()
_, err := gen.Read(buf)
if err != nil {
return nil
}
ret = &ristretto.Scalar{}
ret.FromUniformBytes(buf)
return
}
// dataToVector converts a slice of scalars encoded as bytes to a slice of scalars.
func dataToVector(data [][]byte) ([]*ristretto.Scalar, error) {
ret := make([]*ristretto.Scalar, len(data))
for i, d := range data {
if len(d) != 32 {
return nil, ErrInvalidScalar
}
ret[i] = &ristretto.Scalar{}
if err := ret[i].Decode(d); err != nil {
return nil, ErrInvalidScalar
}
}
return ret, nil
}
// dataToElements converts a slice of scalars encoded as bytes to a slice of elements.
func dataToElements(data [][]byte) ([]*ristretto.Element, error) {
ret := make([]*ristretto.Element, len(data))
for i, d := range data {
if len(d) != 32 {
return nil, ErrInvalidElement
}
ret[i] = &ristretto.Element{}
if err := ret[i].Decode(d); err != nil {
return nil, ErrInvalidElement
}
}
return ret, nil
}
// Data returns the data of the message as serialized bytes
func (m *Message) Data() [][]byte {
data := make([][]byte, len(m.chunk.data))
for i, d := range m.chunk.data {
data[i] = d.Encode(nil)
}
return data
}
// Coefficients returns the coefficients of the message as serialized bytes
func (m *Message) Coefficients() [][]byte {
coefficients := make([][]byte, len(m.chunk.coefficients))
for i, c := range m.chunk.coefficients {
coefficients[i] = c.Encode(nil)
}
return coefficients
}
// Commitments returns the commitments of the message as serialized bytes
func (m *Message) Commitments() [][]byte {
commitments := make([][]byte, len(m.commitments))
for i, c := range m.commitments {
commitments[i] = make([]byte, 0)
commitments[i] = c.Encode(nil)
}
return commitments
}

View File

@@ -0,0 +1,100 @@
package rlnc
import (
"testing"
ristretto "github.com/gtank/ristretto255"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestVerify(t *testing.T) {
// chunks have length 3 and there are two chunks
s11 := randomScalar()
s12 := randomScalar()
s13 := randomScalar()
scalars1 := []*ristretto.Scalar{s11, s12, s13}
s21 := randomScalar()
s22 := randomScalar()
s23 := randomScalar()
scalars2 := []*ristretto.Scalar{s21, s22, s23}
data := [][]*ristretto.Scalar{scalars1, scalars2}
coeff1 := randomScalar()
coeff2 := randomScalar()
coeff3 := randomScalar()
goodCoefficients := []*ristretto.Scalar{coeff1, coeff2}
badCoefficients := []*ristretto.Scalar{coeff1, coeff3}
committer := newCommitter(3)
c1, err := committer.commit(scalars1)
require.NoError(t, err)
c2, err := committer.commit(scalars2)
require.NoError(t, err)
goodCommitments := []*ristretto.Element{c1, c2}
badCommitments := []*ristretto.Element{c1, c1}
goodData, err := scalarLC(goodCoefficients, data)
require.NoError(t, err)
badData, err := scalarLC(badCoefficients, data)
require.NoError(t, err)
tests := []struct {
name string
message *Message
expected bool
}{
{
name: "valid message",
message: &Message{
chunk: chunk{
data: goodData,
coefficients: goodCoefficients,
},
commitments: goodCommitments,
},
expected: true,
},
{
name: "invalid coefficients",
message: &Message{
chunk: chunk{
data: goodData,
coefficients: badCoefficients,
},
commitments: goodCommitments,
},
expected: false,
},
{
name: "invalid data",
message: &Message{
chunk: chunk{
data: badData,
coefficients: goodCoefficients,
},
commitments: goodCommitments,
},
expected: false,
},
{
name: "invalid commitments",
message: &Message{
chunk: chunk{
data: goodData,
coefficients: goodCoefficients,
},
commitments: badCommitments,
},
expected: false,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
result := tt.message.Verify(committer)
require.Equal(t, tt.expected, result)
})
}
}

View File

@@ -0,0 +1,361 @@
package rlnc
import (
"sync"
ristretto "github.com/gtank/ristretto255"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/sirupsen/logrus"
)
// rlncBlockSuffix is a byte that is added to the end of the block to mark it's end.
var rlncBlockSuffix = byte(0xfe)
// Node represents a node in the RLNC network. It keeps the data it holds as a matrix of scalars
// as well as the commitments to the data. The coefficients and a partial inversion of the corresponding
// matrix is kept in the echelon object. The committer keeps the trusted setup generators
type Node struct {
chunks [][]*ristretto.Scalar
commitments []*ristretto.Element
echelon *echelon
committer *Committer
slot primitives.Slot // repeated but convenient for the validator client
proposerIndex primitives.ValidatorIndex // repeated but convenient for the validator client
parentRoot []byte // repeated but convenient for the validator client
signature []byte // repeated but convenient for the validator client
}
func NewNode(committer *Committer, size uint) *Node {
return &Node{
chunks: make([][]*ristretto.Scalar, 0),
echelon: newEchelon(size),
committer: committer,
}
}
func NewNodeFromChunkedBlock(committer *Committer, blk *ethpb.ChunkedBeaconBlock) (*Node, error) {
size := uint(len(blk.Chunks))
chunks := make([][]*ristretto.Scalar, size)
var err error
for i, c := range blk.Chunks {
chunks[i], err = dataToVector(c.Data)
if err != nil {
return nil, errors.Wrap(err, "could not convert chunk data to scalar")
}
}
commitments, err := dataToElements(blk.Header.Commitments)
if err != nil {
return nil, errors.Wrap(err, "could not convert chunk commitments to Ristretto elements")
}
return &Node{
chunks: chunks,
commitments: commitments,
echelon: newIdentityEchelon(size),
committer: committer,
slot: blk.Header.Slot,
proposerIndex: blk.Header.ProposerIndex,
parentRoot: blk.Header.ParentRoot,
signature: blk.Signature,
}, nil
}
func (n *Node) GetChunkedBlock(blk *ethpb.GenericSignedBeaconBlock) *ethpb.ChunkedBeaconBlock {
chunks := make([]*ethpb.BeaconBlockChunkData, len(n.chunks))
for i, c := range n.Data() {
chunks[i] = &ethpb.BeaconBlockChunkData{
Data: c,
}
}
header := &ethpb.BeaconBlockChunkHeader{
Slot: n.Slot(),
ProposerIndex: n.ProposerIndex(),
ParentRoot: n.ParentRoot(),
Commitments: n.Commitments(),
}
return &ethpb.ChunkedBeaconBlock{
Header: header,
Chunks: chunks,
Signature: n.Signature(),
Block: blk,
}
}
// NewSource creates a new node that holds all the data already chunked and committed.
// It is called by a broadcasting node starting the RLNC process.
func NewSource(committer *Committer, size uint, data []byte) (*Node, error) {
chunks := blockToChunks(size, data)
commitments, err := computeCommitments(committer, chunks)
if err != nil {
return nil, err
}
return &Node{
chunks: chunks,
commitments: commitments,
echelon: newIdentityEchelon(size),
committer: committer,
}, nil
}
// SetSlot sets the slot of the node.
func (n *Node) SetSlot(slot primitives.Slot) {
n.slot = slot
}
// SetProposerIndex sets the proposer index of the node.
func (n *Node) SetProposerIndex(proposerIndex primitives.ValidatorIndex) {
n.proposerIndex = proposerIndex
}
// SetSignature sets the signature of the node.
func (n *Node) SetSignature(signature []byte) {
n.signature = signature
}
// SetParentRoot sets the parent root of the node.
func (n *Node) SetParentRoot(parentRoot []byte) {
n.parentRoot = parentRoot
}
// Slot returns the slot of the node.
func (n *Node) Slot() primitives.Slot {
return n.slot
}
// ProposerIndex returns the proposer index of the node.
func (n *Node) ProposerIndex() primitives.ValidatorIndex {
return n.proposerIndex
}
// Signature returns the signature of the node.
func (n *Node) Signature() []byte {
return n.signature
}
// ParentRoot returns the parent root of the node.
func (n *Node) ParentRoot() []byte {
return n.parentRoot
}
// computeCommitments computes the commitments of the data in the node.
func computeCommitments(c *Committer, data [][]*ristretto.Scalar) (commitments []*ristretto.Element, err error) {
if len(data) == 0 {
return nil, nil
}
commitments = make([]*ristretto.Element, len(data))
var wg sync.WaitGroup
errChan := make(chan error, 1)
var once sync.Once
for i, d := range data {
wg.Add(1)
go func(i int, d []*ristretto.Scalar) {
defer wg.Done()
commitment, err := c.commit(d)
if err != nil {
once.Do(func() {
errChan <- err
})
return
}
commitments[i] = commitment
}(i, d)
}
wg.Wait()
select {
case err := <-errChan:
return nil, err
default:
return commitments, nil
}
}
func blockToChunks(numChunks uint, data []byte) [][]*ristretto.Scalar {
chunks := make([][]*ristretto.Scalar, numChunks)
chunkSize := ((uint(len(data))+numChunks-1)/numChunks + 30) / 31 * 31
for i := uint(0); i < numChunks; i++ {
start := i * chunkSize
end := start + chunkSize
if end > uint(len(data)) {
end = uint(len(data))
}
// The last chunk can actually be fully zeroes.
if start > uint(len(data)) {
start = end
}
chunk := data[start:end]
// Pad the chunk with zeroes if necessary
if uint(len(chunk)) < chunkSize {
paddedChunk := make([]byte, chunkSize)
copy(paddedChunk, chunk)
chunk = paddedChunk
}
chunks[i] = bytesToVector(chunk)
}
return chunks
}
// bytesToVector converts a byte slice to a vector of scalars. The input is expected to be a multiple of 31 bytes.
// We take the very naive approach of decoding each 31 bytes of data to a scalar of 32 bytes,
// this way we are guaranteed to get a valid little Endian encoding of the scalar.
// TODO: The actual network encoding should take care of adding 5 zero bits every 256 bits so that
// we can avoid copying memory every time here. This is a temporary solution.
func bytesToVector(data []byte) []*ristretto.Scalar {
ret := make([]*ristretto.Scalar, len(data)/31)
buf := [32]byte{}
for i := 0; i < len(data); i += 31 {
ret[i/31] = ristretto.NewScalar()
copy(buf[:], data[i:i+31])
if err := ret[i/31].Decode(buf[:]); err != nil {
logrus.Error("failed to decode scalar")
}
}
return ret
}
func (n *Node) generateRandomCoeffs() []*ristretto.Scalar {
coeffs := make([]*ristretto.Scalar, len(n.chunks))
for i := 0; i < len(n.chunks); i++ {
coeffs[i] = randomScalar()
}
return coeffs
}
func (n *Node) chunkLC(scalars []*ristretto.Scalar) (chunk, error) {
if len(scalars) != len(n.chunks) {
return chunk{}, ErrInvalidSize
}
data, err := scalarLC(scalars, n.chunks)
if err != nil {
return chunk{}, err
}
coefficients, err := scalarLC(scalars, n.echelon.coefficients)
if err != nil {
return chunk{}, err
}
return chunk{
data: data,
coefficients: coefficients,
}, nil
}
func (n *Node) PrepareMessage() (*Message, error) {
if len(n.chunks) == 0 {
return nil, ErrNoData
}
scalars := n.generateRandomCoeffs()
chunk, err := n.chunkLC(scalars)
if err != nil {
return nil, err
}
return &Message{
chunk: chunk,
commitments: n.commitments,
}, nil
}
// checkExistingCommitments returns true if the commitments are the same as the ones in the node or the node didn't have any.
func (n *Node) checkExistingCommitments(c []*ristretto.Element) bool {
if len(n.commitments) == 0 {
return true
}
if len(c) != len(n.commitments) {
return false
}
for i := 0; i < len(c); i++ {
if c[i].Equal(n.commitments[i]) != 1 {
return false
}
}
return true
}
// checkExistingChunks checks that the incoming chunk has the same size as the preexisting ones if any.
func (n *Node) checkExistingChunks(c []*ristretto.Scalar) bool {
if len(n.chunks) == 0 {
return true
}
return len(c) == len(n.chunks[0])
}
func (n *Node) receive(message *Message) error {
if !n.checkExistingCommitments(message.commitments) {
return ErrIncorrectCommitments
}
if !n.checkExistingChunks(message.chunk.data) {
return ErrInvalidSize
}
if len(message.chunk.coefficients) != len(message.commitments) {
return ErrInvalidSize
}
if !message.Verify(n.committer) {
return ErrInvalidMessage
}
if !n.echelon.addRow(message.chunk.coefficients) {
return ErrLinearlyDependentMessage
}
n.chunks = append(n.chunks, message.chunk.data)
if len(n.commitments) == 0 {
n.commitments = make([]*ristretto.Element, len(message.commitments))
for i, c := range message.commitments {
n.commitments[i] = &ristretto.Element{}
*n.commitments[i] = *c
}
}
return nil
}
// stripSuffix returns the slice before the last appearance of suffix, returns the whole data if suffix is not found
func stripSuffix(data []byte, suffix byte) []byte {
for i := len(data) - 1; i > 0; i-- {
if data[i] == suffix {
return data[:i]
}
}
return data
}
func (n *Node) decode() ([]byte, error) {
inverse, err := n.echelon.inverse()
if err != nil {
return nil, err
}
ret := make([]byte, 0, len(n.chunks)*len(n.chunks[0])*31+1)
prod := &ristretto.Scalar{}
for i := 0; i < len(inverse); i++ {
for k := 0; k < len(n.chunks[0]); k++ {
entry := ristretto.NewScalar()
for j := 0; j < len(inverse); j++ {
prod = prod.Multiply(inverse[i][j], n.chunks[j][k])
entry = entry.Add(entry, prod)
}
ret = entry.Encode(ret)[:len(ret)+31] // len(ret) is computed before the append.
}
}
return stripSuffix(ret, rlncBlockSuffix), nil
}
// Data returns the data of the chunks in a node as serialized bytes
func (m *Node) Data() [][][]byte {
ret := make([][][]byte, len(m.chunks))
for j, c := range m.chunks {
ret[j] = make([][]byte, len(c))
for i, d := range c {
ret[j][i] = d.Encode(nil)
}
}
return ret
}
// Commitments returns the commitments of the chunks in a node as serialized bytes
func (m *Node) Commitments() [][]byte {
ret := make([][]byte, len(m.commitments))
for j, c := range m.commitments {
ret[j] = c.Encode(nil)
}
return ret
}

View File

@@ -0,0 +1,166 @@
package rlnc
import (
"crypto/rand"
"encoding/hex"
"testing"
ristretto "github.com/gtank/ristretto255"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestPrepareMessage(t *testing.T) {
numChunks := uint(10)
chunkSize := uint(100)
committer := newCommitter(chunkSize)
block := make([]byte, numChunks*chunkSize*31)
_, err := rand.Read(block)
require.NoError(t, err)
node, err := NewSource(committer, numChunks, block)
require.NoError(t, err)
message, err := node.PrepareMessage()
require.NoError(t, err)
require.NotNil(t, message)
require.Equal(t, true, message.Verify(committer))
emptyNode := NewNode(committer, numChunks)
_, err = emptyNode.PrepareMessage()
require.ErrorIs(t, ErrNoData, err)
}
func TestReceive(t *testing.T) {
numChunks := uint(2)
chunkSize := uint(100)
committer := newCommitter(chunkSize)
block := make([]byte, numChunks*chunkSize*31)
_, err := rand.Read(block)
require.NoError(t, err)
node, err := NewSource(committer, numChunks, block)
require.NoError(t, err)
// Send one message
message, err := node.PrepareMessage()
require.NoError(t, err)
require.NotNil(t, message)
receiver := NewNode(committer, numChunks)
require.NoError(t, receiver.receive(message))
require.Equal(t, 1, len(receiver.chunks))
// Send another message
message, err = node.PrepareMessage()
require.NoError(t, err)
require.NotNil(t, message)
require.NoError(t, receiver.receive(message))
require.Equal(t, 2, len(receiver.chunks))
// The third one should fail
message, err = node.PrepareMessage()
require.NoError(t, err)
require.NotNil(t, message)
require.ErrorIs(t, ErrLinearlyDependentMessage, receiver.receive(message))
require.Equal(t, 2, len(receiver.chunks))
}
func TestDecode(t *testing.T) {
numChunks := uint(3)
chunkSize := uint(10)
committer := newCommitter(chunkSize)
block := make([]byte, numChunks*chunkSize*31)
_, err := rand.Read(block)
block[len(block)-1] = 0xfe // termination byte
require.NoError(t, err)
node, err := NewSource(committer, numChunks, block)
require.NoError(t, err)
emptyNode := NewNode(committer, numChunks)
for i := 0; i < int(numChunks); i++ {
_, err = emptyNode.decode()
require.ErrorIs(t, ErrNoData, err)
message, err := node.PrepareMessage()
require.NoError(t, err)
require.NoError(t, emptyNode.receive(message))
}
decoded, err := emptyNode.decode()
require.NoError(t, err)
require.DeepEqual(t, block[:len(block)-1], decoded)
}
func TestDecodeLarge(t *testing.T) {
numChunks := uint(10)
blockHex := "bc0e0401000901007b0907f0f500c1441f3332515f27b5371829016890ec25b7b6464ce1781f863a2d50bca20d13f464e4da6b4ee0f208dad1d890023d3bef4be7084148d9e709d06a727e5133e454000000a63c7a7a1e189e8b7ccb767bd808b97a6ef7e7bf445ea5b663f882102031d0f05cbdcf7cb691dd0769ca73771b0353a20a38e2e2872b1d034caf1b4d392e49d11730939b544208663c99b4eca4e6684bc0b52ad8da7d5b19399e4badd867fe0fd70a234731285c6804c2a4f56711ddb8c82c99740f207854891028af34e27e5e00000000000000006d58a19c0870ceeaa225e8edf1fcbca5ee993c26c3d8a81adc2d2903c857128d54616b6f79616b69000dfc3e010008440100150408ec04000504f079ffffffff8d0941132d8b818e3688d3fedaf35f06607329a11609c8988bc9beaba7620280eb1b638500f1b0be7851eeb718bd8c150fb7a0d93de240ec9633ef61ab0cb299fc2d7c9ebf7b257b708b2f09c880ceff9f56f57ba61b31b25aa7d837b57571ddec04000010000000f6000000dc010000c2020000e400199e00020d0b7eac010d27a201007e5000f0658c61ab8ddf9bca9e5f197b4c109ba711c47bd3ce1ee1817c34a62b0878c3d3d0743605ad2c921c1e374f17fc3e056c300e90274033765e377682d42ddf3803bd245566f7fcb05ee5152b318c7132df9e2f1909162f52d6ce5c3258ed1a4fd731560fe400000011af00030d097e96000d27a201007e5000f061823821d2cfa82d8d780ed0876c4d0c318f1b3fd4f1fd737786d0a21bdcef7c606bbad4131d8506ac553bf1abd1a187f911310bbb4a4ce878cf60b75540d5187b0a1ef67624cb1d3e39c66dbf932f1b0916d97d7f1b20ee2bf0cbc963f5e2efab9e062ee60011017e960011289e01007e5000f06192395c50082d15bc3c4e7fe88b105805813975c19a458fca626098365c255f377d9e245037c1b621750d263c565166510a43fa90e4478fb0cce3dde050db2b0f81ce2b20e8615defccab99423795f2d5edc31b2844602f913e416bd764f81abaf0062ee60091667e96009ede0011017e5000f065878aa8c6899a9ed25b66fcaee48033b6c1f16d23e1565327fcea6f9cc42b5731a46d9402a64d50314fca6e08b72d62a718c8324d00a2365190093221e51a109dadaed0f38b7e8ee402b557c7d4ba5b5775858a2190ac997dbc8bd9c336371f58520c6d58a19c6e6404f055ab4de8ffccf7b19aa6d7d4ccc4c82f091ebd5715b978681cf010a0b67a38242ae933d7238681678b4e14f1598387b70d3beaf7c956e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b42100003100fe0100fe0100fe0100d601007e7401516e0880c3c9110b05010c7b66986701091cfc010000c0702734010c5e01008c114188951f5052ea700e7c99d32aabb6c5ba121b04d8d1ec722bb7d9d58306ccfc010000fe"
block, err := hex.DecodeString(blockHex)
require.NoError(t, err)
chunkSize := ((uint(len(block))+numChunks-1)/numChunks + 30) / 31 * 31
committer := newCommitter(chunkSize)
node, err := NewSource(committer, numChunks, block)
require.NoError(t, err)
emptyNode := NewNode(committer, numChunks)
for i := 0; i < int(numChunks); i++ {
_, err = emptyNode.decode()
require.ErrorIs(t, ErrNoData, err)
message, err := node.PrepareMessage()
require.NoError(t, err)
require.NoError(t, emptyNode.receive(message))
}
decoded, err := emptyNode.decode()
require.NoError(t, err)
// Remove termination byte
require.DeepEqual(t, block[:len(block)-1], decoded)
}
func TestOneScalar(t *testing.T) {
scalarHex := "1f5052ea700e7c99d32aabb6c5ba121b04d8d1ec722bb7d9d58306ccfc010000"
stringBytes := []byte{0x1f, 0x50, 0x52, 0xea, 0x70, 0xe, 0x7c, 0x99, 0xd3, 0x2a, 0xab, 0xb6, 0xc5, 0xba, 0x12, 0x1b, 0x4, 0xd8, 0xd1, 0xec, 0x72, 0x2b, 0xb7, 0xd9, 0xd5, 0x83, 0x6, 0xcc, 0xfc, 0x1, 0x0, 0x0}
scalarBytes, err := hex.DecodeString(scalarHex)
require.DeepEqual(t, stringBytes, scalarBytes)
require.NoError(t, err)
scalar := &ristretto.Scalar{}
require.NoError(t, scalar.Decode(scalarBytes))
obtained := scalar.Encode(nil)
require.DeepEqual(t, scalarBytes, obtained)
}
func BenchmarkChunkMSM_6MB(b *testing.B) {
numChunks := uint(10)
chunkSize := uint(19660) // 6MB Blocks
committer, err := LoadTrustedSetup()
require.NoError(b, err)
block := make([]byte, numChunks*chunkSize*31)
_, err = rand.Read(block)
require.NoError(b, err)
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := NewSource(committer, numChunks, block)
require.NoError(b, err)
}
}
func BenchmarkChunkMSM_2MB(b *testing.B) {
numChunks := uint(10)
chunkSize := uint(6554) // 2MB Blocks
committer, err := LoadTrustedSetup()
require.NoError(b, err)
block := make([]byte, numChunks*chunkSize*31)
_, err = rand.Read(block)
require.NoError(b, err)
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := NewSource(committer, numChunks, block)
require.NoError(b, err)
}
}
func BenchmarkChunkMSM_200KB(b *testing.B) {
numChunks := uint(10)
chunkSize := uint(640) // 200KB Blocks
committer, err := LoadTrustedSetup()
require.NoError(b, err)
block := make([]byte, numChunks*chunkSize*31)
_, err = rand.Read(block)
require.NoError(b, err)
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := NewSource(committer, numChunks, block)
require.NoError(b, err)
}
}

View File

@@ -0,0 +1,23 @@
package rlnc
import (
_ "embed"
"encoding/json"
ristretto "github.com/gtank/ristretto255"
"github.com/pkg/errors"
)
var (
//go:embed trusted_setup.json
embeddedTrustedSetup []byte // 311KB
)
func LoadTrustedSetup() (*Committer, error) {
var elements []*ristretto.Element
err := json.Unmarshal(embeddedTrustedSetup, &elements)
if err != nil {
return nil, errors.Wrap(err, "could not parse trusted setup JSON")
}
return &Committer{elements}, nil
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,15 @@
package rlnc
import (
"testing"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestTrustedSetup(t *testing.T) {
// Load the trusted setup.
committer, err := LoadTrustedSetup()
require.NoError(t, err)
require.NotNil(t, committer)
require.Equal(t, maxChunkSize, uint(committer.Num()))
}

View File

@@ -37,6 +37,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/backfill/coverage"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
lruwrpr "github.com/prysmaticlabs/prysm/v5/cache/lru"
"github.com/prysmaticlabs/prysm/v5/config/params"
@@ -102,6 +103,7 @@ type config struct {
clock *startup.Clock
stateNotifier statefeed.Notifier
blobStorage *filesystem.BlobStorage
chunkCommitter *rlnc.Committer
}
// This defines the interface for interacting with block chain service
@@ -164,6 +166,7 @@ type Service struct {
newBlobVerifier verification.NewBlobVerifier
availableBlocker coverage.AvailableBlocker
ctxMap ContextByteVersions
blockChunkCache *rlnc.BlockChunkCache
}
// NewService initializes new regular sync service.
@@ -210,6 +213,7 @@ func NewService(ctx context.Context, opts ...Option) *Service {
r.subHandler = newSubTopicHandler()
r.rateLimiter = newRateLimiter(r.cfg.p2p)
r.initCaches()
async.RunEvery(ctx, 10*time.Minute, r.startChunkPruner)
return r
}
@@ -291,6 +295,7 @@ func (s *Service) initCaches() {
s.seenAttesterSlashingCache = make(map[uint64]bool)
s.seenProposerSlashingCache = lruwrpr.New(seenProposerSlashingSize)
s.badBlockCache = lruwrpr.New(badBlockSize)
s.blockChunkCache = rlnc.NewBlockChunkCache(s.cfg.chunkCommitter)
}
func (s *Service) waitForChainStart() {

View File

@@ -79,12 +79,21 @@ func (s *Service) activeSyncSubnetIndices(currentSlot primitives.Slot) []uint64
// Register PubSub subscribers
func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
s.subscribe(
p2p.BlockSubnetTopicFormat,
s.validateBeaconBlockPubSub,
s.beaconBlockSubscriber,
digest,
)
if features.Get().UseRLNC {
s.subscribe(
p2p.RLNCTopicFormat,
s.validateBeaconBlockChunkPubSub,
s.beaconBlockChunkSubscriber,
digest,
)
} else {
s.subscribe(
p2p.BlockSubnetTopicFormat,
s.validateBeaconBlockPubSub,
s.beaconBlockSubscriber,
digest,
)
}
s.subscribe(
p2p.AggregateAndProofSubnetTopicFormat,
s.validateAggregateAndProof,

View File

@@ -0,0 +1,246 @@
package sync
import (
"context"
"fmt"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
fastssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
core_chunks "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/chunks"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/encoder"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/chunks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"github.com/sirupsen/logrus"
"google.golang.org/protobuf/proto"
)
// beaconBlockChunkSubscriber is a noop since all syncing happens at the validation step
func (s *Service) beaconBlockChunkSubscriber(_ context.Context, _ proto.Message) error {
return nil
}
func (s *Service) validateBeaconBlockChunkPubSub(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
ctx, span := trace.StartSpan(ctx, "sync.validateBeaconBlockChunkPubSub")
defer span.End()
m, err := s.decodePubsubMessage(msg)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, errors.Wrap(err, "Could not decode message")
}
wrappedChunk, err := chunks.NewBlockChunk(m)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, errors.Wrap(err, "Could not create chunk object")
}
// It's fine to use the same lock for both block and chunk validation
s.validateBlockLock.Lock()
defer s.validateBlockLock.Unlock()
chunk := interfaces.ReadOnlyBeaconBlockChunk(wrappedChunk)
if chunk.IsNil() {
return pubsub.ValidationReject, errors.New("chunk is nil")
}
// Ignore the chunk if we have seen the block
if s.hasSeenBlockIndexSlot(chunk.Slot(), chunk.ProposerIndex()) {
return pubsub.ValidationIgnore, nil
}
// Check if parent is a bad block and then reject the chunk.
if s.hasBadBlock(chunk.ParentRoot()) {
err := fmt.Errorf("received chunk that has an invalid parent %#x", chunk.ParentRoot())
log.WithError(err).Debug("Received block with an invalid parent")
return pubsub.ValidationReject, err
}
// Be lenient in handling early blocks. Instead of discarding blocks arriving later than
// MAXIMUM_GOSSIP_CLOCK_DISPARITY in future, we tolerate blocks arriving at max two slots
// earlier (SECONDS_PER_SLOT * 2 seconds). Queue such blocks and process them at the right slot.
genesisTime := uint64(s.cfg.clock.GenesisTime().Unix())
if err := slots.VerifyTime(genesisTime, chunk.Slot(), earlyBlockProcessingTolerance); err != nil {
log.WithError(err).Debug("Ignored chunk: could not verify slot time")
return pubsub.ValidationIgnore, nil
}
cp := s.cfg.chain.FinalizedCheckpt()
startSlot, err := slots.EpochStart(cp.Epoch)
if err != nil {
log.WithError(err).Debug("Ignored block: could not calculate epoch start slot")
return pubsub.ValidationIgnore, nil
}
if startSlot >= chunk.Slot() {
err := fmt.Errorf("finalized slot %d greater or equal to block slot %d", startSlot, chunk.Slot())
log.Debug(err)
return pubsub.ValidationIgnore, err
}
if !s.cfg.chain.HasBlock(ctx, chunk.ParentRoot()) {
// TODO: implement pending chunk storage
return pubsub.ValidationIgnore, err
}
// We ignore messages instead of accepting them to avoid rebroadcasting by gossipsub.
err = s.validateBeaconBlockChunk(ctx, chunk)
if errors.Is(err, rlnc.ErrLinearlyDependentMessage) {
log.Debug("ignoring linearly dependent message")
return pubsub.ValidationIgnore, nil
} else if err != nil {
// TODO: cook up a slashing object if the error is ErrIncorrectCommitments
log.WithError(err).Debug("Could not validate beacon block chunk")
return pubsub.ValidationReject, err
}
pr := chunk.ParentRoot()
logFields := logrus.Fields{
"chunkSlot": chunk.Slot(),
"proposerIndex": chunk.ProposerIndex(),
"parentRoot": fmt.Sprintf("%#x", pr[:8]),
}
log.WithFields(logFields).Debug("Received block chunk")
// If the block can be recovered, send it to the blockchain package
go s.reconstructBlockFromChunk(ctx, chunk)
go s.broadcastBlockChunk(ctx, chunk)
return pubsub.ValidationIgnore, nil
}
func (s *Service) validateBeaconBlockChunk(ctx context.Context, chunk interfaces.ReadOnlyBeaconBlockChunk) error {
if !s.cfg.chain.InForkchoice(chunk.ParentRoot()) {
return blockchain.ErrNotDescendantOfFinalized
}
err := s.blockChunkCache.AddChunk(chunk)
if err == nil {
return nil
}
if errors.Is(err, rlnc.ErrSignatureNotVerified) {
parentState, err := s.cfg.stateGen.StateByRoot(ctx, chunk.ParentRoot())
if err != nil {
s.blockChunkCache.RemoveNode(chunk) // Node is guaranteed to have a single chunk
return err
}
if err := core_chunks.VerifyChunkSignatureUsingCurrentFork(parentState, chunk); err != nil {
s.blockChunkCache.RemoveNode(chunk) // Node is guaranteed to have a single chunk
return err
}
return nil
}
return err
}
// startChunkPruner starts a goroutine that prunes the block chunk cache every epoch.
func (s *Service) startChunkPruner() {
cp := s.cfg.chain.FinalizedCheckpt()
fSlot, err := slots.EpochStart(cp.Epoch)
if err != nil {
log.WithError(err).Debug("could not prune the chunk cache: could not calculate epoch start slot")
} else {
s.blockChunkCache.Prune(fSlot)
}
}
func (s *Service) logReceivedBlock(blk interfaces.ReadOnlySignedBeaconBlock) {
pr := blk.Block().ParentRoot()
parentRoot := fmt.Sprintf("%#x", pr[:8])
since := time.Since(slots.StartTime(uint64(s.cfg.chain.GenesisTime().Unix()), blk.Block().Slot()))
log.WithFields(logrus.Fields{
"slot": blk.Block().Slot(),
"parentRoot": parentRoot,
"sinceSlotStart": since,
}).Debug("Received block")
}
func (s *Service) reconstructBlockFromChunk(ctx context.Context, chunk interfaces.ReadOnlyBeaconBlockChunk) {
data, err := s.blockChunkCache.GetBlockData(chunk.Slot(), chunk.ProposerIndex())
if err != nil {
return
}
msg := p2p.GossipTopicMappings(p2p.BlockSubnetTopicFormat, slots.ToEpoch(chunk.Slot()))
e := &encoder.SszNetworkEncoder{}
msgSSZ, ok := msg.(fastssz.Unmarshaler)
if !ok {
logrus.Error("Could not convert message to fastssz.Unmarshaler")
return
}
if err := e.DecodeGossip(data, msgSSZ); err != nil {
logrus.WithError(err).Error("Could not decode block data")
return
}
// We overwrite the signature in the block to keep it to be the original one in the database
// Unfortunately to do this without reflection we make extra copies of the full block. We could switch over the type instead.
blk, err := blocks.NewSignedBeaconBlock(msg)
if err != nil {
logrus.WithError(err).Error("Could not create signed beacon block")
return
}
// return early if we have already synced the block, this can happen by contention with the chunk syncing
if s.hasSeenBlockIndexSlot(chunk.Slot(), chunk.ProposerIndex()) {
return
}
sig := chunk.Signature()
blk.SetSignature(sig[:])
protoBlock, err := blk.Proto()
if err != nil {
logrus.WithError(err).Error("Could not convert block to protomessage")
return
}
logrus.WithFields(logrus.Fields{"slot": chunk.Slot(), "proposerIndex": chunk.ProposerIndex()}).Info("decoded beacon block")
// Broadcast the block on a feed to notify other services in the beacon node
// of a received block (even if it does not process correctly through a state transition).
s.cfg.blockNotifier.BlockFeed().Send(&feed.Event{
Type: blockfeed.ReceivedBlock,
Data: &blockfeed.ReceivedBlockData{
SignedBlock: blk,
},
})
// log the received block data
s.logReceivedBlock(blk)
// create a new context to process the block
go func() {
slotCtx, cancel := context.WithTimeout(context.Background(), time.Duration(params.BeaconConfig().SecondsPerSlot)*time.Second)
defer cancel()
if err := s.beaconBlockSubscriber(slotCtx, protoBlock); err != nil {
logrus.WithError(err).Error("Could not handle p2p pubsub")
}
}()
}
func (s *Service) broadcastBlockChunk(ctx context.Context, chunk interfaces.ReadOnlyBeaconBlockChunk) {
meshSize := features.Get().RLNCMeshSize
messages := make([]*ethpb.BeaconBlockChunk, 0, meshSize)
for i := uint(0); i < meshSize; i++ {
msg, err := s.blockChunkCache.PrepareMessage(chunk)
if err != nil {
log.WithError(err).Error("could not prepare message")
return
}
messages = append(messages, msg)
}
if err := s.cfg.p2p.BroadcastBlockChunks(ctx, messages); err != nil {
log.WithError(err).Error("chunk broadcast failed")
}
}

View File

@@ -271,8 +271,11 @@ func (s *Service) validatePhase0Block(ctx context.Context, blk interfaces.ReadOn
return nil, err
}
if err := blocks.VerifyBlockSignatureUsingCurrentFork(parentState, blk, blockRoot); err != nil {
return nil, err
if !features.Get().UseRLNC {
// TODO: verify the signature when using RLNC.
if err := blocks.VerifyBlockSignatureUsingCurrentFork(parentState, blk, blockRoot); err != nil {
return nil, err
}
}
// In the event the block is more than an epoch ahead from its
// parent state, we have to advance the state forward.

View File

@@ -92,8 +92,11 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
return pubsub.ValidationIgnore, err
}
if err := vf.ValidProposerSignature(ctx); err != nil {
return pubsub.ValidationReject, err
// TODO: verify the proposer signature here by fetching it from the chunks cache, or putting the blob in a queue
if !features.Get().UseRLNC {
if err := vf.ValidProposerSignature(ctx); err != nil {
return pubsub.ValidationReject, err
}
}
if err := vf.SidecarParentValid(s.hasBadBlock); err != nil {

View File

@@ -0,0 +1,3 @@
### Fixed
- Fixed proposing at genesis when starting post Bellatrix

3
changelog/potuz_rlnc.md Normal file
View File

@@ -0,0 +1,3 @@
### Added
- Use random linear network coding for block propagation

View File

@@ -38,6 +38,8 @@ const disabledFeatureFlag = "Disabled feature flag"
// Flags is a struct to represent which features the client will perform on runtime.
type Flags struct {
RLNCNumChunks uint // RLNCNumChunks specifies the number of chunks to use for RLNC encoding.
RLNCMeshSize uint // RLNCMeshSize specifies the size of the mesh to use for RLNC encoding.
// Feature related flags.
EnableExperimentalState bool // EnableExperimentalState turns on the latest and greatest (but potentially unstable) changes to the beacon state.
WriteSSZStateTransitions bool // WriteSSZStateTransitions to tmp directory.
@@ -50,6 +52,7 @@ type Flags struct {
EnableBeaconRESTApi bool // EnableBeaconRESTApi enables experimental usage of the beacon REST API by the validator when querying a beacon node
DisableCommitteeAwarePacking bool // DisableCommitteeAwarePacking changes the attestation packing algorithm to one that is not aware of attesting committees.
EnableExperimentalAttestationPool bool // EnableExperimentalAttestationPool enables an experimental attestation pool design.
UseRLNC bool // UseRLNC enables the use of random linear network coding for gossiping.
// 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
@@ -86,7 +89,8 @@ type Flags struct {
KeystoreImportDebounceInterval time.Duration
// AggregateIntervals specifies the time durations at which we aggregate attestations preparing for forkchoice.
AggregateIntervals [3]time.Duration
AggregateIntervals [3]time.Duration
DelayBlockBroadcast time.Duration // DelayBlockBroadcast is the time duration to delay block broadcast.
}
var featureConfig *Flags
@@ -272,6 +276,39 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
logEnabled(enableExperimentalAttestationPool)
cfg.EnableExperimentalAttestationPool = true
}
if ctx.IsSet(useRLNC.Name) {
logEnabled(useRLNC)
cfg.UseRLNC = true
cfg.RLNCNumChunks = 10
cfg.RLNCMeshSize = 80
}
if ctx.IsSet(rlncBlockChunks.Name) {
if !cfg.UseRLNC {
logEnabled(useRLNC)
cfg.UseRLNC = true
}
logEnabled(rlncBlockChunks)
if ctx.Int(rlncBlockChunks.Name) < 2 {
return fmt.Errorf("rlncBlockChunks must be greater than 1")
}
cfg.RLNCNumChunks = uint(ctx.Int(rlncBlockChunks.Name))
}
if ctx.IsSet(rlncMeshSize.Name) {
if !cfg.UseRLNC {
logEnabled(useRLNC)
cfg.UseRLNC = true
}
logEnabled(rlncMeshSize)
if ctx.Int(rlncMeshSize.Name) < int(cfg.RLNCNumChunks) {
return fmt.Errorf("rlncMeshSize must be greater than the number of chunks")
}
cfg.RLNCMeshSize = uint(ctx.Int(rlncMeshSize.Name))
}
if ctx.IsSet(delayBlockBroadcast.Name) {
logEnabled(delayBlockBroadcast)
cfg.DelayBlockBroadcast = ctx.Duration(delayBlockBroadcast.Name)
}
cfg.AggregateIntervals = [3]time.Duration{aggregateFirstInterval.Value, aggregateSecondInterval.Value, aggregateThirdInterval.Value}
Init(cfg)
@@ -310,6 +347,21 @@ func ConfigureValidator(ctx *cli.Context) error {
logEnabled(EnableBeaconRESTApi)
cfg.EnableBeaconRESTApi = true
}
if ctx.IsSet(useRLNC.Name) {
logEnabled(useRLNC)
cfg.UseRLNC = true
}
if ctx.IsSet(rlncBlockChunks.Name) {
if !cfg.UseRLNC {
logEnabled(useRLNC)
cfg.UseRLNC = true
}
logEnabled(rlncBlockChunks)
if ctx.Int(rlncBlockChunks.Name) < 2 {
return fmt.Errorf("rlncBlockChunks must be greater than 1")
}
cfg.RLNCNumChunks = uint(ctx.Int(rlncBlockChunks.Name))
}
cfg.KeystoreImportDebounceInterval = ctx.Duration(dynamicKeyReloadDebounceInterval.Name)
Init(cfg)
return nil

View File

@@ -178,11 +178,35 @@ var (
Name: "enable-experimental-attestation-pool",
Usage: "Enables an experimental attestation pool design.",
}
useRLNC = &cli.BoolFlag{
Name: "use-rlnc",
Usage: "Experimental: enables the use of random linear network coding for gossiping.",
Hidden: true,
}
rlncBlockChunks = &cli.IntFlag{
Name: "rlnc-block-chunks",
Usage: "Experimental: number of chunks to split the block into for RLNC.",
Value: 10,
Hidden: true,
}
rlncMeshSize = &cli.IntFlag{
Name: "rlnc-mesh-size",
Usage: "Experimental: number of peers to send RLNC chunks to.",
Value: 80,
Hidden: true,
}
// delayBlockBroadcast is a flag for delaying the block broadcast to the network.
delayBlockBroadcast = &cli.DurationFlag{
Name: "delay-block-broadcast",
Usage: "Delays the block broadcast to the network.",
Value: 0,
}
)
// devModeFlags holds list of flags that are set when development mode is on.
var devModeFlags = []cli.Flag{
backfill.EnableExperimentalBackfill,
useRLNC,
}
// ValidatorFlags contains a list of all the feature flags that apply to the validator client.
@@ -197,11 +221,14 @@ var ValidatorFlags = append(deprecatedFlags, []cli.Flag{
EnableMinimalSlashingProtection,
enableDoppelGangerProtection,
EnableBeaconRESTApi,
useRLNC,
rlncBlockChunks,
}...)
// E2EValidatorFlags contains a list of the validator feature flags to be tested in E2E.
var E2EValidatorFlags = []string{
"--enable-doppelganger",
"--use-rlnc",
}
// BeaconChainFlags contains a list of all the feature flags that apply to the beacon-chain client.
@@ -236,6 +263,10 @@ var BeaconChainFlags = append(deprecatedBeaconFlags, append(deprecatedFlags, []c
DisableCommitteeAwarePacking,
EnableDiscoveryReboot,
enableExperimentalAttestationPool,
useRLNC,
rlncBlockChunks,
rlncMeshSize,
delayBlockBroadcast,
}...)...)
// E2EBeaconChainFlags contains a list of the beacon chain feature flags to be tested in E2E.

View File

@@ -0,0 +1,19 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
srcs = [
"beacon_block_chunk.go",
"error.go",
],
importpath = "github.com/prysmaticlabs/prysm/v5/consensus-types/chunks",
visibility = ["//visibility:public"],
deps = [
"//config/params:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"//time/slots:go_default_library",
],
)

View File

@@ -0,0 +1,182 @@
package chunks
import (
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
var _ interfaces.BeaconBlockChunk = &BeaconBlockChunk{}
type BeaconBlockChunk struct {
chunk *ethpb.BeaconBlockChunk
headerRoot [32]byte
version int
}
// Slot returns the slot of the beacon block chunk.
func (b *BeaconBlockChunk) Slot() primitives.Slot {
return b.chunk.Header.Slot
}
// ProposerIndex returns the proposer index of the beacon block chunk.
func (b *BeaconBlockChunk) ProposerIndex() primitives.ValidatorIndex {
return b.chunk.Header.ProposerIndex
}
// ParentRoot returns the parent root of the beacon block chunk.
func (b *BeaconBlockChunk) ParentRoot() [32]byte {
return [32]byte(b.chunk.Header.ParentRoot)
}
// Commitments returns the commitments of the beacon block chunk.
func (b *BeaconBlockChunk) Commitments() [][]byte {
cmts := make([][]byte, len(b.chunk.Header.Commitments))
for i, cmt := range b.chunk.Header.Commitments {
cmts[i] = make([]byte, len(cmt))
copy(cmts[i], cmt)
}
return cmts
}
// Signature returns the signature of the beacon block chunk.
func (b *BeaconBlockChunk) Signature() [96]byte {
return [96]byte(b.chunk.Signature)
}
// IsNil returns true if the beacon block chunk is nil.
func (b *BeaconBlockChunk) IsNil() bool {
if b == nil || b.chunk == nil || b.chunk.Header == nil {
return true
}
if b.chunk.Header.ParentRoot == nil {
return true
}
if b.chunk.Header.Commitments == nil {
return true
}
if b.chunk.Data == nil {
return true
}
return b.chunk.Signature == nil
}
// Data returns the data of the beacon block chunk.
func (b *BeaconBlockChunk) Data() [][]byte {
data := make([][]byte, len(b.chunk.Data))
for i, d := range b.chunk.Data {
data[i] = make([]byte, len(d))
copy(data[i], d)
}
return data
}
// Coefficients returns the coefficients of the beacon block chunk.
func (b *BeaconBlockChunk) Coefficients() [][]byte {
coefficients := make([][]byte, len(b.chunk.Coefficients))
for i, c := range b.chunk.Coefficients {
coefficients[i] = make([]byte, len(c))
copy(coefficients[i], c)
}
return coefficients
}
// Version returns the version of the beacon block chunk.
func (b *BeaconBlockChunk) Version() int {
return b.version
}
// HeaderRoot returns the root of the beacon block chunk header
func (b *BeaconBlockChunk) HeaderRoot() [32]byte {
return b.headerRoot
}
// Header returns a copy of the header of the beacon block chunk.
func (b *BeaconBlockChunk) Header() *ethpb.BeaconBlockChunkHeader {
root := b.ParentRoot()
return &ethpb.BeaconBlockChunkHeader{
Slot: b.chunk.Header.Slot,
ProposerIndex: b.chunk.Header.ProposerIndex,
ParentRoot: root[:],
Commitments: b.Commitments(),
}
}
func NewBlockChunk(i interface{}) (*BeaconBlockChunk, error) {
switch b := i.(type) {
case nil:
return nil, ErrNilObject
case *ethpb.BeaconBlockChunk:
root, err := b.Header.HashTreeRoot()
if err != nil {
return nil, err
}
return &BeaconBlockChunk{chunk: b, headerRoot: root, version: slotToVersion(b.Header.Slot)}, nil
default:
return nil, ErrInvalidType
}
}
func slotToVersion(slot primitives.Slot) int {
epoch := slots.ToEpoch(slot)
cfg := params.BeaconConfig()
if epoch < cfg.AltairForkEpoch {
return version.Phase0
}
if epoch < cfg.BellatrixForkEpoch {
return version.Altair
}
if epoch < cfg.CapellaForkEpoch {
return version.Bellatrix
}
if epoch < cfg.DenebForkEpoch {
return version.Capella
}
if epoch < cfg.ElectraForkEpoch {
return version.Deneb
}
return version.Electra
}
// SetParentRoot sets the parent root of the beacon block chunk.
func (b *BeaconBlockChunk) SetParentRoot(root []byte) {
b.chunk.Header.ParentRoot = root
}
// SetCommitments sets the commitments of the beacon block chunk.
func (b *BeaconBlockChunk) SetCommitments(commitments [][]byte) {
b.chunk.Header.Commitments = commitments
}
// SetCoefficients sets the coefficients of the beacon block chunk.
func (b *BeaconBlockChunk) SetCoefficients(coefficients [][]byte) {
b.chunk.Coefficients = coefficients
}
// SetData sets the data of the beacon block chunk.
func (b *BeaconBlockChunk) SetData(data [][]byte) {
b.chunk.Data = data
}
// SetSignature sets the signature of the beacon block chunk.
func (b *BeaconBlockChunk) SetSignature(signature [96]byte) {
b.chunk.Signature = signature[:]
}
// SetSlot sets the slot of the beacon block chunk.
func (b *BeaconBlockChunk) SetSlot(slot primitives.Slot) {
b.chunk.Header.Slot = slot
}
// SetProposerIndex sets the proposer index of the beacon block chunk.
func (b *BeaconBlockChunk) SetProposerIndex(index primitives.ValidatorIndex) {
b.chunk.Header.ProposerIndex = index
}
// SetVersion sets the version of the beacon block chunk.
func (b *BeaconBlockChunk) SetVersion(version int) {
b.version = version
}

View File

@@ -0,0 +1,6 @@
package chunks
import "errors"
var ErrNilObject = errors.New("nil object") // ErrNilObject is returned when a nil object is received instead of a valid chunk
var ErrInvalidType = errors.New("invalid type") // ErrInvalidType is returned when an invalid type is received instead of a valid chunk

View File

@@ -4,6 +4,7 @@ go_library(
name = "go_default_library",
srcs = [
"beacon_block.go",
"beacon_block_chunk.go",
"error.go",
"light_client.go",
"utils.go",

View File

@@ -0,0 +1,35 @@
package interfaces
import (
field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
// ReadOnlyBeaconBlockChunk is an interface describing the method set of
// a signed beacon block chunk
type ReadOnlyBeaconBlockChunk interface {
IsNil() bool
Version() int
Slot() primitives.Slot
ProposerIndex() primitives.ValidatorIndex
ParentRoot() [field_params.RootLength]byte
Commitments() [][]byte
Signature() [field_params.BLSSignatureLength]byte
Header() *ethpb.BeaconBlockChunkHeader
HeaderRoot() [field_params.RootLength]byte
Data() [][]byte
Coefficients() [][]byte
}
type BeaconBlockChunk interface {
ReadOnlyBeaconBlockChunk
SetParentRoot([]byte)
SetProposerIndex(idx primitives.ValidatorIndex)
SetSlot(slot primitives.Slot)
SetSignature(sig [96]byte)
SetVersion(version int)
SetCommitments(commitments [][]byte)
SetData(data [][]byte)
SetCoefficients(coefficients [][]byte)
}

View File

@@ -1347,6 +1347,12 @@ def prysm_deps():
sum = "h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1QAp/SlnNrZhI=",
version = "v1.9.5",
)
go_repository(
name = "com_github_gtank_ristretto255",
importpath = "github.com/gtank/ristretto255",
sum = "h1:JEqUCPA1NvLq5DwYtuzigd7ss8fwbYay9fi4/5uMzcc=",
version = "v0.1.2",
)
go_repository(
name = "com_github_guptarohit_asciigraph",
importpath = "github.com/guptarohit/asciigraph",
@@ -1957,8 +1963,9 @@ def prysm_deps():
name = "com_github_libp2p_go_libp2p_pubsub",
build_file_proto_mode = "disable_global",
importpath = "github.com/libp2p/go-libp2p-pubsub",
sum = "h1:Z1fzkKuRAd+U1oxbTbWUtTTzAWaX21jqoLxVIStzp+Y=",
version = "v0.12.1-0.20241230202526-bf5b58384331",
replace = "github.com/nisdas/go-libp2p-pubsub",
sum = "h1:eqQDbjtvXnkZcpC7BayY5FeIzpSGHeDyn53EZ9qVha0=",
version = "v0.3.3-0.20250127130457-2becfc6d889e",
)
go_repository(
name = "com_github_libp2p_go_libp2p_testing",

3
go.mod
View File

@@ -29,6 +29,7 @@ require (
github.com/gostaticanalysis/comment v1.4.2
github.com/grpc-ecosystem/go-grpc-middleware v1.2.2
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
github.com/gtank/ristretto255 v0.1.2
github.com/hashicorp/golang-lru v0.5.5-0.20210104140557-80c98217689d
github.com/herumi/bls-eth-go-binary v1.31.0
github.com/holiman/uint256 v1.3.2
@@ -281,3 +282,5 @@ require (
)
replace github.com/json-iterator/go => github.com/prestonvanloon/go v1.1.7-0.20190722034630-4f2e55fcf87b
replace github.com/libp2p/go-libp2p-pubsub => github.com/nisdas/go-libp2p-pubsub v0.3.3-0.20250127130457-2becfc6d889e

6
go.sum
View File

@@ -450,6 +450,8 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92Bcuy
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw=
github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
github.com/gtank/ristretto255 v0.1.2 h1:JEqUCPA1NvLq5DwYtuzigd7ss8fwbYay9fi4/5uMzcc=
github.com/gtank/ristretto255 v0.1.2/go.mod h1:Ph5OpO6c7xKUGROZfWVLiJf9icMDwUeIvY4OmlYW69o=
github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE=
github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
@@ -587,8 +589,6 @@ github.com/libp2p/go-libp2p-asn-util v0.4.1 h1:xqL7++IKD9TBFMgnLPZR6/6iYhawHKHl9
github.com/libp2p/go-libp2p-asn-util v0.4.1/go.mod h1:d/NI6XZ9qxw67b4e+NgpQexCIiFYJjErASrYW4PFDN8=
github.com/libp2p/go-libp2p-mplex v0.9.0 h1:R58pDRAmuBXkYugbSSXR9wrTX3+1pFM1xP2bLuodIq8=
github.com/libp2p/go-libp2p-mplex v0.9.0/go.mod h1:ro1i4kuwiFT+uMPbIDIFkcLs1KRbNp0QwnUXM+P64Og=
github.com/libp2p/go-libp2p-pubsub v0.12.1-0.20241230202526-bf5b58384331 h1:Z1fzkKuRAd+U1oxbTbWUtTTzAWaX21jqoLxVIStzp+Y=
github.com/libp2p/go-libp2p-pubsub v0.12.1-0.20241230202526-bf5b58384331/go.mod h1:m0gpUOyrXKXdE7c8FNQ9/HLfWbxaEw7xku45w+PaqZo=
github.com/libp2p/go-libp2p-testing v0.12.0 h1:EPvBb4kKMWO29qP4mZGyhVzUyR25dvfUIK5WDu6iPUA=
github.com/libp2p/go-libp2p-testing v0.12.0/go.mod h1:KcGDRXyN7sQCllucn1cOOS+Dmm7ujhfEyXQL5lvkcPg=
github.com/libp2p/go-mplex v0.7.0 h1:BDhFZdlk5tbr0oyFq/xv/NPGfjbnrsDam1EvutpBDbY=
@@ -731,6 +731,8 @@ github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxzi
github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c=
github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo=
github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM=
github.com/nisdas/go-libp2p-pubsub v0.3.3-0.20250127130457-2becfc6d889e h1:eqQDbjtvXnkZcpC7BayY5FeIzpSGHeDyn53EZ9qVha0=
github.com/nisdas/go-libp2p-pubsub v0.3.3-0.20250127130457-2becfc6d889e/go.mod h1:m0gpUOyrXKXdE7c8FNQ9/HLfWbxaEw7xku45w+PaqZo=
github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
github.com/nxadm/tail v1.4.11 h1:8feyoE3OzPrcshW5/MJ4sGESc5cqmGkGCWlco4l0bqY=
github.com/nxadm/tail v1.4.11/go.mod h1:OTaG3NK980DZzxbRq6lEuzgU+mug70nY11sMd4JXXHc=

View File

@@ -51,7 +51,9 @@ ssz_phase0_objs = [
"AttestationData",
"AttesterSlashing",
"BeaconBlock",
"BeaconBlockChunk",
"BeaconBlockHeader",
"BeaconBlockChunkHeader",
"BeaconState",
"Checkpoint",
"Deposit",
@@ -385,6 +387,7 @@ ssz_proto_files(
srcs = [
"attestation.proto",
"beacon_block.proto",
"beacon_block_chunk.proto",
"beacon_state.proto",
"blobs.proto",
"data_columns.proto",

472
proto/prysm/v1alpha1/beacon_block_chunk.pb.go generated Executable file
View File

@@ -0,0 +1,472 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.25.1
// source: proto/prysm/v1alpha1/beacon_block_chunk.proto
package eth
import (
reflect "reflect"
sync "sync"
github_com_prysmaticlabs_prysm_v5_consensus_types_primitives "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
_ "github.com/prysmaticlabs/prysm/v5/proto/eth/ext"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
)
const (
// Verify that this generated code is sufficiently up-to-date.
_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
// Verify that runtime/protoimpl is sufficiently up-to-date.
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
type BeaconBlockChunk struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Data [][]byte `protobuf:"bytes,1,rep,name=data,proto3" json:"data,omitempty" ssz-max:"32768" ssz-size:"?,32"`
Coefficients [][]byte `protobuf:"bytes,2,rep,name=coefficients,proto3" json:"coefficients,omitempty" ssz-max:"128" ssz-size:"?,32"`
Header *BeaconBlockChunkHeader `protobuf:"bytes,3,opt,name=header,proto3" json:"header,omitempty"`
Signature []byte `protobuf:"bytes,4,opt,name=signature,proto3" json:"signature,omitempty" ssz-size:"96"`
}
func (x *BeaconBlockChunk) Reset() {
*x = BeaconBlockChunk{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[0]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BeaconBlockChunk) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BeaconBlockChunk) ProtoMessage() {}
func (x *BeaconBlockChunk) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[0]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BeaconBlockChunk.ProtoReflect.Descriptor instead.
func (*BeaconBlockChunk) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescGZIP(), []int{0}
}
func (x *BeaconBlockChunk) GetData() [][]byte {
if x != nil {
return x.Data
}
return nil
}
func (x *BeaconBlockChunk) GetCoefficients() [][]byte {
if x != nil {
return x.Coefficients
}
return nil
}
func (x *BeaconBlockChunk) GetHeader() *BeaconBlockChunkHeader {
if x != nil {
return x.Header
}
return nil
}
func (x *BeaconBlockChunk) GetSignature() []byte {
if x != nil {
return x.Signature
}
return nil
}
type BeaconBlockChunkHeader struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Slot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"`
ProposerIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,2,opt,name=proposer_index,json=proposerIndex,proto3" json:"proposer_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"`
ParentRoot []byte `protobuf:"bytes,3,opt,name=parent_root,json=parentRoot,proto3" json:"parent_root,omitempty" ssz-size:"32"`
Commitments [][]byte `protobuf:"bytes,4,rep,name=commitments,proto3" json:"commitments,omitempty" ssz-max:"128" ssz-size:"?,32"`
}
func (x *BeaconBlockChunkHeader) Reset() {
*x = BeaconBlockChunkHeader{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BeaconBlockChunkHeader) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BeaconBlockChunkHeader) ProtoMessage() {}
func (x *BeaconBlockChunkHeader) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_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 BeaconBlockChunkHeader.ProtoReflect.Descriptor instead.
func (*BeaconBlockChunkHeader) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescGZIP(), []int{1}
}
func (x *BeaconBlockChunkHeader) GetSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot {
if x != nil {
return x.Slot
}
return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0)
}
func (x *BeaconBlockChunkHeader) GetProposerIndex() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex {
if x != nil {
return x.ProposerIndex
}
return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(0)
}
func (x *BeaconBlockChunkHeader) GetParentRoot() []byte {
if x != nil {
return x.ParentRoot
}
return nil
}
func (x *BeaconBlockChunkHeader) GetCommitments() [][]byte {
if x != nil {
return x.Commitments
}
return nil
}
type BeaconBlockChunkData struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Data [][]byte `protobuf:"bytes,1,rep,name=data,proto3" json:"data,omitempty" ssz-max:"32768" ssz-size:"?,32"`
}
func (x *BeaconBlockChunkData) Reset() {
*x = BeaconBlockChunkData{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[2]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BeaconBlockChunkData) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BeaconBlockChunkData) ProtoMessage() {}
func (x *BeaconBlockChunkData) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[2]
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 BeaconBlockChunkData.ProtoReflect.Descriptor instead.
func (*BeaconBlockChunkData) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescGZIP(), []int{2}
}
func (x *BeaconBlockChunkData) GetData() [][]byte {
if x != nil {
return x.Data
}
return nil
}
type ChunkedBeaconBlock struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Header *BeaconBlockChunkHeader `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"`
Chunks []*BeaconBlockChunkData `protobuf:"bytes,2,rep,name=chunks,proto3" json:"chunks,omitempty"`
Signature []byte `protobuf:"bytes,3,opt,name=signature,proto3" json:"signature,omitempty" ssz-size:"96"`
Block *GenericSignedBeaconBlock `protobuf:"bytes,4,opt,name=block,proto3" json:"block,omitempty"`
}
func (x *ChunkedBeaconBlock) Reset() {
*x = ChunkedBeaconBlock{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[3]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ChunkedBeaconBlock) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ChunkedBeaconBlock) ProtoMessage() {}
func (x *ChunkedBeaconBlock) ProtoReflect() protoreflect.Message {
mi := &file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[3]
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 ChunkedBeaconBlock.ProtoReflect.Descriptor instead.
func (*ChunkedBeaconBlock) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescGZIP(), []int{3}
}
func (x *ChunkedBeaconBlock) GetHeader() *BeaconBlockChunkHeader {
if x != nil {
return x.Header
}
return nil
}
func (x *ChunkedBeaconBlock) GetChunks() []*BeaconBlockChunkData {
if x != nil {
return x.Chunks
}
return nil
}
func (x *ChunkedBeaconBlock) GetSignature() []byte {
if x != nil {
return x.Signature
}
return nil
}
func (x *ChunkedBeaconBlock) GetBlock() *GenericSignedBeaconBlock {
if x != nil {
return x.Block
}
return nil
}
var File_proto_prysm_v1alpha1_beacon_block_chunk_proto protoreflect.FileDescriptor
var file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDesc = []byte{
0x0a, 0x2d, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12,
0x15, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x1a, 0x1b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x65, 0x78, 0x74, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xdb, 0x01, 0x0a,
0x10, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x68, 0x75, 0x6e,
0x6b, 0x12, 0x25, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x42,
0x11, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x33, 0x32, 0x92, 0xb5, 0x18, 0x05, 0x33, 0x32, 0x37,
0x36, 0x38, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x33, 0x0a, 0x0c, 0x63, 0x6f, 0x65, 0x66,
0x66, 0x69, 0x63, 0x69, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x0f,
0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x33, 0x32, 0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52,
0x0c, 0x63, 0x6f, 0x65, 0x66, 0x66, 0x69, 0x63, 0x69, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x45, 0x0a,
0x06, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65,
0x61, 0x64, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52,
0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0xc7, 0x02, 0x0a, 0x16, 0x42,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48,
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20,
0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62,
0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65,
0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69,
0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74,
0x12, 0x76, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64,
0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74,
0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f,
0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f,
0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64,
0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f,
0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65,
0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a,
0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x6f,
0x74, 0x12, 0x31, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73,
0x18, 0x04, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x0f, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x33, 0x32,
0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d,
0x65, 0x6e, 0x74, 0x73, 0x22, 0x3d, 0x0a, 0x14, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x44, 0x61, 0x74, 0x61, 0x12, 0x25, 0x0a, 0x04,
0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x11, 0x8a, 0xb5, 0x18, 0x04,
0x3f, 0x2c, 0x33, 0x32, 0x92, 0xb5, 0x18, 0x05, 0x33, 0x32, 0x37, 0x36, 0x38, 0x52, 0x04, 0x64,
0x61, 0x74, 0x61, 0x22, 0x8d, 0x02, 0x0a, 0x12, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x42,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x45, 0x0a, 0x06, 0x68, 0x65,
0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x68,
0x75, 0x6e, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x65,
0x72, 0x12, 0x43, 0x0a, 0x06, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28,
0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06,
0x63, 0x68, 0x75, 0x6e, 0x6b, 0x73, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74,
0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39,
0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x45, 0x0a, 0x05,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70,
0x68, 0x61, 0x31, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x69, 0x63, 0x53, 0x69, 0x67, 0x6e, 0x65,
0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x05, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x42, 0xa0, 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, 0x15, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x68,
0x75, 0x6e, 0x6b, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63,
0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02,
0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescOnce sync.Once
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescData = file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDesc
)
func file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescGZIP() []byte {
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescOnce.Do(func() {
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescData)
})
return file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDescData
}
var file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes = make([]protoimpl.MessageInfo, 4)
var file_proto_prysm_v1alpha1_beacon_block_chunk_proto_goTypes = []interface{}{
(*BeaconBlockChunk)(nil), // 0: ethereum.eth.v1alpha1.BeaconBlockChunk
(*BeaconBlockChunkHeader)(nil), // 1: ethereum.eth.v1alpha1.BeaconBlockChunkHeader
(*BeaconBlockChunkData)(nil), // 2: ethereum.eth.v1alpha1.BeaconBlockChunkData
(*ChunkedBeaconBlock)(nil), // 3: ethereum.eth.v1alpha1.ChunkedBeaconBlock
(*GenericSignedBeaconBlock)(nil), // 4: ethereum.eth.v1alpha1.GenericSignedBeaconBlock
}
var file_proto_prysm_v1alpha1_beacon_block_chunk_proto_depIdxs = []int32{
1, // 0: ethereum.eth.v1alpha1.BeaconBlockChunk.header:type_name -> ethereum.eth.v1alpha1.BeaconBlockChunkHeader
1, // 1: ethereum.eth.v1alpha1.ChunkedBeaconBlock.header:type_name -> ethereum.eth.v1alpha1.BeaconBlockChunkHeader
2, // 2: ethereum.eth.v1alpha1.ChunkedBeaconBlock.chunks:type_name -> ethereum.eth.v1alpha1.BeaconBlockChunkData
4, // 3: ethereum.eth.v1alpha1.ChunkedBeaconBlock.block:type_name -> ethereum.eth.v1alpha1.GenericSignedBeaconBlock
4, // [4:4] is the sub-list for method output_type
4, // [4:4] is the sub-list for method input_type
4, // [4:4] is the sub-list for extension type_name
4, // [4:4] is the sub-list for extension extendee
0, // [0:4] is the sub-list for field type_name
}
func init() { file_proto_prysm_v1alpha1_beacon_block_chunk_proto_init() }
func file_proto_prysm_v1alpha1_beacon_block_chunk_proto_init() {
if File_proto_prysm_v1alpha1_beacon_block_chunk_proto != nil {
return
}
file_proto_prysm_v1alpha1_beacon_block_proto_init()
if !protoimpl.UnsafeEnabled {
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BeaconBlockChunk); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BeaconBlockChunkHeader); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BeaconBlockChunkData); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ChunkedBeaconBlock); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
}
type x struct{}
out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDesc,
NumEnums: 0,
NumMessages: 4,
NumExtensions: 0,
NumServices: 0,
},
GoTypes: file_proto_prysm_v1alpha1_beacon_block_chunk_proto_goTypes,
DependencyIndexes: file_proto_prysm_v1alpha1_beacon_block_chunk_proto_depIdxs,
MessageInfos: file_proto_prysm_v1alpha1_beacon_block_chunk_proto_msgTypes,
}.Build()
File_proto_prysm_v1alpha1_beacon_block_chunk_proto = out.File
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_rawDesc = nil
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_goTypes = nil
file_proto_prysm_v1alpha1_beacon_block_chunk_proto_depIdxs = nil
}

View File

@@ -0,0 +1,51 @@
// Copyright 2024 Prysmatic Labs.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
syntax = "proto3";
package ethereum.eth.v1alpha1;
import "proto/eth/ext/options.proto";
import "proto/prysm/v1alpha1/beacon_block.proto";
option csharp_namespace = "Ethereum.Eth.v1alpha1";
option go_package = "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1;eth";
option java_multiple_files = true;
option java_outer_classname = "BeaconBlockChunkProto";
option java_package = "org.ethereum.eth.v1alpha1";
option php_namespace = "Ethereum\\Eth\\v1alpha1";
message BeaconBlockChunk {
repeated bytes data = 1 [(ethereum.eth.ext.ssz_size) = "?,32", (ethereum.eth.ext.ssz_max) = "32768"];
repeated bytes coefficients = 2 [(ethereum.eth.ext.ssz_size) = "?,32", (ethereum.eth.ext.ssz_max) = "128"];
BeaconBlockChunkHeader header = 3;
bytes signature = 4 [(ethereum.eth.ext.ssz_size) = "96"];
}
message BeaconBlockChunkHeader {
uint64 slot = 1 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"];
uint64 proposer_index = 2 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"];
bytes parent_root = 3 [(ethereum.eth.ext.ssz_size) = "32"];
repeated bytes commitments = 4 [(ethereum.eth.ext.ssz_size) = "?,32", (ethereum.eth.ext.ssz_max) = "128"];
}
message BeaconBlockChunkData {
repeated bytes data = 1 [(ethereum.eth.ext.ssz_size) = "?,32", (ethereum.eth.ext.ssz_max) = "32768"];
}
message ChunkedBeaconBlock {
BeaconBlockChunkHeader header = 1;
repeated BeaconBlockChunkData chunks = 2;
bytes signature = 3 [(ethereum.eth.ext.ssz_size) = "96"];
GenericSignedBeaconBlock block = 4;
}

View File

@@ -2244,6 +2244,383 @@ func (d *Deposit_Data) HashTreeRootWith(hh *ssz.Hasher) (err error) {
return
}
// MarshalSSZ ssz marshals the BeaconBlockChunk object
func (b *BeaconBlockChunk) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(b)
}
// MarshalSSZTo ssz marshals the BeaconBlockChunk object to a target array
func (b *BeaconBlockChunk) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
offset := int(108)
// Offset (0) 'Data'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.Data) * 32
// Offset (1) 'Coefficients'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.Coefficients) * 32
// Offset (2) 'Header'
dst = ssz.WriteOffset(dst, offset)
if b.Header == nil {
b.Header = new(BeaconBlockChunkHeader)
}
offset += b.Header.SizeSSZ()
// Field (3) 'Signature'
if size := len(b.Signature); size != 96 {
err = ssz.ErrBytesLengthFn("--.Signature", size, 96)
return
}
dst = append(dst, b.Signature...)
// Field (0) 'Data'
if size := len(b.Data); size > 32768 {
err = ssz.ErrListTooBigFn("--.Data", size, 32768)
return
}
for ii := 0; ii < len(b.Data); ii++ {
if size := len(b.Data[ii]); size != 32 {
err = ssz.ErrBytesLengthFn("--.Data[ii]", size, 32)
return
}
dst = append(dst, b.Data[ii]...)
}
// Field (1) 'Coefficients'
if size := len(b.Coefficients); size > 128 {
err = ssz.ErrListTooBigFn("--.Coefficients", size, 128)
return
}
for ii := 0; ii < len(b.Coefficients); ii++ {
if size := len(b.Coefficients[ii]); size != 32 {
err = ssz.ErrBytesLengthFn("--.Coefficients[ii]", size, 32)
return
}
dst = append(dst, b.Coefficients[ii]...)
}
// Field (2) 'Header'
if dst, err = b.Header.MarshalSSZTo(dst); err != nil {
return
}
return
}
// UnmarshalSSZ ssz unmarshals the BeaconBlockChunk object
func (b *BeaconBlockChunk) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size < 108 {
return ssz.ErrSize
}
tail := buf
var o0, o1, o2 uint64
// Offset (0) 'Data'
if o0 = ssz.ReadOffset(buf[0:4]); o0 > size {
return ssz.ErrOffset
}
if o0 != 108 {
return ssz.ErrInvalidVariableOffset
}
// Offset (1) 'Coefficients'
if o1 = ssz.ReadOffset(buf[4:8]); o1 > size || o0 > o1 {
return ssz.ErrOffset
}
// Offset (2) 'Header'
if o2 = ssz.ReadOffset(buf[8:12]); o2 > size || o1 > o2 {
return ssz.ErrOffset
}
// Field (3) 'Signature'
if cap(b.Signature) == 0 {
b.Signature = make([]byte, 0, len(buf[12:108]))
}
b.Signature = append(b.Signature, buf[12:108]...)
// Field (0) 'Data'
{
buf = tail[o0:o1]
num, err := ssz.DivideInt2(len(buf), 32, 32768)
if err != nil {
return err
}
b.Data = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(b.Data[ii]) == 0 {
b.Data[ii] = make([]byte, 0, len(buf[ii*32:(ii+1)*32]))
}
b.Data[ii] = append(b.Data[ii], buf[ii*32:(ii+1)*32]...)
}
}
// Field (1) 'Coefficients'
{
buf = tail[o1:o2]
num, err := ssz.DivideInt2(len(buf), 32, 128)
if err != nil {
return err
}
b.Coefficients = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(b.Coefficients[ii]) == 0 {
b.Coefficients[ii] = make([]byte, 0, len(buf[ii*32:(ii+1)*32]))
}
b.Coefficients[ii] = append(b.Coefficients[ii], buf[ii*32:(ii+1)*32]...)
}
}
// Field (2) 'Header'
{
buf = tail[o2:]
if b.Header == nil {
b.Header = new(BeaconBlockChunkHeader)
}
if err = b.Header.UnmarshalSSZ(buf); err != nil {
return err
}
}
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the BeaconBlockChunk object
func (b *BeaconBlockChunk) SizeSSZ() (size int) {
size = 108
// Field (0) 'Data'
size += len(b.Data) * 32
// Field (1) 'Coefficients'
size += len(b.Coefficients) * 32
// Field (2) 'Header'
if b.Header == nil {
b.Header = new(BeaconBlockChunkHeader)
}
size += b.Header.SizeSSZ()
return
}
// HashTreeRoot ssz hashes the BeaconBlockChunk object
func (b *BeaconBlockChunk) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(b)
}
// HashTreeRootWith ssz hashes the BeaconBlockChunk object with a hasher
func (b *BeaconBlockChunk) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'Data'
{
if size := len(b.Data); size > 32768 {
err = ssz.ErrListTooBigFn("--.Data", size, 32768)
return
}
subIndx := hh.Index()
for _, i := range b.Data {
if len(i) != 32 {
err = ssz.ErrBytesLength
return
}
hh.Append(i)
}
numItems := uint64(len(b.Data))
hh.MerkleizeWithMixin(subIndx, numItems, 32768)
}
// Field (1) 'Coefficients'
{
if size := len(b.Coefficients); size > 128 {
err = ssz.ErrListTooBigFn("--.Coefficients", size, 128)
return
}
subIndx := hh.Index()
for _, i := range b.Coefficients {
if len(i) != 32 {
err = ssz.ErrBytesLength
return
}
hh.Append(i)
}
numItems := uint64(len(b.Coefficients))
hh.MerkleizeWithMixin(subIndx, numItems, 128)
}
// Field (2) 'Header'
if err = b.Header.HashTreeRootWith(hh); err != nil {
return
}
// Field (3) 'Signature'
if size := len(b.Signature); size != 96 {
err = ssz.ErrBytesLengthFn("--.Signature", size, 96)
return
}
hh.PutBytes(b.Signature)
hh.Merkleize(indx)
return
}
// MarshalSSZ ssz marshals the BeaconBlockChunkHeader object
func (b *BeaconBlockChunkHeader) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(b)
}
// MarshalSSZTo ssz marshals the BeaconBlockChunkHeader object to a target array
func (b *BeaconBlockChunkHeader) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
offset := int(52)
// Field (0) 'Slot'
dst = ssz.MarshalUint64(dst, uint64(b.Slot))
// Field (1) 'ProposerIndex'
dst = ssz.MarshalUint64(dst, uint64(b.ProposerIndex))
// Field (2) 'ParentRoot'
if size := len(b.ParentRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.ParentRoot", size, 32)
return
}
dst = append(dst, b.ParentRoot...)
// Offset (3) 'Commitments'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.Commitments) * 32
// Field (3) 'Commitments'
if size := len(b.Commitments); size > 128 {
err = ssz.ErrListTooBigFn("--.Commitments", size, 128)
return
}
for ii := 0; ii < len(b.Commitments); ii++ {
if size := len(b.Commitments[ii]); size != 32 {
err = ssz.ErrBytesLengthFn("--.Commitments[ii]", size, 32)
return
}
dst = append(dst, b.Commitments[ii]...)
}
return
}
// UnmarshalSSZ ssz unmarshals the BeaconBlockChunkHeader object
func (b *BeaconBlockChunkHeader) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size < 52 {
return ssz.ErrSize
}
tail := buf
var o3 uint64
// Field (0) 'Slot'
b.Slot = github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[0:8]))
// Field (1) 'ProposerIndex'
b.ProposerIndex = github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(ssz.UnmarshallUint64(buf[8:16]))
// Field (2) 'ParentRoot'
if cap(b.ParentRoot) == 0 {
b.ParentRoot = make([]byte, 0, len(buf[16:48]))
}
b.ParentRoot = append(b.ParentRoot, buf[16:48]...)
// Offset (3) 'Commitments'
if o3 = ssz.ReadOffset(buf[48:52]); o3 > size {
return ssz.ErrOffset
}
if o3 != 52 {
return ssz.ErrInvalidVariableOffset
}
// Field (3) 'Commitments'
{
buf = tail[o3:]
num, err := ssz.DivideInt2(len(buf), 32, 128)
if err != nil {
return err
}
b.Commitments = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(b.Commitments[ii]) == 0 {
b.Commitments[ii] = make([]byte, 0, len(buf[ii*32:(ii+1)*32]))
}
b.Commitments[ii] = append(b.Commitments[ii], buf[ii*32:(ii+1)*32]...)
}
}
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the BeaconBlockChunkHeader object
func (b *BeaconBlockChunkHeader) SizeSSZ() (size int) {
size = 52
// Field (3) 'Commitments'
size += len(b.Commitments) * 32
return
}
// HashTreeRoot ssz hashes the BeaconBlockChunkHeader object
func (b *BeaconBlockChunkHeader) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(b)
}
// HashTreeRootWith ssz hashes the BeaconBlockChunkHeader object with a hasher
func (b *BeaconBlockChunkHeader) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'Slot'
hh.PutUint64(uint64(b.Slot))
// Field (1) 'ProposerIndex'
hh.PutUint64(uint64(b.ProposerIndex))
// Field (2) 'ParentRoot'
if size := len(b.ParentRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.ParentRoot", size, 32)
return
}
hh.PutBytes(b.ParentRoot)
// Field (3) 'Commitments'
{
if size := len(b.Commitments); size > 128 {
err = ssz.ErrListTooBigFn("--.Commitments", size, 128)
return
}
subIndx := hh.Index()
for _, i := range b.Commitments {
if len(i) != 32 {
err = ssz.ErrBytesLength
return
}
hh.Append(i)
}
numItems := uint64(len(b.Commitments))
hh.MerkleizeWithMixin(subIndx, numItems, 128)
}
hh.Merkleize(indx)
return
}
// MarshalSSZ ssz marshals the BeaconState object
func (b *BeaconState) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(b)

View File

@@ -109,6 +109,7 @@ type SignRequest struct {
// *SignRequest_AggregateAttestationAndProofElectra
// *SignRequest_BlockFulu
// *SignRequest_BlindedBlockFulu
// *SignRequest_BeaconBlockChunkHeader
Object isSignRequest_Object `protobuf_oneof:"object"`
SigningSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,6,opt,name=signing_slot,json=signingSlot,proto3" json:"signing_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"`
}
@@ -327,6 +328,13 @@ func (x *SignRequest) GetBlindedBlockFulu() *v1alpha1.BlindedBeaconBlockFulu {
return nil
}
func (x *SignRequest) GetBeaconBlockChunkHeader() *v1alpha1.BeaconBlockChunkHeader {
if x, ok := x.GetObject().(*SignRequest_BeaconBlockChunkHeader); ok {
return x.BeaconBlockChunkHeader
}
return nil
}
func (x *SignRequest) GetSigningSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot {
if x != nil {
return x.SigningSlot
@@ -426,6 +434,10 @@ type SignRequest_BlindedBlockFulu struct {
BlindedBlockFulu *v1alpha1.BlindedBeaconBlockFulu `protobuf:"bytes,122,opt,name=blinded_block_fulu,json=blindedBlockFulu,proto3,oneof"`
}
type SignRequest_BeaconBlockChunkHeader struct {
BeaconBlockChunkHeader *v1alpha1.BeaconBlockChunkHeader `protobuf:"bytes,123,opt,name=beacon_block_chunk_header,json=beaconBlockChunkHeader,proto3,oneof"`
}
func (*SignRequest_Block) isSignRequest_Object() {}
func (*SignRequest_AttestationData) isSignRequest_Object() {}
@@ -470,6 +482,8 @@ func (*SignRequest_BlockFulu) isSignRequest_Object() {}
func (*SignRequest_BlindedBlockFulu) isSignRequest_Object() {}
func (*SignRequest_BeaconBlockChunkHeader) isSignRequest_Object() {}
type SignResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -724,221 +738,231 @@ var file_proto_prysm_v1alpha1_validator_client_keymanager_proto_rawDesc = []byte
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x62, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a,
0x29, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69,
0x74, 0x74, 0x65, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xe5, 0x11, 0x0a, 0x0b, 0x53,
0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x75,
0x62, 0x6c, 0x69, 0x63, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09,
0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x4b, 0x65, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x69, 0x67,
0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52,
0x0b, 0x73, 0x69, 0x67, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x29, 0x0a, 0x10,
0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x3a, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x18, 0x65, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x00, 0x52, 0x05, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x12, 0x53, 0x0a, 0x10, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x66, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x0f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61,
0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x7c, 0x0a, 0x1f, 0x61, 0x67, 0x67, 0x72,
0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x67, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x33, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67,
0x61, 0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e,
0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x48, 0x00, 0x52, 0x1c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67,
0x61, 0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e,
0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x3a, 0x0a, 0x04, 0x65, 0x78, 0x69, 0x74, 0x18, 0x68,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x56, 0x6f, 0x6c,
0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x48, 0x00, 0x52, 0x04, 0x65, 0x78,
0x69, 0x74, 0x12, 0x5b, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x69, 0x20, 0x01, 0x28, 0x04,
0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70,
0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 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, 0x48, 0x00, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12,
0x5e, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x6a, 0x20, 0x01, 0x28, 0x04, 0x42, 0x46,
0x82, 0xb5, 0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70,
0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 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, 0x48, 0x00, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12,
0x4d, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x61, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x18,
0x6b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2d, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x29,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c,
0x70, 0x68, 0x61, 0x31, 0x2f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74,
0x74, 0x65, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd1, 0x12, 0x0a, 0x0b, 0x53, 0x69,
0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x75, 0x62,
0x6c, 0x69, 0x63, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x70,
0x75, 0x62, 0x6c, 0x69, 0x63, 0x4b, 0x65, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x69, 0x67, 0x6e,
0x69, 0x6e, 0x67, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b,
0x73, 0x69, 0x67, 0x6e, 0x69, 0x6e, 0x67, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x73,
0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18,
0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x3a, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18,
0x65, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x48,
0x00, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x12, 0x79,
0x0a, 0x1e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f,
0x72, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61,
0x18, 0x6c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53,
0x79, 0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1b, 0x73, 0x79,
0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x63, 0x0a, 0x16, 0x63, 0x6f, 0x6e,
0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72,
0x6f, 0x6f, 0x66, 0x18, 0x6d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e,
0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x48, 0x00, 0x52, 0x14, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69,
0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x37,
0x0a, 0x17, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x6e, 0x20, 0x01, 0x28, 0x0c, 0x48,
0x00, 0x52, 0x14, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x56, 0x0a, 0x0f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x5f, 0x62, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x18, 0x6f, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x48, 0x00, 0x52,
0x0e, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x12,
0x6c, 0x0a, 0x17, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x5f, 0x62, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x18, 0x70, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x32, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64,
0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61,
0x74, 0x72, 0x69, 0x78, 0x48, 0x00, 0x52, 0x15, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x12, 0x54, 0x0a,
0x0c, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x71, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x56, 0x31, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x12, 0x50, 0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x61, 0x70,
0x65, 0x6c, 0x6c, 0x61, 0x18, 0x72, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61,
0x70, 0x65, 0x6c, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61,
0x70, 0x65, 0x6c, 0x6c, 0x61, 0x12, 0x66, 0x0a, 0x15, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64,
0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x18, 0x73,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69,
0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43,
0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x13, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65,
0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x12, 0x4a, 0x0a,
0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x6e, 0x65, 0x62, 0x18, 0x74, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x48, 0x00, 0x52, 0x0a, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x60, 0x0a, 0x13, 0x62, 0x6c, 0x69,
0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x6e, 0x65, 0x62,
0x18, 0x75, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42,
0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x48, 0x00, 0x52, 0x11, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65,
0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x50, 0x0a, 0x0d, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x18, 0x76, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x48, 0x00, 0x52,
0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x66, 0x0a,
0x15, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65,
0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x18, 0x77, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x00, 0x52, 0x05, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x12, 0x53, 0x0a, 0x10, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x66, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c,
0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x48, 0x00,
0x52, 0x13, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c,
0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x92, 0x01, 0x0a, 0x27, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67,
0x61, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x5f, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x72,
0x61, 0x18, 0x78, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e,
0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61,
0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x45, 0x6c, 0x65, 0x63,
0x74, 0x72, 0x61, 0x48, 0x00, 0x52, 0x23, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65,
0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72,
0x6f, 0x6f, 0x66, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x47, 0x0a, 0x0a, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x66, 0x75, 0x6c, 0x75, 0x18, 0x79, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x46,
0x75, 0x6c, 0x75, 0x12, 0x5d, 0x0a, 0x12, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x66, 0x75, 0x6c, 0x75, 0x18, 0x7a, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x0f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x7c, 0x0a, 0x1f, 0x61, 0x67, 0x67, 0x72, 0x65,
0x67, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x67, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x33, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,
0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64,
0x50, 0x72, 0x6f, 0x6f, 0x66, 0x48, 0x00, 0x52, 0x1c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,
0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64,
0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x3a, 0x0a, 0x04, 0x65, 0x78, 0x69, 0x74, 0x18, 0x68, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x56, 0x6f, 0x6c, 0x75,
0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x48, 0x00, 0x52, 0x04, 0x65, 0x78, 0x69,
0x74, 0x12, 0x5b, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x69, 0x20, 0x01, 0x28, 0x04, 0x42,
0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 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, 0x48, 0x00, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x5e,
0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x6a, 0x20, 0x01, 0x28, 0x04, 0x42, 0x46, 0x82,
0xb5, 0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x2f, 0x76, 0x35, 0x2f, 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, 0x48, 0x00, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x4d,
0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x61, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x18, 0x6b,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x48, 0x00,
0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x12, 0x79, 0x0a,
0x1e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f, 0x72,
0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18,
0x6c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x79,
0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1b, 0x73, 0x79, 0x6e,
0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x63, 0x0a, 0x16, 0x63, 0x6f, 0x6e, 0x74,
0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f,
0x6f, 0x66, 0x18, 0x6d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64,
0x50, 0x72, 0x6f, 0x6f, 0x66, 0x48, 0x00, 0x52, 0x14, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62,
0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x37, 0x0a,
0x17, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x6e, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00,
0x52, 0x14, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x56, 0x0a, 0x0f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x62, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x18, 0x6f, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x48, 0x00, 0x52, 0x0e,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x12, 0x6c,
0x0a, 0x17, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x62, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x18, 0x70, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x32, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x48, 0x00,
0x52, 0x10, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75,
0x6c, 0x75, 0x12, 0x68, 0x0a, 0x0c, 0x73, 0x69, 0x67, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x6c,
0x6f, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74,
0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 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,
0x0b, 0x73, 0x69, 0x67, 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x6c, 0x6f, 0x74, 0x42, 0x08, 0x0a, 0x06,
0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05,
0x10, 0x06, 0x22, 0xb7, 0x01, 0x0a, 0x0c, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
0x65, 0x12, 0x4b, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0e, 0x32, 0x33, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c,
0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e,
0x76, 0x32, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e,
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x3c,
0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e,
0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x43, 0x43, 0x45, 0x45, 0x44,
0x45, 0x44, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4e, 0x49, 0x45, 0x44, 0x10, 0x02,
0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x22, 0xb3, 0x01, 0x0a,
0x15, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x50,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65,
0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66,
0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x47, 0x0a, 0x07, 0x62,
0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x65,
0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74,
0x72, 0x69, 0x78, 0x48, 0x00, 0x52, 0x15, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x42, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x72, 0x69, 0x78, 0x12, 0x54, 0x0a, 0x0c,
0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x71, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64,
0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x56, 0x31, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x12, 0x50, 0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x61, 0x70, 0x65,
0x6c, 0x6c, 0x61, 0x18, 0x72, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61, 0x70,
0x65, 0x6c, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61, 0x70,
0x65, 0x6c, 0x6c, 0x61, 0x12, 0x66, 0x0a, 0x15, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x18, 0x73, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e,
0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61,
0x70, 0x65, 0x6c, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x13, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x12, 0x4a, 0x0a, 0x0b,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x6e, 0x65, 0x62, 0x18, 0x74, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x48, 0x00, 0x52, 0x0a, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x60, 0x0a, 0x13, 0x62, 0x6c, 0x69, 0x6e,
0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x6e, 0x65, 0x62, 0x18,
0x75, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c,
0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b,
0x44, 0x65, 0x6e, 0x65, 0x62, 0x48, 0x00, 0x52, 0x11, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x50, 0x0a, 0x0d, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x18, 0x76, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x48, 0x00, 0x52, 0x0c,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x66, 0x0a, 0x15,
0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c,
0x65, 0x63, 0x74, 0x72, 0x61, 0x18, 0x77, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70,
0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x48, 0x00, 0x52,
0x13, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x65,
0x63, 0x74, 0x72, 0x61, 0x12, 0x92, 0x01, 0x0a, 0x27, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,
0x74, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61,
0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x5f, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61,
0x18, 0x78, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x41,
0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x45, 0x6c, 0x65, 0x63, 0x74,
0x72, 0x61, 0x48, 0x00, 0x52, 0x23, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41,
0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f,
0x6f, 0x66, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x47, 0x0a, 0x0a, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x5f, 0x66, 0x75, 0x6c, 0x75, 0x18, 0x79, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x46, 0x75, 0x6c, 0x75, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75,
0x6c, 0x75, 0x12, 0x5d, 0x0a, 0x12, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x66, 0x75, 0x6c, 0x75, 0x18, 0x7a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x48, 0x00, 0x52,
0x10, 0x62, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c,
0x75, 0x12, 0x6a, 0x0a, 0x19, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x7b,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61,
0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x65, 0x61,
0x64, 0x65, 0x72, 0x48, 0x00, 0x52, 0x16, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f,
0x63, 0x6b, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x68, 0x0a,
0x0c, 0x73, 0x69, 0x67, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x06, 0x20,
0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62,
0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 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, 0x0b, 0x73, 0x69, 0x67, 0x6e,
0x69, 0x6e, 0x67, 0x53, 0x6c, 0x6f, 0x74, 0x42, 0x08, 0x0a, 0x06, 0x6f, 0x62, 0x6a, 0x65, 0x63,
0x74, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, 0x22, 0xb7, 0x01,
0x0a, 0x0c, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c,
0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x4b, 0x0a, 0x06,
0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x33, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f,
0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x75,
0x69, 0x6c, 0x64, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x07, 0x62, 0x75, 0x69,
0x6c, 0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x08, 0x67, 0x72, 0x61, 0x66, 0x66, 0x69, 0x74, 0x69,
0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x08, 0x67, 0x72, 0x61, 0x66, 0x66, 0x69,
0x74, 0x69, 0x88, 0x01, 0x01, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x67, 0x72, 0x61, 0x66, 0x66, 0x69,
0x74, 0x69, 0x22, 0xa6, 0x01, 0x0a, 0x0d, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x43, 0x6f,
0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18,
0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x12, 0x63,
0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28,
0x04, 0x42, 0x46, 0x82, 0xb5, 0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f,
0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f,
0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73,
0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x6f, 0x72, 0x2e, 0x55, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69,
0x6d, 0x69, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6c, 0x61, 0x79, 0x73, 0x18, 0x03, 0x20,
0x03, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x6c, 0x61, 0x79, 0x73, 0x22, 0xe7, 0x02, 0x0a, 0x17,
0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73,
0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x74, 0x0a, 0x0f, 0x70, 0x72, 0x6f, 0x70, 0x6f,
0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,
0x32, 0x4b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76,
0x32, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e,
0x67, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73,
0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x70,
0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x5c, 0x0a,
0x0e, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18,
0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75,
0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x4f,
0x70, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x0d, 0x64, 0x65,
0x66, 0x61, 0x75, 0x6c, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x1a, 0x78, 0x0a, 0x13, 0x50,
0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74,
0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
0x03, 0x6b, 0x65, 0x79, 0x12, 0x4b, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76,
0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x73, 0x2e, 0x76, 0x32, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x4f, 0x70, 0x74,
0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0xce, 0x01, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74,
0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x69,
0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75,
0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x3c, 0x0a, 0x06, 0x53, 0x74, 0x61,
0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00,
0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x43, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x01, 0x12,
0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4e, 0x49, 0x45, 0x44, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46,
0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x22, 0xb3, 0x01, 0x0a, 0x15, 0x50, 0x72, 0x6f, 0x70,
0x6f, 0x73, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65,
0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x65, 0x65, 0x52, 0x65, 0x63,
0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x47, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65,
0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63,
0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72,
0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x12,
0x1f, 0x0a, 0x08, 0x67, 0x72, 0x61, 0x66, 0x66, 0x69, 0x74, 0x69, 0x18, 0x03, 0x20, 0x01, 0x28,
0x09, 0x48, 0x00, 0x52, 0x08, 0x67, 0x72, 0x61, 0x66, 0x66, 0x69, 0x74, 0x69, 0x88, 0x01, 0x01,
0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x67, 0x72, 0x61, 0x66, 0x66, 0x69, 0x74, 0x69, 0x22, 0xa6, 0x01,
0x0a, 0x0d, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12,
0x18, 0x0a, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08,
0x52, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x12, 0x63, 0x0a, 0x09, 0x67, 0x61, 0x73,
0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x46, 0x82, 0xb5,
0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79,
0x70, 0x65, 0x73, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x55, 0x69,
0x6e, 0x74, 0x36, 0x34, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x16,
0x0a, 0x06, 0x72, 0x65, 0x6c, 0x61, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06,
0x72, 0x65, 0x6c, 0x61, 0x79, 0x73, 0x22, 0xe7, 0x02, 0x0a, 0x17, 0x50, 0x72, 0x6f, 0x70, 0x6f,
0x73, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x12, 0x74, 0x0a, 0x0f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x63,
0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4b, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72,
0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x42, 0x0f, 0x4b, 0x65,
0x79, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a,
0x53, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f,
0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76,
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f,
0x72, 0x2d, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x3b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x6f, 0x72, 0x70, 0x62, 0xaa, 0x02, 0x1e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e,
0x74, 0x73, 0x2e, 0x56, 0x32, 0xca, 0x02, 0x1e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x5c, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5c, 0x41, 0x63, 0x63, 0x6f, 0x75,
0x6e, 0x74, 0x73, 0x5c, 0x56, 0x32, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x2e, 0x50, 0x72, 0x6f,
0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x50, 0x61, 0x79,
0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e,
0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73,
0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x5c, 0x0a, 0x0e, 0x64, 0x65, 0x66, 0x61,
0x75, 0x6c, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x35, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76,
0x32, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74,
0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x1a, 0x78, 0x0a, 0x13, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73,
0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a,
0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12,
0x4b, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61,
0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x2e,
0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61,
0x79, 0x6c, 0x6f, 0x61, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
0x42, 0xce, 0x01, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63, 0x6f,
0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x42, 0x0f, 0x4b, 0x65, 0x79, 0x6d, 0x61, 0x6e, 0x61,
0x67, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x53, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63,
0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2d, 0x63, 0x6c, 0x69,
0x65, 0x6e, 0x74, 0x3b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x70, 0x62, 0xaa,
0x02, 0x1e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64,
0x61, 0x74, 0x6f, 0x72, 0x2e, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x56, 0x32,
0xca, 0x02, 0x1e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x56, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x5c, 0x41, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x5c, 0x56,
0x32, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -982,6 +1006,7 @@ var file_proto_prysm_v1alpha1_validator_client_keymanager_proto_goTypes = []inte
(*v1alpha1.AggregateAttestationAndProofElectra)(nil), // 23: ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra
(*v1alpha1.BeaconBlockFulu)(nil), // 24: ethereum.eth.v1alpha1.BeaconBlockFulu
(*v1alpha1.BlindedBeaconBlockFulu)(nil), // 25: ethereum.eth.v1alpha1.BlindedBeaconBlockFulu
(*v1alpha1.BeaconBlockChunkHeader)(nil), // 26: ethereum.eth.v1alpha1.BeaconBlockChunkHeader
}
var file_proto_prysm_v1alpha1_validator_client_keymanager_proto_depIdxs = []int32{
7, // 0: ethereum.validator.accounts.v2.SignRequest.block:type_name -> ethereum.eth.v1alpha1.BeaconBlock
@@ -1003,16 +1028,17 @@ var file_proto_prysm_v1alpha1_validator_client_keymanager_proto_depIdxs = []int3
23, // 16: ethereum.validator.accounts.v2.SignRequest.aggregate_attestation_and_proof_electra:type_name -> ethereum.eth.v1alpha1.AggregateAttestationAndProofElectra
24, // 17: ethereum.validator.accounts.v2.SignRequest.block_fulu:type_name -> ethereum.eth.v1alpha1.BeaconBlockFulu
25, // 18: ethereum.validator.accounts.v2.SignRequest.blinded_block_fulu:type_name -> ethereum.eth.v1alpha1.BlindedBeaconBlockFulu
0, // 19: ethereum.validator.accounts.v2.SignResponse.status:type_name -> ethereum.validator.accounts.v2.SignResponse.Status
4, // 20: ethereum.validator.accounts.v2.ProposerOptionPayload.builder:type_name -> ethereum.validator.accounts.v2.BuilderConfig
6, // 21: ethereum.validator.accounts.v2.ProposerSettingsPayload.proposer_config:type_name -> ethereum.validator.accounts.v2.ProposerSettingsPayload.ProposerConfigEntry
3, // 22: ethereum.validator.accounts.v2.ProposerSettingsPayload.default_config:type_name -> ethereum.validator.accounts.v2.ProposerOptionPayload
3, // 23: ethereum.validator.accounts.v2.ProposerSettingsPayload.ProposerConfigEntry.value:type_name -> ethereum.validator.accounts.v2.ProposerOptionPayload
24, // [24:24] is the sub-list for method output_type
24, // [24:24] is the sub-list for method input_type
24, // [24:24] is the sub-list for extension type_name
24, // [24:24] is the sub-list for extension extendee
0, // [0:24] is the sub-list for field type_name
26, // 19: ethereum.validator.accounts.v2.SignRequest.beacon_block_chunk_header:type_name -> ethereum.eth.v1alpha1.BeaconBlockChunkHeader
0, // 20: ethereum.validator.accounts.v2.SignResponse.status:type_name -> ethereum.validator.accounts.v2.SignResponse.Status
4, // 21: ethereum.validator.accounts.v2.ProposerOptionPayload.builder:type_name -> ethereum.validator.accounts.v2.BuilderConfig
6, // 22: ethereum.validator.accounts.v2.ProposerSettingsPayload.proposer_config:type_name -> ethereum.validator.accounts.v2.ProposerSettingsPayload.ProposerConfigEntry
3, // 23: ethereum.validator.accounts.v2.ProposerSettingsPayload.default_config:type_name -> ethereum.validator.accounts.v2.ProposerOptionPayload
3, // 24: ethereum.validator.accounts.v2.ProposerSettingsPayload.ProposerConfigEntry.value:type_name -> ethereum.validator.accounts.v2.ProposerOptionPayload
25, // [25:25] is the sub-list for method output_type
25, // [25:25] is the sub-list for method input_type
25, // [25:25] is the sub-list for extension type_name
25, // [25:25] is the sub-list for extension extendee
0, // [0:25] is the sub-list for field type_name
}
func init() { file_proto_prysm_v1alpha1_validator_client_keymanager_proto_init() }
@@ -1105,6 +1131,7 @@ func file_proto_prysm_v1alpha1_validator_client_keymanager_proto_init() {
(*SignRequest_AggregateAttestationAndProofElectra)(nil),
(*SignRequest_BlockFulu)(nil),
(*SignRequest_BlindedBlockFulu)(nil),
(*SignRequest_BeaconBlockChunkHeader)(nil),
}
file_proto_prysm_v1alpha1_validator_client_keymanager_proto_msgTypes[2].OneofWrappers = []interface{}{}
type x struct{}

View File

@@ -5,6 +5,7 @@ import "proto/eth/ext/options.proto";
import "proto/prysm/v1alpha1/attestation.proto";
import "proto/prysm/v1alpha1/beacon_block.proto";
import "proto/prysm/v1alpha1/beacon_state.proto";
import "proto/prysm/v1alpha1/beacon_block_chunk.proto";
import "proto/prysm/v1alpha1/sync_committee.proto";
option csharp_namespace = "Ethereum.Validator.Accounts.V2";
@@ -69,6 +70,9 @@ message SignRequest {
// Fulu objects.
ethereum.eth.v1alpha1.BeaconBlockFulu block_fulu = 121;
ethereum.eth.v1alpha1.BlindedBeaconBlockFulu blinded_block_fulu = 122;
// RLNC objects.
ethereum.eth.v1alpha1.BeaconBlockChunkHeader beacon_block_chunk_header = 123;
}
reserved 4, 5; // Reserving old, deleted fields.

File diff suppressed because it is too large Load Diff

View File

@@ -22,6 +22,7 @@ import "google/protobuf/wrappers.proto";
import "proto/eth/ext/options.proto";
import "proto/prysm/v1alpha1/beacon_block.proto";
import "proto/prysm/v1alpha1/beacon_block_chunk.proto";
import "proto/prysm/v1alpha1/sync_committee.proto";
import "proto/prysm/v1alpha1/attestation.proto";
@@ -138,6 +139,18 @@ service BeaconNodeValidator {
};
}
// Sends a newly signed beacon block chunked to the beacon node.
//
// The validator sends the newly signed beacon block chunked to the beacon node so the beacon block can
// be included in the beacon chain. The beacon node is expected to validate and process the
// beacon block into its state.
rpc ProposeChunkedBlock(ChunkedBeaconBlock) returns (ProposeResponse) {
option (google.api.http) = {
post: "/eth/v1alpha2/validator/chunked_block"
body: "*"
};
}
// Sends the newly signed beacon block to beacon node.
//
// The validator sends the newly signed beacon block to the beacon node so the beacon block can

View File

@@ -283,9 +283,9 @@ func (node *BeaconNode) Start(ctx context.Context) error {
// on our features or the beacon index is a multiplier of 2 (idea is to split nodes
// equally down the line with one group having feature flags and the other without
// feature flags; this is to allow A-B testing on new features)
if !config.TestFeature || index != 1 {
args = append(args, features.E2EBeaconChainFlags...)
}
// Temporarily default to e2e flags enabled
args = append(args, features.E2EBeaconChainFlags...)
if config.UseBuilder {
args = append(args, fmt.Sprintf("--%s=%s:%d", flags.MevRelayEndpoint.Name, "http://127.0.0.1", e2e.TestParams.Ports.Eth1ProxyPort+index))
}

View File

@@ -343,6 +343,26 @@ func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeAttestationElectra(a
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeAttestationElectra", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ProposeAttestationElectra), varargs...)
}
// ProposeChunkedBlock mocks base method.
func (m *MockBeaconNodeValidatorClient) ProposeChunkedBlock(arg0 context.Context, arg1 *eth.ChunkedBeaconBlock, arg2 ...grpc.CallOption) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ProposeChunkedBlock", varargs...)
ret0, _ := ret[0].(*eth.ProposeResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// ProposeChunkedBlock indicates an expected call of ProposeChunkedBlock.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeChunkedBlock(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeChunkedBlock", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ProposeChunkedBlock), varargs...)
}
// ProposeBeaconBlock mocks base method.
func (m *MockBeaconNodeValidatorClient) ProposeBeaconBlock(arg0 context.Context, arg1 *eth.GenericSignedBeaconBlock, arg2 ...grpc.CallOption) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()

View File

@@ -267,6 +267,21 @@ func (mr *MockBeaconNodeValidatorServerMockRecorder) ProposeAttestationElectra(a
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeAttestationElectra", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).ProposeAttestationElectra), arg0, arg1)
}
// ProposeChunkedBlock mocks base method.
func (m *MockBeaconNodeValidatorServer) ProposeChunkedBlock(arg0 context.Context, arg1 *eth.ChunkedBeaconBlock) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "ProposeChunkedBlock", arg0, arg1)
ret0, _ := ret[0].(*eth.ProposeResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// ProposeChunkedBlock indicates an expected call of ProposeChunkedBlock.
func (mr *MockBeaconNodeValidatorServerMockRecorder) ProposeChunkedBlock(arg0, arg1 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeChunkedBlock", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).ProposeChunkedBlock), arg0, arg1)
}
// ProposeBeaconBlock mocks base method.
func (m *MockBeaconNodeValidatorServer) ProposeBeaconBlock(arg0 context.Context, arg1 *eth.GenericSignedBeaconBlock) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()

View File

@@ -252,6 +252,21 @@ func (mr *MockValidatorClientMockRecorder) ProposeAttestationElectra(arg0, arg1
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeAttestationElectra", reflect.TypeOf((*MockValidatorClient)(nil).ProposeAttestationElectra), arg0, arg1)
}
// ProposeChunkedBlock mocks base method.
func (m *MockValidatorClient) ProposeChunkedBlock(arg0 context.Context, arg1 *eth.ChunkedBeaconBlock) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "ProposeChunkedBlock", arg0, arg1)
ret0, _ := ret[0].(*eth.ProposeResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// ProposeChunkedBlock indicates an expected call of ProposeChunkedBeaconBlock.
func (mr *MockValidatorClientMockRecorder) ProposeChunkedBlock(arg0, arg1 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeChunkedBlock", reflect.TypeOf((*MockValidatorClient)(nil).ProposeChunkedBlock), arg0, arg1)
}
// ProposeBeaconBlock mocks base method.
func (m *MockValidatorClient) ProposeBeaconBlock(arg0 context.Context, arg1 *eth.GenericSignedBeaconBlock) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()

View File

@@ -1,6 +1,7 @@
package slots
import (
"context"
"fmt"
"math"
"time"
@@ -18,6 +19,23 @@ import (
// incoming objects. (24 mins with mainnet spec)
const MaxSlotBuffer = uint64(1 << 7)
// WaitUntil waits until the given slot plus a delay has been attained or the context is canceled.
func WaitUntil(ctx context.Context, genesis time.Time, slot primitives.Slot, delay time.Duration) error {
deadline := startFromTime(genesis, slot).Add(delay)
waitTime := time.Until(deadline)
if waitTime <= 0 {
return nil
}
timer := time.NewTimer(waitTime)
defer timer.Stop()
select {
case <-timer.C:
return nil
case <-ctx.Done():
return ctx.Err()
}
}
// startFromTime returns the slot start in terms of genesis time.Time
func startFromTime(genesis time.Time, slot primitives.Slot) time.Time {
duration := time.Second * time.Duration(slot.Mul(params.BeaconConfig().SecondsPerSlot))

View File

@@ -11,6 +11,7 @@ go_library(
"multiple_endpoints_grpc_resolver.go",
"propose.go",
"registration.go",
"rlnc.go",
"runner.go",
"service.go",
"sync_committee.go",
@@ -33,6 +34,8 @@ go_library(
"//beacon-chain/builder:go_default_library",
"//beacon-chain/core/altair:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/p2p/encoder:go_default_library",
"//beacon-chain/sync/rlnc:go_default_library",
"//cache/lru:go_default_library",
"//cmd:go_default_library",
"//config/features:go_default_library",

View File

@@ -163,6 +163,15 @@ func (c *beaconApiValidatorClient) ProposeAttestationElectra(ctx context.Context
})
}
func (c *beaconApiValidatorClient) ProposeChunkedBlock(ctx context.Context, in *ethpb.ChunkedBeaconBlock) (*ethpb.ProposeResponse, error) {
ctx, span := trace.StartSpan(ctx, "beacon-api.ProposeChunkedBlock")
defer span.End()
return wrapInMetrics[*ethpb.ProposeResponse]("ProposeChunkedBlock", func() (*ethpb.ProposeResponse, error) {
return c.proposeChunkedBlock(ctx, in)
})
}
func (c *beaconApiValidatorClient) ProposeBeaconBlock(ctx context.Context, in *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error) {
ctx, span := trace.StartSpan(ctx, "beacon-api.ProposeBeaconBlock")
defer span.End()

View File

@@ -14,6 +14,10 @@ import (
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
func (c *beaconApiValidatorClient) proposeChunkedBlock(ctx context.Context, in *ethpb.ChunkedBeaconBlock) (*ethpb.ProposeResponse, error) {
return nil, nil
}
// nolint:gocognit
func (c *beaconApiValidatorClient) proposeBeaconBlock(ctx context.Context, in *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error) {
var consensusVersion string

View File

@@ -75,6 +75,10 @@ func (c *grpcValidatorClient) ProposeAttestationElectra(ctx context.Context, in
return c.beaconNodeValidatorClient.ProposeAttestationElectra(ctx, in)
}
func (c *grpcValidatorClient) ProposeChunkedBlock(ctx context.Context, in *ethpb.ChunkedBeaconBlock) (*ethpb.ProposeResponse, error) {
return c.beaconNodeValidatorClient.ProposeChunkedBlock(ctx, in)
}
func (c *grpcValidatorClient) ProposeBeaconBlock(ctx context.Context, in *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error) {
return c.beaconNodeValidatorClient.ProposeBeaconBlock(ctx, in)
}

View File

@@ -128,6 +128,7 @@ type ValidatorClient interface {
ValidatorStatus(ctx context.Context, in *ethpb.ValidatorStatusRequest) (*ethpb.ValidatorStatusResponse, error)
MultipleValidatorStatus(ctx context.Context, in *ethpb.MultipleValidatorStatusRequest) (*ethpb.MultipleValidatorStatusResponse, error)
BeaconBlock(ctx context.Context, in *ethpb.BlockRequest) (*ethpb.GenericBeaconBlock, error)
ProposeChunkedBlock(ctx context.Context, in *ethpb.ChunkedBeaconBlock) (*ethpb.ProposeResponse, error)
ProposeBeaconBlock(ctx context.Context, in *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error)
PrepareBeaconProposer(ctx context.Context, in *ethpb.PrepareBeaconProposerRequest) (*empty.Empty, error)
FeeRecipientByPubKey(ctx context.Context, in *ethpb.FeeRecipientByPubKeyRequest) (*ethpb.FeeRecipientByPubKeyResponse, error)

View File

@@ -11,6 +11,8 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/async"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/config/proposer"
@@ -102,21 +104,38 @@ func (v *validator) ProposeBlock(ctx context.Context, slot primitives.Slot, pubK
return
}
sig, signingRoot, err := v.signBlock(ctx, pubKey, epoch, slot, wb)
if err != nil {
log.WithError(err).Error("Failed to sign block")
if v.emitAccountMetrics {
ValidatorProposeFailVec.WithLabelValues(fmtKey).Inc()
var sig []byte
var signingRoot [32]byte
var node *rlnc.Node
var blk interfaces.SignedBeaconBlock
if features.Get().UseRLNC {
blk, err = blocks.BuildSignedBeaconBlock(wb, make([]byte, 96))
if err != nil {
log.WithError(err).Error("Failed to build signed beacon block")
return
}
return
}
blk, err := blocks.BuildSignedBeaconBlock(wb, sig)
if err != nil {
log.WithError(err).Error("Failed to build signed beacon block")
return
sig, signingRoot, node, err = v.createSignedChunks(ctx, pubKey, epoch, slot, blk)
if err != nil {
log.WithError(err).Error("Failed to sign block")
return
}
blk.SetSignature(sig)
} else {
sig, signingRoot, err = v.signBlock(ctx, pubKey, epoch, slot, wb)
if err != nil {
log.WithError(err).Error("Failed to sign block")
if v.emitAccountMetrics {
ValidatorProposeFailVec.WithLabelValues(fmtKey).Inc()
}
return
}
blk, err = blocks.BuildSignedBeaconBlock(wb, sig)
if err != nil {
log.WithError(err).Error("Failed to build signed beacon block")
return
}
}
if err := v.db.SlashableProposalCheck(ctx, pubKey, blk, signingRoot, v.emitAccountMetrics, ValidatorProposeFailVec); err != nil {
log.WithFields(
blockLogFields(pubKey, wb, nil),
@@ -168,13 +187,22 @@ func (v *validator) ProposeBlock(ctx context.Context, slot primitives.Slot, pubK
}
}
blkResp, err := v.validatorClient.ProposeBeaconBlock(ctx, genericSignedBlock)
if err != nil {
log.WithField("slot", slot).WithError(err).Error("Failed to propose block")
if v.emitAccountMetrics {
ValidatorProposeFailVec.WithLabelValues(fmtKey).Inc()
var blkResp *ethpb.ProposeResponse
if features.Get().UseRLNC {
blkResp, err = v.validatorClient.ProposeChunkedBlock(ctx, node.GetChunkedBlock(genericSignedBlock))
if err != nil {
log.WithField("slot", slot).WithError(err).Error("Failed to propose block")
return
}
} else {
blkResp, err = v.validatorClient.ProposeBeaconBlock(ctx, genericSignedBlock)
if err != nil {
log.WithField("slot", slot).WithError(err).Error("Failed to propose block")
if v.emitAccountMetrics {
ValidatorProposeFailVec.WithLabelValues(fmtKey).Inc()
}
return
}
return
}
span.SetAttributes(

83
validator/client/rlnc.go Normal file
View File

@@ -0,0 +1,83 @@
package client
import (
"bytes"
"context"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/encoder"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
validatorpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1/validator-client"
"github.com/sirupsen/logrus"
)
// rlncBlockSuffix is a byte that is added to the end of the block to mark it's end.
var rlncBlockSuffix = byte(0xfe)
func (v *validator) createSignedChunks(
ctx context.Context,
pubKey [fieldparams.BLSPubkeyLength]byte,
epoch primitives.Epoch,
slot primitives.Slot,
b interfaces.ReadOnlySignedBeaconBlock,
) ([]byte, [32]byte, *rlnc.Node, error) {
ctx, span := trace.StartSpan(ctx, "validator.createSignedChunks")
defer span.End()
domain, err := v.domainData(ctx, epoch, params.BeaconConfig().DomainBeaconProposer[:])
if err != nil {
return nil, [32]byte{}, nil, errors.Wrap(err, domainDataErr)
}
if domain == nil {
return nil, [32]byte{}, nil, errors.New(domainDataErr)
}
e := &encoder.SszNetworkEncoder{}
buf := new(bytes.Buffer)
if _, err := e.EncodeGossip(buf, b); err != nil {
logrus.WithError(err).Error("Could not encode block data")
return nil, [32]byte{}, nil, errors.Wrap(err, domainDataErr)
}
buf.WriteByte(rlncBlockSuffix)
node, err := rlnc.NewSource(v.committer, features.Get().RLNCNumChunks, buf.Bytes())
if err != nil {
return nil, [32]byte{}, nil, errors.Wrap(err, "could not create source node")
}
node.SetProposerIndex(b.Block().ProposerIndex())
node.SetSlot(slot)
parentRoot := b.Block().ParentRoot()
node.SetParentRoot(parentRoot[:])
header := &ethpb.BeaconBlockChunkHeader{
Slot: slot,
ProposerIndex: b.Block().ProposerIndex(),
ParentRoot: parentRoot[:],
Commitments: node.Commitments(),
}
signingRoot, err := signing.ComputeSigningRoot(header, domain.SignatureDomain)
if err != nil {
return nil, [32]byte{}, nil, errors.Wrap(err, signingRootErr)
}
req := &validatorpb.SignRequest_BeaconBlockChunkHeader{
BeaconBlockChunkHeader: header,
}
sig, err := v.km.Sign(ctx, &validatorpb.SignRequest{
PublicKey: pubKey[:],
SigningRoot: signingRoot[:],
SignatureDomain: domain.SignatureDomain,
Object: req,
SigningSlot: slot,
})
if err != nil {
return nil, [32]byte{}, nil, errors.Wrap(err, "could not sign block proposal")
}
node.SetSignature(sig.Marshal())
return sig.Marshal(), signingRoot, node, nil
}

View File

@@ -14,7 +14,9 @@ import (
"github.com/pkg/errors"
grpcutil "github.com/prysmaticlabs/prysm/v5/api/grpc"
"github.com/prysmaticlabs/prysm/v5/async/event"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
lruwrpr "github.com/prysmaticlabs/prysm/v5/cache/lru"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/config/proposer"
@@ -178,6 +180,13 @@ func (v *ValidatorService) Start() {
)
validatorClient := validatorclientfactory.NewValidatorClient(v.conn, restHandler)
var committer *rlnc.Committer
if features.Get().UseRLNC {
committer, err = rlnc.LoadTrustedSetup()
if err != nil {
log.WithError(err).Error("Could not load the RLNC trusted setup")
}
}
valStruct := &validator{
slotFeed: new(event.Feed),
@@ -215,6 +224,7 @@ func (v *ValidatorService) Start() {
emitAccountMetrics: v.emitAccountMetrics,
useWeb: v.useWeb,
distributed: v.distributed,
committer: committer,
}
v.validator = valStruct

View File

@@ -26,6 +26,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
"github.com/prysmaticlabs/prysm/v5/async/event"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/altair"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/rlnc"
"github.com/prysmaticlabs/prysm/v5/cmd"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
@@ -112,6 +113,7 @@ type validator struct {
blacklistedPubkeysLock sync.RWMutex
attSelectionLock sync.Mutex
dutiesLock sync.RWMutex
committer *rlnc.Committer
}
type validatorStatus struct {