Compare commits

...

7 Commits

Author SHA1 Message Date
Potuz
8a22df902f Check for nil duties on checkDependentRoots (#15241) 2025-05-01 19:16:55 +00:00
Potuz
5e3a5b877a Remove slot from UpdateDuties (#15223)
* Remove slot from `UpdateDuties`

* Fix slot to epoch conversion

* Add gossip clock disparity

* do not use disparity

* check for nil duties

---------

Co-authored-by: terence tsao <terence@prysmaticlabs.com>
Co-authored-by: james-prysm <90280386+james-prysm@users.noreply.github.com>
2025-04-29 18:25:09 +00:00
Bastin
b69c71d65a Add LC p2p validators and subscribers (#15214)
* add and pass lcstore to sync service

* validator for optimistic updates

* validator for finality updates

* subscribers

* gossip scorings

* tmp - add validation test

* optimistic update validation tests

* finality update validation tests

* tests for subscribers

* deps

* changelog entry

* play around with config cleanup

* better logs

* better logs 2

* better logs 3

* address comments

* add comments
2025-04-29 14:07:17 +00:00
Manu NALEPA
1298dc3a46 PeerDAS: Add needed proto files and corresponding generated code. (#15187)
* PeerDAS: Add needed proto files and corresponding generated code.

* Fix Nishant's comment.

* `max_cell_proofs_length.size`: Set to `CELLS_PER_EXT_BLOB * MAX_BLOB_COMMITMENTS_PER_BLOCK`.

* `BlobsBundleV2`: Add comment.
2025-04-28 16:08:28 +00:00
Potuz
3c463d8171 Pass dependent roots in block events (#15227)
* Pass dependent roots in block events

* Check for empty roots
2025-04-28 00:00:48 +00:00
kasey
0a48fafc71 extend payload attribute deadline to proposal slot (#15230)
* extend payload attribute deadline to proposal slot

* Update beacon-chain/blockchain/execution_engine.go

Co-authored-by: Potuz <potuz@prysmaticlabs.com>

* Terence's feedback

* skip past proposal slots

* lint and skip events during init sync

* finagle test to not trigger old event error

* fix blockchain tests that panic without SyncChecker

---------

Co-authored-by: Kasey <kasey@users.noreply.github.com>
Co-authored-by: Potuz <potuz@prysmaticlabs.com>
2025-04-27 23:59:37 +00:00
terence
28cd59c9b7 Fix payload attribute proposer calculation (#15228)
Process slots if epoch is greater than head state epoch
2025-04-27 17:52:09 +00:00
61 changed files with 2315 additions and 477 deletions

View File

@@ -184,7 +184,11 @@ func (s *Service) notifyForkchoiceUpdate(ctx context.Context, arg *fcuConfig) (*
return payloadID, nil
}
func firePayloadAttributesEvent(f event.SubscriberSender, block interfaces.ReadOnlySignedBeaconBlock, root [32]byte, nextSlot primitives.Slot) {
func (s *Service) firePayloadAttributesEvent(f event.SubscriberSender, block interfaces.ReadOnlySignedBeaconBlock, root [32]byte, nextSlot primitives.Slot) {
// If we're syncing a block in the past and init-sync is still running, we shouldn't fire this event.
if !s.cfg.SyncChecker.Synced() {
return
}
// the fcu args have differing amounts of completeness based on the code path,
// and there is work we only want to do if a client is actually listening to the events beacon api endpoint.
// temporary solution: just fire a blank event and fill in the details in the api handler.

View File

@@ -102,7 +102,7 @@ func (s *Service) forkchoiceUpdateWithExecution(ctx context.Context, args *fcuCo
log.WithError(err).Error("could not save head")
}
go firePayloadAttributesEvent(s.cfg.StateNotifier.StateFeed(), args.headBlock, args.headRoot, s.CurrentSlot()+1)
go s.firePayloadAttributesEvent(s.cfg.StateNotifier.StateFeed(), args.headBlock, args.headRoot, s.CurrentSlot()+1)
// Only need to prune attestations from pool if the head has changed.
s.pruneAttsFromPool(s.ctx, args.headState, args.headBlock)

View File

@@ -20,6 +20,7 @@ func testServiceOptsWithDB(t *testing.T) []Option {
WithForkChoiceStore(fcs),
WithClockSynchronizer(cs),
WithStateNotifier(&mock.MockStateNotifier{RecordEvents: true}),
WithSyncChecker(&mock.MockSyncChecker{}),
}
}

View File

@@ -735,7 +735,7 @@ func (s *Service) lateBlockTasks(ctx context.Context) {
}
// notifyForkchoiceUpdate fires the payload attribute event. But in this case, we won't
// call notifyForkchoiceUpdate, so the event is fired here.
go firePayloadAttributesEvent(s.cfg.StateNotifier.StateFeed(), headBlock, headRoot, s.CurrentSlot()+1)
go s.firePayloadAttributesEvent(s.cfg.StateNotifier.StateFeed(), headBlock, headRoot, s.CurrentSlot()+1)
return
}

View File

@@ -103,15 +103,29 @@ func (s *Service) sendStateFeedOnBlock(cfg *postBlockProcessConfig) {
log.WithError(err).Debug("Could not check if block is optimistic")
optimistic = true
}
currEpoch := slots.ToEpoch(s.CurrentSlot())
currDependenRoot, err := s.cfg.ForkChoiceStore.DependentRoot(currEpoch)
if err != nil {
log.WithError(err).Debug("Could not get dependent root")
}
prevDependentRoot := [32]byte{}
if currEpoch > 0 {
prevDependentRoot, err = s.cfg.ForkChoiceStore.DependentRoot(currEpoch - 1)
if err != nil {
log.WithError(err).Debug("Could not get previous dependent root")
}
}
// Send notification of the processed block to the state feed.
s.cfg.StateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.BlockProcessed,
Data: &statefeed.BlockProcessedData{
Slot: cfg.roblock.Block().Slot(),
BlockRoot: cfg.roblock.Root(),
SignedBlock: cfg.roblock,
Verified: true,
Optimistic: optimistic,
Slot: cfg.roblock.Block().Slot(),
BlockRoot: cfg.roblock.Root(),
SignedBlock: cfg.roblock,
CurrDependentRoot: currDependenRoot,
PrevDependentRoot: prevDependentRoot,
Verified: true,
Optimistic: optimistic,
},
})
}

View File

@@ -719,3 +719,14 @@ func (c *ChainService) ReceiveBlob(_ context.Context, b blocks.VerifiedROBlob) e
func (c *ChainService) TargetRootForEpoch(_ [32]byte, _ primitives.Epoch) ([32]byte, error) {
return c.TargetRoot, nil
}
// MockSyncChecker is a mock implementation of blockchain.Checker.
// We can't make an assertion here that this is true because that would create a circular dependency.
type MockSyncChecker struct {
synced bool
}
// Synced satisfies the blockchain.Checker interface.
func (m *MockSyncChecker) Synced() bool {
return m.synced
}

View File

@@ -11,6 +11,8 @@ const (
// ReceivedBlockData is the data sent with ReceivedBlock events.
type ReceivedBlockData struct {
SignedBlock interfaces.ReadOnlySignedBeaconBlock
IsOptimistic bool
SignedBlock interfaces.ReadOnlySignedBeaconBlock
CurrDependentRoot [32]byte
PrevDependentRoot [32]byte
IsOptimistic bool
}

View File

@@ -43,6 +43,10 @@ type BlockProcessedData struct {
BlockRoot [32]byte
// SignedBlock is the physical processed block.
SignedBlock interfaces.ReadOnlySignedBeaconBlock
// CurrDependentRoot is the current dependent root
CurrDependentRoot [32]byte
// PrevDependentRoot is the previous dependent root
PrevDependentRoot [32]byte
// Verified is true if the block's BLS contents have been verified.
Verified bool
// Optimistic is true if the block is optimistic.

View File

@@ -865,6 +865,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithVerifierWaiter(b.verifyInitWaiter),
regularsync.WithAvailableBlocker(bFillStore),
regularsync.WithSlasherEnabled(b.slasherEnabled),
regularsync.WithLightClientStore(b.lcStore),
)
return b.services.RegisterService(rs)
}

View File

@@ -44,6 +44,12 @@ const (
// blsToExecutionChangeWeight specifies the scoring weight that we apply to
// our bls to execution topic.
blsToExecutionChangeWeight = 0.05
// lightClientOptimisticUpdateWeight specifies the scoring weight that we apply to
// our light client optimistic update topic.
lightClientOptimisticUpdateWeight = 0.05
// lightClientFinalityUpdateWeight specifies the scoring weight that we apply to
// our light client finality update topic.
lightClientFinalityUpdateWeight = 0.05
// maxInMeshScore describes the max score a peer can attain from being in the mesh.
maxInMeshScore = 10
@@ -124,6 +130,10 @@ func (s *Service) topicScoreParams(topic string) (*pubsub.TopicScoreParams, erro
case strings.Contains(topic, GossipBlobSidecarMessage):
// TODO(Deneb): Using the default block scoring. But this should be updated.
return defaultBlockTopicParams(), nil
case strings.Contains(topic, GossipLightClientOptimisticUpdateMessage):
return defaultLightClientOptimisticUpdateTopicParams(), nil
case strings.Contains(topic, GossipLightClientFinalityUpdateMessage):
return defaultLightClientFinalityUpdateTopicParams(), nil
default:
return nil, errors.Errorf("unrecognized topic provided for parameter registration: %s", topic)
}
@@ -503,6 +513,50 @@ func defaultBlsToExecutionChangeTopicParams() *pubsub.TopicScoreParams {
}
}
func defaultLightClientOptimisticUpdateTopicParams() *pubsub.TopicScoreParams {
return &pubsub.TopicScoreParams{
TopicWeight: lightClientOptimisticUpdateWeight,
TimeInMeshWeight: maxInMeshScore / inMeshCap(),
TimeInMeshQuantum: inMeshTime(),
TimeInMeshCap: inMeshCap(),
FirstMessageDeliveriesWeight: 2,
FirstMessageDeliveriesDecay: scoreDecay(oneHundredEpochs),
FirstMessageDeliveriesCap: 5,
MeshMessageDeliveriesWeight: 0,
MeshMessageDeliveriesDecay: 0,
MeshMessageDeliveriesCap: 0,
MeshMessageDeliveriesThreshold: 0,
MeshMessageDeliveriesWindow: 0,
MeshMessageDeliveriesActivation: 0,
MeshFailurePenaltyWeight: 0,
MeshFailurePenaltyDecay: 0,
InvalidMessageDeliveriesWeight: -2000,
InvalidMessageDeliveriesDecay: scoreDecay(invalidDecayPeriod),
}
}
func defaultLightClientFinalityUpdateTopicParams() *pubsub.TopicScoreParams {
return &pubsub.TopicScoreParams{
TopicWeight: lightClientFinalityUpdateWeight,
TimeInMeshWeight: maxInMeshScore / inMeshCap(),
TimeInMeshQuantum: inMeshTime(),
TimeInMeshCap: inMeshCap(),
FirstMessageDeliveriesWeight: 2,
FirstMessageDeliveriesDecay: scoreDecay(oneHundredEpochs),
FirstMessageDeliveriesCap: 5,
MeshMessageDeliveriesWeight: 0,
MeshMessageDeliveriesDecay: 0,
MeshMessageDeliveriesCap: 0,
MeshMessageDeliveriesThreshold: 0,
MeshMessageDeliveriesWindow: 0,
MeshMessageDeliveriesActivation: 0,
MeshFailurePenaltyWeight: 0,
MeshFailurePenaltyDecay: 0,
InvalidMessageDeliveriesWeight: -2000,
InvalidMessageDeliveriesDecay: scoreDecay(invalidDecayPeriod),
}
}
func oneSlotDuration() time.Duration {
return time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
}

View File

@@ -75,4 +75,6 @@ func TestLoggingParameters(_ *testing.T) {
logGossipParameters("testing", defaultAttesterSlashingTopicParams())
logGossipParameters("testing", defaultProposerSlashingTopicParams())
logGossipParameters("testing", defaultVoluntaryExitTopicParams())
logGossipParameters("testing", defaultLightClientOptimisticUpdateTopicParams())
logGossipParameters("testing", defaultLightClientFinalityUpdateTopicParams())
}

View File

@@ -73,6 +73,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateAltair)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateAltair)
assert.Equal(t, true, ok)
// Bellatrix Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, bellatrixForkEpoch)
@@ -87,6 +93,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateAltair)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateAltair)
assert.Equal(t, true, ok)
// Capella Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, capellaForkEpoch)
@@ -101,6 +113,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateCapella)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateCapella)
assert.Equal(t, true, ok)
// Deneb Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, denebForkEpoch)
@@ -115,6 +133,12 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateDeneb)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateDeneb)
assert.Equal(t, true, ok)
// Electra Fork
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, electraForkEpoch)
@@ -129,4 +153,10 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProofElectra)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientOptimisticUpdateTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.LightClientOptimisticUpdateDeneb)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(LightClientFinalityUpdateTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.LightClientFinalityUpdateElectra)
assert.Equal(t, true, ok)
}

View File

@@ -21,6 +21,7 @@ go_library(
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/light-client:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -4,6 +4,7 @@ import (
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
lightclientConsensusTypes "github.com/OffchainLabs/prysm/v6/consensus-types/light-client"
"github.com/OffchainLabs/prysm/v6/consensus-types/wrapper"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
enginev1 "github.com/OffchainLabs/prysm/v6/proto/engine/v1"
@@ -36,6 +37,12 @@ var (
// AttesterSlashingMap maps the fork-version to the underlying data type for that particular
// fork period.
AttesterSlashingMap map[[4]byte]func() (ethpb.AttSlashing, error)
// LightClientOptimisticUpdateMap maps the fork-version to the underlying data type for that
// particular fork period.
LightClientOptimisticUpdateMap map[[4]byte]func() (interfaces.LightClientOptimisticUpdate, error)
// LightClientFinalityUpdateMap maps the fork-version to the underlying data type for that
// particular fork period.
LightClientFinalityUpdateMap map[[4]byte]func() (interfaces.LightClientFinalityUpdate, error)
)
// InitializeDataMaps initializes all the relevant object maps. This function is called to
@@ -179,4 +186,42 @@ func InitializeDataMaps() {
return &ethpb.AttesterSlashingElectra{}, nil
},
}
// Reset our light client optimistic update map.
LightClientOptimisticUpdateMap = map[[4]byte]func() (interfaces.LightClientOptimisticUpdate, error){
bytesutil.ToBytes4(params.BeaconConfig().AltairForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().BellatrixForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().CapellaForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateCapella(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().DenebForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateDeneb(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().ElectraForkVersion): func() (interfaces.LightClientOptimisticUpdate, error) {
return lightclientConsensusTypes.NewEmptyOptimisticUpdateDeneb(), nil
},
}
// Reset our light client finality update map.
LightClientFinalityUpdateMap = map[[4]byte]func() (interfaces.LightClientFinalityUpdate, error){
bytesutil.ToBytes4(params.BeaconConfig().AltairForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().BellatrixForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateAltair(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().CapellaForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateCapella(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().DenebForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateDeneb(), nil
},
bytesutil.ToBytes4(params.BeaconConfig().ElectraForkVersion): func() (interfaces.LightClientFinalityUpdate, error) {
return lightclientConsensusTypes.NewEmptyFinalityUpdateElectra(), nil
},
}
}

View File

@@ -37,7 +37,6 @@ import (
)
const DefaultEventFeedDepth = 1000
const payloadAttributeTimeout = 2 * time.Second
const (
InvalidTopic = "__invalid__"
@@ -627,6 +626,7 @@ func (s *Server) lazyReaderForEvent(ctx context.Context, event *feed.Event, topi
}
var errUnsupportedPayloadAttribute = errors.New("cannot compute payload attributes pre-Bellatrix")
var errPayloadAttributeExpired = errors.New("skipping payload attribute event for past slot")
func (s *Server) computePayloadAttributes(ctx context.Context, st state.ReadOnlyBeaconState, root [32]byte, proposer primitives.ValidatorIndex, timestamp uint64, randao []byte) (payloadattribute.Attributer, error) {
v := st.Version()
@@ -711,7 +711,7 @@ func (s *Server) fillEventData(ctx context.Context, ev payloadattribute.EventDat
return ev, errors.Wrap(err, "could not get head state")
}
// double check that we need to process_slots, just in case we got here via a hot state cache miss.
if slots.ToEpoch(st.Slot()) == pse {
if slots.ToEpoch(st.Slot()) < pse {
start, err := slots.EpochStart(pse)
if err != nil {
return ev, errors.Wrap(err, "invalid state slot; could not compute epoch start")
@@ -750,7 +750,11 @@ func (s *Server) fillEventData(ctx context.Context, ev payloadattribute.EventDat
// This event stream is intended to be used by builders and relays.
// Parent fields are based on state at N_{current_slot}, while the rest of fields are based on state of N_{current_slot + 1}
func (s *Server) payloadAttributesReader(ctx context.Context, ev payloadattribute.EventData) (lazyReader, error) {
ctx, cancel := context.WithTimeout(ctx, payloadAttributeTimeout)
deadline := slots.BeginsAt(ev.ProposalSlot, s.ChainInfoFetcher.GenesisTime())
if deadline.Before(time.Now()) {
return nil, errors.Wrapf(errPayloadAttributeExpired, "proposal slot time %d", deadline.Unix())
}
ctx, cancel := context.WithDeadline(ctx, deadline)
edc := make(chan asyncPayloadAttrData)
go func() {
d := asyncPayloadAttrData{}

View File

@@ -523,11 +523,14 @@ func TestStreamEvents_OperationsEvents(t *testing.T) {
// to avoid slot processing
require.NoError(t, st.SetSlot(currentSlot+1))
b := tc.getBlock()
genesis := time.Now()
require.NoError(t, st.SetGenesisTime(uint64(genesis.Unix())))
mockChainService := &mockChain.ChainService{
Root: make([]byte, 32),
State: st,
Block: b,
Slot: &currentSlot,
Root: make([]byte, 32),
State: st,
Block: b,
Slot: &currentSlot,
Genesis: genesis,
}
headRoot, err := b.Block().HashTreeRoot()
require.NoError(t, err)
@@ -557,7 +560,7 @@ func TestStreamEvents_OperationsEvents(t *testing.T) {
Type: statefeed.PayloadAttributes,
Data: payloadattribute.EventData{
ProposerIndex: 0,
ProposalSlot: 0,
ProposalSlot: mockChainService.CurrentSlot() + 1,
ParentBlockNumber: 0,
ParentBlockHash: make([]byte, 32),
HeadBlock: b,

View File

@@ -9,7 +9,6 @@ import (
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/time/slots"
"github.com/pkg/errors"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@@ -67,6 +66,7 @@ func (vs *Server) StreamSlots(req *ethpb.StreamSlotsRequest, stream ethpb.Beacon
select {
case ev := <-ch:
var s primitives.Slot
var currDependentRoot, prevDependentRoot [32]byte
if req.VerifiedOnly {
if ev.Type != statefeed.BlockProcessed {
continue
@@ -76,6 +76,8 @@ func (vs *Server) StreamSlots(req *ethpb.StreamSlotsRequest, stream ethpb.Beacon
continue
}
s = data.Slot
currDependentRoot = data.CurrDependentRoot
prevDependentRoot = data.PrevDependentRoot
} else {
if ev.Type != blockfeed.ReceivedBlock {
continue
@@ -85,24 +87,14 @@ func (vs *Server) StreamSlots(req *ethpb.StreamSlotsRequest, stream ethpb.Beacon
continue
}
s = data.SignedBlock.Block().Slot()
}
currEpoch := slots.ToEpoch(s)
currDepRoot, err := vs.ForkchoiceFetcher.DependentRoot(currEpoch)
if err != nil {
return status.Errorf(codes.Internal, "Could not get dependent root: %v", err)
}
prevDepRoot := currDepRoot
if currEpoch > 0 {
prevDepRoot, err = vs.ForkchoiceFetcher.DependentRoot(currEpoch - 1)
if err != nil {
return status.Errorf(codes.Internal, "Could not get dependent root: %v", err)
}
currDependentRoot = data.CurrDependentRoot
prevDependentRoot = data.PrevDependentRoot
}
if err := stream.Send(
&ethpb.StreamSlotsResponse{
Slot: s,
PreviousDutyDependentRoot: prevDepRoot[:],
CurrentDutyDependentRoot: currDepRoot[:],
PreviousDutyDependentRoot: prevDependentRoot[:],
CurrentDutyDependentRoot: currDependentRoot[:],
}); err != nil {
return status.Errorf(codes.Unavailable, "Could not send over stream: %v", err)
}

View File

@@ -38,6 +38,7 @@ go_library(
"subscriber_blob_sidecar.go",
"subscriber_bls_to_execution_change.go",
"subscriber_handlers.go",
"subscriber_light_client.go",
"subscriber_sync_committee_message.go",
"subscriber_sync_contribution_proof.go",
"subscription_topic_handler.go",
@@ -47,6 +48,7 @@ go_library(
"validate_beacon_blocks.go",
"validate_blob.go",
"validate_bls_to_execution_change.go",
"validate_light_client.go",
"validate_proposer_slashing.go",
"validate_sync_committee_message.go",
"validate_sync_contribution_proof.go",
@@ -71,6 +73,7 @@ go_library(
"//beacon-chain/core/feed/operation:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/core/transition/interop:go_default_library",
@@ -101,6 +104,7 @@ go_library(
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/light-client:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",
"//container/leaky-bucket:go_default_library",
@@ -182,6 +186,7 @@ go_test(
"validate_beacon_blocks_test.go",
"validate_blob_test.go",
"validate_bls_to_execution_change_test.go",
"validate_light_client_test.go",
"validate_proposer_slashing_test.go",
"validate_sync_committee_message_test.go",
"validate_sync_contribution_proof_test.go",
@@ -198,6 +203,7 @@ go_test(
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/operation:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",

View File

@@ -89,6 +89,10 @@ func extractValidDataTypeFromTopic(topic string, digest []byte, clock *startup.C
return extractDataTypeFromTypeMap(types.AggregateAttestationMap, digest, clock)
case p2p.AttesterSlashingSubnetTopicFormat:
return extractDataTypeFromTypeMap(types.AttesterSlashingMap, digest, clock)
case p2p.LightClientOptimisticUpdateTopicFormat:
return extractDataTypeFromTypeMap(types.LightClientOptimisticUpdateMap, digest, clock)
case p2p.LightClientFinalityUpdateTopicFormat:
return extractDataTypeFromTypeMap(types.LightClientFinalityUpdateMap, digest, clock)
}
return nil, nil
}

View File

@@ -6,6 +6,7 @@ import (
blockfeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/block"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/operation"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/execution"
@@ -196,3 +197,11 @@ func WithSlasherEnabled(enabled bool) Option {
return nil
}
}
// WithLightClientStore allows the sync package to access light client data.
func WithLightClientStore(lcs *lightClient.Store) Option {
return func(s *Service) error {
s.lcStore = lcs
return nil
}
}

View File

@@ -9,6 +9,7 @@ import (
"sync"
"time"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
lru "github.com/hashicorp/golang-lru"
pubsub "github.com/libp2p/go-libp2p-pubsub"
libp2pcore "github.com/libp2p/go-libp2p/core"
@@ -165,6 +166,7 @@ type Service struct {
availableBlocker coverage.AvailableBlocker
ctxMap ContextByteVersions
slasherEnabled bool
lcStore *lightClient.Store
}
// NewService initializes new regular sync service.

View File

@@ -0,0 +1,66 @@
package sync
import (
"context"
"fmt"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightclientTypes "github.com/OffchainLabs/prysm/v6/consensus-types/light-client"
"github.com/sirupsen/logrus"
"google.golang.org/protobuf/proto"
)
func (s *Service) lightClientOptimisticUpdateSubscriber(_ context.Context, msg proto.Message) error {
update, err := lightclientTypes.NewWrappedOptimisticUpdate(msg)
if err != nil {
return err
}
attestedHeaderRoot, err := update.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return err
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", update.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", update.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("Saving newly received light client optimistic update.")
s.lcStore.SetLastOptimisticUpdate(update)
s.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.LightClientOptimisticUpdate,
Data: update,
})
return nil
}
func (s *Service) lightClientFinalityUpdateSubscriber(_ context.Context, msg proto.Message) error {
update, err := lightclientTypes.NewWrappedFinalityUpdate(msg)
if err != nil {
return err
}
attestedHeaderRoot, err := update.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return err
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", update.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", update.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("Saving newly received light client finality update.")
s.lcStore.SetLastFinalityUpdate(update)
s.cfg.stateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.LightClientFinalityUpdate,
Data: update,
})
return nil
}

View File

@@ -11,6 +11,7 @@ import (
"github.com/OffchainLabs/prysm/v6/async/abool"
mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/signing"
db "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/operations/slashings"
@@ -27,6 +28,7 @@ import (
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/network/forks"
pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/assert"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
@@ -638,3 +640,137 @@ func createPeer(t *testing.T, topics ...string) *p2ptest.TestP2P {
}
return p
}
func TestSubscribe_ReceivesLCOptimisticUpdate(t *testing.T) {
origNC := params.BeaconConfig()
// restore network config after test completes
defer func() {
params.OverrideBeaconConfig(origNC)
}()
params.SetupTestConfigCleanup(t)
p2pService := p2ptest.NewTestP2P(t)
ctx := context.Background()
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
genesisDrift := slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals
chainService := &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0),
}
d := db.SetupDB(t)
r := Service{
ctx: ctx,
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
beaconDB: d,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: &lightClient.Store{},
subHandler: newSubTopicHandler(),
}
topic := p2p.LightClientOptimisticUpdateTopicFormat
var wg sync.WaitGroup
wg.Add(1)
var err error
p2pService.Digest, err = r.currentForkDigest()
require.NoError(t, err)
r.subscribe(topic, r.validateLightClientOptimisticUpdate, func(ctx context.Context, msg proto.Message) error {
require.NoError(t, r.lightClientOptimisticUpdateSubscriber(ctx, msg))
wg.Done()
return nil
}, p2pService.Digest)
r.markForChainStart()
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority())
update, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err, "Error generating light client optimistic update")
p2pService.ReceivePubSub(topic, update.Proto())
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")
}
u := r.lcStore.LastOptimisticUpdate()
assert.DeepEqual(t, update.Proto(), u.Proto())
}
func TestSubscribe_ReceivesLCFinalityUpdate(t *testing.T) {
origNC := params.BeaconConfig()
// restore network config after test completes
defer func() {
params.OverrideBeaconConfig(origNC)
}()
params.SetupTestConfigCleanup(t)
p2pService := p2ptest.NewTestP2P(t)
ctx := context.Background()
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
genesisDrift := slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals
chainService := &mockChain.ChainService{
ValidatorsRoot: [32]byte{'A'},
Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0),
}
d := db.SetupDB(t)
r := Service{
ctx: ctx,
cfg: &config{
p2p: p2pService,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: chainService,
beaconDB: d,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
stateNotifier: &mockChain.MockStateNotifier{},
},
chainStarted: abool.New(),
lcStore: &lightClient.Store{},
subHandler: newSubTopicHandler(),
}
topic := p2p.LightClientFinalityUpdateTopicFormat
var wg sync.WaitGroup
wg.Add(1)
var err error
p2pService.Digest, err = r.currentForkDigest()
require.NoError(t, err)
r.subscribe(topic, r.validateLightClientFinalityUpdate, func(ctx context.Context, msg proto.Message) error {
require.NoError(t, r.lightClientFinalityUpdateSubscriber(ctx, msg))
wg.Done()
return nil
}, p2pService.Digest)
r.markForChainStart()
l := util.NewTestLightClient(t, version.Altair, util.WithSupermajority())
update, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err, "Error generating light client finality update")
p2pService.ReceivePubSub(topic, update.Proto())
if util.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")
}
u := r.lcStore.LastFinalityUpdate()
assert.DeepEqual(t, update.Proto(), u.Proto())
}

View File

@@ -0,0 +1,160 @@
package sync
import (
"context"
"fmt"
"time"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
"github.com/OffchainLabs/prysm/v6/time/slots"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/sirupsen/logrus"
)
func (s *Service) validateLightClientOptimisticUpdate(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Ignore updates while syncing
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
_, span := trace.StartSpan(ctx, "sync.validateLightClientOptimisticUpdate")
defer span.End()
m, err := s.decodePubsubMessage(msg)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
newUpdate, ok := m.(interfaces.LightClientOptimisticUpdate)
if !ok {
return pubsub.ValidationReject, errWrongMessage
}
attestedHeaderRoot, err := newUpdate.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return pubsub.ValidationIgnore, err
}
// [IGNORE] The optimistic_update is received after the block at signature_slot was given enough time
// to propagate through the network -- i.e. validate that one-third of optimistic_update.signature_slot
// has transpired (SECONDS_PER_SLOT / INTERVALS_PER_SLOT seconds after the start of the slot,
// with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance)
earliestValidTime := slots.StartTime(uint64(s.cfg.clock.GenesisTime().Unix()), newUpdate.SignatureSlot()).
Add(time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot/params.BeaconConfig().IntervalsPerSlot)).
Add(-params.BeaconConfig().MaximumGossipClockDisparityDuration())
if s.cfg.clock.Now().Before(earliestValidTime) {
log.Debug("Newly received light client optimistic update ignored. not enough time passed for block to propagate")
return pubsub.ValidationIgnore, nil
}
lastStoredUpdate := s.lcStore.LastOptimisticUpdate()
if lastStoredUpdate != nil {
lastUpdateSlot := lastStoredUpdate.AttestedHeader().Beacon().Slot
newUpdateSlot := newUpdate.AttestedHeader().Beacon().Slot
// [IGNORE] The attested_header.beacon.slot is greater than that of all previously forwarded optimistic_updates
if newUpdateSlot <= lastUpdateSlot {
log.Debug("Newly received light client optimistic update ignored. new update is older than stored update")
return pubsub.ValidationIgnore, nil
}
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", newUpdate.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", newUpdate.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("New gossiped light client optimistic update validated.")
msg.ValidatorData = newUpdate.Proto()
return pubsub.ValidationAccept, nil
}
func (s *Service) validateLightClientFinalityUpdate(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
// Ignore updates while syncing
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
_, span := trace.StartSpan(ctx, "sync.validateLightClientFinalityUpdate")
defer span.End()
m, err := s.decodePubsubMessage(msg)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
newUpdate, ok := m.(interfaces.LightClientFinalityUpdate)
if !ok {
return pubsub.ValidationReject, errWrongMessage
}
attestedHeaderRoot, err := newUpdate.AttestedHeader().Beacon().HashTreeRoot()
if err != nil {
return pubsub.ValidationIgnore, err
}
// [IGNORE] The optimistic_update is received after the block at signature_slot was given enough time
// to propagate through the network -- i.e. validate that one-third of optimistic_update.signature_slot
// has transpired (SECONDS_PER_SLOT / INTERVALS_PER_SLOT seconds after the start of the slot,
// with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance)
earliestValidTime := slots.StartTime(uint64(s.cfg.clock.GenesisTime().Unix()), newUpdate.SignatureSlot()).
Add(time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot/params.BeaconConfig().IntervalsPerSlot)).
Add(-params.BeaconConfig().MaximumGossipClockDisparityDuration())
if s.cfg.clock.Now().Before(earliestValidTime) {
log.Debug("Newly received light client finality update ignored. not enough time passed for block to propagate")
return pubsub.ValidationIgnore, nil
}
lastStoredUpdate := s.lcStore.LastFinalityUpdate()
if lastStoredUpdate != nil {
lastUpdateSlot := lastStoredUpdate.FinalizedHeader().Beacon().Slot
newUpdateSlot := newUpdate.FinalizedHeader().Beacon().Slot
// [IGNORE] The finalized_header.beacon.slot is greater than that of all previously forwarded finality_updates,
// or it matches the highest previously forwarded slot and also has a sync_aggregate indicating supermajority (> 2/3)
// sync committee participation while the previously forwarded finality_update for that slot did not indicate supermajority
lastUpdateHasSupermajority := lightClient.UpdateHasSupermajority(lastStoredUpdate.SyncAggregate())
newUpdateHasSupermajority := lightClient.UpdateHasSupermajority(newUpdate.SyncAggregate())
if newUpdateSlot < lastUpdateSlot {
log.Debug("Newly received light client finality update ignored. new update is older than stored update")
return pubsub.ValidationIgnore, nil
}
if newUpdateSlot == lastUpdateSlot && (lastUpdateHasSupermajority || !newUpdateHasSupermajority) {
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", newUpdate.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", newUpdate.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("Newly received light client finality update ignored. no supermajority advantage.")
return pubsub.ValidationIgnore, nil
}
}
log.WithFields(logrus.Fields{
"attestedSlot": fmt.Sprintf("%d", newUpdate.AttestedHeader().Beacon().Slot),
"signatureSlot": fmt.Sprintf("%d", newUpdate.SignatureSlot()),
"attestedHeaderRoot": fmt.Sprintf("%x", attestedHeaderRoot),
}).Debug("New gossiped light client finality update validated.")
msg.ValidatorData = newUpdate.Proto()
return pubsub.ValidationAccept, nil
}

View File

@@ -0,0 +1,284 @@
package sync
import (
"bytes"
"context"
"testing"
"time"
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/runtime/version"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pb "github.com/libp2p/go-libp2p-pubsub/pb"
)
func TestValidateLightClientOptimisticUpdate_NilMessageOrTopic(t *testing.T) {
params.SetupTestConfigCleanup(t)
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}}}
_, err := s.validateLightClientOptimisticUpdate(ctx, "", nil)
require.ErrorIs(t, err, errNilPubsubMessage)
_, err = s.validateLightClientOptimisticUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{}})
require.ErrorIs(t, err, errNilPubsubMessage)
emptyTopic := ""
_, err = s.validateLightClientOptimisticUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{
Topic: &emptyTopic,
}})
require.ErrorIs(t, err, errNilPubsubMessage)
}
func TestValidateLightClientOptimisticUpdate(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.BellatrixForkEpoch = 2
cfg.CapellaForkEpoch = 3
cfg.DenebForkEpoch = 4
cfg.ElectraForkEpoch = 5
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
cfg.ForkVersionSchedule[[4]byte{2, 0, 0, 0}] = 2
cfg.ForkVersionSchedule[[4]byte{3, 0, 0, 0}] = 3
cfg.ForkVersionSchedule[[4]byte{4, 0, 0, 0}] = 4
cfg.ForkVersionSchedule[[4]byte{5, 0, 0, 0}] = 5
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
tests := []struct {
name string
genesisDrift int
oldUpdateOptions []util.LightClientOption
newUpdateOptions []util.LightClientOption
expectedResult pubsub.ValidationResult
expectedErr error
}{
{
name: "no previous update",
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationAccept,
},
{
name: "not enough time passed",
genesisDrift: -secondsPerSlot / slotIntervals,
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update has no age advantage",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update is better - younger",
genesisDrift: secondsPerSlot,
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithIncreasedAttestedSlot(1)},
expectedResult: pubsub.ValidationAccept,
},
}
for _, test := range tests {
for v := 1; v < 6; v++ {
t.Run(test.name+"_"+version.String(v), func(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
// drift back appropriate number of epochs based on fork + 2 slots for signature slot + time for gossip propagation + any extra drift
genesisDrift := v*slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals + test.genesisDrift
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0)}
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}, lcStore: &lightClient.Store{}}
var oldUpdate interfaces.LightClientOptimisticUpdate
var err error
if test.oldUpdateOptions != nil {
l := util.NewTestLightClient(t, v, test.oldUpdateOptions...)
oldUpdate, err = lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
s.lcStore.SetLastOptimisticUpdate(oldUpdate)
}
l := util.NewTestLightClient(t, v, test.newUpdateOptions...)
newUpdate, err := lightClient.NewLightClientOptimisticUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock)
require.NoError(t, err)
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, newUpdate)
require.NoError(t, err)
topic := p2p.LightClientOptimisticUpdateTopicFormat
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
r, err := s.validateLightClientOptimisticUpdate(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
if test.expectedErr != nil {
require.ErrorIs(t, err, test.expectedErr)
} else {
require.NoError(t, err)
require.Equal(t, test.expectedResult, r)
}
})
}
}
}
func TestValidateLightClientFinalityUpdate_NilMessageOrTopic(t *testing.T) {
params.SetupTestConfigCleanup(t)
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}}}
_, err := s.validateLightClientFinalityUpdate(ctx, "", nil)
require.ErrorIs(t, err, errNilPubsubMessage)
_, err = s.validateLightClientFinalityUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{}})
require.ErrorIs(t, err, errNilPubsubMessage)
emptyTopic := ""
_, err = s.validateLightClientFinalityUpdate(ctx, "", &pubsub.Message{Message: &pb.Message{
Topic: &emptyTopic,
}})
require.ErrorIs(t, err, errNilPubsubMessage)
}
func TestValidateLightClientFinalityUpdate(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = 1
cfg.BellatrixForkEpoch = 2
cfg.CapellaForkEpoch = 3
cfg.DenebForkEpoch = 4
cfg.ElectraForkEpoch = 5
cfg.ForkVersionSchedule[[4]byte{1, 0, 0, 0}] = 1
cfg.ForkVersionSchedule[[4]byte{2, 0, 0, 0}] = 2
cfg.ForkVersionSchedule[[4]byte{3, 0, 0, 0}] = 3
cfg.ForkVersionSchedule[[4]byte{4, 0, 0, 0}] = 4
cfg.ForkVersionSchedule[[4]byte{5, 0, 0, 0}] = 5
params.OverrideBeaconConfig(cfg)
secondsPerSlot := int(params.BeaconConfig().SecondsPerSlot)
slotIntervals := int(params.BeaconConfig().IntervalsPerSlot)
slotsPerEpoch := int(params.BeaconConfig().SlotsPerEpoch)
tests := []struct {
name string
genesisDrift int
oldUpdateOptions []util.LightClientOption
newUpdateOptions []util.LightClientOption
expectedResult pubsub.ValidationResult
expectedErr error
}{
{
name: "no previous update",
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationAccept,
},
{
name: "not enough time passed",
genesisDrift: -secondsPerSlot / slotIntervals,
oldUpdateOptions: nil,
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update has no advantage",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "new update is better - age",
genesisDrift: secondsPerSlot,
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithIncreasedFinalizedSlot(1)},
expectedResult: pubsub.ValidationAccept,
},
{
name: "new update is better - supermajority",
oldUpdateOptions: []util.LightClientOption{},
newUpdateOptions: []util.LightClientOption{util.WithSupermajority()},
expectedResult: pubsub.ValidationAccept,
},
{
name: "old update is better - supermajority",
oldUpdateOptions: []util.LightClientOption{util.WithSupermajority()},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
{
name: "old update is better - age",
oldUpdateOptions: []util.LightClientOption{util.WithIncreasedAttestedSlot(1)},
newUpdateOptions: []util.LightClientOption{},
expectedResult: pubsub.ValidationIgnore,
},
}
for _, test := range tests {
for v := 1; v < 6; v++ {
t.Run(test.name+"_"+version.String(v), func(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
// drift back appropriate number of epochs based on fork + 2 slots for signature slot + time for gossip propagation + any extra drift
genesisDrift := v*slotsPerEpoch*secondsPerSlot + 2*secondsPerSlot + secondsPerSlot/slotIntervals + test.genesisDrift
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(genesisDrift), 0)}
s := &Service{cfg: &config{p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}, lcStore: &lightClient.Store{}}
var oldUpdate interfaces.LightClientFinalityUpdate
var err error
if test.oldUpdateOptions != nil {
l := util.NewTestLightClient(t, v, test.oldUpdateOptions...)
oldUpdate, err = lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
s.lcStore.SetLastFinalityUpdate(oldUpdate)
}
l := util.NewTestLightClient(t, v, test.newUpdateOptions...)
newUpdate, err := lightClient.NewLightClientFinalityUpdateFromBeaconState(l.Ctx, l.State.Slot(), l.State, l.Block, l.AttestedState, l.AttestedBlock, l.FinalizedBlock)
require.NoError(t, err)
buf := new(bytes.Buffer)
_, err = p.Encoding().EncodeGossip(buf, newUpdate)
require.NoError(t, err)
topic := p2p.LightClientFinalityUpdateTopicFormat
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
r, err := s.validateLightClientFinalityUpdate(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
if test.expectedErr != nil {
require.ErrorIs(t, err, test.expectedErr)
} else {
require.NoError(t, err)
require.Equal(t, test.expectedResult, r)
}
})
}
}
}

View File

@@ -0,0 +1,3 @@
### Added
- Add light client p2p validator and subscriber functions.

View File

@@ -0,0 +1,2 @@
### Fixed
- extend the payload attribute computation deadline to the beginning of the proposal slot.

View File

@@ -0,0 +1,3 @@
### Added
- PeerDAS: Add needed proto files and corresponding generated code.

View File

@@ -0,0 +1,3 @@
### Ignored
- Check for uninitialized duties on `checkDependentRoot`

View File

@@ -0,0 +1,3 @@
### Ignored
- Add dependent roots in block events.

View File

@@ -0,0 +1,3 @@
### Changed
- Removed the slot from `UpdateDuties`.

3
changelog/tt_ramen.md Normal file
View File

@@ -0,0 +1,3 @@
### Fixed
- Process slots across epoch for payload attribute event.

View File

@@ -99,6 +99,12 @@ type finalityUpdateAltair struct {
finalityBranch interfaces.LightClientFinalityBranch
}
// NewEmptyFinalityUpdateAltair normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateAltair() interfaces.LightClientFinalityUpdate {
return &finalityUpdateAltair{}
}
func (u *finalityUpdateAltair) IsNil() bool {
return u == nil || u.p == nil
}
@@ -205,6 +211,12 @@ type finalityUpdateCapella struct {
finalityBranch interfaces.LightClientFinalityBranch
}
// NewEmptyFinalityUpdateCapella normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateCapella() interfaces.LightClientFinalityUpdate {
return &finalityUpdateCapella{}
}
func (u *finalityUpdateCapella) IsNil() bool {
return u == nil || u.p == nil
}
@@ -311,6 +323,12 @@ type finalityUpdateDeneb struct {
finalityBranch interfaces.LightClientFinalityBranch
}
// NewEmptyFinalityUpdateDeneb normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateDeneb() interfaces.LightClientFinalityUpdate {
return &finalityUpdateDeneb{}
}
func (u *finalityUpdateDeneb) IsNil() bool {
return u == nil || u.p == nil
}
@@ -417,6 +435,12 @@ type finalityUpdateElectra struct {
finalityBranch interfaces.LightClientFinalityBranchElectra
}
// NewEmptyFinalityUpdateElectra normally should never be called and NewFinalityUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyFinalityUpdateElectra() interfaces.LightClientFinalityUpdate {
return &finalityUpdateElectra{}
}
func (u *finalityUpdateElectra) IsNil() bool {
return u == nil || u.p == nil
}

View File

@@ -78,6 +78,12 @@ type optimisticUpdateAltair struct {
attestedHeader interfaces.LightClientHeader
}
// NewEmptyOptimisticUpdateAltair normally should never be called and NewOptimisticUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyOptimisticUpdateAltair() interfaces.LightClientOptimisticUpdate {
return &optimisticUpdateAltair{}
}
func (u *optimisticUpdateAltair) IsNil() bool {
return u == nil || u.p == nil
}
@@ -156,6 +162,12 @@ type optimisticUpdateCapella struct {
attestedHeader interfaces.LightClientHeader
}
// NewEmptyOptimisticUpdateCapella normally should never be called and NewOptimisticUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyOptimisticUpdateCapella() interfaces.LightClientOptimisticUpdate {
return &optimisticUpdateCapella{}
}
func (u *optimisticUpdateCapella) IsNil() bool {
return u == nil || u.p == nil
}
@@ -234,6 +246,12 @@ type optimisticUpdateDeneb struct {
attestedHeader interfaces.LightClientHeader
}
// NewEmptyOptimisticUpdateDeneb normally should never be called and NewOptimisticUpdateFromUpdate should be used instead.
// This function exists only for scenarios where an empty struct is required.
func NewEmptyOptimisticUpdateDeneb() interfaces.LightClientOptimisticUpdate {
return &optimisticUpdateDeneb{}
}
func (u *optimisticUpdateDeneb) IsNil() bool {
return u == nil || u.p == nil
}

View File

@@ -2,14 +2,13 @@
# Common
##############################################################################
load("@rules_proto//proto:defs.bzl", "proto_library")
##############################################################################
# Go
##############################################################################
# gazelle:ignore
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("@rules_proto//proto:defs.bzl", "proto_library")
load("//proto:ssz_proto_library.bzl", "ssz_proto_files")
load("//tools:ssz.bzl", "SSZ_DEPS", "ssz_gen_marshal")
@@ -30,8 +29,8 @@ proto_library(
##############################################################################
ssz_gen_marshal(
name = "ssz_generated_files",
go_proto = ":go_proto",
out = "engine.ssz.go",
go_proto = ":go_proto",
includes = [
"//consensus-types/primitives:go_default_library",
],
@@ -42,9 +41,10 @@ ssz_gen_marshal(
"ExecutionPayloadHeaderCapella",
"ExecutionPayloadHeaderDeneb",
"ExecutionPayloadDeneb",
'ExecutionPayloadDenebAndBlobsBundle',
"ExecutionPayloadDenebAndBlobsBundle",
"BlindedBlobsBundle",
"BlobsBundle",
"BlobsBundleV2",
"Withdrawal",
"WithdrawalRequest",
"DepositRequest",
@@ -75,8 +75,10 @@ go_proto_library(
go_library(
name = "go_default_library",
srcs = [
"blobs_bundle.go",
"electra.go",
"execution_engine.go",
"fulu.go",
"json_marshal_unmarshal.go",
":ssz_generated_files", # keep
],
@@ -111,8 +113,9 @@ go_library(
ssz_proto_files(
name = "ssz_proto_files",
srcs = [
"electra.proto",
"execution_engine.proto",
"electra.proto",
"fulu.proto",
],
config = select({
"//conditions:default": "mainnet",
@@ -124,8 +127,8 @@ ssz_proto_files(
go_test(
name = "go_default_test",
srcs = [
"export_test.go",
"execution_engine_fuzz_test.go",
"export_test.go",
"json_marshal_unmarshal_test.go",
],
embed = [":go_default_library"],
@@ -135,10 +138,10 @@ go_test(
"//consensus-types/primitives:go_default_library",
"//encoding/bytesutil:go_default_library",
"//testing/require:go_default_library",
"@com_github_google_gofuzz//:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@com_github_google_gofuzz//:go_default_library",
"@com_github_holiman_uint256//:go_default_library",
],
)

View File

@@ -0,0 +1,7 @@
package enginev1
type BlobsBundler interface {
GetKzgCommitments() [][]byte
GetProofs() [][]byte
GetBlobs() [][]byte
}

View File

@@ -3222,3 +3222,233 @@ func (b *BlobsBundle) HashTreeRootWith(hh *ssz.Hasher) (err error) {
hh.Merkleize(indx)
return
}
// MarshalSSZ ssz marshals the BlobsBundleV2 object
func (b *BlobsBundleV2) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(b)
}
// MarshalSSZTo ssz marshals the BlobsBundleV2 object to a target array
func (b *BlobsBundleV2) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
offset := int(12)
// Offset (0) 'KzgCommitments'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.KzgCommitments) * 48
// Offset (1) 'Proofs'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.Proofs) * 48
// Offset (2) 'Blobs'
dst = ssz.WriteOffset(dst, offset)
offset += len(b.Blobs) * 131072
// Field (0) 'KzgCommitments'
if size := len(b.KzgCommitments); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgCommitments", size, 4096)
return
}
for ii := 0; ii < len(b.KzgCommitments); ii++ {
if size := len(b.KzgCommitments[ii]); size != 48 {
err = ssz.ErrBytesLengthFn("--.KzgCommitments[ii]", size, 48)
return
}
dst = append(dst, b.KzgCommitments[ii]...)
}
// Field (1) 'Proofs'
if size := len(b.Proofs); size > 524288 {
err = ssz.ErrListTooBigFn("--.Proofs", size, 524288)
return
}
for ii := 0; ii < len(b.Proofs); ii++ {
if size := len(b.Proofs[ii]); size != 48 {
err = ssz.ErrBytesLengthFn("--.Proofs[ii]", size, 48)
return
}
dst = append(dst, b.Proofs[ii]...)
}
// Field (2) 'Blobs'
if size := len(b.Blobs); size > 4096 {
err = ssz.ErrListTooBigFn("--.Blobs", size, 4096)
return
}
for ii := 0; ii < len(b.Blobs); ii++ {
if size := len(b.Blobs[ii]); size != 131072 {
err = ssz.ErrBytesLengthFn("--.Blobs[ii]", size, 131072)
return
}
dst = append(dst, b.Blobs[ii]...)
}
return
}
// UnmarshalSSZ ssz unmarshals the BlobsBundleV2 object
func (b *BlobsBundleV2) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size < 12 {
return ssz.ErrSize
}
tail := buf
var o0, o1, o2 uint64
// Offset (0) 'KzgCommitments'
if o0 = ssz.ReadOffset(buf[0:4]); o0 > size {
return ssz.ErrOffset
}
if o0 != 12 {
return ssz.ErrInvalidVariableOffset
}
// Offset (1) 'Proofs'
if o1 = ssz.ReadOffset(buf[4:8]); o1 > size || o0 > o1 {
return ssz.ErrOffset
}
// Offset (2) 'Blobs'
if o2 = ssz.ReadOffset(buf[8:12]); o2 > size || o1 > o2 {
return ssz.ErrOffset
}
// Field (0) 'KzgCommitments'
{
buf = tail[o0:o1]
num, err := ssz.DivideInt2(len(buf), 48, 4096)
if err != nil {
return err
}
b.KzgCommitments = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(b.KzgCommitments[ii]) == 0 {
b.KzgCommitments[ii] = make([]byte, 0, len(buf[ii*48:(ii+1)*48]))
}
b.KzgCommitments[ii] = append(b.KzgCommitments[ii], buf[ii*48:(ii+1)*48]...)
}
}
// Field (1) 'Proofs'
{
buf = tail[o1:o2]
num, err := ssz.DivideInt2(len(buf), 48, 524288)
if err != nil {
return err
}
b.Proofs = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(b.Proofs[ii]) == 0 {
b.Proofs[ii] = make([]byte, 0, len(buf[ii*48:(ii+1)*48]))
}
b.Proofs[ii] = append(b.Proofs[ii], buf[ii*48:(ii+1)*48]...)
}
}
// Field (2) 'Blobs'
{
buf = tail[o2:]
num, err := ssz.DivideInt2(len(buf), 131072, 4096)
if err != nil {
return err
}
b.Blobs = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(b.Blobs[ii]) == 0 {
b.Blobs[ii] = make([]byte, 0, len(buf[ii*131072:(ii+1)*131072]))
}
b.Blobs[ii] = append(b.Blobs[ii], buf[ii*131072:(ii+1)*131072]...)
}
}
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the BlobsBundleV2 object
func (b *BlobsBundleV2) SizeSSZ() (size int) {
size = 12
// Field (0) 'KzgCommitments'
size += len(b.KzgCommitments) * 48
// Field (1) 'Proofs'
size += len(b.Proofs) * 48
// Field (2) 'Blobs'
size += len(b.Blobs) * 131072
return
}
// HashTreeRoot ssz hashes the BlobsBundleV2 object
func (b *BlobsBundleV2) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(b)
}
// HashTreeRootWith ssz hashes the BlobsBundleV2 object with a hasher
func (b *BlobsBundleV2) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'KzgCommitments'
{
if size := len(b.KzgCommitments); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgCommitments", size, 4096)
return
}
subIndx := hh.Index()
for _, i := range b.KzgCommitments {
if len(i) != 48 {
err = ssz.ErrBytesLength
return
}
hh.PutBytes(i)
}
numItems := uint64(len(b.KzgCommitments))
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
}
// Field (1) 'Proofs'
{
if size := len(b.Proofs); size > 524288 {
err = ssz.ErrListTooBigFn("--.Proofs", size, 524288)
return
}
subIndx := hh.Index()
for _, i := range b.Proofs {
if len(i) != 48 {
err = ssz.ErrBytesLength
return
}
hh.PutBytes(i)
}
numItems := uint64(len(b.Proofs))
hh.MerkleizeWithMixin(subIndx, numItems, 524288)
}
// Field (2) 'Blobs'
{
if size := len(b.Blobs); size > 4096 {
err = ssz.ErrListTooBigFn("--.Blobs", size, 4096)
return
}
subIndx := hh.Index()
for _, i := range b.Blobs {
if len(i) != 131072 {
err = ssz.ErrBytesLength
return
}
hh.PutBytes(i)
}
numItems := uint64(len(b.Blobs))
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
}
hh.Merkleize(indx)
return
}

View File

@@ -1705,6 +1705,69 @@ func (x *BlobsBundle) GetBlobs() [][]byte {
return nil
}
type BlobsBundleV2 struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
KzgCommitments [][]byte `protobuf:"bytes,1,rep,name=kzg_commitments,json=kzgCommitments,proto3" json:"kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"`
Proofs [][]byte `protobuf:"bytes,2,rep,name=proofs,proto3" json:"proofs,omitempty" ssz-max:"524288" ssz-size:"?,48"`
Blobs [][]byte `protobuf:"bytes,3,rep,name=blobs,proto3" json:"blobs,omitempty" ssz-max:"4096" ssz-size:"?,131072"`
}
func (x *BlobsBundleV2) Reset() {
*x = BlobsBundleV2{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[16]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BlobsBundleV2) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BlobsBundleV2) ProtoMessage() {}
func (x *BlobsBundleV2) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[16]
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 BlobsBundleV2.ProtoReflect.Descriptor instead.
func (*BlobsBundleV2) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{16}
}
func (x *BlobsBundleV2) GetKzgCommitments() [][]byte {
if x != nil {
return x.KzgCommitments
}
return nil
}
func (x *BlobsBundleV2) GetProofs() [][]byte {
if x != nil {
return x.Proofs
}
return nil
}
func (x *BlobsBundleV2) GetBlobs() [][]byte {
if x != nil {
return x.Blobs
}
return nil
}
type Blob struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1716,7 +1779,7 @@ type Blob struct {
func (x *Blob) Reset() {
*x = Blob{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[16]
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[17]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1729,7 +1792,7 @@ func (x *Blob) String() string {
func (*Blob) ProtoMessage() {}
func (x *Blob) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[16]
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[17]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1742,7 +1805,7 @@ func (x *Blob) ProtoReflect() protoreflect.Message {
// Deprecated: Use Blob.ProtoReflect.Descriptor instead.
func (*Blob) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{16}
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{17}
}
func (x *Blob) GetData() []byte {
@@ -1764,7 +1827,7 @@ type BlobAndProof struct {
func (x *BlobAndProof) Reset() {
*x = BlobAndProof{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[17]
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[18]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1777,7 +1840,7 @@ func (x *BlobAndProof) String() string {
func (*BlobAndProof) ProtoMessage() {}
func (x *BlobAndProof) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[17]
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[18]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1790,7 +1853,7 @@ func (x *BlobAndProof) ProtoReflect() protoreflect.Message {
// Deprecated: Use BlobAndProof.ProtoReflect.Descriptor instead.
func (*BlobAndProof) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{17}
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{18}
}
func (x *BlobAndProof) GetBlob() []byte {
@@ -1807,6 +1870,61 @@ func (x *BlobAndProof) GetKzgProof() []byte {
return nil
}
type BlobAndProofV2 struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Blob []byte `protobuf:"bytes,1,opt,name=blob,proto3" json:"blob,omitempty" ssz-size:"131072"`
KzgProofs [][]byte `protobuf:"bytes,2,rep,name=kzg_proofs,json=kzgProofs,proto3" json:"kzg_proofs,omitempty" ssz-max:"524288" ssz-size:"48"`
}
func (x *BlobAndProofV2) Reset() {
*x = BlobAndProofV2{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[19]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BlobAndProofV2) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BlobAndProofV2) ProtoMessage() {}
func (x *BlobAndProofV2) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[19]
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 BlobAndProofV2.ProtoReflect.Descriptor instead.
func (*BlobAndProofV2) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{19}
}
func (x *BlobAndProofV2) GetBlob() []byte {
if x != nil {
return x.Blob
}
return nil
}
func (x *BlobAndProofV2) GetKzgProofs() [][]byte {
if x != nil {
return x.KzgProofs
}
return nil
}
var File_proto_engine_v1_execution_engine_proto protoreflect.FileDescriptor
var file_proto_engine_v1_execution_engine_proto_rawDesc = []byte{
@@ -2191,24 +2309,41 @@ var file_proto_engine_v1_execution_engine_proto_rawDesc = []byte{
0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x2a, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73,
0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x14, 0x8a, 0xb5, 0x18, 0x08, 0x3f, 0x2c, 0x31, 0x33,
0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x05, 0x62, 0x6c,
0x6f, 0x62, 0x73, 0x22, 0x26, 0x0a, 0x04, 0x42, 0x6c, 0x6f, 0x62, 0x12, 0x1e, 0x0a, 0x04, 0x64,
0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0x8a, 0xb5, 0x18, 0x06, 0x31,
0x33, 0x31, 0x30, 0x37, 0x32, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x53, 0x0a, 0x0c, 0x42,
0x6c, 0x6f, 0x62, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x1e, 0x0a, 0x04, 0x62,
0x6c, 0x6f, 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0x8a, 0xb5, 0x18, 0x06, 0x31,
0x33, 0x31, 0x30, 0x37, 0x32, 0x52, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x12, 0x23, 0x0a, 0x09, 0x6b,
0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x08, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66,
0x42, 0x95, 0x01, 0x0a, 0x16, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x14, 0x45, 0x78, 0x65,
0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x50, 0x72, 0x6f, 0x74,
0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x6e, 0x67, 0x69,
0x6e, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x76, 0x31, 0xaa, 0x02,
0x12, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65,
0x2e, 0x56, 0x31, 0xca, 0x02, 0x12, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45,
0x6e, 0x67, 0x69, 0x6e, 0x65, 0x5c, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x6f, 0x62, 0x73, 0x22, 0xa2, 0x01, 0x0a, 0x0d, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e,
0x64, 0x6c, 0x65, 0x56, 0x32, 0x12, 0x39, 0x0a, 0x0f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10,
0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36,
0x52, 0x0e, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73,
0x12, 0x2a, 0x0a, 0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c,
0x42, 0x12, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x06, 0x35, 0x32,
0x34, 0x32, 0x38, 0x38, 0x52, 0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x2a, 0x0a, 0x05,
0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x14, 0x8a, 0xb5, 0x18,
0x08, 0x3f, 0x2c, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39,
0x36, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x26, 0x0a, 0x04, 0x42, 0x6c, 0x6f, 0x62,
0x12, 0x1e, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a,
0x8a, 0xb5, 0x18, 0x06, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61,
0x22, 0x53, 0x0a, 0x0c, 0x42, 0x6c, 0x6f, 0x62, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66,
0x12, 0x1e, 0x0a, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a,
0x8a, 0xb5, 0x18, 0x06, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x52, 0x04, 0x62, 0x6c, 0x6f, 0x62,
0x12, 0x23, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x08, 0x6b, 0x7a, 0x67,
0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, 0x61, 0x0a, 0x0e, 0x42, 0x6c, 0x6f, 0x62, 0x41, 0x6e, 0x64,
0x50, 0x72, 0x6f, 0x6f, 0x66, 0x56, 0x32, 0x12, 0x1e, 0x0a, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0x8a, 0xb5, 0x18, 0x06, 0x31, 0x33, 0x31, 0x30, 0x37,
0x32, 0x52, 0x04, 0x62, 0x6c, 0x6f, 0x62, 0x12, 0x2f, 0x0a, 0x0a, 0x6b, 0x7a, 0x67, 0x5f, 0x70,
0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18,
0x02, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x06, 0x35, 0x32, 0x34, 0x32, 0x38, 0x38, 0x52, 0x09, 0x6b,
0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x42, 0x95, 0x01, 0x0a, 0x16, 0x6f, 0x72, 0x67,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65,
0x2e, 0x76, 0x31, 0x42, 0x14, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e,
0x67, 0x69, 0x6e, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e,
0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e,
0x67, 0x69, 0x6e, 0x65, 0x76, 0x31, 0xaa, 0x02, 0x12, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x12, 0x45, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x5c, 0x76, 0x31,
0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -2224,7 +2359,7 @@ func file_proto_engine_v1_execution_engine_proto_rawDescGZIP() []byte {
}
var file_proto_engine_v1_execution_engine_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
var file_proto_engine_v1_execution_engine_proto_msgTypes = make([]protoimpl.MessageInfo, 18)
var file_proto_engine_v1_execution_engine_proto_msgTypes = make([]protoimpl.MessageInfo, 20)
var file_proto_engine_v1_execution_engine_proto_goTypes = []interface{}{
(PayloadStatus_Status)(0), // 0: ethereum.engine.v1.PayloadStatus.Status
(*ExecutionPayload)(nil), // 1: ethereum.engine.v1.ExecutionPayload
@@ -2243,8 +2378,10 @@ var file_proto_engine_v1_execution_engine_proto_goTypes = []interface{}{
(*ForkchoiceState)(nil), // 14: ethereum.engine.v1.ForkchoiceState
(*Withdrawal)(nil), // 15: ethereum.engine.v1.Withdrawal
(*BlobsBundle)(nil), // 16: ethereum.engine.v1.BlobsBundle
(*Blob)(nil), // 17: ethereum.engine.v1.Blob
(*BlobAndProof)(nil), // 18: ethereum.engine.v1.BlobAndProof
(*BlobsBundleV2)(nil), // 17: ethereum.engine.v1.BlobsBundleV2
(*Blob)(nil), // 18: ethereum.engine.v1.Blob
(*BlobAndProof)(nil), // 19: ethereum.engine.v1.BlobAndProof
(*BlobAndProofV2)(nil), // 20: ethereum.engine.v1.BlobAndProofV2
}
var file_proto_engine_v1_execution_engine_proto_depIdxs = []int32{
15, // 0: ethereum.engine.v1.ExecutionPayloadCapella.withdrawals:type_name -> ethereum.engine.v1.Withdrawal
@@ -2463,7 +2600,7 @@ func file_proto_engine_v1_execution_engine_proto_init() {
}
}
file_proto_engine_v1_execution_engine_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*Blob); i {
switch v := v.(*BlobsBundleV2); i {
case 0:
return &v.state
case 1:
@@ -2475,6 +2612,18 @@ func file_proto_engine_v1_execution_engine_proto_init() {
}
}
file_proto_engine_v1_execution_engine_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*Blob); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_proto_engine_v1_execution_engine_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlobAndProof); i {
case 0:
return &v.state
@@ -2486,6 +2635,18 @@ func file_proto_engine_v1_execution_engine_proto_init() {
return nil
}
}
file_proto_engine_v1_execution_engine_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BlobAndProofV2); 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{
@@ -2493,7 +2654,7 @@ func file_proto_engine_v1_execution_engine_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_engine_v1_execution_engine_proto_rawDesc,
NumEnums: 1,
NumMessages: 18,
NumMessages: 20,
NumExtensions: 0,
NumServices: 0,
},

View File

@@ -227,8 +227,8 @@ message Withdrawal {
}
// BlobsBundle is retrieved through engine-api from the execution layer client.
// It consists of the necessary components for constructing a blobs sidecar
// object to gossip through p2p.
// It consists of the necessary components for constructing blob sidecars
// objects to gossip through p2p.
message BlobsBundle {
// The KZG commitments of the blobs.
repeated bytes kzg_commitments = 1 [
@@ -249,13 +249,45 @@ message BlobsBundle {
];
}
// BlobsBundleV2 is retrieved through engine-api from the execution layer client.
// It consists of the necessary components for constructing data column sidecars
// objects to gossip through p2p.
// It is introduced in Fulu network upgrade.
message BlobsBundleV2 {
repeated bytes kzg_commitments = 1 [
(ethereum.eth.ext.ssz_size) = "?,48",
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
repeated bytes proofs = 2 [
(ethereum.eth.ext.ssz_size) = "?,48",
(ethereum.eth.ext.ssz_max) = "max_cell_proofs_length.size" // Changed in EIP-7594
];
repeated bytes blobs = 3 [
(ethereum.eth.ext.ssz_size) = "?,blob.size",
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
}
// Blob contains the data that is to be committed on chain.
message Blob {
// The blob bytes.
bytes data = 1 [ (ethereum.eth.ext.ssz_size) = "blob.size" ];
}
// BlobAndProofV2 consists of the blob and the cell proofs for each cell in the blob.
message BlobAndProof {
bytes blob = 1 [ (ethereum.eth.ext.ssz_size) = "blob.size" ];
bytes kzg_proof = 2 [ (ethereum.eth.ext.ssz_size) = "48" ];
}
// BlobAndProofV2 consists of the blob and the cell proofs for each cell in the blob.
// It is introduced in Fulu network upgrade.
message BlobAndProofV2 {
bytes blob = 1 [ (ethereum.eth.ext.ssz_size) = "blob.size" ];
repeated bytes kzg_proofs = 2 [
(ethereum.eth.ext.ssz_size) = "48",
(ethereum.eth.ext.ssz_max) = "max_cell_proofs_length.size"
];
}

43
proto/engine/v1/fulu.go Normal file
View File

@@ -0,0 +1,43 @@
package enginev1
import (
"github.com/pkg/errors"
)
func (ebe *ExecutionBundleFulu) GetDecodedExecutionRequests() (*ExecutionRequests, error) {
requests := &ExecutionRequests{}
var prevTypeNum *uint8
for i := range ebe.ExecutionRequests {
requestType, requestListInSSZBytes, err := decodeExecutionRequest(ebe.ExecutionRequests[i])
if err != nil {
return nil, err
}
if prevTypeNum != nil && *prevTypeNum >= requestType {
return nil, errors.New("invalid execution request type order or duplicate requests, requests should be in sorted order and unique")
}
prevTypeNum = &requestType
switch requestType {
case DepositRequestType:
drs, err := unmarshalDeposits(requestListInSSZBytes)
if err != nil {
return nil, err
}
requests.Deposits = drs
case WithdrawalRequestType:
wrs, err := unmarshalWithdrawals(requestListInSSZBytes)
if err != nil {
return nil, err
}
requests.Withdrawals = wrs
case ConsolidationRequestType:
crs, err := unmarshalConsolidations(requestListInSSZBytes)
if err != nil {
return nil, err
}
requests.Consolidations = crs
default:
return nil, errors.Errorf("unsupported request type %d", requestType)
}
}
return requests, nil
}

210
proto/engine/v1/fulu.pb.go generated Executable file
View File

@@ -0,0 +1,210 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.33.0
// protoc v3.21.7
// source: proto/engine/v1/fulu.proto
package enginev1
import (
reflect "reflect"
sync "sync"
_ "github.com/OffchainLabs/prysm/v6/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 ExecutionBundleFulu struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Payload *ExecutionPayloadDeneb `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"`
Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
BlobsBundle *BlobsBundleV2 `protobuf:"bytes,3,opt,name=blobs_bundle,json=blobsBundle,proto3" json:"blobs_bundle,omitempty"`
ShouldOverrideBuilder bool `protobuf:"varint,4,opt,name=should_override_builder,json=shouldOverrideBuilder,proto3" json:"should_override_builder,omitempty"`
ExecutionRequests [][]byte `protobuf:"bytes,5,rep,name=execution_requests,json=executionRequests,proto3" json:"execution_requests,omitempty"`
}
func (x *ExecutionBundleFulu) Reset() {
*x = ExecutionBundleFulu{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_fulu_proto_msgTypes[0]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ExecutionBundleFulu) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ExecutionBundleFulu) ProtoMessage() {}
func (x *ExecutionBundleFulu) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_fulu_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 ExecutionBundleFulu.ProtoReflect.Descriptor instead.
func (*ExecutionBundleFulu) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_fulu_proto_rawDescGZIP(), []int{0}
}
func (x *ExecutionBundleFulu) GetPayload() *ExecutionPayloadDeneb {
if x != nil {
return x.Payload
}
return nil
}
func (x *ExecutionBundleFulu) GetValue() []byte {
if x != nil {
return x.Value
}
return nil
}
func (x *ExecutionBundleFulu) GetBlobsBundle() *BlobsBundleV2 {
if x != nil {
return x.BlobsBundle
}
return nil
}
func (x *ExecutionBundleFulu) GetShouldOverrideBuilder() bool {
if x != nil {
return x.ShouldOverrideBuilder
}
return false
}
func (x *ExecutionBundleFulu) GetExecutionRequests() [][]byte {
if x != nil {
return x.ExecutionRequests
}
return nil
}
var File_proto_engine_v1_fulu_proto protoreflect.FileDescriptor
var file_proto_engine_v1_fulu_proto_rawDesc = []byte{
0x0a, 0x1a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2f, 0x76,
0x31, 0x2f, 0x66, 0x75, 0x6c, 0x75, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 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, 0x26, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x65,
0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x9d, 0x02, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74,
0x69, 0x6f, 0x6e, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x43, 0x0a,
0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65,
0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79,
0x6c, 0x6f, 0x61, 0x64, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x62,
0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65,
0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x56,
0x32, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x36,
0x0a, 0x17, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64,
0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52,
0x15, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x42,
0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x12, 0x2d, 0x0a, 0x12, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03,
0x28, 0x0c, 0x52, 0x11, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x73, 0x42, 0x8e, 0x01, 0x0a, 0x16, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
0x42, 0x0c, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 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, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65,
0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x76, 0x31, 0xaa, 0x02, 0x12, 0x45,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x56,
0x31, 0xca, 0x02, 0x12, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x6e, 0x67,
0x69, 0x6e, 0x65, 0x5c, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
file_proto_engine_v1_fulu_proto_rawDescOnce sync.Once
file_proto_engine_v1_fulu_proto_rawDescData = file_proto_engine_v1_fulu_proto_rawDesc
)
func file_proto_engine_v1_fulu_proto_rawDescGZIP() []byte {
file_proto_engine_v1_fulu_proto_rawDescOnce.Do(func() {
file_proto_engine_v1_fulu_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_engine_v1_fulu_proto_rawDescData)
})
return file_proto_engine_v1_fulu_proto_rawDescData
}
var file_proto_engine_v1_fulu_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
var file_proto_engine_v1_fulu_proto_goTypes = []interface{}{
(*ExecutionBundleFulu)(nil), // 0: ethereum.engine.v1.ExecutionBundleFulu
(*ExecutionPayloadDeneb)(nil), // 1: ethereum.engine.v1.ExecutionPayloadDeneb
(*BlobsBundleV2)(nil), // 2: ethereum.engine.v1.BlobsBundleV2
}
var file_proto_engine_v1_fulu_proto_depIdxs = []int32{
1, // 0: ethereum.engine.v1.ExecutionBundleFulu.payload:type_name -> ethereum.engine.v1.ExecutionPayloadDeneb
2, // 1: ethereum.engine.v1.ExecutionBundleFulu.blobs_bundle:type_name -> ethereum.engine.v1.BlobsBundleV2
2, // [2:2] is the sub-list for method output_type
2, // [2:2] is the sub-list for method input_type
2, // [2:2] is the sub-list for extension type_name
2, // [2:2] is the sub-list for extension extendee
0, // [0:2] is the sub-list for field type_name
}
func init() { file_proto_engine_v1_fulu_proto_init() }
func file_proto_engine_v1_fulu_proto_init() {
if File_proto_engine_v1_fulu_proto != nil {
return
}
file_proto_engine_v1_execution_engine_proto_init()
if !protoimpl.UnsafeEnabled {
file_proto_engine_v1_fulu_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ExecutionBundleFulu); 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_engine_v1_fulu_proto_rawDesc,
NumEnums: 0,
NumMessages: 1,
NumExtensions: 0,
NumServices: 0,
},
GoTypes: file_proto_engine_v1_fulu_proto_goTypes,
DependencyIndexes: file_proto_engine_v1_fulu_proto_depIdxs,
MessageInfos: file_proto_engine_v1_fulu_proto_msgTypes,
}.Build()
File_proto_engine_v1_fulu_proto = out.File
file_proto_engine_v1_fulu_proto_rawDesc = nil
file_proto_engine_v1_fulu_proto_goTypes = nil
file_proto_engine_v1_fulu_proto_depIdxs = nil
}

View File

@@ -0,0 +1,21 @@
syntax = "proto3";
package ethereum.engine.v1;
import "proto/eth/ext/options.proto";
import "proto/engine/v1/execution_engine.proto";
option csharp_namespace = "Ethereum.Engine.V1";
option go_package = "github.com/prysmaticlabs/prysm/v5/proto/engine/v1;enginev1";
option java_multiple_files = true;
option java_outer_classname = "ElectraProto";
option java_package = "org.ethereum.engine.v1";
option php_namespace = "Ethereum\\Engine\\v1";
message ExecutionBundleFulu {
ExecutionPayloadDeneb payload = 1;
bytes value = 2;
BlobsBundleV2 blobs_bundle = 3;
bool should_override_builder = 4;
repeated bytes execution_requests = 5;
}

View File

@@ -305,6 +305,14 @@ type GetPayloadV4ResponseJson struct {
ExecutionRequests []hexutil.Bytes `json:"executionRequests"`
}
type GetPayloadV5ResponseJson struct {
ExecutionPayload *ExecutionPayloadDenebJSON `json:"executionPayload"`
BlockValue string `json:"blockValue"`
BlobsBundle *BlobBundleV2JSON `json:"blobsBundle"`
ShouldOverrideBuilder bool `json:"shouldOverrideBuilder"`
ExecutionRequests []hexutil.Bytes `json:"executionRequests"`
}
// ExecutionPayloadBody represents the engine API ExecutionPayloadV1 or ExecutionPayloadV2 type.
type ExecutionPayloadBody struct {
Transactions []hexutil.Bytes `json:"transactions"`
@@ -838,6 +846,20 @@ func (b BlobBundleJSON) ToProto() *BlobsBundle {
}
}
type BlobBundleV2JSON struct {
Commitments []hexutil.Bytes `json:"commitments"`
Proofs []hexutil.Bytes `json:"proofs"`
Blobs []hexutil.Bytes `json:"blobs"`
}
func (b BlobBundleV2JSON) ToProto() *BlobsBundleV2 {
return &BlobsBundleV2{
KzgCommitments: bytesutil.SafeCopy2dHexUtilBytes(b.Commitments),
Proofs: bytesutil.SafeCopy2dHexUtilBytes(b.Proofs),
Blobs: bytesutil.SafeCopy2dHexUtilBytes(b.Blobs),
}
}
type BlobAndProofJson struct {
Blob hexutil.Bytes `json:"blob"`
KzgProof hexutil.Bytes `json:"proof"`
@@ -1256,6 +1278,137 @@ func (e *ExecutionBundleElectra) UnmarshalJSON(enc []byte) error {
return nil
}
func (e *ExecutionBundleFulu) UnmarshalJSON(enc []byte) error {
dec := GetPayloadV5ResponseJson{}
if err := json.Unmarshal(enc, &dec); err != nil {
return err
}
if dec.ExecutionPayload.ParentHash == nil {
return errors.New("missing required field 'parentHash' for ExecutionPayload")
}
if dec.ExecutionPayload.FeeRecipient == nil {
return errors.New("missing required field 'feeRecipient' for ExecutionPayload")
}
if dec.ExecutionPayload.StateRoot == nil {
return errors.New("missing required field 'stateRoot' for ExecutionPayload")
}
if dec.ExecutionPayload.ReceiptsRoot == nil {
return errors.New("missing required field 'receiptsRoot' for ExecutableDataV1")
}
if dec.ExecutionPayload.LogsBloom == nil {
return errors.New("missing required field 'logsBloom' for ExecutionPayload")
}
if dec.ExecutionPayload.PrevRandao == nil {
return errors.New("missing required field 'prevRandao' for ExecutionPayload")
}
if dec.ExecutionPayload.ExtraData == nil {
return errors.New("missing required field 'extraData' for ExecutionPayload")
}
if dec.ExecutionPayload.BlockHash == nil {
return errors.New("missing required field 'blockHash' for ExecutionPayload")
}
if dec.ExecutionPayload.Transactions == nil {
return errors.New("missing required field 'transactions' for ExecutionPayload")
}
if dec.ExecutionPayload.BlockNumber == nil {
return errors.New("missing required field 'blockNumber' for ExecutionPayload")
}
if dec.ExecutionPayload.Timestamp == nil {
return errors.New("missing required field 'timestamp' for ExecutionPayload")
}
if dec.ExecutionPayload.GasUsed == nil {
return errors.New("missing required field 'gasUsed' for ExecutionPayload")
}
if dec.ExecutionPayload.GasLimit == nil {
return errors.New("missing required field 'gasLimit' for ExecutionPayload")
}
if dec.ExecutionPayload.BlobGasUsed == nil {
return errors.New("missing required field 'blobGasUsed' for ExecutionPayload")
}
if dec.ExecutionPayload.ExcessBlobGas == nil {
return errors.New("missing required field 'excessBlobGas' for ExecutionPayload")
}
*e = ExecutionBundleFulu{Payload: &ExecutionPayloadDeneb{}}
e.Payload.ParentHash = dec.ExecutionPayload.ParentHash.Bytes()
e.Payload.FeeRecipient = dec.ExecutionPayload.FeeRecipient.Bytes()
e.Payload.StateRoot = dec.ExecutionPayload.StateRoot.Bytes()
e.Payload.ReceiptsRoot = dec.ExecutionPayload.ReceiptsRoot.Bytes()
e.Payload.LogsBloom = *dec.ExecutionPayload.LogsBloom
e.Payload.PrevRandao = dec.ExecutionPayload.PrevRandao.Bytes()
e.Payload.BlockNumber = uint64(*dec.ExecutionPayload.BlockNumber)
e.Payload.GasLimit = uint64(*dec.ExecutionPayload.GasLimit)
e.Payload.GasUsed = uint64(*dec.ExecutionPayload.GasUsed)
e.Payload.Timestamp = uint64(*dec.ExecutionPayload.Timestamp)
e.Payload.ExtraData = dec.ExecutionPayload.ExtraData
baseFee, err := hexutil.DecodeBig(dec.ExecutionPayload.BaseFeePerGas)
if err != nil {
return err
}
e.Payload.BaseFeePerGas = bytesutil.PadTo(bytesutil.ReverseByteOrder(baseFee.Bytes()), fieldparams.RootLength)
e.Payload.ExcessBlobGas = uint64(*dec.ExecutionPayload.ExcessBlobGas)
e.Payload.BlobGasUsed = uint64(*dec.ExecutionPayload.BlobGasUsed)
e.Payload.BlockHash = dec.ExecutionPayload.BlockHash.Bytes()
transactions := make([][]byte, len(dec.ExecutionPayload.Transactions))
for i, tx := range dec.ExecutionPayload.Transactions {
transactions[i] = tx
}
e.Payload.Transactions = transactions
if dec.ExecutionPayload.Withdrawals == nil {
dec.ExecutionPayload.Withdrawals = make([]*Withdrawal, 0)
}
e.Payload.Withdrawals = dec.ExecutionPayload.Withdrawals
v, err := hexutil.DecodeBig(dec.BlockValue)
if err != nil {
return err
}
e.Value = bytesutil.PadTo(bytesutil.ReverseByteOrder(v.Bytes()), fieldparams.RootLength)
if dec.BlobsBundle == nil {
return nil
}
e.BlobsBundle = &BlobsBundleV2{}
commitments := make([][]byte, len(dec.BlobsBundle.Commitments))
for i, kzg := range dec.BlobsBundle.Commitments {
k := kzg
commitments[i] = bytesutil.PadTo(k[:], fieldparams.BLSPubkeyLength)
}
e.BlobsBundle.KzgCommitments = commitments
proofs := make([][]byte, len(dec.BlobsBundle.Proofs))
for i, proof := range dec.BlobsBundle.Proofs {
p := proof
proofs[i] = bytesutil.PadTo(p[:], fieldparams.BLSPubkeyLength)
}
e.BlobsBundle.Proofs = proofs
blobs := make([][]byte, len(dec.BlobsBundle.Blobs))
for i, blob := range dec.BlobsBundle.Blobs {
b := make([]byte, fieldparams.BlobLength)
copy(b, blob)
blobs[i] = b
}
e.BlobsBundle.Blobs = blobs
e.ShouldOverrideBuilder = dec.ShouldOverrideBuilder
requests := make([][]byte, len(dec.ExecutionRequests))
for i, request := range dec.ExecutionRequests {
r := make([]byte, len(request))
copy(r, request)
requests[i] = r
}
e.ExecutionRequests = requests
return nil
}
// RecastHexutilByteSlice converts a []hexutil.Bytes to a [][]byte
func RecastHexutilByteSlice(h []hexutil.Bytes) [][]byte {
r := make([][]byte, len(h))
@@ -1282,3 +1435,28 @@ func (b *BlobAndProof) UnmarshalJSON(enc []byte) error {
return nil
}
type BlobAndProofV2Json struct {
Blob hexutil.Bytes `json:"blob"`
KzgProofs []hexutil.Bytes `json:"proofs"`
}
func (b *BlobAndProofV2) UnmarshalJSON(enc []byte) error {
var dec *BlobAndProofV2Json
if err := json.Unmarshal(enc, &dec); err != nil {
return err
}
blob := make([]byte, fieldparams.BlobLength)
copy(blob, dec.Blob)
b.Blob = blob
proofs := make([][]byte, len(dec.KzgProofs))
for i, proof := range dec.KzgProofs {
p := proof
proofs[i] = bytesutil.PadTo(p[:], fieldparams.BLSPubkeyLength)
}
b.KzgProofs = proofs
return nil
}

View File

@@ -0,0 +1,36 @@
// Copyright 2024 Offchain Labs.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
syntax = "proto3";
package ethereum.eth.v1alpha1;
import "proto/eth/ext/options.proto";
import "proto/prysm/v1alpha1/beacon_block.proto";
option csharp_namespace = "Ethereum.Eth.v1alpha1";
option go_package = "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1;eth";
option java_multiple_files = true;
option java_outer_classname = "DataColumnsProto";
option java_package = "org.ethereum.eth.v1alpha1";
option php_namespace = "Ethereum\\Eth\\v1alpha1";
message DataColumnSidecar {
uint64 index = 1;
repeated bytes column = 2 [(ethereum.eth.ext.ssz_size) = "?,bytes_per_cell.size", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"];
repeated bytes kzg_commitments = 3 [(ethereum.eth.ext.ssz_size) = "?,48", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"];
repeated bytes kzg_proofs = 4 [(ethereum.eth.ext.ssz_size) = "?,48", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"];
SignedBeaconBlockHeader signed_block_header = 5;
repeated bytes kzg_commitments_inclusion_proof = 6 [(ethereum.eth.ext.ssz_size) = "kzg_commitments_inclusion_proof_depth.size,32"];
}

View File

@@ -4865,7 +4865,7 @@ type SignedBeaconBlockContentsFulu struct {
unknownFields protoimpl.UnknownFields
Block *SignedBeaconBlockFulu `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"`
KzgProofs [][]byte `protobuf:"bytes,2,rep,name=kzg_proofs,json=kzgProofs,proto3" json:"kzg_proofs,omitempty" ssz-max:"4096" ssz-size:"?,48"`
KzgProofs [][]byte `protobuf:"bytes,2,rep,name=kzg_proofs,json=kzgProofs,proto3" json:"kzg_proofs,omitempty" ssz-max:"524288" ssz-size:"?,48"`
Blobs [][]byte `protobuf:"bytes,3,rep,name=blobs,proto3" json:"blobs,omitempty" ssz-max:"4096" ssz-size:"?,131072"`
}
@@ -4983,7 +4983,7 @@ type BeaconBlockContentsFulu struct {
unknownFields protoimpl.UnknownFields
Block *BeaconBlockElectra `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"`
KzgProofs [][]byte `protobuf:"bytes,2,rep,name=kzg_proofs,json=kzgProofs,proto3" json:"kzg_proofs,omitempty" ssz-max:"4096" ssz-size:"?,48"`
KzgProofs [][]byte `protobuf:"bytes,2,rep,name=kzg_proofs,json=kzgProofs,proto3" json:"kzg_proofs,omitempty" ssz-max:"524288" ssz-size:"?,48"`
Blobs [][]byte `protobuf:"bytes,3,rep,name=blobs,proto3" json:"blobs,omitempty" ssz-max:"4096" ssz-size:"?,131072"`
}
@@ -6598,83 +6598,83 @@ var file_proto_prysm_v1alpha1_beacon_block_proto_rawDesc = []byte{
0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x52, 0x04, 0x64,
0x61, 0x74, 0x61, 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, 0x22, 0xc0, 0x01, 0x0a, 0x1d, 0x53, 0x69,
0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0xc2, 0x01, 0x0a, 0x1d, 0x53, 0x69,
0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43,
0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x42, 0x0a, 0x05, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x12,
0x2f, 0x0a, 0x0a, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20,
0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18,
0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x09, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x73,
0x12, 0x2a, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42,
0x14, 0x8a, 0xb5, 0x18, 0x08, 0x3f, 0x2c, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18,
0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x7e, 0x0a, 0x15,
0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x3f, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01,
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, 0x52,
0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74,
0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39,
0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0xb7, 0x01, 0x0a,
0x17, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6f, 0x6e, 0x74,
0x65, 0x6e, 0x74, 0x73, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x3f, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x18, 0x01, 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, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x2f, 0x0a, 0x0a, 0x6b, 0x7a, 0x67,
0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a,
0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52,
0x09, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x2a, 0x0a, 0x05, 0x62, 0x6c,
0x6f, 0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x14, 0x8a, 0xb5, 0x18, 0x08, 0x3f,
0x2c, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52,
0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x8d, 0x01, 0x0a, 0x1c, 0x53, 0x69, 0x67, 0x6e, 0x65,
0x64, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x47, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61,
0x67, 0x65, 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, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67,
0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x83, 0x03, 0x0a, 0x16, 0x42, 0x6c, 0x69, 0x6e, 0x64,
0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c,
0x75, 0x12, 0x58, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42,
0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d,
0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73,
0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x75, 0x0a, 0x0e, 0x70,
0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20,
0x01, 0x28, 0x04, 0x42, 0x4e, 0x82, 0xb5, 0x18, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x31, 0x0a, 0x0a, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20,
0x03, 0x28, 0x0c, 0x42, 0x12, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18,
0x06, 0x35, 0x32, 0x34, 0x32, 0x38, 0x38, 0x52, 0x09, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f,
0x66, 0x73, 0x12, 0x2a, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28,
0x0c, 0x42, 0x14, 0x8a, 0xb5, 0x18, 0x08, 0x3f, 0x2c, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92,
0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x7e,
0x0a, 0x15, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c,
0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x3f, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x18, 0x01, 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, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e,
0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0xb9,
0x01, 0x0a, 0x17, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6f,
0x6e, 0x74, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x3f, 0x0a, 0x05, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x18, 0x01, 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, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x31, 0x0a, 0x0a, 0x6b,
0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x42,
0x12, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x06, 0x35, 0x32, 0x34,
0x32, 0x38, 0x38, 0x52, 0x09, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x2a,
0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x14, 0x8a,
0xb5, 0x18, 0x08, 0x3f, 0x2c, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18, 0x04, 0x34,
0x30, 0x39, 0x36, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x8d, 0x01, 0x0a, 0x1c, 0x53,
0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x47, 0x0a, 0x07, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 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, 0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x46, 0x75, 0x6c, 0x75, 0x52, 0x07, 0x6d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52,
0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x83, 0x03, 0x0a, 0x16, 0x42,
0x6c, 0x69, 0x6e, 0x64, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x46, 0x75, 0x6c, 0x75, 0x12, 0x58, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20,
0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73,
0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e,
0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74,
0x69, 0x76, 0x65, 0x73, 0x2e, 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, 0x25, 0x0a, 0x0a, 0x73,
0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f,
0x6f, 0x74, 0x12, 0x48, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x34, 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, 0x6f, 0x64, 0x79, 0x45,
0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x42, 0x9a, 0x01, 0x0a,
0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, 0x42, 0x65, 0x61, 0x63,
0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39,
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68,
0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36,
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68,
0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x33,
0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12,
0x75, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65,
0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4e, 0x82, 0xb5, 0x18, 0x4a, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e,
0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f,
0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72,
0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 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,
0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61,
0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x48, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x05,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 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,
0x6f, 0x64, 0x79, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79,
0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10,
0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x74, 0x6f,
0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f,
0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15,
0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@@ -1188,11 +1188,11 @@ message SignedBeaconBlockContentsFulu {
SignedBeaconBlockFulu block = 1;
repeated bytes kzg_proofs = 2 [
(ethereum.eth.ext.ssz_size) = "?,48",
(ethereum.eth.ext.ssz_max) = "4096"
(ethereum.eth.ext.ssz_max) = "max_cell_proofs_length.size"
];
repeated bytes blobs = 3 [
(ethereum.eth.ext.ssz_size) = "?,blob.size",
(ethereum.eth.ext.ssz_max) = "4096"
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
}
@@ -1208,11 +1208,11 @@ message BeaconBlockContentsFulu {
BeaconBlockElectra block = 1;
repeated bytes kzg_proofs = 2 [
(ethereum.eth.ext.ssz_size) = "?,48",
(ethereum.eth.ext.ssz_max) = "4096"
(ethereum.eth.ext.ssz_max) = "max_cell_proofs_length.size"
];
repeated bytes blobs = 3 [
(ethereum.eth.ext.ssz_size) = "?,blob.size",
(ethereum.eth.ext.ssz_max) = "4096"
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
}
@@ -1245,4 +1245,3 @@ message BlindedBeaconBlockFulu {
// The blinded beacon block body.
BlindedBeaconBlockBodyElectra body = 5;
}

View File

@@ -27,10 +27,10 @@ type DataColumnSidecar struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ColumnIndex uint64 `protobuf:"varint,1,opt,name=column_index,json=columnIndex,proto3" json:"column_index,omitempty"`
DataColumn [][]byte `protobuf:"bytes,2,rep,name=data_column,json=dataColumn,proto3" json:"data_column,omitempty" ssz-max:"4096" ssz-size:"?,2048"`
Index uint64 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"`
Column [][]byte `protobuf:"bytes,2,rep,name=column,proto3" json:"column,omitempty" ssz-max:"4096" ssz-size:"?,2048"`
KzgCommitments [][]byte `protobuf:"bytes,3,rep,name=kzg_commitments,json=kzgCommitments,proto3" json:"kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"`
KzgProof [][]byte `protobuf:"bytes,4,rep,name=kzg_proof,json=kzgProof,proto3" json:"kzg_proof,omitempty" ssz-max:"4096" ssz-size:"?,48"`
KzgProofs [][]byte `protobuf:"bytes,4,rep,name=kzg_proofs,json=kzgProofs,proto3" json:"kzg_proofs,omitempty" ssz-max:"4096" ssz-size:"?,48"`
SignedBlockHeader *SignedBeaconBlockHeader `protobuf:"bytes,5,opt,name=signed_block_header,json=signedBlockHeader,proto3" json:"signed_block_header,omitempty"`
KzgCommitmentsInclusionProof [][]byte `protobuf:"bytes,6,rep,name=kzg_commitments_inclusion_proof,json=kzgCommitmentsInclusionProof,proto3" json:"kzg_commitments_inclusion_proof,omitempty" ssz-size:"4,32"`
}
@@ -67,16 +67,16 @@ func (*DataColumnSidecar) Descriptor() ([]byte, []int) {
return file_proto_prysm_v1alpha1_data_columns_proto_rawDescGZIP(), []int{0}
}
func (x *DataColumnSidecar) GetColumnIndex() uint64 {
func (x *DataColumnSidecar) GetIndex() uint64 {
if x != nil {
return x.ColumnIndex
return x.Index
}
return 0
}
func (x *DataColumnSidecar) GetDataColumn() [][]byte {
func (x *DataColumnSidecar) GetColumn() [][]byte {
if x != nil {
return x.DataColumn
return x.Column
}
return nil
}
@@ -88,9 +88,9 @@ func (x *DataColumnSidecar) GetKzgCommitments() [][]byte {
return nil
}
func (x *DataColumnSidecar) GetKzgProof() [][]byte {
func (x *DataColumnSidecar) GetKzgProofs() [][]byte {
if x != nil {
return x.KzgProof
return x.KzgProofs
}
return nil
}
@@ -114,8 +114,8 @@ type DataColumnIdentifier struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
BlockRoot []byte `protobuf:"bytes,1,opt,name=block_root,json=blockRoot,proto3" json:"block_root,omitempty" ssz-size:"32"`
ColumnIndex uint64 `protobuf:"varint,2,opt,name=column_index,json=columnIndex,proto3" json:"column_index,omitempty"`
BlockRoot []byte `protobuf:"bytes,1,opt,name=block_root,json=blockRoot,proto3" json:"block_root,omitempty" ssz-size:"32"`
Index uint64 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"`
}
func (x *DataColumnIdentifier) Reset() {
@@ -157,9 +157,9 @@ func (x *DataColumnIdentifier) GetBlockRoot() []byte {
return nil
}
func (x *DataColumnIdentifier) GetColumnIndex() uint64 {
func (x *DataColumnIdentifier) GetIndex() uint64 {
if x != nil {
return x.ColumnIndex
return x.Index
}
return 0
}
@@ -175,48 +175,46 @@ var file_proto_prysm_v1alpha1_data_columns_proto_rawDesc = []byte{
0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70,
0x68, 0x61, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x86, 0x03, 0x0a, 0x11, 0x44, 0x61, 0x74, 0x61, 0x43,
0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x21, 0x0a, 0x0c,
0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01,
0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12,
0x33, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x02,
0x20, 0x03, 0x28, 0x0c, 0x42, 0x12, 0x8a, 0xb5, 0x18, 0x06, 0x3f, 0x2c, 0x32, 0x30, 0x34, 0x38,
0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x6f,
0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x39, 0x0a, 0x0f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d,
0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xf2, 0x02, 0x0a, 0x11, 0x44, 0x61, 0x74, 0x61, 0x43,
0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x12, 0x14, 0x0a, 0x05,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64,
0x65, 0x78, 0x12, 0x2a, 0x0a, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x02, 0x20, 0x03,
0x28, 0x0c, 0x42, 0x12, 0x8a, 0xb5, 0x18, 0x06, 0x3f, 0x2c, 0x32, 0x30, 0x34, 0x38, 0x92, 0xb5,
0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x39,
0x0a, 0x0f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74,
0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34,
0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x0e, 0x6b, 0x7a, 0x67, 0x43, 0x6f,
0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x0a, 0x6b, 0x7a, 0x67,
0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a,
0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52,
0x0e, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12,
0x2d, 0x0a, 0x09, 0x6b, 0x7a, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x04, 0x20, 0x03,
0x28, 0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04,
0x34, 0x30, 0x39, 0x36, 0x52, 0x08, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x12, 0x5e,
0x0a, 0x13, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68,
0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70,
0x68, 0x61, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e,
0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x11, 0x73, 0x69, 0x67,
0x6e, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4f,
0x0a, 0x1f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74,
0x73, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x6f,
0x66, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x08, 0x8a, 0xb5, 0x18, 0x04, 0x34, 0x2c, 0x33,
0x32, 0x52, 0x1c, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74,
0x73, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22,
0x60, 0x0a, 0x14, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x64, 0x65,
0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x21,
0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x6e, 0x64, 0x65,
0x78, 0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42,
0x10, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x50, 0x72, 0x6f, 0x74,
0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02,
0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x09, 0x6b, 0x7a, 0x67, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x5e, 0x0a, 0x13, 0x73, 0x69,
0x67, 0x6e, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65,
0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e,
0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x11, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x1f, 0x6b, 0x7a,
0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x69, 0x6e,
0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x06, 0x20,
0x03, 0x28, 0x0c, 0x42, 0x08, 0x8a, 0xb5, 0x18, 0x04, 0x34, 0x2c, 0x33, 0x32, 0x52, 0x1c, 0x6b,
0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x49, 0x6e, 0x63,
0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x22, 0x53, 0x0a, 0x14, 0x44,
0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66,
0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f,
0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10,
0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f,
0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f,
0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15,
0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@@ -26,8 +26,8 @@ option java_package = "org.ethereum.eth.v1alpha1";
option php_namespace = "Ethereum\\Eth\\v1alpha1";
message DataColumnSidecar {
uint64 column_index = 1;
repeated bytes data_column = 2 [
uint64 index = 1;
repeated bytes column = 2 [
(ethereum.eth.ext.ssz_size) = "?,bytes_per_cell.size",
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
@@ -35,7 +35,7 @@ message DataColumnSidecar {
(ethereum.eth.ext.ssz_size) = "?,48",
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
repeated bytes kzg_proof = 4 [
repeated bytes kzg_proofs = 4 [
(ethereum.eth.ext.ssz_size) = "?,48",
(ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"
];
@@ -47,5 +47,5 @@ message DataColumnSidecar {
message DataColumnIdentifier {
bytes block_root = 1 [ (ethereum.eth.ext.ssz_size) = "32" ];
uint64 column_index = 2;
uint64 index = 2;
}

View File

@@ -744,6 +744,7 @@ type DebugPeerResponse_PeerInfo struct {
ProtocolVersion string `protobuf:"bytes,5,opt,name=protocol_version,json=protocolVersion,proto3" json:"protocol_version,omitempty"`
AgentVersion string `protobuf:"bytes,6,opt,name=agent_version,json=agentVersion,proto3" json:"agent_version,omitempty"`
PeerLatency uint64 `protobuf:"varint,7,opt,name=peer_latency,json=peerLatency,proto3" json:"peer_latency,omitempty"`
MetadataV2 *MetaDataV2 `protobuf:"bytes,8,opt,name=metadataV2,proto3" json:"metadataV2,omitempty"`
}
func (x *DebugPeerResponse_PeerInfo) Reset() {
@@ -827,6 +828,13 @@ func (x *DebugPeerResponse_PeerInfo) GetPeerLatency() uint64 {
return 0
}
func (x *DebugPeerResponse_PeerInfo) GetMetadataV2() *MetaDataV2 {
if x != nil {
return x.MetadataV2
}
return nil
}
var File_proto_prysm_v1alpha1_debug_proto protoreflect.FileDescriptor
var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{
@@ -890,8 +898,8 @@ var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{
0x01, 0x20, 0x03, 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, 0x44, 0x65,
0x62, 0x75, 0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52,
0x09, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x3a, 0x02, 0x18, 0x01, 0x22, 0xc3,
0x06, 0x0a, 0x11, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70,
0x09, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x3a, 0x02, 0x18, 0x01, 0x22, 0x86,
0x07, 0x0a, 0x11, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x13, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x69, 0x6e,
0x67, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,
0x09, 0x52, 0x12, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x69, 0x6e, 0x67, 0x41, 0x64, 0x64, 0x72,
@@ -922,7 +930,7 @@ var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{
0x6f, 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f,
0x52, 0x09, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0xc2, 0x02, 0x0a, 0x08,
0x52, 0x09, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x85, 0x03, 0x0a, 0x08,
0x50, 0x65, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x41, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61,
0x64, 0x61, 0x74, 0x61, 0x56, 0x30, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c,
@@ -943,100 +951,105 @@ var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{
0x0c, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a,
0x0c, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x6c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x07, 0x20,
0x01, 0x28, 0x04, 0x52, 0x0b, 0x70, 0x65, 0x65, 0x72, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79,
0x3a, 0x02, 0x18, 0x01, 0x22, 0xc9, 0x03, 0x0a, 0x09, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e,
0x66, 0x6f, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x76, 0x65, 0x72, 0x61, 0x6c, 0x6c, 0x5f, 0x73, 0x63,
0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0c, 0x6f, 0x76, 0x65, 0x72, 0x61,
0x6c, 0x6c, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x63, 0x65,
0x73, 0x73, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28,
0x04, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63,
0x6b, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, 0x76,
0x69, 0x64, 0x65, 0x72, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02,
0x52, 0x12, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x53,
0x63, 0x6f, 0x72, 0x65, 0x12, 0x54, 0x0a, 0x0c, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x73, 0x63,
0x6f, 0x72, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x65, 0x74, 0x68,
0x12, 0x41, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, 0x32, 0x18, 0x08,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4d, 0x65, 0x74,
0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x32, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
0x61, 0x56, 0x32, 0x3a, 0x02, 0x18, 0x01, 0x22, 0xc9, 0x03, 0x0a, 0x09, 0x53, 0x63, 0x6f, 0x72,
0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x76, 0x65, 0x72, 0x61, 0x6c, 0x6c,
0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0c, 0x6f, 0x76,
0x65, 0x72, 0x61, 0x6c, 0x6c, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x72,
0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18, 0x02,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70,
0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x03, 0x20,
0x01, 0x28, 0x02, 0x52, 0x12, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64,
0x65, 0x72, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x54, 0x0a, 0x0c, 0x74, 0x6f, 0x70, 0x69, 0x63,
0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e,
0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79,
0x52, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x12, 0x21, 0x0a,
0x0c, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x05, 0x20,
0x01, 0x28, 0x02, 0x52, 0x0b, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x63, 0x6f, 0x72, 0x65,
0x12, 0x2b, 0x0a, 0x11, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x75, 0x72, 0x5f, 0x70, 0x65,
0x6e, 0x61, 0x6c, 0x74, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, 0x10, 0x62, 0x65, 0x68,
0x61, 0x76, 0x69, 0x6f, 0x75, 0x72, 0x50, 0x65, 0x6e, 0x61, 0x6c, 0x74, 0x79, 0x12, 0x29, 0x0a,
0x10, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x72, 0x72, 0x6f,
0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0x69, 0x0a, 0x10, 0x54, 0x6f, 0x70, 0x69,
0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03,
0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3f,
0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 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, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65,
0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,
0x02, 0x38, 0x01, 0x22, 0xe6, 0x01, 0x0a, 0x12, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f,
0x72, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x20, 0x0a, 0x0c, 0x74, 0x69,
0x6d, 0x65, 0x5f, 0x69, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04,
0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x49, 0x6e, 0x4d, 0x65, 0x73, 0x68, 0x12, 0x38, 0x0a, 0x18,
0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65,
0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x16,
0x66, 0x69, 0x72, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69,
0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x36, 0x0a, 0x17, 0x6d, 0x65, 0x73, 0x68, 0x5f, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65,
0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x15, 0x6d, 0x65, 0x73, 0x68, 0x4d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x3c,
0x0a, 0x1a, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01,
0x28, 0x02, 0x52, 0x18, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61,
0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x32, 0x83, 0x05, 0x0a,
0x05, 0x44, 0x65, 0x62, 0x75, 0x67, 0x12, 0x85, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x42, 0x65,
0x61, 0x63, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 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, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x24, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b,
0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f,
0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x88, 0x02, 0x01, 0x12, 0x7f,
0x0a, 0x08, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x29, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2e, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x54, 0x6f, 0x70,
0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x74,
0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x67, 0x6f,
0x73, 0x73, 0x69, 0x70, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x02,
0x52, 0x0b, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x2b, 0x0a,
0x11, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x75, 0x72, 0x5f, 0x70, 0x65, 0x6e, 0x61, 0x6c,
0x74, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, 0x10, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69,
0x6f, 0x75, 0x72, 0x50, 0x65, 0x6e, 0x61, 0x6c, 0x74, 0x79, 0x12, 0x29, 0x0a, 0x10, 0x76, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x07,
0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x45, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0x69, 0x0a, 0x10, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63,
0x6f, 0x72, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3f, 0x0a, 0x05, 0x76,
0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 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, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x53, 0x6e, 0x61,
0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
0x22, 0xe6, 0x01, 0x0a, 0x12, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x53,
0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x20, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f,
0x69, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x74,
0x69, 0x6d, 0x65, 0x49, 0x6e, 0x4d, 0x65, 0x73, 0x68, 0x12, 0x38, 0x0a, 0x18, 0x66, 0x69, 0x72,
0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76,
0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x16, 0x66, 0x69, 0x72,
0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72,
0x69, 0x65, 0x73, 0x12, 0x36, 0x0a, 0x17, 0x6d, 0x65, 0x73, 0x68, 0x5f, 0x6d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x03,
0x20, 0x01, 0x28, 0x02, 0x52, 0x15, 0x6d, 0x65, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x1a, 0x69,
0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64,
0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x02, 0x52,
0x18, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44,
0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x32, 0x83, 0x05, 0x0a, 0x05, 0x44, 0x65,
0x62, 0x75, 0x67, 0x12, 0x85, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x42, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 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, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x24, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f,
0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62,
0x75, 0x67, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x88, 0x02, 0x01, 0x12, 0x7f, 0x0a, 0x08, 0x47,
0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 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, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x79, 0x52, 0x6f,
0x6f, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x24, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19,
0x61, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42,
0x79, 0x52, 0x6f, 0x6f, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53,
0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x24, 0x82, 0xd3, 0xe4, 0x93, 0x02,
0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x88, 0x02, 0x01, 0x12,
0x7d, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x4c, 0x65, 0x76,
0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x67, 0x69,
0x6e, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16,
0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x26, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1d, 0x22, 0x1b,
0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65,
0x62, 0x75, 0x67, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x88, 0x02, 0x01, 0x12, 0x7d, 0x0a, 0x0f,
0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12,
0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x4c,
0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f,
0x62, 0x75, 0x67, 0x2f, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x88, 0x02, 0x01, 0x12, 0x74,
0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f,
0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d,
0x70, 0x74, 0x79, 0x22, 0x26, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1d, 0x22, 0x1b, 0x2f, 0x65, 0x74,
0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67,
0x2f, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x88, 0x02, 0x01, 0x12, 0x74, 0x0a, 0x09, 0x4c,
0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79,
0x1a, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65,
0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x22, 0x24, 0x82, 0xd3, 0xe4,
0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x88, 0x02,
0x01, 0x12, 0x7c, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x22, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c,
0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65,
0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x23, 0x82, 0xd3, 0xe4, 0x93,
0x02, 0x1a, 0x12, 0x18, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x88, 0x02, 0x01, 0x42,
0x94, 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, 0x0a, 0x44,
0x65, 0x62, 0x75, 0x67, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e,
0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02,
0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x70, 0x74, 0x79, 0x1a, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75,
0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x22, 0x24,
0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72,
0x73, 0x88, 0x02, 0x01, 0x12, 0x7c, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12,
0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76,
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65,
0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75,
0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x23, 0x82,
0xd3, 0xe4, 0x93, 0x02, 0x1a, 0x12, 0x18, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c,
0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x88,
0x02, 0x01, 0x42, 0x94, 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, 0x0a, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39,
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68,
0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36,
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68,
0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x33,
}
var (
@@ -1072,8 +1085,9 @@ var file_proto_prysm_v1alpha1_debug_proto_goTypes = []interface{}{
(*Status)(nil), // 15: ethereum.eth.v1alpha1.Status
(*MetaDataV0)(nil), // 16: ethereum.eth.v1alpha1.MetaDataV0
(*MetaDataV1)(nil), // 17: ethereum.eth.v1alpha1.MetaDataV1
(*emptypb.Empty)(nil), // 18: google.protobuf.Empty
(*PeerRequest)(nil), // 19: ethereum.eth.v1alpha1.PeerRequest
(*MetaDataV2)(nil), // 18: ethereum.eth.v1alpha1.MetaDataV2
(*emptypb.Empty)(nil), // 19: google.protobuf.Empty
(*PeerRequest)(nil), // 20: ethereum.eth.v1alpha1.PeerRequest
}
var file_proto_prysm_v1alpha1_debug_proto_depIdxs = []int32{
0, // 0: ethereum.eth.v1alpha1.LoggingLevelRequest.level:type_name -> ethereum.eth.v1alpha1.LoggingLevelRequest.Level
@@ -1086,22 +1100,23 @@ var file_proto_prysm_v1alpha1_debug_proto_depIdxs = []int32{
12, // 7: ethereum.eth.v1alpha1.ScoreInfo.topic_scores:type_name -> ethereum.eth.v1alpha1.ScoreInfo.TopicScoresEntry
16, // 8: ethereum.eth.v1alpha1.DebugPeerResponse.PeerInfo.metadataV0:type_name -> ethereum.eth.v1alpha1.MetaDataV0
17, // 9: ethereum.eth.v1alpha1.DebugPeerResponse.PeerInfo.metadataV1:type_name -> ethereum.eth.v1alpha1.MetaDataV1
10, // 10: ethereum.eth.v1alpha1.ScoreInfo.TopicScoresEntry.value:type_name -> ethereum.eth.v1alpha1.TopicScoreSnapshot
3, // 11: ethereum.eth.v1alpha1.Debug.GetBeaconState:input_type -> ethereum.eth.v1alpha1.BeaconStateRequest
4, // 12: ethereum.eth.v1alpha1.Debug.GetBlock:input_type -> ethereum.eth.v1alpha1.BlockRequestByRoot
6, // 13: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:input_type -> ethereum.eth.v1alpha1.LoggingLevelRequest
18, // 14: ethereum.eth.v1alpha1.Debug.ListPeers:input_type -> google.protobuf.Empty
19, // 15: ethereum.eth.v1alpha1.Debug.GetPeer:input_type -> ethereum.eth.v1alpha1.PeerRequest
5, // 16: ethereum.eth.v1alpha1.Debug.GetBeaconState:output_type -> ethereum.eth.v1alpha1.SSZResponse
5, // 17: ethereum.eth.v1alpha1.Debug.GetBlock:output_type -> ethereum.eth.v1alpha1.SSZResponse
18, // 18: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:output_type -> google.protobuf.Empty
7, // 19: ethereum.eth.v1alpha1.Debug.ListPeers:output_type -> ethereum.eth.v1alpha1.DebugPeerResponses
8, // 20: ethereum.eth.v1alpha1.Debug.GetPeer:output_type -> ethereum.eth.v1alpha1.DebugPeerResponse
16, // [16:21] is the sub-list for method output_type
11, // [11:16] is the sub-list for method input_type
11, // [11:11] is the sub-list for extension type_name
11, // [11:11] is the sub-list for extension extendee
0, // [0:11] is the sub-list for field type_name
18, // 10: ethereum.eth.v1alpha1.DebugPeerResponse.PeerInfo.metadataV2:type_name -> ethereum.eth.v1alpha1.MetaDataV2
10, // 11: ethereum.eth.v1alpha1.ScoreInfo.TopicScoresEntry.value:type_name -> ethereum.eth.v1alpha1.TopicScoreSnapshot
3, // 12: ethereum.eth.v1alpha1.Debug.GetBeaconState:input_type -> ethereum.eth.v1alpha1.BeaconStateRequest
4, // 13: ethereum.eth.v1alpha1.Debug.GetBlock:input_type -> ethereum.eth.v1alpha1.BlockRequestByRoot
6, // 14: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:input_type -> ethereum.eth.v1alpha1.LoggingLevelRequest
19, // 15: ethereum.eth.v1alpha1.Debug.ListPeers:input_type -> google.protobuf.Empty
20, // 16: ethereum.eth.v1alpha1.Debug.GetPeer:input_type -> ethereum.eth.v1alpha1.PeerRequest
5, // 17: ethereum.eth.v1alpha1.Debug.GetBeaconState:output_type -> ethereum.eth.v1alpha1.SSZResponse
5, // 18: ethereum.eth.v1alpha1.Debug.GetBlock:output_type -> ethereum.eth.v1alpha1.SSZResponse
19, // 19: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:output_type -> google.protobuf.Empty
7, // 20: ethereum.eth.v1alpha1.Debug.ListPeers:output_type -> ethereum.eth.v1alpha1.DebugPeerResponses
8, // 21: ethereum.eth.v1alpha1.Debug.GetPeer:output_type -> ethereum.eth.v1alpha1.DebugPeerResponse
17, // [17:22] is the sub-list for method output_type
12, // [12:17] is the sub-list for method input_type
12, // [12:12] is the sub-list for extension type_name
12, // [12:12] is the sub-list for extension extendee
0, // [0:12] is the sub-list for field type_name
}
func init() { file_proto_prysm_v1alpha1_debug_proto_init() }

View File

@@ -139,6 +139,8 @@ message DebugPeerResponse {
string agent_version = 6;
// Latency of responses from peer(in ms).
uint64 peer_latency = 7;
// Metadata of the peer.
MetaDataV2 metadataV2 = 8;
}
// Listening addresses know of the peer.
repeated string listening_addresses = 1;

View File

@@ -37,8 +37,8 @@ func (s *SignedBeaconBlockContentsFulu) MarshalSSZTo(buf []byte) (dst []byte, er
}
// Field (1) 'KzgProofs'
if size := len(s.KzgProofs); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 4096)
if size := len(s.KzgProofs); size > 524288 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 524288)
return
}
for ii := 0; ii < len(s.KzgProofs); ii++ {
@@ -109,7 +109,7 @@ func (s *SignedBeaconBlockContentsFulu) UnmarshalSSZ(buf []byte) error {
// Field (1) 'KzgProofs'
{
buf = tail[o1:o2]
num, err := ssz.DivideInt2(len(buf), 48, 4096)
num, err := ssz.DivideInt2(len(buf), 48, 524288)
if err != nil {
return err
}
@@ -175,8 +175,8 @@ func (s *SignedBeaconBlockContentsFulu) HashTreeRootWith(hh *ssz.Hasher) (err er
// Field (1) 'KzgProofs'
{
if size := len(s.KzgProofs); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 4096)
if size := len(s.KzgProofs); size > 524288 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 524288)
return
}
subIndx := hh.Index()
@@ -189,7 +189,7 @@ func (s *SignedBeaconBlockContentsFulu) HashTreeRootWith(hh *ssz.Hasher) (err er
}
numItems := uint64(len(s.KzgProofs))
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
hh.MerkleizeWithMixin(subIndx, numItems, 524288)
}
// Field (2) 'Blobs'
@@ -355,8 +355,8 @@ func (b *BeaconBlockContentsFulu) MarshalSSZTo(buf []byte) (dst []byte, err erro
}
// Field (1) 'KzgProofs'
if size := len(b.KzgProofs); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 4096)
if size := len(b.KzgProofs); size > 524288 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 524288)
return
}
for ii := 0; ii < len(b.KzgProofs); ii++ {
@@ -427,7 +427,7 @@ func (b *BeaconBlockContentsFulu) UnmarshalSSZ(buf []byte) error {
// Field (1) 'KzgProofs'
{
buf = tail[o1:o2]
num, err := ssz.DivideInt2(len(buf), 48, 4096)
num, err := ssz.DivideInt2(len(buf), 48, 524288)
if err != nil {
return err
}
@@ -493,8 +493,8 @@ func (b *BeaconBlockContentsFulu) HashTreeRootWith(hh *ssz.Hasher) (err error) {
// Field (1) 'KzgProofs'
{
if size := len(b.KzgProofs); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 4096)
if size := len(b.KzgProofs); size > 524288 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 524288)
return
}
subIndx := hh.Index()
@@ -507,7 +507,7 @@ func (b *BeaconBlockContentsFulu) HashTreeRootWith(hh *ssz.Hasher) (err error) {
}
numItems := uint64(len(b.KzgProofs))
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
hh.MerkleizeWithMixin(subIndx, numItems, 524288)
}
// Field (2) 'Blobs'
@@ -799,20 +799,20 @@ func (d *DataColumnSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
offset := int(356)
// Field (0) 'ColumnIndex'
dst = ssz.MarshalUint64(dst, d.ColumnIndex)
// Field (0) 'Index'
dst = ssz.MarshalUint64(dst, d.Index)
// Offset (1) 'DataColumn'
// Offset (1) 'Column'
dst = ssz.WriteOffset(dst, offset)
offset += len(d.DataColumn) * 2048
offset += len(d.Column) * 2048
// Offset (2) 'KzgCommitments'
dst = ssz.WriteOffset(dst, offset)
offset += len(d.KzgCommitments) * 48
// Offset (3) 'KzgProof'
// Offset (3) 'KzgProofs'
dst = ssz.WriteOffset(dst, offset)
offset += len(d.KzgProof) * 48
offset += len(d.KzgProofs) * 48
// Field (4) 'SignedBlockHeader'
if d.SignedBlockHeader == nil {
@@ -835,17 +835,17 @@ func (d *DataColumnSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = append(dst, d.KzgCommitmentsInclusionProof[ii]...)
}
// Field (1) 'DataColumn'
if size := len(d.DataColumn); size > 4096 {
err = ssz.ErrListTooBigFn("--.DataColumn", size, 4096)
// Field (1) 'Column'
if size := len(d.Column); size > 4096 {
err = ssz.ErrListTooBigFn("--.Column", size, 4096)
return
}
for ii := 0; ii < len(d.DataColumn); ii++ {
if size := len(d.DataColumn[ii]); size != 2048 {
err = ssz.ErrBytesLengthFn("--.DataColumn[ii]", size, 2048)
for ii := 0; ii < len(d.Column); ii++ {
if size := len(d.Column[ii]); size != 2048 {
err = ssz.ErrBytesLengthFn("--.Column[ii]", size, 2048)
return
}
dst = append(dst, d.DataColumn[ii]...)
dst = append(dst, d.Column[ii]...)
}
// Field (2) 'KzgCommitments'
@@ -861,17 +861,17 @@ func (d *DataColumnSidecar) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = append(dst, d.KzgCommitments[ii]...)
}
// Field (3) 'KzgProof'
if size := len(d.KzgProof); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProof", size, 4096)
// Field (3) 'KzgProofs'
if size := len(d.KzgProofs); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 4096)
return
}
for ii := 0; ii < len(d.KzgProof); ii++ {
if size := len(d.KzgProof[ii]); size != 48 {
err = ssz.ErrBytesLengthFn("--.KzgProof[ii]", size, 48)
for ii := 0; ii < len(d.KzgProofs); ii++ {
if size := len(d.KzgProofs[ii]); size != 48 {
err = ssz.ErrBytesLengthFn("--.KzgProofs[ii]", size, 48)
return
}
dst = append(dst, d.KzgProof[ii]...)
dst = append(dst, d.KzgProofs[ii]...)
}
return
@@ -888,10 +888,10 @@ func (d *DataColumnSidecar) UnmarshalSSZ(buf []byte) error {
tail := buf
var o1, o2, o3 uint64
// Field (0) 'ColumnIndex'
d.ColumnIndex = ssz.UnmarshallUint64(buf[0:8])
// Field (0) 'Index'
d.Index = ssz.UnmarshallUint64(buf[0:8])
// Offset (1) 'DataColumn'
// Offset (1) 'Column'
if o1 = ssz.ReadOffset(buf[8:12]); o1 > size {
return ssz.ErrOffset
}
@@ -905,7 +905,7 @@ func (d *DataColumnSidecar) UnmarshalSSZ(buf []byte) error {
return ssz.ErrOffset
}
// Offset (3) 'KzgProof'
// Offset (3) 'KzgProofs'
if o3 = ssz.ReadOffset(buf[16:20]); o3 > size || o2 > o3 {
return ssz.ErrOffset
}
@@ -927,19 +927,19 @@ func (d *DataColumnSidecar) UnmarshalSSZ(buf []byte) error {
d.KzgCommitmentsInclusionProof[ii] = append(d.KzgCommitmentsInclusionProof[ii], buf[228:356][ii*32:(ii+1)*32]...)
}
// Field (1) 'DataColumn'
// Field (1) 'Column'
{
buf = tail[o1:o2]
num, err := ssz.DivideInt2(len(buf), 2048, 4096)
if err != nil {
return err
}
d.DataColumn = make([][]byte, num)
d.Column = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(d.DataColumn[ii]) == 0 {
d.DataColumn[ii] = make([]byte, 0, len(buf[ii*2048:(ii+1)*2048]))
if cap(d.Column[ii]) == 0 {
d.Column[ii] = make([]byte, 0, len(buf[ii*2048:(ii+1)*2048]))
}
d.DataColumn[ii] = append(d.DataColumn[ii], buf[ii*2048:(ii+1)*2048]...)
d.Column[ii] = append(d.Column[ii], buf[ii*2048:(ii+1)*2048]...)
}
}
@@ -959,19 +959,19 @@ func (d *DataColumnSidecar) UnmarshalSSZ(buf []byte) error {
}
}
// Field (3) 'KzgProof'
// Field (3) 'KzgProofs'
{
buf = tail[o3:]
num, err := ssz.DivideInt2(len(buf), 48, 4096)
if err != nil {
return err
}
d.KzgProof = make([][]byte, num)
d.KzgProofs = make([][]byte, num)
for ii := 0; ii < num; ii++ {
if cap(d.KzgProof[ii]) == 0 {
d.KzgProof[ii] = make([]byte, 0, len(buf[ii*48:(ii+1)*48]))
if cap(d.KzgProofs[ii]) == 0 {
d.KzgProofs[ii] = make([]byte, 0, len(buf[ii*48:(ii+1)*48]))
}
d.KzgProof[ii] = append(d.KzgProof[ii], buf[ii*48:(ii+1)*48]...)
d.KzgProofs[ii] = append(d.KzgProofs[ii], buf[ii*48:(ii+1)*48]...)
}
}
return err
@@ -981,14 +981,14 @@ func (d *DataColumnSidecar) UnmarshalSSZ(buf []byte) error {
func (d *DataColumnSidecar) SizeSSZ() (size int) {
size = 356
// Field (1) 'DataColumn'
size += len(d.DataColumn) * 2048
// Field (1) 'Column'
size += len(d.Column) * 2048
// Field (2) 'KzgCommitments'
size += len(d.KzgCommitments) * 48
// Field (3) 'KzgProof'
size += len(d.KzgProof) * 48
// Field (3) 'KzgProofs'
size += len(d.KzgProofs) * 48
return
}
@@ -1002,17 +1002,17 @@ func (d *DataColumnSidecar) HashTreeRoot() ([32]byte, error) {
func (d *DataColumnSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'ColumnIndex'
hh.PutUint64(d.ColumnIndex)
// Field (0) 'Index'
hh.PutUint64(d.Index)
// Field (1) 'DataColumn'
// Field (1) 'Column'
{
if size := len(d.DataColumn); size > 4096 {
err = ssz.ErrListTooBigFn("--.DataColumn", size, 4096)
if size := len(d.Column); size > 4096 {
err = ssz.ErrListTooBigFn("--.Column", size, 4096)
return
}
subIndx := hh.Index()
for _, i := range d.DataColumn {
for _, i := range d.Column {
if len(i) != 2048 {
err = ssz.ErrBytesLength
return
@@ -1020,7 +1020,7 @@ func (d *DataColumnSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
hh.PutBytes(i)
}
numItems := uint64(len(d.DataColumn))
numItems := uint64(len(d.Column))
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
}
@@ -1043,14 +1043,14 @@ func (d *DataColumnSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
}
// Field (3) 'KzgProof'
// Field (3) 'KzgProofs'
{
if size := len(d.KzgProof); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProof", size, 4096)
if size := len(d.KzgProofs); size > 4096 {
err = ssz.ErrListTooBigFn("--.KzgProofs", size, 4096)
return
}
subIndx := hh.Index()
for _, i := range d.KzgProof {
for _, i := range d.KzgProofs {
if len(i) != 48 {
err = ssz.ErrBytesLength
return
@@ -1058,7 +1058,7 @@ func (d *DataColumnSidecar) HashTreeRootWith(hh *ssz.Hasher) (err error) {
hh.PutBytes(i)
}
numItems := uint64(len(d.KzgProof))
numItems := uint64(len(d.KzgProofs))
hh.MerkleizeWithMixin(subIndx, numItems, 4096)
}
@@ -1104,8 +1104,8 @@ func (d *DataColumnIdentifier) MarshalSSZTo(buf []byte) (dst []byte, err error)
}
dst = append(dst, d.BlockRoot...)
// Field (1) 'ColumnIndex'
dst = ssz.MarshalUint64(dst, d.ColumnIndex)
// Field (1) 'Index'
dst = ssz.MarshalUint64(dst, d.Index)
return
}
@@ -1124,8 +1124,8 @@ func (d *DataColumnIdentifier) UnmarshalSSZ(buf []byte) error {
}
d.BlockRoot = append(d.BlockRoot, buf[0:32]...)
// Field (1) 'ColumnIndex'
d.ColumnIndex = ssz.UnmarshallUint64(buf[32:40])
// Field (1) 'Index'
d.Index = ssz.UnmarshallUint64(buf[32:40])
return err
}
@@ -1152,8 +1152,8 @@ func (d *DataColumnIdentifier) HashTreeRootWith(hh *ssz.Hasher) (err error) {
}
hh.PutBytes(d.BlockRoot)
// Field (1) 'ColumnIndex'
hh.PutUint64(d.ColumnIndex)
// Field (1) 'Index'
hh.PutUint64(d.Index)
hh.Merkleize(indx)
return

View File

@@ -685,8 +685,8 @@ func (m *MetaDataV2) MarshalSSZTo(buf []byte) (dst []byte, err error) {
}
dst = append(dst, m.Syncnets...)
// Field (3) 'CustodySubnetCount'
dst = ssz.MarshalUint64(dst, m.CustodySubnetCount)
// Field (3) 'CustodyGroupCount'
dst = ssz.MarshalUint64(dst, m.CustodyGroupCount)
return
}
@@ -714,8 +714,8 @@ func (m *MetaDataV2) UnmarshalSSZ(buf []byte) error {
}
m.Syncnets = append(m.Syncnets, buf[16:17]...)
// Field (3) 'CustodySubnetCount'
m.CustodySubnetCount = ssz.UnmarshallUint64(buf[17:25])
// Field (3) 'CustodyGroupCount'
m.CustodyGroupCount = ssz.UnmarshallUint64(buf[17:25])
return err
}
@@ -752,8 +752,8 @@ func (m *MetaDataV2) HashTreeRootWith(hh *ssz.Hasher) (err error) {
}
hh.PutBytes(m.Syncnets)
// Field (3) 'CustodySubnetCount'
hh.PutUint64(m.CustodySubnetCount)
// Field (3) 'CustodyGroupCount'
hh.PutUint64(m.CustodyGroupCount)
hh.Merkleize(indx)
return

View File

@@ -353,10 +353,10 @@ type MetaDataV2 struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
SeqNumber uint64 `protobuf:"varint,1,opt,name=seq_number,json=seqNumber,proto3" json:"seq_number,omitempty"`
Attnets github_com_prysmaticlabs_go_bitfield.Bitvector64 `protobuf:"bytes,2,opt,name=attnets,proto3" json:"attnets,omitempty" cast-type:"github.com/prysmaticlabs/go-bitfield.Bitvector64" ssz-size:"8"`
Syncnets github_com_prysmaticlabs_go_bitfield.Bitvector4 `protobuf:"bytes,3,opt,name=syncnets,proto3" json:"syncnets,omitempty" cast-type:"github.com/prysmaticlabs/go-bitfield.Bitvector4" ssz-size:"1"`
CustodySubnetCount uint64 `protobuf:"varint,4,opt,name=custody_subnet_count,json=custodySubnetCount,proto3" json:"custody_subnet_count,omitempty"`
SeqNumber uint64 `protobuf:"varint,1,opt,name=seq_number,json=seqNumber,proto3" json:"seq_number,omitempty"`
Attnets github_com_prysmaticlabs_go_bitfield.Bitvector64 `protobuf:"bytes,2,opt,name=attnets,proto3" json:"attnets,omitempty" cast-type:"github.com/prysmaticlabs/go-bitfield.Bitvector64" ssz-size:"8"`
Syncnets github_com_prysmaticlabs_go_bitfield.Bitvector4 `protobuf:"bytes,3,opt,name=syncnets,proto3" json:"syncnets,omitempty" cast-type:"github.com/prysmaticlabs/go-bitfield.Bitvector4" ssz-size:"1"`
CustodyGroupCount uint64 `protobuf:"varint,4,opt,name=custody_group_count,json=custodyGroupCount,proto3" json:"custody_group_count,omitempty"`
}
func (x *MetaDataV2) Reset() {
@@ -412,9 +412,9 @@ func (x *MetaDataV2) GetSyncnets() github_com_prysmaticlabs_go_bitfield.Bitvecto
return github_com_prysmaticlabs_go_bitfield.Bitvector4(nil)
}
func (x *MetaDataV2) GetCustodySubnetCount() uint64 {
func (x *MetaDataV2) GetCustodyGroupCount() uint64 {
if x != nil {
return x.CustodySubnetCount
return x.CustodyGroupCount
}
return 0
}
@@ -616,7 +616,7 @@ var file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc = []byte{
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63,
0x6c, 0x61, 0x62, 0x73, 0x2f, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64,
0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x31,
0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x22, 0x88, 0x02, 0x0a, 0x0a, 0x4d,
0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x22, 0x86, 0x02, 0x0a, 0x0a, 0x4d,
0x65, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x32, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x71,
0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x73,
0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x74, 0x74, 0x6e,
@@ -630,42 +630,42 @@ var file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc = []byte{
0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x67, 0x6f,
0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63,
0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x31, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e,
0x65, 0x74, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x5f, 0x73,
0x75, 0x62, 0x6e, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
0x04, 0x52, 0x12, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x53, 0x75, 0x62, 0x6e, 0x65, 0x74,
0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x97, 0x01, 0x0a, 0x1a, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69,
0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c,
0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69,
0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63,
0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70,
0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09,
0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75,
0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22,
0xc0, 0x01, 0x0a, 0x20, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69,
0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c,
0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69,
0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63,
0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70,
0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09,
0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75,
0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12,
0x21, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04,
0x42, 0x07, 0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d,
0x6e, 0x73, 0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31,
0x42, 0x10, 0x50, 0x32, 0x50, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x50, 0x72, 0x6f,
0x74, 0x6f, 0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa,
0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56,
0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62,
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x65, 0x74, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x5f, 0x67,
0x72, 0x6f, 0x75, 0x70, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04,
0x52, 0x11, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f,
0x75, 0x6e, 0x74, 0x22, 0x97, 0x01, 0x0a, 0x1a, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65,
0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74,
0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xc0, 0x01,
0x0a, 0x20, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65,
0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x12, 0x63, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74,
0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x44, 0x82, 0xb5, 0x18, 0x40, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f, 0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x63, 0x6f, 0x6e,
0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69,
0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74,
0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, 0x0a,
0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x07,
0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
0x42, 0x9a, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10,
0x50, 0x32, 0x50, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f,
0x50, 0x01, 0x5a, 0x39, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x4f,
0x66, 0x66, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x4c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x2f, 0x76, 0x36, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15,
0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@@ -91,7 +91,7 @@ message MetaDataV1 {
seq_number: uint64
attnets: Bitvector[ATTESTATION_SUBNET_COUNT]
syncnets: Bitvector[SYNC_COMMITTEE_SUBNET_COUNT]
custody_subnet_count: uint64
custody_group_count: uint64
)
*/
message MetaDataV2 {
@@ -106,7 +106,7 @@ message MetaDataV2 {
(ethereum.eth.ext.cast_type) =
"github.com/prysmaticlabs/go-bitfield.Bitvector4"
];
uint64 custody_subnet_count = 4;
uint64 custody_group_count = 4;
}
/*

View File

@@ -25,6 +25,7 @@ mainnet = {
"extra_data.size": "32",
"max_blobs_per_block.size": "6",
"max_blob_commitments.size": "4096",
"max_cell_proofs_length.size": "524288", # CELLS_PER_EXT_BLOB * MAX_BLOB_COMMITMENTS_PER_BLOCK
"kzg_commitment_inclusion_proof_depth.size": "17",
"max_withdrawal_requests_per_payload.size": "16",
"max_deposit_requests_per_payload.size": "8192",
@@ -62,6 +63,7 @@ minimal = {
"extra_data.size": "32",
"max_blobs_per_block.size": "6",
"max_blob_commitments.size": "32",
"max_cell_proofs_length.size": "524288", # CELLS_PER_EXT_BLOB * MAX_BLOB_COMMITMENTS_PER_BLOCK
"kzg_commitment_inclusion_proof_depth.size": "10",
"max_withdrawal_requests_per_payload.size": "2",
"max_deposit_requests_per_payload.size": "4",

View File

@@ -43,7 +43,7 @@ type Validator interface {
NextSlot() <-chan primitives.Slot
SlotDeadline(slot primitives.Slot) time.Time
LogValidatorGainsAndLosses(ctx context.Context, slot primitives.Slot) error
UpdateDuties(ctx context.Context, slot primitives.Slot) error
UpdateDuties(ctx context.Context) error
RolesAt(ctx context.Context, slot primitives.Slot) (map[[fieldparams.BLSPubkeyLength]byte][]ValidatorRole, error) // validator pubKey -> roles
SubmitAttestation(ctx context.Context, slot primitives.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)
ProposeBlock(ctx context.Context, slot primitives.Slot, pubKey [fieldparams.BLSPubkeyLength]byte)

View File

@@ -43,7 +43,7 @@ func run(ctx context.Context, v iface.Validator) {
if err != nil {
return // Exit if context is canceled.
}
if err := v.UpdateDuties(ctx, headSlot); err != nil {
if err := v.UpdateDuties(ctx); err != nil {
handleAssignmentError(err, headSlot)
}
eventsChan := make(chan *event.Event, 1)
@@ -89,11 +89,13 @@ func run(ctx context.Context, v iface.Validator) {
// Keep trying to update assignments if they are nil or if we are past an
// epoch transition in the beacon node's state.
if err := v.UpdateDuties(slotCtx, slot); err != nil {
handleAssignmentError(err, slot)
span.End()
cancel()
continue
if slots.IsEpochStart(slot) {
if err := v.UpdateDuties(slotCtx); err != nil {
handleAssignmentError(err, slot)
span.End()
cancel()
continue
}
}
// call push proposer settings often to account for the following edge cases:
@@ -125,7 +127,7 @@ func run(ctx context.Context, v iface.Validator) {
log.WithError(err).Error("Failed to re initialize validator and get head slot")
continue
}
if err := v.UpdateDuties(ctx, headSlot); err != nil {
if err := v.UpdateDuties(ctx); err != nil {
handleAssignmentError(err, headSlot)
continue
}

View File

@@ -116,7 +116,6 @@ func TestUpdateDuties_NextSlot(t *testing.T) {
run(ctx, v)
require.Equal(t, true, v.UpdateDutiesCalled, "Expected UpdateAssignments(%d) to be called", slot)
assert.Equal(t, uint64(slot), v.UpdateDutiesArg1, "UpdateAssignments was called with wrong argument")
}
func TestUpdateDuties_HandlesError(t *testing.T) {

View File

@@ -137,9 +137,8 @@ func (fv *FakeValidator) NextSlot() <-chan primitives.Slot {
}
// UpdateDuties for mocking.
func (fv *FakeValidator) UpdateDuties(_ context.Context, slot primitives.Slot) error {
func (fv *FakeValidator) UpdateDuties(_ context.Context) error {
fv.UpdateDutiesCalled = true
fv.UpdateDutiesArg1 = uint64(slot)
return fv.UpdateDutiesRet
}

View File

@@ -514,13 +514,11 @@ func retrieveLatestRecord(recs []*dbCommon.AttestationRecord) *dbCommon.Attestat
// UpdateDuties checks the slot number to determine if the validator's
// list of upcoming assignments needs to be updated. For example, at the
// beginning of a new epoch.
func (v *validator) UpdateDuties(ctx context.Context, slot primitives.Slot) error {
if !slots.IsEpochStart(slot) && v.duties != nil {
// Do nothing if not epoch start AND assignments already exist.
return nil
}
func (v *validator) UpdateDuties(ctx context.Context) error {
// Set deadline to end of epoch.
ss, err := slots.EpochStart(primitives.Epoch(slots.CurrentSlot(v.genesisTime) + 1))
epoch := slots.ToEpoch(slots.CurrentSlot(v.genesisTime) + 1)
ss, err := slots.EpochStart(epoch + 1)
if err != nil {
return err
}
@@ -550,13 +548,13 @@ func (v *validator) UpdateDuties(ctx context.Context, slot primitives.Slot) erro
v.blacklistedPubkeysLock.RUnlock()
req := &ethpb.DutiesRequest{
Epoch: primitives.Epoch(slot / params.BeaconConfig().SlotsPerEpoch),
Epoch: epoch,
PublicKeys: bytesutil.FromBytes48Array(filteredKeys),
}
// If duties is nil it means we have had no prior duties and just started up.
resp, err := v.validatorClient.Duties(ctx, req)
if err != nil {
if err != nil || resp == nil {
v.dutiesLock.Lock()
v.duties = nil // Clear assignments so we know to retry the request.
v.dutiesLock.Unlock()
@@ -566,7 +564,7 @@ func (v *validator) UpdateDuties(ctx context.Context, slot primitives.Slot) erro
v.dutiesLock.Lock()
v.duties = resp
v.logDuties(slot, v.duties.CurrentEpochDuties, v.duties.NextEpochDuties)
v.logDuties(ss-params.BeaconConfig().SlotsPerEpoch, v.duties.CurrentEpochDuties, v.duties.NextEpochDuties)
v.dutiesLock.Unlock()
allExitedCounter := 0
@@ -689,6 +687,10 @@ func (v *validator) RolesAt(ctx context.Context, slot primitives.Slot) (map[[fie
v.dutiesLock.RLock()
defer v.dutiesLock.RUnlock()
if v.duties == nil {
return nil, errors.New("validator duties are not initialized")
}
var (
rolesAt = make(map[[fieldparams.BLSPubkeyLength]byte][]iface.ValidatorRole)
@@ -1144,22 +1146,18 @@ func (v *validator) checkDependentRoots(ctx context.Context, head *structs.HeadE
if head == nil {
return errors.New("received empty head event")
}
if v.duties == nil {
return errors.New("duties are not initialized")
}
prevDepedentRoot, err := bytesutil.DecodeHexWithLength(head.PreviousDutyDependentRoot, fieldparams.RootLength)
if err != nil {
return errors.Wrap(err, "failed to decode previous duty dependent root")
}
uintSlot, err := strconv.ParseUint(head.Slot, 10, 64)
if err != nil {
return errors.Wrap(err, "Failed to parse slot")
}
slot := primitives.Slot(uintSlot)
currEpochStart, err := slots.EpochStart(slots.ToEpoch(slot))
if err != nil {
return err
if bytes.Equal(prevDepedentRoot, params.BeaconConfig().ZeroHash[:]) {
return nil
}
if !bytes.Equal(prevDepedentRoot, v.duties.PrevDependentRoot) {
if err := v.UpdateDuties(ctx, currEpochStart); err != nil {
if err := v.UpdateDuties(ctx); err != nil {
return errors.Wrap(err, "failed to update duties")
}
log.Info("Updated duties due to previous dependent root change")
@@ -1169,8 +1167,11 @@ func (v *validator) checkDependentRoots(ctx context.Context, head *structs.HeadE
if err != nil {
return errors.Wrap(err, "failed to decode current duty dependent root")
}
if bytes.Equal(currDepedentRoot, params.BeaconConfig().ZeroHash[:]) {
return nil
}
if !bytes.Equal(currDepedentRoot, v.duties.CurrDependentRoot) {
if err := v.UpdateDuties(ctx, currEpochStart); err != nil {
if err := v.UpdateDuties(ctx); err != nil {
return errors.Wrap(err, "failed to update duties")
}
log.Info("Updated duties due to current dependent root change")

View File

@@ -392,8 +392,8 @@ func TestUpdateDuties_DoesNothingWhenNotEpochStart_AlreadyExistingAssignments(t
defer ctrl.Finish()
client := validatormock.NewMockValidatorClient(ctrl)
slot := primitives.Slot(1)
v := validator{
km: newMockKeymanager(t, randKeypair(t)),
validatorClient: client,
duties: &ethpb.ValidatorDutiesContainer{
CurrentEpochDuties: []*ethpb.ValidatorDuty{
@@ -402,14 +402,20 @@ func TestUpdateDuties_DoesNothingWhenNotEpochStart_AlreadyExistingAssignments(t
CommitteeIndex: 20,
},
},
NextEpochDuties: []*ethpb.ValidatorDuty{
{
AttesterSlot: 10,
CommitteeIndex: 20,
},
},
},
}
client.EXPECT().Duties(
gomock.Any(),
gomock.Any(),
).Times(0)
).Times(1)
assert.NoError(t, v.UpdateDuties(context.Background(), slot), "Could not update assignments")
assert.NoError(t, v.UpdateDuties(context.Background()), "Could not update assignments")
}
func TestUpdateDuties_ReturnsError(t *testing.T) {
@@ -436,7 +442,7 @@ func TestUpdateDuties_ReturnsError(t *testing.T) {
gomock.Any(),
).Return(nil, expected)
assert.ErrorContains(t, expected.Error(), v.UpdateDuties(context.Background(), params.BeaconConfig().SlotsPerEpoch))
assert.ErrorContains(t, expected.Error(), v.UpdateDuties(context.Background()))
assert.Equal(t, (*ethpb.ValidatorDutiesContainer)(nil), v.duties, "Assignments should have been cleared on failure")
}
@@ -445,7 +451,6 @@ func TestUpdateDuties_OK(t *testing.T) {
defer ctrl.Finish()
client := validatormock.NewMockValidatorClient(ctrl)
slot := params.BeaconConfig().SlotsPerEpoch
resp := &ethpb.ValidatorDutiesContainer{
CurrentEpochDuties: []*ethpb.ValidatorDuty{
{
@@ -479,7 +484,7 @@ func TestUpdateDuties_OK(t *testing.T) {
return nil, nil
})
require.NoError(t, v.UpdateDuties(context.Background(), slot), "Could not update assignments")
require.NoError(t, v.UpdateDuties(context.Background()), "Could not update assignments")
util.WaitTimeout(&wg, 2*time.Second)
@@ -494,7 +499,6 @@ func TestUpdateDuties_OK_FilterBlacklistedPublicKeys(t *testing.T) {
ctrl := gomock.NewController(t)
defer ctrl.Finish()
client := validatormock.NewMockValidatorClient(ctrl)
slot := params.BeaconConfig().SlotsPerEpoch
numValidators := 10
km := genMockKeymanager(t, numValidators)
@@ -527,7 +531,7 @@ func TestUpdateDuties_OK_FilterBlacklistedPublicKeys(t *testing.T) {
return nil, nil
})
require.NoError(t, v.UpdateDuties(context.Background(), slot), "Could not update assignments")
require.NoError(t, v.UpdateDuties(context.Background()), "Could not update assignments")
util.WaitTimeout(&wg, 2*time.Second)
@@ -541,7 +545,6 @@ func TestUpdateDuties_AllValidatorsExited(t *testing.T) {
defer ctrl.Finish()
client := validatormock.NewMockValidatorClient(ctrl)
slot := params.BeaconConfig().SlotsPerEpoch
resp := &ethpb.ValidatorDutiesContainer{
CurrentEpochDuties: []*ethpb.ValidatorDuty{
{
@@ -573,7 +576,7 @@ func TestUpdateDuties_AllValidatorsExited(t *testing.T) {
gomock.Any(),
).Return(resp, nil)
err := v.UpdateDuties(context.Background(), slot)
err := v.UpdateDuties(context.Background())
require.ErrorContains(t, ErrValidatorsAllExited.Error(), err)
}
@@ -659,7 +662,7 @@ func TestUpdateDuties_Distributed(t *testing.T) {
return nil, nil
})
require.NoError(t, v.UpdateDuties(context.Background(), slot), "Could not update assignments")
require.NoError(t, v.UpdateDuties(context.Background()), "Could not update assignments")
util.WaitTimeout(&wg, 2*time.Second)
require.Equal(t, 2, len(v.attSelections))
}