From 629afd7c5b5bcc9f0549b5f6f97ddd9dbe543b45 Mon Sep 17 00:00:00 2001 From: Potuz Date: Mon, 28 Oct 2024 09:56:07 -0300 Subject: [PATCH] Process Execution Payload Envelope in Chain Service (#14295) Adds the processing of execution payload envelope Corrects the protos for attestations and slashings in Electra versions Adds generators of full blocks for Electra --- beacon-chain/blockchain/BUILD.bazel | 6 + beacon-chain/blockchain/chain_info.go | 5 + ...ntly_syncing_execution_payload_envelope.go | 27 + .../blockchain/execution_engine_test.go | 4 +- beacon-chain/blockchain/metrics.go | 4 + .../blockchain/receive_attestation.go | 2 +- .../receive_execution_payload_envelope.go | 166 ++++ ...receive_execution_payload_envelope_test.go | 104 +++ beacon-chain/blockchain/service.go | 4 +- beacon-chain/blockchain/service_test.go | 4 +- beacon-chain/core/blocks/genesis.go | 37 + beacon-chain/core/epbs/BUILD.bazel | 24 +- .../core/epbs/execution_payload_envelope.go | 126 +++ .../epbs/execution_payload_envelope_test.go | 105 +++ beacon-chain/p2p/BUILD.bazel | 2 + beacon-chain/p2p/broadcaster_test.go | 3 +- beacon-chain/rpc/eth/events/BUILD.bazel | 7 +- beacon-chain/rpc/eth/events/events.go | 1 + beacon-chain/state/state-native/state_trie.go | 1 - .../state/state-native/state_trie_epbs.go | 11 +- .../state/state-native/types/types.go | 4 - consensus-types/blocks/getters.go | 10 - consensus-types/blocks/proto.go | 8 +- consensus-types/blocks/proto_epbs_test.go | 8 +- consensus-types/blocks/proto_test.go | 60 ++ consensus-types/epbs/BUILD.bazel | 25 +- .../epbs/execution_payload_envelope.go | 145 ++++ consensus-types/interfaces/BUILD.bazel | 2 + .../interfaces/execution_payload_envelope.go | 31 + consensus-types/mock/BUILD.bazel | 2 - encoding/ssz/htrutils.go | 26 + proto/engine/v1/epbs.pb.go | 801 +++--------------- proto/engine/v1/epbs.proto | 2 +- proto/prysm/v1alpha1/beacon_block.pb.go | 143 ++-- proto/prysm/v1alpha1/beacon_block.proto | 4 +- proto/prysm/v1alpha1/beacon_state.pb.go | 366 ++++---- .../v1alpha1/blind_payload_envelope.pb.go | 93 +- testing/util/BUILD.bazel | 2 + testing/util/block.go | 9 +- testing/util/epbs_block.go | 260 ++++++ testing/util/epbs_state.go | 273 ++++++ testing/util/helpers.go | 8 + testing/util/random/epbs.go | 13 +- 43 files changed, 1849 insertions(+), 1089 deletions(-) create mode 100644 beacon-chain/blockchain/currently_syncing_execution_payload_envelope.go create mode 100644 beacon-chain/blockchain/receive_execution_payload_envelope.go create mode 100644 beacon-chain/blockchain/receive_execution_payload_envelope_test.go create mode 100644 beacon-chain/core/epbs/execution_payload_envelope.go create mode 100644 beacon-chain/core/epbs/execution_payload_envelope_test.go create mode 100644 consensus-types/epbs/execution_payload_envelope.go create mode 100644 consensus-types/interfaces/execution_payload_envelope.go create mode 100644 testing/util/epbs_block.go create mode 100644 testing/util/epbs_state.go diff --git a/beacon-chain/blockchain/BUILD.bazel b/beacon-chain/blockchain/BUILD.bazel index c12a3d1340bd..a866849843b9 100644 --- a/beacon-chain/blockchain/BUILD.bazel +++ b/beacon-chain/blockchain/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "chain_info.go", "chain_info_forkchoice.go", "currently_syncing_block.go", + "currently_syncing_execution_payload_envelope.go", "defragment.go", "error.go", "execution_engine.go", @@ -25,6 +26,7 @@ go_library( "receive_attestation.go", "receive_blob.go", "receive_block.go", + "receive_execution_payload_envelope.go", "service.go", "tracked_proposer.go", "weak_subjectivity_checks.go", @@ -43,6 +45,7 @@ go_library( "//beacon-chain/cache:go_default_library", "//beacon-chain/core/altair:go_default_library", "//beacon-chain/core/blocks:go_default_library", + "//beacon-chain/core/epbs:go_default_library", "//beacon-chain/core/epoch/precompute:go_default_library", "//beacon-chain/core/feed:go_default_library", "//beacon-chain/core/feed/state:go_default_library", @@ -97,6 +100,7 @@ go_library( "@com_github_prometheus_client_golang//prometheus:go_default_library", "@com_github_prometheus_client_golang//prometheus/promauto:go_default_library", "@com_github_sirupsen_logrus//:go_default_library", + "@io_opencensus_go//trace:go_default_library", "@org_golang_x_sync//errgroup:go_default_library", ], ) @@ -124,6 +128,7 @@ go_test( "process_block_test.go", "receive_attestation_test.go", "receive_block_test.go", + "receive_execution_payload_envelope_test.go", "service_norace_test.go", "service_test.go", "setup_test.go", @@ -165,6 +170,7 @@ go_test( "//config/fieldparams:go_default_library", "//config/params:go_default_library", "//consensus-types/blocks:go_default_library", + "//consensus-types/epbs:go_default_library", "//consensus-types/interfaces:go_default_library", "//consensus-types/primitives:go_default_library", "//container/trie:go_default_library", diff --git a/beacon-chain/blockchain/chain_info.go b/beacon-chain/blockchain/chain_info.go index 47ba3ff86529..9f59da66cc94 100644 --- a/beacon-chain/blockchain/chain_info.go +++ b/beacon-chain/blockchain/chain_info.go @@ -531,6 +531,11 @@ func (s *Service) recoverStateSummary(ctx context.Context, blockRoot [32]byte) ( return nil, errBlockDoesNotExist } +// PayloadBeingSynced returns whether the payload for the block with the given root is currently being synced +func (s *Service) PayloadBeingSynced(root [32]byte) bool { + return s.payloadBeingSynced.isSyncing(root) +} + // BlockBeingSynced returns whether the block with the given root is currently being synced func (s *Service) BlockBeingSynced(root [32]byte) bool { return s.blockBeingSynced.isSyncing(root) diff --git a/beacon-chain/blockchain/currently_syncing_execution_payload_envelope.go b/beacon-chain/blockchain/currently_syncing_execution_payload_envelope.go new file mode 100644 index 000000000000..df15abcb1c58 --- /dev/null +++ b/beacon-chain/blockchain/currently_syncing_execution_payload_envelope.go @@ -0,0 +1,27 @@ +package blockchain + +import "sync" + +type currentlySyncingPayload struct { + sync.Mutex + roots map[[32]byte]struct{} +} + +func (b *currentlySyncingPayload) set(root [32]byte) { + b.Lock() + defer b.Unlock() + b.roots[root] = struct{}{} +} + +func (b *currentlySyncingPayload) unset(root [32]byte) { + b.Lock() + defer b.Unlock() + delete(b.roots, root) +} + +func (b *currentlySyncingPayload) isSyncing(root [32]byte) bool { + b.Lock() + defer b.Unlock() + _, ok := b.roots[root] + return ok +} diff --git a/beacon-chain/blockchain/execution_engine_test.go b/beacon-chain/blockchain/execution_engine_test.go index ce8b132bc465..aae09a0205f5 100644 --- a/beacon-chain/blockchain/execution_engine_test.go +++ b/beacon-chain/blockchain/execution_engine_test.go @@ -1056,8 +1056,8 @@ func TestService_removeInvalidBlockAndState(t *testing.T) { require.NoError(t, service.removeInvalidBlockAndState(ctx, [][32]byte{r1, r2})) - require.Equal(t, false, service.hasBlock(ctx, r1)) - require.Equal(t, false, service.hasBlock(ctx, r2)) + require.Equal(t, false, service.chainHasBlock(ctx, r1)) + require.Equal(t, false, service.chainHasBlock(ctx, r2)) require.Equal(t, false, service.cfg.BeaconDB.HasStateSummary(ctx, r1)) require.Equal(t, false, service.cfg.BeaconDB.HasStateSummary(ctx, r2)) has, err := service.cfg.StateGen.HasState(ctx, r1) diff --git a/beacon-chain/blockchain/metrics.go b/beacon-chain/blockchain/metrics.go index 8abcce3a2c52..551a48b8a4c4 100644 --- a/beacon-chain/blockchain/metrics.go +++ b/beacon-chain/blockchain/metrics.go @@ -182,6 +182,10 @@ var ( Name: "chain_service_processing_milliseconds", Help: "Total time to call a chain service in ReceiveBlock()", }) + executionEngineProcessingTime = promauto.NewSummary(prometheus.SummaryOpts{ + Name: "execution_engine_processing_milliseconds", + Help: "Total time to process an execution payload envelope in ReceiveExecutionPayloadEnvelope()", + }) dataAvailWaitedTime = promauto.NewSummary(prometheus.SummaryOpts{ Name: "da_waited_time_milliseconds", Help: "Total time spent waiting for a data availability check in ReceiveBlock()", diff --git a/beacon-chain/blockchain/receive_attestation.go b/beacon-chain/blockchain/receive_attestation.go index e4776eae4d37..e8138121368c 100644 --- a/beacon-chain/blockchain/receive_attestation.go +++ b/beacon-chain/blockchain/receive_attestation.go @@ -177,7 +177,7 @@ func (s *Service) processAttestations(ctx context.Context, disparity time.Durati } hasState := s.cfg.BeaconDB.HasStateSummary(ctx, bytesutil.ToBytes32(a.GetData().BeaconBlockRoot)) - hasBlock := s.hasBlock(ctx, bytesutil.ToBytes32(a.GetData().BeaconBlockRoot)) + hasBlock := s.chainHasBlock(ctx, bytesutil.ToBytes32(a.GetData().BeaconBlockRoot)) if !(hasState && hasBlock) { continue } diff --git a/beacon-chain/blockchain/receive_execution_payload_envelope.go b/beacon-chain/blockchain/receive_execution_payload_envelope.go new file mode 100644 index 000000000000..ff8aa44b5fe7 --- /dev/null +++ b/beacon-chain/blockchain/receive_execution_payload_envelope.go @@ -0,0 +1,166 @@ +package blockchain + +import ( + "context" + "fmt" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/pkg/errors" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/epbs" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/das" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/state" + "github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces" + "github.com/prysmaticlabs/prysm/v5/encoding/bytesutil" + "github.com/sirupsen/logrus" + "go.opencensus.io/trace" + "golang.org/x/sync/errgroup" +) + +// ReceiveExecutionPayloadEnvelope is a function that defines the operations (minus pubsub) +// that are performed on a received execution payload envelope. The operations consist of: +// 1. Validate the payload, apply state transition. +// 2. Apply fork choice to the processed payload +// 3. Save latest head info +func (s *Service) ReceiveExecutionPayloadEnvelope(ctx context.Context, envelope interfaces.ROExecutionPayloadEnvelope, _ das.AvailabilityStore) error { + receivedTime := time.Now() + root, err := envelope.BeaconBlockRoot() + if err != nil { + return errors.Wrap(err, "could not get beacon block root") + } + s.payloadBeingSynced.set(root) + defer s.payloadBeingSynced.unset(root) + + preState, err := s.getPayloadEnvelopePrestate(ctx, envelope) + if err != nil { + return errors.Wrap(err, "could not get prestate") + } + + eg, _ := errgroup.WithContext(ctx) + var postState state.BeaconState + eg.Go(func() error { + if err := epbs.ValidatePayloadStateTransition(ctx, preState, envelope); err != nil { + return errors.Wrap(err, "failed to validate consensus state transition function") + } + return nil + }) + var isValidPayload bool + eg.Go(func() error { + var err error + isValidPayload, err = s.validateExecutionOnEnvelope(ctx, envelope) + if err != nil { + return errors.Wrap(err, "could not notify the engine of the new payload") + } + return nil + }) + + if err := eg.Wait(); err != nil { + return err + } + _ = isValidPayload + _ = postState + daStartTime := time.Now() + // TODO: Add DA check + daWaitedTime := time.Since(daStartTime) + dataAvailWaitedTime.Observe(float64(daWaitedTime.Milliseconds())) + // TODO: Add Head update, cache handling, postProcessing + timeWithoutDaWait := time.Since(receivedTime) - daWaitedTime + executionEngineProcessingTime.Observe(float64(timeWithoutDaWait.Milliseconds())) + return nil +} + +// notifyNewPayload signals execution engine on a new payload. +// It returns true if the EL has returned VALID for the block +func (s *Service) notifyNewEnvelope(ctx context.Context, envelope interfaces.ROExecutionPayloadEnvelope) (bool, error) { + ctx, span := trace.StartSpan(ctx, "blockChain.notifyNewPayload") + defer span.End() + + payload, err := envelope.Execution() + if err != nil { + return false, errors.Wrap(err, "could not get execution payload") + } + + var lastValidHash []byte + var versionedHashes []common.Hash + versionedHashes, err = envelope.VersionedHashes() + if err != nil { + return false, errors.Wrap(err, "could not get versioned hashes to feed the engine") + } + root, err := envelope.BeaconBlockRoot() + if err != nil { + return false, errors.Wrap(err, "could not get beacon block root") + } + parentRoot, err := s.ParentRoot(root) + if err != nil { + return false, errors.Wrap(err, "could not get parent block root") + } + pr := common.Hash(parentRoot) + lastValidHash, err = s.cfg.ExecutionEngineCaller.NewPayload(ctx, payload, versionedHashes, &pr) + switch { + case err == nil: + newPayloadValidNodeCount.Inc() + return true, nil + case errors.Is(err, execution.ErrAcceptedSyncingPayloadStatus): + newPayloadOptimisticNodeCount.Inc() + log.WithFields(logrus.Fields{ + "payloadBlockHash": fmt.Sprintf("%#x", bytesutil.Trunc(payload.BlockHash())), + }).Info("Called new payload with optimistic block") + return false, nil + case errors.Is(err, execution.ErrInvalidPayloadStatus): + lvh := bytesutil.ToBytes32(lastValidHash) + return false, invalidBlock{ + error: ErrInvalidPayload, + lastValidHash: lvh, + } + default: + return false, errors.WithMessage(ErrUndefinedExecutionEngineError, err.Error()) + } +} + +// validateExecutionOnEnvelope notifies the engine of the incoming execution payload and returns true if the payload is valid +func (s *Service) validateExecutionOnEnvelope(ctx context.Context, e interfaces.ROExecutionPayloadEnvelope) (bool, error) { + isValidPayload, err := s.notifyNewEnvelope(ctx, e) + if err == nil { + return isValidPayload, nil + } + blockRoot, rootErr := e.BeaconBlockRoot() + if rootErr != nil { + return false, errors.Wrap(rootErr, "could not get beacon block root") + } + parentRoot, rootErr := s.ParentRoot(blockRoot) + if rootErr != nil { + return false, errors.Wrap(rootErr, "could not get parent block root") + } + s.cfg.ForkChoiceStore.Lock() + err = s.handleInvalidExecutionError(ctx, err, blockRoot, parentRoot) + s.cfg.ForkChoiceStore.Unlock() + return false, err +} + +func (s *Service) getPayloadEnvelopePrestate(ctx context.Context, e interfaces.ROExecutionPayloadEnvelope) (state.BeaconState, error) { + ctx, span := trace.StartSpan(ctx, "blockChain.getPayloadEnvelopePreState") + defer span.End() + + // Verify incoming payload has a valid pre state. + root, err := e.BeaconBlockRoot() + if err != nil { + return nil, errors.Wrap(err, "could not get beacon block root") + } + // Verify the referred block is known to forkchoice + if !s.InForkchoice(root) { + return nil, errors.New("Cannot import execution payload envelope for unknown block") + } + if err := s.verifyBlkPreState(ctx, root); err != nil { + return nil, errors.Wrap(err, "could not verify payload prestate") + } + + preState, err := s.cfg.StateGen.StateByRoot(ctx, root) + if err != nil { + return nil, errors.Wrap(err, "could not get pre state") + } + if preState == nil || preState.IsNil() { + return nil, errors.Wrap(err, "nil pre state") + } + return preState, nil +} diff --git a/beacon-chain/blockchain/receive_execution_payload_envelope_test.go b/beacon-chain/blockchain/receive_execution_payload_envelope_test.go new file mode 100644 index 000000000000..ffb7e234547c --- /dev/null +++ b/beacon-chain/blockchain/receive_execution_payload_envelope_test.go @@ -0,0 +1,104 @@ +package blockchain + +import ( + "testing" + + "github.com/prysmaticlabs/prysm/v5/beacon-chain/cache" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/das" + mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing" + forkchoicetypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/forkchoice/types" + "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs" + enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" + "github.com/prysmaticlabs/prysm/v5/testing/require" + "github.com/prysmaticlabs/prysm/v5/testing/util" +) + +func Test_getPayloadEnvelopePrestate(t *testing.T) { + service, tr := minimalTestService(t) + ctx, fcs := tr.ctx, tr.fcs + + gs, _ := util.DeterministicGenesisStateEpbs(t, 32) + require.NoError(t, service.saveGenesisData(ctx, gs)) + require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot})) + + p := &enginev1.ExecutionPayloadEnvelope{ + Payload: &enginev1.ExecutionPayloadElectra{}, + BeaconBlockRoot: service.originBlockRoot[:], + } + e, err := epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + + _, err = service.getPayloadEnvelopePrestate(ctx, e) + require.NoError(t, err) +} + +func Test_notifyNewEnvelope(t *testing.T) { + service, tr := minimalTestService(t, WithPayloadIDCache(cache.NewPayloadIDCache())) + ctx, fcs := tr.ctx, tr.fcs + gs, _ := util.DeterministicGenesisStateEpbs(t, 32) + require.NoError(t, service.saveGenesisData(ctx, gs)) + require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot})) + p := &enginev1.ExecutionPayloadEnvelope{ + Payload: &enginev1.ExecutionPayloadElectra{}, + BeaconBlockRoot: service.originBlockRoot[:], + } + e, err := epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + engine := &mockExecution.EngineClient{} + service.cfg.ExecutionEngineCaller = engine + isValidPayload, err := service.notifyNewEnvelope(ctx, e) + require.NoError(t, err) + require.Equal(t, true, isValidPayload) +} + +func Test_validateExecutionOnEnvelope(t *testing.T) { + service, tr := minimalTestService(t, WithPayloadIDCache(cache.NewPayloadIDCache())) + ctx, fcs := tr.ctx, tr.fcs + gs, _ := util.DeterministicGenesisStateEpbs(t, 32) + require.NoError(t, service.saveGenesisData(ctx, gs)) + require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot})) + p := &enginev1.ExecutionPayloadEnvelope{ + Payload: &enginev1.ExecutionPayloadElectra{}, + BeaconBlockRoot: service.originBlockRoot[:], + } + e, err := epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + engine := &mockExecution.EngineClient{} + service.cfg.ExecutionEngineCaller = engine + isValidPayload, err := service.validateExecutionOnEnvelope(ctx, e) + require.NoError(t, err) + require.Equal(t, true, isValidPayload) +} + +func Test_ReceiveExecutionPayloadEnvelope(t *testing.T) { + service, tr := minimalTestService(t, WithPayloadIDCache(cache.NewPayloadIDCache())) + ctx, fcs := tr.ctx, tr.fcs + gs, _ := util.DeterministicGenesisStateEpbs(t, 32) + require.NoError(t, service.saveGenesisData(ctx, gs)) + require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot})) + post := gs.Copy() + p := &enginev1.ExecutionPayloadEnvelope{ + Payload: &enginev1.ExecutionPayloadElectra{ + ParentHash: make([]byte, 32), + BlockHash: make([]byte, 32), + }, + BeaconBlockRoot: service.originBlockRoot[:], + BlobKzgCommitments: make([][]byte, 0), + StateRoot: make([]byte, 32), + } + e, err := epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + das := &das.MockAvailabilityStore{} + + blockHeader := post.LatestBlockHeader() + prevStateRoot, err := post.HashTreeRoot(ctx) + require.NoError(t, err) + blockHeader.StateRoot = prevStateRoot[:] + require.NoError(t, post.SetLatestBlockHeader(blockHeader)) + stRoot, err := post.HashTreeRoot(ctx) + require.NoError(t, err) + p.StateRoot = stRoot[:] + engine := &mockExecution.EngineClient{} + service.cfg.ExecutionEngineCaller = engine + require.NoError(t, service.ReceiveExecutionPayloadEnvelope(ctx, e, das)) +} diff --git a/beacon-chain/blockchain/service.go b/beacon-chain/blockchain/service.go index c984a2f79750..8d7f3a97f444 100644 --- a/beacon-chain/blockchain/service.go +++ b/beacon-chain/blockchain/service.go @@ -66,6 +66,7 @@ type Service struct { syncComplete chan struct{} blobNotifiers *blobNotifierMap blockBeingSynced *currentlySyncingBlock + payloadBeingSynced *currentlySyncingPayload blobStorage *filesystem.BlobStorage lastPublishedLightClientEpoch primitives.Epoch } @@ -180,6 +181,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) { blobNotifiers: bn, cfg: &config{}, blockBeingSynced: ¤tlySyncingBlock{roots: make(map[[32]byte]struct{})}, + payloadBeingSynced: ¤tlySyncingPayload{roots: make(map[[32]byte]struct{})}, } for _, opt := range opts { if err := opt(srv); err != nil { @@ -555,7 +557,7 @@ func (s *Service) saveGenesisData(ctx context.Context, genesisState state.Beacon // 2.) Check DB. // Checking 1.) is ten times faster than checking 2.) // this function requires a lock in forkchoice -func (s *Service) hasBlock(ctx context.Context, root [32]byte) bool { +func (s *Service) chainHasBlock(ctx context.Context, root [32]byte) bool { if s.cfg.ForkChoiceStore.HasNode(root) { return true } diff --git a/beacon-chain/blockchain/service_test.go b/beacon-chain/blockchain/service_test.go index e511559ae5a4..2f255b6fc8a1 100644 --- a/beacon-chain/blockchain/service_test.go +++ b/beacon-chain/blockchain/service_test.go @@ -386,8 +386,8 @@ func TestHasBlock_ForkChoiceAndDB_DoublyLinkedTree(t *testing.T) { require.NoError(t, err) require.NoError(t, s.cfg.ForkChoiceStore.InsertNode(ctx, beaconState, roblock)) - assert.Equal(t, false, s.hasBlock(ctx, [32]byte{}), "Should not have block") - assert.Equal(t, true, s.hasBlock(ctx, r), "Should have block") + assert.Equal(t, false, s.chainHasBlock(ctx, [32]byte{}), "Should not have block") + assert.Equal(t, true, s.chainHasBlock(ctx, r), "Should have block") } func TestServiceStop_SaveCachedBlocks(t *testing.T) { diff --git a/beacon-chain/core/blocks/genesis.go b/beacon-chain/core/blocks/genesis.go index d3d687faa534..03e4f3072e2a 100644 --- a/beacon-chain/core/blocks/genesis.go +++ b/beacon-chain/core/blocks/genesis.go @@ -12,6 +12,7 @@ import ( "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks" "github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces" "github.com/prysmaticlabs/prysm/v5/encoding/bytesutil" + "github.com/prysmaticlabs/prysm/v5/encoding/ssz" enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" ) @@ -222,6 +223,42 @@ func NewGenesisBlockForState(ctx context.Context, st state.BeaconState) (interfa }, Signature: params.BeaconConfig().EmptySignature[:], }) + case *ethpb.BeaconStateEPBS: + kzgs := make([][]byte, 0) + kzgRoot, err := ssz.KzgCommitmentsRoot(kzgs) + if err != nil { + return nil, err + } + return blocks.NewSignedBeaconBlock(ðpb.SignedBeaconBlockEpbs{ + Block: ðpb.BeaconBlockEpbs{ + ParentRoot: params.BeaconConfig().ZeroHash[:], + StateRoot: root[:], + Body: ðpb.BeaconBlockBodyEpbs{ + RandaoReveal: make([]byte, 96), + Eth1Data: ðpb.Eth1Data{ + DepositRoot: make([]byte, 32), + BlockHash: make([]byte, 32), + }, + Graffiti: make([]byte, 32), + SyncAggregate: ðpb.SyncAggregate{ + SyncCommitteeBits: make([]byte, fieldparams.SyncCommitteeLength/8), + SyncCommitteeSignature: make([]byte, fieldparams.BLSSignatureLength), + }, + SignedExecutionPayloadHeader: &enginev1.SignedExecutionPayloadHeader{ + Message: &enginev1.ExecutionPayloadHeaderEPBS{ + ParentBlockHash: make([]byte, 32), + ParentBlockRoot: make([]byte, 32), + BlockHash: make([]byte, 32), + BlobKzgCommitmentsRoot: kzgRoot[:], + }, + Signature: make([]byte, 96), + }, + BlsToExecutionChanges: make([]*ethpb.SignedBLSToExecutionChange, 0), + PayloadAttestations: make([]*ethpb.PayloadAttestation, 0), + }, + }, + Signature: params.BeaconConfig().EmptySignature[:], + }) default: return nil, ErrUnrecognizedState } diff --git a/beacon-chain/core/epbs/BUILD.bazel b/beacon-chain/core/epbs/BUILD.bazel index f42848eb1b20..7e86283d47b2 100644 --- a/beacon-chain/core/epbs/BUILD.bazel +++ b/beacon-chain/core/epbs/BUILD.bazel @@ -2,18 +2,36 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test") go_library( name = "go_default_library", - srcs = ["attestation.go"], + srcs = [ + "attestation.go", + "execution_payload_envelope.go", + ], importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/epbs", visibility = ["//visibility:public"], + deps = [ + "//beacon-chain/core/electra:go_default_library", + "//beacon-chain/state:go_default_library", + "//consensus-types/interfaces:go_default_library", + "//proto/engine/v1:go_default_library", + "@com_github_pkg_errors//:go_default_library", + ], ) go_test( name = "go_default_test", - srcs = ["attestation_test.go"], + srcs = [ + "attestation_test.go", + "execution_payload_envelope_test.go", + ], + embed = [":go_default_library"], deps = [ - ":go_default_library", "//beacon-chain/core/altair:go_default_library", + "//beacon-chain/state/state-native:go_default_library", "//config/params:go_default_library", + "//consensus-types/epbs:go_default_library", + "//proto/engine/v1:go_default_library", + "//proto/prysm/v1alpha1:go_default_library", "//testing/require:go_default_library", + "//testing/util:go_default_library", ], ) diff --git a/beacon-chain/core/epbs/execution_payload_envelope.go b/beacon-chain/core/epbs/execution_payload_envelope.go new file mode 100644 index 000000000000..e36e3633bb6d --- /dev/null +++ b/beacon-chain/core/epbs/execution_payload_envelope.go @@ -0,0 +1,126 @@ +package epbs + +import ( + "context" + "fmt" + + "github.com/pkg/errors" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/electra" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/state" + "github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces" + enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" +) + +// ValidatePayloadStateTransition performs the process_execution_payload +// function. +func ValidatePayloadStateTransition( + ctx context.Context, + preState state.BeaconState, + envelope interfaces.ROExecutionPayloadEnvelope, +) error { + if err := validateAgainstHeader(ctx, preState, envelope); err != nil { + return err + } + committedHeader, err := preState.LatestExecutionPayloadHeaderEPBS() + if err != nil { + return err + } + if err := validateAgainstCommittedBid(committedHeader, envelope); err != nil { + return err + } + if err := ProcessPayloadStateTransition(ctx, preState, envelope); err != nil { + return err + } + return checkPostStateRoot(ctx, preState, envelope) +} + +func ProcessPayloadStateTransition( + ctx context.Context, + preState state.BeaconState, + envelope interfaces.ROExecutionPayloadEnvelope, +) error { + er := envelope.ExecutionRequests() + preState, err := electra.ProcessDepositRequests(ctx, preState, er.Deposits) + if err != nil { + return errors.Wrap(err, "could not process deposit receipts") + } + preState, err = electra.ProcessWithdrawalRequests(ctx, preState, er.Withdrawals) + if err != nil { + return errors.Wrap(err, "could not process ercution layer withdrawal requests") + } + if err := electra.ProcessConsolidationRequests(ctx, preState, er.Consolidations); err != nil { + return errors.Wrap(err, "could not process consolidation requests") + } + payload, err := envelope.Execution() + if err != nil { + return errors.Wrap(err, "could not get execution payload") + } + if err := preState.SetLatestBlockHash(payload.BlockHash()); err != nil { + return err + } + return preState.SetLatestFullSlot(preState.Slot()) +} + +func validateAgainstHeader( + ctx context.Context, + preState state.BeaconState, + envelope interfaces.ROExecutionPayloadEnvelope, +) error { + blockHeader := preState.LatestBlockHeader() + if blockHeader == nil { + return errors.New("invalid nil latest block header") + } + if len(blockHeader.StateRoot) == 0 || [32]byte(blockHeader.StateRoot) == [32]byte{} { + prevStateRoot, err := preState.HashTreeRoot(ctx) + if err != nil { + return errors.Wrap(err, "could not compute previous state root") + } + blockHeader.StateRoot = prevStateRoot[:] + if err := preState.SetLatestBlockHeader(blockHeader); err != nil { + return errors.Wrap(err, "could not set latest block header") + } + } + blockHeaderRoot, err := blockHeader.HashTreeRoot() + if err != nil { + return err + } + beaconBlockRoot := envelope.BeaconBlockRoot() + if blockHeaderRoot != beaconBlockRoot { + return fmt.Errorf("beacon block root does not match previous header, got: %#x wanted: %#x", beaconBlockRoot, blockHeaderRoot) + } + return nil +} + +func validateAgainstCommittedBid( + committedHeader *enginev1.ExecutionPayloadHeaderEPBS, + envelope interfaces.ROExecutionPayloadEnvelope, +) error { + builderIndex := envelope.BuilderIndex() + if committedHeader.BuilderIndex != builderIndex { + return errors.New("builder index does not match committed header") + } + kzgRoot, err := envelope.BlobKzgCommitmentsRoot() + if err != nil { + return err + } + if [32]byte(committedHeader.BlobKzgCommitmentsRoot) != kzgRoot { + return errors.New("blob KZG commitments root does not match committed header") + } + return nil +} + +func checkPostStateRoot( + ctx context.Context, + preState state.BeaconState, + envelope interfaces.ROExecutionPayloadEnvelope, +) error { + stateRoot, err := preState.HashTreeRoot(ctx) + if err != nil { + return err + } + envelopeStateRoot := envelope.StateRoot() + if stateRoot != envelopeStateRoot { + return errors.New("state root mismatch") + } + return nil +} diff --git a/beacon-chain/core/epbs/execution_payload_envelope_test.go b/beacon-chain/core/epbs/execution_payload_envelope_test.go new file mode 100644 index 000000000000..69c0391c774f --- /dev/null +++ b/beacon-chain/core/epbs/execution_payload_envelope_test.go @@ -0,0 +1,105 @@ +package epbs + +import ( + "context" + "testing" + + state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native" + consensus_epbs "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs" + enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" + ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" + "github.com/prysmaticlabs/prysm/v5/testing/require" + "github.com/prysmaticlabs/prysm/v5/testing/util" +) + +func TestProcessPayloadStateTransition(t *testing.T) { + bh := [32]byte{'h'} + payload := &enginev1.ExecutionPayloadElectra{BlockHash: bh[:]} + p := &enginev1.ExecutionPayloadEnvelope{Payload: payload} + e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + stpb := ðpb.BeaconStateEPBS{Slot: 3} + st, err := state_native.InitializeFromProtoUnsafeEpbs(stpb) + require.NoError(t, err) + ctx := context.Background() + + lbh, err := st.LatestBlockHash() + require.NoError(t, err) + require.Equal(t, [32]byte{}, [32]byte(lbh)) + + require.NoError(t, processPayloadStateTransition(ctx, st, e)) + + lbh, err = st.LatestBlockHash() + require.NoError(t, err) + require.Equal(t, bh, [32]byte(lbh)) + + lfs, err := st.LatestFullSlot() + require.NoError(t, err) + require.Equal(t, lfs, st.Slot()) +} + +func Test_validateAgainstHeader(t *testing.T) { + bh := [32]byte{'h'} + payload := &enginev1.ExecutionPayloadElectra{BlockHash: bh[:]} + p := &enginev1.ExecutionPayloadEnvelope{Payload: payload} + e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + stpb := ðpb.BeaconStateEPBS{Slot: 3} + st, err := state_native.InitializeFromProtoUnsafeEpbs(stpb) + require.NoError(t, err) + ctx := context.Background() + require.ErrorContains(t, "invalid nil latest block header", validateAgainstHeader(ctx, st, e)) + + prest, _ := util.DeterministicGenesisStateEpbs(t, 64) + require.ErrorContains(t, "attempted to wrap nil object", validateAgainstHeader(ctx, prest, e)) + + br := [32]byte{'r'} + p.BeaconBlockRoot = br[:] + require.ErrorContains(t, "beacon block root does not match previous header", validateAgainstHeader(ctx, prest, e)) + + header := prest.LatestBlockHeader() + require.NoError(t, err) + headerRoot, err := header.HashTreeRoot() + require.NoError(t, err) + p.BeaconBlockRoot = headerRoot[:] + require.NoError(t, validateAgainstHeader(ctx, prest, e)) +} + +func Test_validateAgainstCommittedBid(t *testing.T) { + payload := &enginev1.ExecutionPayloadElectra{} + p := &enginev1.ExecutionPayloadEnvelope{Payload: payload, BuilderIndex: 1} + e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + h := &enginev1.ExecutionPayloadHeaderEPBS{} + require.ErrorContains(t, "builder index does not match committed header", validateAgainstCommittedBid(h, e)) + + h.BuilderIndex = 1 + require.ErrorContains(t, "attempted to wrap nil object", validateAgainstCommittedBid(h, e)) + p.BlobKzgCommitments = make([][]byte, 6) + for i := range p.BlobKzgCommitments { + p.BlobKzgCommitments[i] = make([]byte, 48) + } + h.BlobKzgCommitmentsRoot = make([]byte, 32) + require.ErrorContains(t, "blob KZG commitments root does not match committed header", validateAgainstCommittedBid(h, e)) + + root, err := e.BlobKzgCommitmentsRoot() + require.NoError(t, err) + h.BlobKzgCommitmentsRoot = root[:] + require.NoError(t, validateAgainstCommittedBid(h, e)) +} + +func TestCheckPostStateRoot(t *testing.T) { + payload := &enginev1.ExecutionPayloadElectra{} + p := &enginev1.ExecutionPayloadEnvelope{Payload: payload, BuilderIndex: 1} + e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p) + require.NoError(t, err) + ctx := context.Background() + st, _ := util.DeterministicGenesisStateEpbs(t, 64) + require.ErrorContains(t, "attempted to wrap nil object", checkPostStateRoot(ctx, st, e)) + p.StateRoot = make([]byte, 32) + require.ErrorContains(t, "state root mismatch", checkPostStateRoot(ctx, st, e)) + root, err := st.HashTreeRoot(ctx) + require.NoError(t, err) + p.StateRoot = root[:] + require.NoError(t, checkPostStateRoot(ctx, st, e)) +} diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index 7a5698220976..14b7ee6303b8 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -188,9 +188,11 @@ go_test( "@com_github_libp2p_go_libp2p_pubsub//pb:go_default_library", "@com_github_multiformats_go_multiaddr//:go_default_library", "@com_github_pkg_errors//:go_default_library", + "@com_github_prysmaticlabs_fastssz//:go_default_library", "@com_github_prysmaticlabs_go_bitfield//:go_default_library", "@com_github_sirupsen_logrus//:go_default_library", "@com_github_sirupsen_logrus//hooks/test:go_default_library", "@org_golang_google_protobuf//proto:go_default_library", + "@org_golang_google_protobuf//reflect/protoreflect:go_default_library", ], ) diff --git a/beacon-chain/p2p/broadcaster_test.go b/beacon-chain/p2p/broadcaster_test.go index 9dbdc677aee8..643fef093952 100644 --- a/beacon-chain/p2p/broadcaster_test.go +++ b/beacon-chain/p2p/broadcaster_test.go @@ -581,7 +581,8 @@ func testBroadcast(t *testing.T, topicFormat string, msg interface{}) { incomingMessage, err := subscription.Next(ctx) require.NoError(t, err) - result := msg.(ssz.Unmarshaler) + result, ok := msg.(ssz.Unmarshaler) + require.Equal(t, true, ok) require.NoError(t, s1.Encoding().DecodeGossip(incomingMessage.Data, result)) require.DeepEqual(t, result, msg) }() diff --git a/beacon-chain/rpc/eth/events/BUILD.bazel b/beacon-chain/rpc/eth/events/BUILD.bazel index ad6b09eb30bd..498d88405fca 100644 --- a/beacon-chain/rpc/eth/events/BUILD.bazel +++ b/beacon-chain/rpc/eth/events/BUILD.bazel @@ -31,16 +31,12 @@ go_library( "//time/slots:go_default_library", "@com_github_ethereum_go_ethereum//common/hexutil:go_default_library", "@com_github_pkg_errors//:go_default_library", - "@com_github_sirupsen_logrus//:go_default_library", ], ) go_test( name = "go_default_test", - srcs = [ - "events_test.go", - "http_test.go", - ], + srcs = ["events_test.go"], embed = [":go_default_library"], deps = [ "//beacon-chain/blockchain/testing:go_default_library", @@ -55,6 +51,7 @@ go_test( "//consensus-types/primitives:go_default_library", "//proto/eth/v1:go_default_library", "//proto/prysm/v1alpha1:go_default_library", + "//testing/assert:go_default_library", "//testing/require:go_default_library", "//testing/util:go_default_library", "@com_github_ethereum_go_ethereum//common:go_default_library", diff --git a/beacon-chain/rpc/eth/events/events.go b/beacon-chain/rpc/eth/events/events.go index 92071fda9d2a..b555069a097f 100644 --- a/beacon-chain/rpc/eth/events/events.go +++ b/beacon-chain/rpc/eth/events/events.go @@ -11,6 +11,7 @@ import ( "github.com/ethereum/go-ethereum/common/hexutil" "github.com/pkg/errors" + "github.com/prysmaticlabs/prysm/v4/beacon-chain/blockchain" "github.com/prysmaticlabs/prysm/v5/api" "github.com/prysmaticlabs/prysm/v5/api/server/structs" "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed" diff --git a/beacon-chain/state/state-native/state_trie.go b/beacon-chain/state/state-native/state_trie.go index 22d930220d53..4a84e03102aa 100644 --- a/beacon-chain/state/state-native/state_trie.go +++ b/beacon-chain/state/state-native/state_trie.go @@ -943,7 +943,6 @@ func (b *BeaconState) Copy() state.BeaconState { latestExecutionPayloadHeader: b.latestExecutionPayloadHeader.Copy(), latestExecutionPayloadHeaderCapella: b.latestExecutionPayloadHeaderCapella.Copy(), latestExecutionPayloadHeaderDeneb: b.latestExecutionPayloadHeaderDeneb.Copy(), - executionPayloadHeader: b.executionPayloadHeaderVal(), latestExecutionPayloadHeaderEPBS: b.executionPayloadHeaderVal(), id: types.Enumerator.Inc(), diff --git a/beacon-chain/state/state-native/state_trie_epbs.go b/beacon-chain/state/state-native/state_trie_epbs.go index d358261cc8a2..37784be02589 100644 --- a/beacon-chain/state/state-native/state_trie_epbs.go +++ b/beacon-chain/state/state-native/state_trie_epbs.go @@ -69,11 +69,12 @@ func InitializeFromProtoUnsafeEpbs(st *ethpb.BeaconStateEPBS) (*BeaconState, err latestExecutionPayloadHeaderEPBS: st.LatestExecutionPayloadHeader, lastWithdrawalsRoot: bytesutil.ToBytes32(st.LastWithdrawalsRoot), - dirtyFields: make(map[types.FieldIndex]bool, fieldCount), - dirtyIndices: make(map[types.FieldIndex][]uint64, fieldCount), - stateFieldLeaves: make(map[types.FieldIndex]*fieldtrie.FieldTrie, fieldCount), - rebuildTrie: make(map[types.FieldIndex]bool, fieldCount), - valMapHandler: stateutil.NewValMapHandler(st.Validators), + dirtyFields: make(map[types.FieldIndex]bool, fieldCount), + dirtyIndices: make(map[types.FieldIndex][]uint64, fieldCount), + stateFieldLeaves: make(map[types.FieldIndex]*fieldtrie.FieldTrie, fieldCount), + rebuildTrie: make(map[types.FieldIndex]bool, fieldCount), + valMapHandler: stateutil.NewValMapHandler(st.Validators), + validatorIndexCache: newFinalizedValidatorIndexCache(), // only used in post-electra and only populates when finalizing, otherwise it falls back to processing the full validator set } if features.Get().EnableExperimentalState { diff --git a/beacon-chain/state/state-native/types/types.go b/beacon-chain/state/state-native/types/types.go index 7aaaceec33af..782bd56280a4 100644 --- a/beacon-chain/state/state-native/types/types.go +++ b/beacon-chain/state/state-native/types/types.go @@ -261,11 +261,7 @@ const ( LatestExecutionPayloadHeader LatestExecutionPayloadHeaderCapella LatestExecutionPayloadHeaderDeneb -<<<<<<< HEAD -======= - LatestExecutionPayloadHeaderElectra ExecutionPayloadHeader ->>>>>>> 666b5c4cdb (Remove inclusion list from epbs (#14188)) NextWithdrawalIndex NextWithdrawalValidatorIndex HistoricalSummaries diff --git a/consensus-types/blocks/getters.go b/consensus-types/blocks/getters.go index 331c95b8bfa2..6ca52fc754f8 100644 --- a/consensus-types/blocks/getters.go +++ b/consensus-types/blocks/getters.go @@ -1176,16 +1176,6 @@ func (b *BeaconBlockBody) ExecutionRequests() (*enginev1.ExecutionRequests, erro return b.executionRequests, nil } -// PayloadAttestations returns the payload attestations in the block. -func (b *BeaconBlockBody) PayloadAttestations() []*eth.PayloadAttestation { - return b.payloadAttestations -} - -// SignedExecutionPayloadHeader returns the signed execution payload header in the block. -func (b *BeaconBlockBody) SignedExecutionPayloadHeader() *enginev1.SignedExecutionPayloadHeader { - return b.signedExecutionPayloadHeader -} - // Version returns the version of the beacon block body func (b *BeaconBlockBody) Version() int { return b.version diff --git a/consensus-types/blocks/proto.go b/consensus-types/blocks/proto.go index 5805881334da..69077177ac0a 100644 --- a/consensus-types/blocks/proto.go +++ b/consensus-types/blocks/proto.go @@ -586,8 +586,8 @@ func (b *BeaconBlockBody) Proto() (proto.Message, error) { Eth1Data: b.eth1Data, Graffiti: b.graffiti[:], ProposerSlashings: b.proposerSlashings, - AttesterSlashings: b.attesterSlashings, - Attestations: b.attestations, + AttesterSlashings: b.attesterSlashingsElectra, + Attestations: b.attestationsElectra, Deposits: b.deposits, VoluntaryExits: b.voluntaryExits, SyncAggregate: b.syncAggregate, @@ -1188,8 +1188,8 @@ func initBlockBodyFromProtoEpbs(pb *eth.BeaconBlockBodyEpbs) (*BeaconBlockBody, eth1Data: pb.Eth1Data, graffiti: bytesutil.ToBytes32(pb.Graffiti), proposerSlashings: pb.ProposerSlashings, - attesterSlashings: pb.AttesterSlashings, - attestations: pb.Attestations, + attesterSlashingsElectra: pb.AttesterSlashings, + attestationsElectra: pb.Attestations, deposits: pb.Deposits, voluntaryExits: pb.VoluntaryExits, syncAggregate: pb.SyncAggregate, diff --git a/consensus-types/blocks/proto_epbs_test.go b/consensus-types/blocks/proto_epbs_test.go index 44bdd2a75d3b..5fab09336ad1 100644 --- a/consensus-types/blocks/proto_epbs_test.go +++ b/consensus-types/blocks/proto_epbs_test.go @@ -72,8 +72,8 @@ func bodyEpbs() *BeaconBlockBody { }, graffiti: f.root, proposerSlashings: f.proposerSlashings, - attesterSlashings: f.attesterSlashings, - attestations: f.atts, + attesterSlashingsElectra: f.attesterSlashingsElectra, + attestationsElectra: f.attsElectra, deposits: f.deposits, voluntaryExits: f.voluntaryExits, syncAggregate: f.syncAggregate, @@ -95,8 +95,8 @@ func bodyPbEpbs() *eth.BeaconBlockBodyEpbs { }, Graffiti: f.root[:], ProposerSlashings: f.proposerSlashings, - AttesterSlashings: f.attesterSlashings, - Attestations: f.atts, + AttesterSlashings: f.attesterSlashingsElectra, + Attestations: f.attsElectra, Deposits: f.deposits, VoluntaryExits: f.voluntaryExits, SyncAggregate: f.syncAggregate, diff --git a/consensus-types/blocks/proto_test.go b/consensus-types/blocks/proto_test.go index e35333a8542c..e98da274abdc 100644 --- a/consensus-types/blocks/proto_test.go +++ b/consensus-types/blocks/proto_test.go @@ -19,8 +19,10 @@ type fields struct { sig [96]byte deposits []*eth.Deposit atts []*eth.Attestation + attsElectra []*eth.AttestationElectra proposerSlashings []*eth.ProposerSlashing attesterSlashings []*eth.AttesterSlashing + attesterSlashingsElectra []*eth.AttesterSlashingElectra voluntaryExits []*eth.SignedVoluntaryExit syncAggregate *eth.SyncAggregate execPayload *enginev1.ExecutionPayload @@ -1552,6 +1554,26 @@ func getFields() fields { }, } } + attsElectra := make([]*eth.AttestationElectra, 8) + for i := range attsElectra { + attsElectra[i] = ð.AttestationElectra{} + attsElectra[i].Signature = sig[:] + attsElectra[i].AggregationBits = bitfield.NewBitlist(1) + attsElectra[i].CommitteeBits = primitives.NewAttestationCommitteeBits() + attsElectra[i].Data = ð.AttestationData{ + Slot: 128, + CommitteeIndex: 128, + BeaconBlockRoot: root[:], + Source: ð.Checkpoint{ + Epoch: 128, + Root: root[:], + }, + Target: ð.Checkpoint{ + Epoch: 128, + Root: root[:], + }, + } + } proposerSlashing := ð.ProposerSlashing{ Header_1: ð.SignedBeaconBlockHeader{ Header: ð.BeaconBlockHeader{ @@ -1610,6 +1632,42 @@ func getFields() fields { Signature: sig[:], }, } + attesterSlashingElectra := ð.AttesterSlashingElectra{ + Attestation_1: ð.IndexedAttestationElectra{ + AttestingIndices: []uint64{1, 2, 8}, + Data: ð.AttestationData{ + Slot: 128, + CommitteeIndex: 128, + BeaconBlockRoot: root[:], + Source: ð.Checkpoint{ + Epoch: 128, + Root: root[:], + }, + Target: ð.Checkpoint{ + Epoch: 128, + Root: root[:], + }, + }, + Signature: sig[:], + }, + Attestation_2: ð.IndexedAttestationElectra{ + AttestingIndices: []uint64{1, 2, 8}, + Data: ð.AttestationData{ + Slot: 128, + CommitteeIndex: 128, + BeaconBlockRoot: root[:], + Source: ð.Checkpoint{ + Epoch: 128, + Root: root[:], + }, + Target: ð.Checkpoint{ + Epoch: 128, + Root: root[:], + }, + }, + Signature: sig[:], + }, + } voluntaryExit := ð.SignedVoluntaryExit{ Exit: ð.VoluntaryExit{ Epoch: 128, @@ -1800,8 +1858,10 @@ func getFields() fields { sig: sig, deposits: deposits, atts: atts, + attsElectra: attsElectra, proposerSlashings: []*eth.ProposerSlashing{proposerSlashing}, attesterSlashings: []*eth.AttesterSlashing{attesterSlashing}, + attesterSlashingsElectra: []*eth.AttesterSlashingElectra{attesterSlashingElectra}, voluntaryExits: []*eth.SignedVoluntaryExit{voluntaryExit}, syncAggregate: syncAggregate, execPayload: execPayload, diff --git a/consensus-types/epbs/BUILD.bazel b/consensus-types/epbs/BUILD.bazel index 8d2f8cb29f6e..80378a3dfa43 100644 --- a/consensus-types/epbs/BUILD.bazel +++ b/consensus-types/epbs/BUILD.bazel @@ -1,12 +1,33 @@ -load("@prysm//tools/go:def.bzl", "go_library") +load("@prysm//tools/go:def.bzl", "go_library", "go_test") go_library( name = "go_default_library", - srcs = ["indexed_payload_attestation.go"], + srcs = [ + "execution_payload_envelope.go", + "indexed_payload_attestation.go", + ], importpath = "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs", visibility = ["//visibility:public"], + deps = [ + "//config/fieldparams:go_default_library", + "//consensus-types:go_default_library", + "//consensus-types/blocks:go_default_library", + "//consensus-types/interfaces:go_default_library", + "//consensus-types/primitives:go_default_library", + "//encoding/ssz:go_default_library", + "//proto/engine/v1:go_default_library", + "//proto/prysm/v1alpha1:go_default_library", + "@com_github_ethereum_go_ethereum//common:go_default_library", + ], +) + +go_test( + name = "go_default_test", + srcs = ["indexed_payload_attestation_test.go"], + embed = [":go_default_library"], deps = [ "//consensus-types/primitives:go_default_library", "//proto/prysm/v1alpha1:go_default_library", + "//testing/require:go_default_library", ], ) diff --git a/consensus-types/epbs/execution_payload_envelope.go b/consensus-types/epbs/execution_payload_envelope.go new file mode 100644 index 000000000000..ffed38a6e3e8 --- /dev/null +++ b/consensus-types/epbs/execution_payload_envelope.go @@ -0,0 +1,145 @@ +package epbs + +import ( + "github.com/ethereum/go-ethereum/common" + field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams" + consensus_types "github.com/prysmaticlabs/prysm/v5/consensus-types" + "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks" + "github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces" + "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives" + "github.com/prysmaticlabs/prysm/v5/encoding/ssz" + enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" +) + +type signedExecutionPayloadEnvelope struct { + s *enginev1.SignedExecutionPayloadEnvelope +} + +type executionPayloadEnvelope struct { + p *enginev1.ExecutionPayloadEnvelope +} + +// WrappedROSignedExecutionPayloadEnvelope is a constructor which wraps a +// protobuf signed execution payload envelope into an interface. +func WrappedROSignedExecutionPayloadEnvelope(s *enginev1.SignedExecutionPayloadEnvelope) (interfaces.ROSignedExecutionPayloadEnvelope, error) { + w := signedExecutionPayloadEnvelope{s: s} + if w.IsNil() { + return nil, consensus_types.ErrNilObjectWrapped + } + return w, nil +} + +// WrappedROExecutionPayloadEnvelope is a constructor which wraps a +// protobuf execution payload envelope into an interface. +func WrappedROExecutionPayloadEnvelope(p *enginev1.ExecutionPayloadEnvelope) (interfaces.ROExecutionPayloadEnvelope, error) { + w := executionPayloadEnvelope{p: p} + if w.IsNil() { + return nil, consensus_types.ErrNilObjectWrapped + } + return w, nil +} + +// Envelope returns the wrapped object as an interface +func (s signedExecutionPayloadEnvelope) Envelope() (interfaces.ROExecutionPayloadEnvelope, error) { + return WrappedROExecutionPayloadEnvelope(s.s.Message) +} + +// Signature returns the wrapped value +func (s signedExecutionPayloadEnvelope) Signature() ([field_params.BLSSignatureLength]byte, error) { + if s.IsNil() { + return [field_params.BLSSignatureLength]byte{}, consensus_types.ErrNilObjectWrapped + } + return [field_params.BLSSignatureLength]byte(s.s.Signature), nil +} + +// IsNil returns whether the wrapped value is nil +func (s signedExecutionPayloadEnvelope) IsNil() bool { + return s.s == nil +} + +// IsNil returns whether the wrapped value is nil +func (p executionPayloadEnvelope) IsNil() bool { + return p.p == nil +} + +// IsBlinded returns whether the wrapped value is blinded +func (p executionPayloadEnvelope) IsBlinded() bool { + return !p.IsNil() && p.p.Payload == nil +} + +// Execution returns the wrapped payload as an interface +func (p executionPayloadEnvelope) Execution() (interfaces.ExecutionData, error) { + if p.IsBlinded() { + return nil, consensus_types.ErrNilObjectWrapped + } + return blocks.WrappedExecutionPayloadElectra(p.p.Payload) +} + +// BuilderIndex returns the wrapped value +func (p executionPayloadEnvelope) BuilderIndex() (primitives.ValidatorIndex, error) { + if p.IsNil() { + return 0, consensus_types.ErrNilObjectWrapped + } + return p.p.BuilderIndex, nil +} + +// BeaconBlockRoot returns the wrapped value +func (p executionPayloadEnvelope) BeaconBlockRoot() ([field_params.RootLength]byte, error) { + if p.IsNil() || len(p.p.BeaconBlockRoot) == 0 { + return [field_params.RootLength]byte{}, consensus_types.ErrNilObjectWrapped + } + return [field_params.RootLength]byte(p.p.BeaconBlockRoot), nil +} + +// BlobKzgCommitments returns the wrapped value +func (p executionPayloadEnvelope) BlobKzgCommitments() ([][]byte, error) { + if p.IsNil() { + return nil, consensus_types.ErrNilObjectWrapped + } + commitments := make([][]byte, len(p.p.BlobKzgCommitments)) + for i, commit := range p.p.BlobKzgCommitments { + commitments[i] = make([]byte, len(commit)) + copy(commitments[i], commit) + } + return commitments, nil +} + +// PayloadWithheld returns the wrapped value +func (p executionPayloadEnvelope) PayloadWithheld() (bool, error) { + if p.IsBlinded() { + return false, consensus_types.ErrNilObjectWrapped + } + return p.p.PayloadWithheld, nil +} + +// StateRoot returns the wrapped value +func (p executionPayloadEnvelope) StateRoot() ([field_params.RootLength]byte, error) { + if p.IsNil() || len(p.p.StateRoot) == 0 { + return [field_params.RootLength]byte{}, consensus_types.ErrNilObjectWrapped + } + return [field_params.RootLength]byte(p.p.StateRoot), nil +} + +// VersionedHashes returns the Versioned Hashes of the KZG commitments within +// the envelope +func (p executionPayloadEnvelope) VersionedHashes() ([]common.Hash, error) { + if p.IsNil() { + return nil, consensus_types.ErrNilObjectWrapped + } + + commitments := p.p.BlobKzgCommitments + versionedHashes := make([]common.Hash, len(commitments)) + for i, commitment := range commitments { + versionedHashes[i] = primitives.ConvertKzgCommitmentToVersionedHash(commitment) + } + return versionedHashes, nil +} + +// BlobKzgCommitmentsRoot returns the HTR of the KZG commitments in the payload +func (p executionPayloadEnvelope) BlobKzgCommitmentsRoot() ([field_params.RootLength]byte, error) { + if p.IsNil() || p.p.BlobKzgCommitments == nil { + return [field_params.RootLength]byte{}, consensus_types.ErrNilObjectWrapped + } + + return ssz.KzgCommitmentsRoot(p.p.BlobKzgCommitments) +} diff --git a/consensus-types/interfaces/BUILD.bazel b/consensus-types/interfaces/BUILD.bazel index 51b9c89816c0..c3614b3a17bc 100644 --- a/consensus-types/interfaces/BUILD.bazel +++ b/consensus-types/interfaces/BUILD.bazel @@ -6,6 +6,7 @@ go_library( "beacon_block.go", "error.go", "light_client.go", + "execution_payload_envelope.go", "utils.go", "validator.go", ], @@ -18,6 +19,7 @@ go_library( "//proto/prysm/v1alpha1:go_default_library", "//proto/prysm/v1alpha1/validator-client:go_default_library", "//runtime/version:go_default_library", + "@com_github_ethereum_go_ethereum//common:go_default_library", "@com_github_pkg_errors//:go_default_library", "@com_github_prysmaticlabs_fastssz//:go_default_library", "@org_golang_google_protobuf//proto:go_default_library", diff --git a/consensus-types/interfaces/execution_payload_envelope.go b/consensus-types/interfaces/execution_payload_envelope.go new file mode 100644 index 000000000000..9d8d5a147639 --- /dev/null +++ b/consensus-types/interfaces/execution_payload_envelope.go @@ -0,0 +1,31 @@ +package interfaces + +import ( + "github.com/ethereum/go-ethereum/common" + field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams" + "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives" + enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" +) + +type ROSignedExecutionPayloadEnvelope interface { + Envelope() (ROExecutionPayloadEnvelope, error) + Signature() [field_params.BLSSignatureLength]byte + SigningRoot([]byte) ([32]byte, error) + IsNil() bool +} + +type ROExecutionPayloadEnvelope interface { + Execution() (ExecutionData, error) + ExecutionRequests() *enginev1.ExecutionRequests + BuilderIndex() primitives.ValidatorIndex + BeaconBlockRoot() [field_params.RootLength]byte + BlobKzgCommitments() [][]byte + BlobKzgCommitmentsRoot() ([field_params.RootLength]byte, error) + VersionedHashes() []common.Hash + PayloadWithheld() bool + StateRoot() [field_params.RootLength]byte + SetSlot(primitives.Slot) + Slot() primitives.Slot + IsBlinded() bool + IsNil() bool +} diff --git a/consensus-types/mock/BUILD.bazel b/consensus-types/mock/BUILD.bazel index 935d1f435738..e7ce82febd6f 100644 --- a/consensus-types/mock/BUILD.bazel +++ b/consensus-types/mock/BUILD.bazel @@ -11,8 +11,6 @@ go_library( "//consensus-types/primitives:go_default_library", "//proto/engine/v1:go_default_library", "//proto/eth/v1:go_default_library", - "//proto/engine/v1:go_default_library", - "//proto/eth/v1:go_default_library", "//proto/prysm/v1alpha1:go_default_library", "//proto/prysm/v1alpha1/validator-client:go_default_library", "@com_github_prysmaticlabs_fastssz//:go_default_library", diff --git a/encoding/ssz/htrutils.go b/encoding/ssz/htrutils.go index d0581d47e809..d3a6c5e4a4a0 100644 --- a/encoding/ssz/htrutils.go +++ b/encoding/ssz/htrutils.go @@ -5,6 +5,7 @@ import ( "encoding/binary" "github.com/pkg/errors" + "github.com/prysmaticlabs/gohashtree" fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams" "github.com/prysmaticlabs/prysm/v5/encoding/bytesutil" enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" @@ -116,6 +117,31 @@ func TransactionsRoot(txs [][]byte) ([32]byte, error) { return MixInLength(bytesRoot, bytesRootBufRoot), nil } +// KzgCommitmentsRoot computes the HTR for a list of KZG commitments +func KzgCommitmentsRoot(commitments [][]byte) ([32]byte, error) { + hash := [32]byte{} + leaves := make([][32]byte, 2*len(commitments)) + for i, kzg := range commitments { + copy(leaves[2*i][:], kzg[:32]) + copy(leaves[2*i+1][:], kzg[32:]) + } + if err := gohashtree.Hash(leaves, leaves); err != nil { + return hash, err + } + + bytesRoot, err := BitwiseMerkleize(leaves[:len(commitments)], uint64(len(commitments)), fieldparams.MaxBlobCommitmentsPerBlock) + if err != nil { + return [32]byte{}, errors.Wrap(err, "could not compute merkleization") + } + chunks := make([][32]byte, 2) + chunks[0] = bytesRoot + binary.LittleEndian.PutUint64(chunks[1][24:], uint64(len(commitments))) + if err := gohashtree.Hash(chunks, chunks); err != nil { + return hash, err + } + return chunks[0], nil +} + // WithdrawalSliceRoot computes the HTR of a slice of withdrawals. // The limit parameter is used as input to the bitwise merkleization algorithm. func WithdrawalSliceRoot(withdrawals []*enginev1.Withdrawal, limit uint64) ([32]byte, error) { diff --git a/proto/engine/v1/epbs.pb.go b/proto/engine/v1/epbs.pb.go index d500ec13de5a..45809d30881f 100755 --- a/proto/engine/v1/epbs.pb.go +++ b/proto/engine/v1/epbs.pb.go @@ -23,187 +23,6 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -type InclusionListSummary struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - ProposerIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,1,opt,name=proposer_index,json=proposerIndex,proto3" json:"proposer_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - Slot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,2,opt,name=slot,proto3" json:"slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - Summary [][]byte `protobuf:"bytes,3,rep,name=summary,proto3" json:"summary,omitempty" ssz-max:"1024" ssz-size:"?,20"` -} - -func (x *InclusionListSummary) Reset() { - *x = InclusionListSummary{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *InclusionListSummary) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*InclusionListSummary) ProtoMessage() {} - -func (x *InclusionListSummary) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_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 InclusionListSummary.ProtoReflect.Descriptor instead. -func (*InclusionListSummary) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{0} -} - -func (x *InclusionListSummary) GetProposerIndex() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex { - if x != nil { - return x.ProposerIndex - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(0) -} - -func (x *InclusionListSummary) GetSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { - if x != nil { - return x.Slot - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0) -} - -func (x *InclusionListSummary) GetSummary() [][]byte { - if x != nil { - return x.Summary - } - return nil -} - -type SignedInclusionListSummary struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Message *InclusionListSummary `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` - Signature []byte `protobuf:"bytes,2,opt,name=signature,proto3" json:"signature,omitempty" ssz-size:"96"` -} - -func (x *SignedInclusionListSummary) Reset() { - *x = SignedInclusionListSummary{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *SignedInclusionListSummary) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*SignedInclusionListSummary) ProtoMessage() {} - -func (x *SignedInclusionListSummary) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use SignedInclusionListSummary.ProtoReflect.Descriptor instead. -func (*SignedInclusionListSummary) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{1} -} - -func (x *SignedInclusionListSummary) GetMessage() *InclusionListSummary { - if x != nil { - return x.Message - } - return nil -} - -func (x *SignedInclusionListSummary) GetSignature() []byte { - if x != nil { - return x.Signature - } - return nil -} - -type InclusionList struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - SignedSummary *SignedInclusionListSummary `protobuf:"bytes,1,opt,name=signed_summary,json=signedSummary,proto3" json:"signed_summary,omitempty"` - ParentBlockHash []byte `protobuf:"bytes,2,opt,name=parent_block_hash,json=parentBlockHash,proto3" json:"parent_block_hash,omitempty" ssz-size:"32"` - Transactions [][]byte `protobuf:"bytes,3,rep,name=transactions,proto3" json:"transactions,omitempty" ssz-max:"1024,1073741824" ssz-size:"?,?"` -} - -func (x *InclusionList) Reset() { - *x = InclusionList{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *InclusionList) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*InclusionList) ProtoMessage() {} - -func (x *InclusionList) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use InclusionList.ProtoReflect.Descriptor instead. -func (*InclusionList) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{2} -} - -func (x *InclusionList) GetSignedSummary() *SignedInclusionListSummary { - if x != nil { - return x.SignedSummary - } - return nil -} - -func (x *InclusionList) GetParentBlockHash() []byte { - if x != nil { - return x.ParentBlockHash - } - return nil -} - -func (x *InclusionList) GetTransactions() [][]byte { - if x != nil { - return x.Transactions - } - return nil -} - type ExecutionPayloadHeaderEPBS struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -212,16 +31,17 @@ type ExecutionPayloadHeaderEPBS struct { ParentBlockHash []byte `protobuf:"bytes,1,opt,name=parent_block_hash,json=parentBlockHash,proto3" json:"parent_block_hash,omitempty" ssz-size:"32"` ParentBlockRoot []byte `protobuf:"bytes,2,opt,name=parent_block_root,json=parentBlockRoot,proto3" json:"parent_block_root,omitempty" ssz-size:"32"` BlockHash []byte `protobuf:"bytes,3,opt,name=block_hash,json=blockHash,proto3" json:"block_hash,omitempty" ssz-size:"32"` - BuilderIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,4,opt,name=builder_index,json=builderIndex,proto3" json:"builder_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - Slot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,5,opt,name=slot,proto3" json:"slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - Value uint64 `protobuf:"varint,6,opt,name=value,proto3" json:"value,omitempty"` - BlobKzgCommitmentsRoot []byte `protobuf:"bytes,7,opt,name=blob_kzg_commitments_root,json=blobKzgCommitmentsRoot,proto3" json:"blob_kzg_commitments_root,omitempty" ssz-size:"32"` + GasLimit uint64 `protobuf:"varint,4,opt,name=gas_limit,json=gasLimit,proto3" json:"gas_limit,omitempty"` + BuilderIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,5,opt,name=builder_index,json=builderIndex,proto3" json:"builder_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` + Slot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,6,opt,name=slot,proto3" json:"slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` + Value uint64 `protobuf:"varint,7,opt,name=value,proto3" json:"value,omitempty"` + BlobKzgCommitmentsRoot []byte `protobuf:"bytes,8,opt,name=blob_kzg_commitments_root,json=blobKzgCommitmentsRoot,proto3" json:"blob_kzg_commitments_root,omitempty" ssz-size:"32"` } func (x *ExecutionPayloadHeaderEPBS) Reset() { *x = ExecutionPayloadHeaderEPBS{} if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[3] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -234,7 +54,7 @@ func (x *ExecutionPayloadHeaderEPBS) String() string { func (*ExecutionPayloadHeaderEPBS) ProtoMessage() {} func (x *ExecutionPayloadHeaderEPBS) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[3] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -247,7 +67,7 @@ func (x *ExecutionPayloadHeaderEPBS) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecutionPayloadHeaderEPBS.ProtoReflect.Descriptor instead. func (*ExecutionPayloadHeaderEPBS) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{3} + return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{0} } func (x *ExecutionPayloadHeaderEPBS) GetParentBlockHash() []byte { @@ -271,6 +91,13 @@ func (x *ExecutionPayloadHeaderEPBS) GetBlockHash() []byte { return nil } +func (x *ExecutionPayloadHeaderEPBS) GetGasLimit() uint64 { + if x != nil { + return x.GasLimit + } + return 0 +} + func (x *ExecutionPayloadHeaderEPBS) GetBuilderIndex() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex { if x != nil { return x.BuilderIndex @@ -299,189 +126,6 @@ func (x *ExecutionPayloadHeaderEPBS) GetBlobKzgCommitmentsRoot() []byte { return nil } -type ExecutionPayloadEPBS struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - ParentHash []byte `protobuf:"bytes,1,opt,name=parent_hash,json=parentHash,proto3" json:"parent_hash,omitempty" ssz-size:"32"` - FeeRecipient []byte `protobuf:"bytes,2,opt,name=fee_recipient,json=feeRecipient,proto3" json:"fee_recipient,omitempty" ssz-size:"20"` - StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"` - ReceiptsRoot []byte `protobuf:"bytes,4,opt,name=receipts_root,json=receiptsRoot,proto3" json:"receipts_root,omitempty" ssz-size:"32"` - LogsBloom []byte `protobuf:"bytes,5,opt,name=logs_bloom,json=logsBloom,proto3" json:"logs_bloom,omitempty" ssz-size:"256"` - PrevRandao []byte `protobuf:"bytes,6,opt,name=prev_randao,json=prevRandao,proto3" json:"prev_randao,omitempty" ssz-size:"32"` - BlockNumber uint64 `protobuf:"varint,7,opt,name=block_number,json=blockNumber,proto3" json:"block_number,omitempty"` - GasLimit uint64 `protobuf:"varint,8,opt,name=gas_limit,json=gasLimit,proto3" json:"gas_limit,omitempty"` - GasUsed uint64 `protobuf:"varint,9,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"` - Timestamp uint64 `protobuf:"varint,10,opt,name=timestamp,proto3" json:"timestamp,omitempty"` - ExtraData []byte `protobuf:"bytes,11,opt,name=extra_data,json=extraData,proto3" json:"extra_data,omitempty" ssz-max:"32"` - BaseFeePerGas []byte `protobuf:"bytes,12,opt,name=base_fee_per_gas,json=baseFeePerGas,proto3" json:"base_fee_per_gas,omitempty" ssz-size:"32"` - BlockHash []byte `protobuf:"bytes,13,opt,name=block_hash,json=blockHash,proto3" json:"block_hash,omitempty" ssz-size:"32"` - Transactions [][]byte `protobuf:"bytes,14,rep,name=transactions,proto3" json:"transactions,omitempty" ssz-max:"1048576,1073741824" ssz-size:"?,?"` - Withdrawals []*Withdrawal `protobuf:"bytes,15,rep,name=withdrawals,proto3" json:"withdrawals,omitempty" ssz-max:"16"` - BlobGasUsed uint64 `protobuf:"varint,16,opt,name=blob_gas_used,json=blobGasUsed,proto3" json:"blob_gas_used,omitempty"` - ExcessBlobGas uint64 `protobuf:"varint,17,opt,name=excess_blob_gas,json=excessBlobGas,proto3" json:"excess_blob_gas,omitempty"` - InclusionListSummary [][]byte `protobuf:"bytes,18,rep,name=inclusion_list_summary,json=inclusionListSummary,proto3" json:"inclusion_list_summary,omitempty" ssz-max:"1024" ssz-size:"?,20"` -} - -func (x *ExecutionPayloadEPBS) Reset() { - *x = ExecutionPayloadEPBS{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ExecutionPayloadEPBS) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ExecutionPayloadEPBS) ProtoMessage() {} - -func (x *ExecutionPayloadEPBS) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[4] - 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 ExecutionPayloadEPBS.ProtoReflect.Descriptor instead. -func (*ExecutionPayloadEPBS) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{4} -} - -func (x *ExecutionPayloadEPBS) GetParentHash() []byte { - if x != nil { - return x.ParentHash - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetFeeRecipient() []byte { - if x != nil { - return x.FeeRecipient - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetStateRoot() []byte { - if x != nil { - return x.StateRoot - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetReceiptsRoot() []byte { - if x != nil { - return x.ReceiptsRoot - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetLogsBloom() []byte { - if x != nil { - return x.LogsBloom - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetPrevRandao() []byte { - if x != nil { - return x.PrevRandao - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetBlockNumber() uint64 { - if x != nil { - return x.BlockNumber - } - return 0 -} - -func (x *ExecutionPayloadEPBS) GetGasLimit() uint64 { - if x != nil { - return x.GasLimit - } - return 0 -} - -func (x *ExecutionPayloadEPBS) GetGasUsed() uint64 { - if x != nil { - return x.GasUsed - } - return 0 -} - -func (x *ExecutionPayloadEPBS) GetTimestamp() uint64 { - if x != nil { - return x.Timestamp - } - return 0 -} - -func (x *ExecutionPayloadEPBS) GetExtraData() []byte { - if x != nil { - return x.ExtraData - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetBaseFeePerGas() []byte { - if x != nil { - return x.BaseFeePerGas - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetBlockHash() []byte { - if x != nil { - return x.BlockHash - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetTransactions() [][]byte { - if x != nil { - return x.Transactions - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetWithdrawals() []*Withdrawal { - if x != nil { - return x.Withdrawals - } - return nil -} - -func (x *ExecutionPayloadEPBS) GetBlobGasUsed() uint64 { - if x != nil { - return x.BlobGasUsed - } - return 0 -} - -func (x *ExecutionPayloadEPBS) GetExcessBlobGas() uint64 { - if x != nil { - return x.ExcessBlobGas - } - return 0 -} - -func (x *ExecutionPayloadEPBS) GetInclusionListSummary() [][]byte { - if x != nil { - return x.InclusionListSummary - } - return nil -} - type SignedExecutionPayloadHeader struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -494,7 +138,7 @@ type SignedExecutionPayloadHeader struct { func (x *SignedExecutionPayloadHeader) Reset() { *x = SignedExecutionPayloadHeader{} if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[5] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -507,7 +151,7 @@ func (x *SignedExecutionPayloadHeader) String() string { func (*SignedExecutionPayloadHeader) ProtoMessage() {} func (x *SignedExecutionPayloadHeader) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[5] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -520,7 +164,7 @@ func (x *SignedExecutionPayloadHeader) ProtoReflect() protoreflect.Message { // Deprecated: Use SignedExecutionPayloadHeader.ProtoReflect.Descriptor instead. func (*SignedExecutionPayloadHeader) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{5} + return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{1} } func (x *SignedExecutionPayloadHeader) GetMessage() *ExecutionPayloadHeaderEPBS { @@ -542,21 +186,18 @@ type ExecutionPayloadEnvelope struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Payload *ExecutionPayloadEPBS `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"` - BuilderIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,2,opt,name=builder_index,json=builderIndex,proto3" json:"builder_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - BeaconBlockRoot []byte `protobuf:"bytes,3,opt,name=beacon_block_root,json=beaconBlockRoot,proto3" json:"beacon_block_root,omitempty" ssz-size:"32"` - BlobKzgCommitments [][]byte `protobuf:"bytes,4,rep,name=blob_kzg_commitments,json=blobKzgCommitments,proto3" json:"blob_kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"` - InclusionListProposerIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,5,opt,name=inclusion_list_proposer_index,json=inclusionListProposerIndex,proto3" json:"inclusion_list_proposer_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - InclusionListSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,6,opt,name=inclusion_list_slot,json=inclusionListSlot,proto3" json:"inclusion_list_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - InclusionListSignature []byte `protobuf:"bytes,7,opt,name=inclusion_list_signature,json=inclusionListSignature,proto3" json:"inclusion_list_signature,omitempty" ssz-size:"96"` - PayloadWithheld bool `protobuf:"varint,8,opt,name=payload_withheld,json=payloadWithheld,proto3" json:"payload_withheld,omitempty"` - StateRoot []byte `protobuf:"bytes,9,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"` + Payload *ExecutionPayloadDeneb `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"` + BuilderIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,2,opt,name=builder_index,json=builderIndex,proto3" json:"builder_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` + BeaconBlockRoot []byte `protobuf:"bytes,3,opt,name=beacon_block_root,json=beaconBlockRoot,proto3" json:"beacon_block_root,omitempty" ssz-size:"32"` + BlobKzgCommitments [][]byte `protobuf:"bytes,4,rep,name=blob_kzg_commitments,json=blobKzgCommitments,proto3" json:"blob_kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"` + PayloadWithheld bool `protobuf:"varint,8,opt,name=payload_withheld,json=payloadWithheld,proto3" json:"payload_withheld,omitempty"` + StateRoot []byte `protobuf:"bytes,9,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"` } func (x *ExecutionPayloadEnvelope) Reset() { *x = ExecutionPayloadEnvelope{} if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[6] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -569,7 +210,7 @@ func (x *ExecutionPayloadEnvelope) String() string { func (*ExecutionPayloadEnvelope) ProtoMessage() {} func (x *ExecutionPayloadEnvelope) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[6] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -582,10 +223,10 @@ func (x *ExecutionPayloadEnvelope) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecutionPayloadEnvelope.ProtoReflect.Descriptor instead. func (*ExecutionPayloadEnvelope) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{6} + return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{2} } -func (x *ExecutionPayloadEnvelope) GetPayload() *ExecutionPayloadEPBS { +func (x *ExecutionPayloadEnvelope) GetPayload() *ExecutionPayloadDeneb { if x != nil { return x.Payload } @@ -613,27 +254,6 @@ func (x *ExecutionPayloadEnvelope) GetBlobKzgCommitments() [][]byte { return nil } -func (x *ExecutionPayloadEnvelope) GetInclusionListProposerIndex() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex { - if x != nil { - return x.InclusionListProposerIndex - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(0) -} - -func (x *ExecutionPayloadEnvelope) GetInclusionListSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { - if x != nil { - return x.InclusionListSlot - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0) -} - -func (x *ExecutionPayloadEnvelope) GetInclusionListSignature() []byte { - if x != nil { - return x.InclusionListSignature - } - return nil -} - func (x *ExecutionPayloadEnvelope) GetPayloadWithheld() bool { if x != nil { return x.PayloadWithheld @@ -660,7 +280,7 @@ type SignedExecutionPayloadEnvelope struct { func (x *SignedExecutionPayloadEnvelope) Reset() { *x = SignedExecutionPayloadEnvelope{} if protoimpl.UnsafeEnabled { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[7] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -673,7 +293,7 @@ func (x *SignedExecutionPayloadEnvelope) String() string { func (*SignedExecutionPayloadEnvelope) ProtoMessage() {} func (x *SignedExecutionPayloadEnvelope) ProtoReflect() protoreflect.Message { - mi := &file_proto_engine_v1_epbs_proto_msgTypes[7] + mi := &file_proto_engine_v1_epbs_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -686,7 +306,7 @@ func (x *SignedExecutionPayloadEnvelope) ProtoReflect() protoreflect.Message { // Deprecated: Use SignedExecutionPayloadEnvelope.ProtoReflect.Descriptor instead. func (*SignedExecutionPayloadEnvelope) Descriptor() ([]byte, []int) { - return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{7} + return file_proto_engine_v1_epbs_proto_rawDescGZIP(), []int{3} } func (x *SignedExecutionPayloadEnvelope) GetMessage() *ExecutionPayloadEnvelope { @@ -713,202 +333,92 @@ var file_proto_engine_v1_epbs_proto_rawDesc = []byte{ 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, 0x95, 0x02, 0x0a, 0x14, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, - 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x76, - 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, - 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, - 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, - 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, - 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, - 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xf2, 0x03, 0x0a, 0x1a, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x45, 0x50, 0x42, 0x53, 0x12, 0x32, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, + 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x42, + 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x32, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x25, 0x0a, 0x0a, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, + 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, + 0x61, 0x73, 0x68, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, + 0x12, 0x74, 0x0a, 0x0d, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, + 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, + 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, + 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0c, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, + 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, - 0x74, 0x12, 0x2a, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x03, - 0x28, 0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x32, 0x30, 0x92, 0xb5, 0x18, 0x04, - 0x31, 0x30, 0x32, 0x34, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x86, 0x01, - 0x0a, 0x1a, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, - 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x42, 0x0a, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, - 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, - 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, - 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 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, 0xda, 0x01, 0x0a, 0x0d, 0x49, 0x6e, 0x63, 0x6c, 0x75, - 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x55, 0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e, - 0x65, 0x64, 0x5f, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, - 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x49, 0x6e, 0x63, 0x6c, - 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, - 0x52, 0x0d, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, - 0x32, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, - 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, - 0x33, 0x32, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, - 0x61, 0x73, 0x68, 0x12, 0x3e, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x1a, 0x8a, 0xb5, 0x18, 0x03, 0x3f, - 0x2c, 0x3f, 0x92, 0xb5, 0x18, 0x0f, 0x31, 0x30, 0x32, 0x34, 0x2c, 0x31, 0x30, 0x37, 0x33, 0x37, - 0x34, 0x31, 0x38, 0x32, 0x34, 0x52, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x22, 0xd5, 0x03, 0x0a, 0x1a, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x50, - 0x42, 0x53, 0x12, 0x32, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x6c, 0x6f, - 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, - 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x32, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, - 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x65, 0x6e, - 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, - 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, - 0x68, 0x12, 0x74, 0x0a, 0x0d, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, - 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, - 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, - 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0c, 0x62, 0x75, 0x69, 0x6c, 0x64, - 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x59, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, - 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, - 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, - 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, - 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, - 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x41, 0x0a, 0x19, 0x62, 0x6c, 0x6f, 0x62, - 0x5f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, - 0x02, 0x33, 0x32, 0x52, 0x16, 0x62, 0x6c, 0x6f, 0x62, 0x4b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, - 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xaa, 0x06, 0x0a, 0x14, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, - 0x45, 0x50, 0x42, 0x53, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, - 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, - 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x2b, 0x0a, - 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x0c, 0x66, 0x65, - 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, - 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, - 0x74, 0x12, 0x2b, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x5f, 0x72, 0x6f, - 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, - 0x52, 0x0c, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, - 0x0a, 0x0a, 0x6c, 0x6f, 0x67, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0c, 0x42, 0x07, 0x8a, 0xb5, 0x18, 0x03, 0x32, 0x35, 0x36, 0x52, 0x09, 0x6c, 0x6f, 0x67, - 0x73, 0x42, 0x6c, 0x6f, 0x6f, 0x6d, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, - 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, - 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, - 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, - 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, - 0x19, 0x0a, 0x08, 0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x07, 0x67, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, - 0x61, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x92, 0xb5, - 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x65, 0x78, 0x74, 0x72, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, - 0x2f, 0x0a, 0x10, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, - 0x67, 0x61, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, - 0x32, 0x52, 0x0d, 0x62, 0x61, 0x73, 0x65, 0x46, 0x65, 0x65, 0x50, 0x65, 0x72, 0x47, 0x61, 0x73, - 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, - 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x41, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x1d, 0x8a, - 0xb5, 0x18, 0x03, 0x3f, 0x2c, 0x3f, 0x92, 0xb5, 0x18, 0x12, 0x31, 0x30, 0x34, 0x38, 0x35, 0x37, - 0x36, 0x2c, 0x31, 0x30, 0x37, 0x33, 0x37, 0x34, 0x31, 0x38, 0x32, 0x34, 0x52, 0x0c, 0x74, 0x72, - 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x48, 0x0a, 0x0b, 0x77, 0x69, - 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x1e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x42, - 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, - 0x77, 0x61, 0x6c, 0x73, 0x12, 0x22, 0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, - 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, - 0x62, 0x47, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x78, 0x63, 0x65, - 0x73, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, - 0x12, 0x46, 0x0a, 0x16, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, - 0x73, 0x74, 0x5f, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x12, 0x20, 0x03, 0x28, 0x0c, - 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x32, 0x30, 0x92, 0xb5, 0x18, 0x04, 0x31, 0x30, - 0x32, 0x34, 0x52, 0x14, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, - 0x74, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x8e, 0x01, 0x0a, 0x1c, 0x53, 0x69, 0x67, - 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x48, 0x0a, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, - 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x50, 0x42, 0x53, 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, 0xec, 0x05, 0x0a, 0x18, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6e, - 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 0x12, 0x42, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 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, 0x45, 0x50, 0x42, - 0x53, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x74, 0x0a, 0x0d, 0x62, 0x75, - 0x69, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, - 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, - 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, - 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, - 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x52, 0x0c, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x12, 0x32, 0x0a, 0x11, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, - 0x02, 0x33, 0x32, 0x52, 0x0f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, - 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x42, 0x0a, 0x14, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x6b, 0x7a, 0x67, - 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 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, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x4b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x92, 0x01, 0x0a, 0x1d, 0x69, 0x6e, 0x63, - 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x70, - 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, - 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, - 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, - 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, - 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x52, 0x1a, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x75, 0x0a, - 0x13, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, - 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, - 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, - 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, - 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, - 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, - 0x74, 0x52, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, - 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x40, 0x0a, 0x18, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, - 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x16, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x57, 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, - 0x64, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, - 0x09, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x8e, 0x01, 0x0a, 0x1e, 0x53, 0x69, 0x67, - 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 0x12, 0x46, 0x0a, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x65, + 0x74, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x41, 0x0a, 0x19, 0x62, 0x6c, 0x6f, 0x62, 0x5f, + 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, + 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, + 0x33, 0x32, 0x52, 0x16, 0x62, 0x6c, 0x6f, 0x62, 0x4b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, + 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x8e, 0x01, 0x0a, 0x1c, 0x53, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x48, 0x0a, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 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, 0x42, 0x96, 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, 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, + 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x50, 0x42, 0x53, 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, 0x9f, 0x03, 0x0a, 0x18, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 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, 0x74, 0x0a, + 0x0d, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, + 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, + 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, + 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, + 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0c, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x12, 0x32, 0x0a, 0x11, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, + 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, + 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x42, 0x0a, 0x14, 0x62, 0x6c, 0x6f, 0x62, 0x5f, + 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 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, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x4b, 0x7a, 0x67, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x70, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x57, 0x69, + 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, + 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, + 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x8e, 0x01, + 0x0a, 0x1e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, + 0x12, 0x46, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2c, 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, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 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, 0x42, 0x96, + 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, 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 ( @@ -923,30 +433,23 @@ func file_proto_engine_v1_epbs_proto_rawDescGZIP() []byte { return file_proto_engine_v1_epbs_proto_rawDescData } -var file_proto_engine_v1_epbs_proto_msgTypes = make([]protoimpl.MessageInfo, 8) +var file_proto_engine_v1_epbs_proto_msgTypes = make([]protoimpl.MessageInfo, 4) var file_proto_engine_v1_epbs_proto_goTypes = []interface{}{ - (*InclusionListSummary)(nil), // 0: ethereum.engine.v1.InclusionListSummary - (*SignedInclusionListSummary)(nil), // 1: ethereum.engine.v1.SignedInclusionListSummary - (*InclusionList)(nil), // 2: ethereum.engine.v1.InclusionList - (*ExecutionPayloadHeaderEPBS)(nil), // 3: ethereum.engine.v1.ExecutionPayloadHeaderEPBS - (*ExecutionPayloadEPBS)(nil), // 4: ethereum.engine.v1.ExecutionPayloadEPBS - (*SignedExecutionPayloadHeader)(nil), // 5: ethereum.engine.v1.SignedExecutionPayloadHeader - (*ExecutionPayloadEnvelope)(nil), // 6: ethereum.engine.v1.ExecutionPayloadEnvelope - (*SignedExecutionPayloadEnvelope)(nil), // 7: ethereum.engine.v1.SignedExecutionPayloadEnvelope - (*Withdrawal)(nil), // 8: ethereum.engine.v1.Withdrawal + (*ExecutionPayloadHeaderEPBS)(nil), // 0: ethereum.engine.v1.ExecutionPayloadHeaderEPBS + (*SignedExecutionPayloadHeader)(nil), // 1: ethereum.engine.v1.SignedExecutionPayloadHeader + (*ExecutionPayloadEnvelope)(nil), // 2: ethereum.engine.v1.ExecutionPayloadEnvelope + (*SignedExecutionPayloadEnvelope)(nil), // 3: ethereum.engine.v1.SignedExecutionPayloadEnvelope + (*ExecutionPayloadDeneb)(nil), // 4: ethereum.engine.v1.ExecutionPayloadDeneb } var file_proto_engine_v1_epbs_proto_depIdxs = []int32{ - 0, // 0: ethereum.engine.v1.SignedInclusionListSummary.message:type_name -> ethereum.engine.v1.InclusionListSummary - 1, // 1: ethereum.engine.v1.InclusionList.signed_summary:type_name -> ethereum.engine.v1.SignedInclusionListSummary - 8, // 2: ethereum.engine.v1.ExecutionPayloadEPBS.withdrawals:type_name -> ethereum.engine.v1.Withdrawal - 3, // 3: ethereum.engine.v1.SignedExecutionPayloadHeader.message:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderEPBS - 4, // 4: ethereum.engine.v1.ExecutionPayloadEnvelope.payload:type_name -> ethereum.engine.v1.ExecutionPayloadEPBS - 6, // 5: ethereum.engine.v1.SignedExecutionPayloadEnvelope.message:type_name -> ethereum.engine.v1.ExecutionPayloadEnvelope - 6, // [6:6] is the sub-list for method output_type - 6, // [6:6] is the sub-list for method input_type - 6, // [6:6] is the sub-list for extension type_name - 6, // [6:6] is the sub-list for extension extendee - 0, // [0:6] is the sub-list for field type_name + 0, // 0: ethereum.engine.v1.SignedExecutionPayloadHeader.message:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderEPBS + 4, // 1: ethereum.engine.v1.ExecutionPayloadEnvelope.payload:type_name -> ethereum.engine.v1.ExecutionPayloadDeneb + 2, // 2: ethereum.engine.v1.SignedExecutionPayloadEnvelope.message:type_name -> ethereum.engine.v1.ExecutionPayloadEnvelope + 3, // [3:3] is the sub-list for method output_type + 3, // [3:3] is the sub-list for method input_type + 3, // [3:3] is the sub-list for extension type_name + 3, // [3:3] is the sub-list for extension extendee + 0, // [0:3] is the sub-list for field type_name } func init() { file_proto_engine_v1_epbs_proto_init() } @@ -957,42 +460,6 @@ func file_proto_engine_v1_epbs_proto_init() { file_proto_engine_v1_execution_engine_proto_init() if !protoimpl.UnsafeEnabled { file_proto_engine_v1_epbs_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InclusionListSummary); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_engine_v1_epbs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SignedInclusionListSummary); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_engine_v1_epbs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InclusionList); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_engine_v1_epbs_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecutionPayloadHeaderEPBS); i { case 0: return &v.state @@ -1004,19 +471,7 @@ func file_proto_engine_v1_epbs_proto_init() { return nil } } - file_proto_engine_v1_epbs_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutionPayloadEPBS); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_engine_v1_epbs_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + file_proto_engine_v1_epbs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SignedExecutionPayloadHeader); i { case 0: return &v.state @@ -1028,7 +483,7 @@ func file_proto_engine_v1_epbs_proto_init() { return nil } } - file_proto_engine_v1_epbs_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + file_proto_engine_v1_epbs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecutionPayloadEnvelope); i { case 0: return &v.state @@ -1040,7 +495,7 @@ func file_proto_engine_v1_epbs_proto_init() { return nil } } - file_proto_engine_v1_epbs_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + file_proto_engine_v1_epbs_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SignedExecutionPayloadEnvelope); i { case 0: return &v.state @@ -1059,7 +514,7 @@ func file_proto_engine_v1_epbs_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_engine_v1_epbs_proto_rawDesc, NumEnums: 0, - NumMessages: 8, + NumMessages: 4, NumExtensions: 0, NumServices: 0, }, diff --git a/proto/engine/v1/epbs.proto b/proto/engine/v1/epbs.proto index 3918d4fa8b77..9f1db496fcfc 100644 --- a/proto/engine/v1/epbs.proto +++ b/proto/engine/v1/epbs.proto @@ -42,7 +42,7 @@ message SignedExecutionPayloadHeader{ } message ExecutionPayloadEnvelope { - ExecutionPayloadElectra payload = 1; + ExecutionPayloadDeneb payload = 1; uint64 builder_index = 2 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"]; bytes beacon_block_root = 3 [(ethereum.eth.ext.ssz_size) = "32"]; repeated bytes blob_kzg_commitments = 4 [(ethereum.eth.ext.ssz_size) = "?,48", (ethereum.eth.ext.ssz_max) = "max_blob_commitments.size"]; diff --git a/proto/prysm/v1alpha1/beacon_block.pb.go b/proto/prysm/v1alpha1/beacon_block.pb.go index 6ecc040ace78..6a3c44a3e113 100755 --- a/proto/prysm/v1alpha1/beacon_block.pb.go +++ b/proto/prysm/v1alpha1/beacon_block.pb.go @@ -4782,8 +4782,8 @@ type BeaconBlockBodyEpbs struct { Eth1Data *Eth1Data `protobuf:"bytes,2,opt,name=eth1_data,json=eth1Data,proto3" json:"eth1_data,omitempty"` Graffiti []byte `protobuf:"bytes,3,opt,name=graffiti,proto3" json:"graffiti,omitempty" ssz-size:"32"` ProposerSlashings []*ProposerSlashing `protobuf:"bytes,4,rep,name=proposer_slashings,json=proposerSlashings,proto3" json:"proposer_slashings,omitempty" ssz-max:"16"` - AttesterSlashings []*AttesterSlashing `protobuf:"bytes,5,rep,name=attester_slashings,json=attesterSlashings,proto3" json:"attester_slashings,omitempty" ssz-max:"2"` - Attestations []*Attestation `protobuf:"bytes,6,rep,name=attestations,proto3" json:"attestations,omitempty" ssz-max:"128"` + AttesterSlashings []*AttesterSlashingElectra `protobuf:"bytes,5,rep,name=attester_slashings,json=attesterSlashings,proto3" json:"attester_slashings,omitempty" ssz-max:"2"` + Attestations []*AttestationElectra `protobuf:"bytes,6,rep,name=attestations,proto3" json:"attestations,omitempty" ssz-max:"128"` Deposits []*Deposit `protobuf:"bytes,7,rep,name=deposits,proto3" json:"deposits,omitempty" ssz-max:"16"` VoluntaryExits []*SignedVoluntaryExit `protobuf:"bytes,8,rep,name=voluntary_exits,json=voluntaryExits,proto3" json:"voluntary_exits,omitempty" ssz-max:"16"` SyncAggregate *SyncAggregate `protobuf:"bytes,9,opt,name=sync_aggregate,json=syncAggregate,proto3" json:"sync_aggregate,omitempty"` @@ -4852,14 +4852,14 @@ func (x *BeaconBlockBodyEpbs) GetProposerSlashings() []*ProposerSlashing { return nil } -func (x *BeaconBlockBodyEpbs) GetAttesterSlashings() []*AttesterSlashing { +func (x *BeaconBlockBodyEpbs) GetAttesterSlashings() []*AttesterSlashingElectra { if x != nil { return x.AttesterSlashings } return nil } -func (x *BeaconBlockBodyEpbs) GetAttestations() []*Attestation { +func (x *BeaconBlockBodyEpbs) GetAttestations() []*AttestationElectra { if x != nil { return x.Attestations } @@ -6376,7 +6376,7 @@ var file_proto_prysm_v1alpha1_beacon_block_proto_rawDesc = []byte{ 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, - 0x6f, 0x64, 0x79, 0x45, 0x70, 0x62, 0x73, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x22, 0xf4, 0x07, + 0x6f, 0x64, 0x79, 0x45, 0x70, 0x62, 0x73, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x22, 0x82, 0x08, 0x0a, 0x13, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x6f, 0x64, 0x79, 0x45, 0x70, 0x62, 0x73, 0x12, 0x2b, 0x0a, 0x0d, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x5f, 0x72, 0x65, 0x76, 0x65, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, @@ -6393,72 +6393,73 @@ var file_proto_prysm_v1alpha1_beacon_block_proto_rawDesc = []byte{ 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x11, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, - 0x69, 0x6e, 0x67, 0x73, 0x12, 0x5d, 0x0a, 0x12, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, + 0x69, 0x6e, 0x67, 0x73, 0x12, 0x64, 0x0a, 0x12, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x27, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, + 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, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, - 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x42, 0x05, 0x92, 0xb5, 0x18, 0x01, 0x32, - 0x52, 0x11, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, - 0x6e, 0x67, 0x73, 0x12, 0x4f, 0x0a, 0x0c, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, - 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, - 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x07, 0x92, - 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x0c, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x42, 0x0a, 0x08, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x73, - 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 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, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x08, - 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x73, 0x12, 0x5b, 0x0a, 0x0f, 0x76, 0x6f, 0x6c, 0x75, - 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x2a, 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, - 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x42, 0x06, 0x92, - 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x0e, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, - 0x45, 0x78, 0x69, 0x74, 0x73, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, - 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, - 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, - 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x52, 0x0d, 0x73, 0x79, 0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, - 0x74, 0x65, 0x12, 0x72, 0x0a, 0x18, 0x62, 0x6c, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x18, 0x0a, - 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, 0x69, 0x67, - 0x6e, 0x65, 0x64, 0x42, 0x4c, 0x53, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, - 0x15, 0x62, 0x6c, 0x73, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x12, 0x77, 0x0a, 0x1f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, - 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x30, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x52, 0x1c, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, - 0x63, 0x0a, 0x14, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x03, 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, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, - 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x05, 0x92, 0xb5, 0x18, 0x01, 0x34, 0x52, - 0x13, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x7b, 0x0a, 0x15, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, - 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x70, 0x62, 0x73, 0x12, 0x3c, 0x0a, - 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x65, - 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, - 0x45, 0x70, 0x62, 0x73, 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, 0x42, 0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, - 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, - 0x10, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x74, - 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, - 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, - 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, - 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, - 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, - 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, + 0x42, 0x05, 0x92, 0xb5, 0x18, 0x01, 0x32, 0x52, 0x11, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, + 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x56, 0x0a, 0x0c, 0x61, 0x74, + 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, 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, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x42, 0x07, 0x92, 0xb5, 0x18, + 0x03, 0x31, 0x32, 0x38, 0x52, 0x0c, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x42, 0x0a, 0x08, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x73, 0x18, 0x07, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 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, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x08, 0x64, 0x65, + 0x70, 0x6f, 0x73, 0x69, 0x74, 0x73, 0x12, 0x5b, 0x0a, 0x0f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, + 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x2a, 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, 0x56, 0x6f, + 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x42, 0x06, 0x92, 0xb5, 0x18, + 0x02, 0x31, 0x36, 0x52, 0x0e, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, + 0x69, 0x74, 0x73, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, + 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x65, 0x74, + 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x65, 0x52, 0x0d, 0x73, 0x79, 0x6e, 0x63, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, + 0x12, 0x72, 0x0a, 0x18, 0x62, 0x6c, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x73, 0x18, 0x0a, 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, 0x69, 0x67, 0x6e, 0x65, + 0x64, 0x42, 0x4c, 0x53, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x68, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x15, 0x62, + 0x6c, 0x73, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x68, 0x61, + 0x6e, 0x67, 0x65, 0x73, 0x12, 0x77, 0x0a, 0x1f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, + 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, + 0x1c, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x63, 0x0a, + 0x14, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x03, 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, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, + 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x05, 0x92, 0xb5, 0x18, 0x01, 0x34, 0x52, 0x13, 0x70, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x22, 0x7b, 0x0a, 0x15, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x65, 0x61, 0x63, + 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x70, 0x62, 0x73, 0x12, 0x3c, 0x0a, 0x05, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x65, 0x74, 0x68, + 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, + 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x70, + 0x62, 0x73, 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, 0x42, + 0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, + 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, 0x42, + 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, + 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, + 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, + 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, + 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, + 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, + 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, + 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -6708,8 +6709,8 @@ var file_proto_prysm_v1alpha1_beacon_block_proto_depIdxs = []int32{ 60, // 152: ethereum.eth.v1alpha1.BeaconBlockEpbs.body:type_name -> ethereum.eth.v1alpha1.BeaconBlockBodyEpbs 14, // 153: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.eth1_data:type_name -> ethereum.eth.v1alpha1.Eth1Data 8, // 154: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.proposer_slashings:type_name -> ethereum.eth.v1alpha1.ProposerSlashing - 9, // 155: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.attester_slashings:type_name -> ethereum.eth.v1alpha1.AttesterSlashing - 63, // 156: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.attestations:type_name -> ethereum.eth.v1alpha1.Attestation + 10, // 155: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.attester_slashings:type_name -> ethereum.eth.v1alpha1.AttesterSlashingElectra + 72, // 156: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.attestations:type_name -> ethereum.eth.v1alpha1.AttestationElectra 11, // 157: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.deposits:type_name -> ethereum.eth.v1alpha1.Deposit 13, // 158: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.voluntary_exits:type_name -> ethereum.eth.v1alpha1.SignedVoluntaryExit 19, // 159: ethereum.eth.v1alpha1.BeaconBlockBodyEpbs.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate diff --git a/proto/prysm/v1alpha1/beacon_block.proto b/proto/prysm/v1alpha1/beacon_block.proto index 492c62651015..a6a7a7685f0d 100644 --- a/proto/prysm/v1alpha1/beacon_block.proto +++ b/proto/prysm/v1alpha1/beacon_block.proto @@ -997,10 +997,10 @@ message BeaconBlockBodyEpbs { repeated ProposerSlashing proposer_slashings = 4 [(ethereum.eth.ext.ssz_max) = "16"]; // At most MAX_ATTESTER_SLASHINGS. - repeated AttesterSlashing attester_slashings = 5 [(ethereum.eth.ext.ssz_max) = "2"]; + repeated AttesterSlashingElectra attester_slashings = 5 [(ethereum.eth.ext.ssz_max) = "2"]; // At most MAX_ATTESTATIONS. - repeated Attestation attestations = 6 [(ethereum.eth.ext.ssz_max) = "128"]; + repeated AttestationElectra attestations = 6 [(ethereum.eth.ext.ssz_max) = "128"]; // At most MAX_DEPOSITS. repeated Deposit deposits = 7 [(ethereum.eth.ext.ssz_max) = "16"]; diff --git a/proto/prysm/v1alpha1/beacon_state.pb.go b/proto/prysm/v1alpha1/beacon_state.pb.go index b000c9e0d236..ff78a016970b 100755 --- a/proto/prysm/v1alpha1/beacon_state.pb.go +++ b/proto/prysm/v1alpha1/beacon_state.pb.go @@ -2198,6 +2198,7 @@ type BeaconStateEPBS struct { InactivityScores []uint64 `protobuf:"varint,9001,rep,packed,name=inactivity_scores,json=inactivityScores,proto3" json:"inactivity_scores,omitempty" ssz-max:"1099511627776"` CurrentSyncCommittee *SyncCommittee `protobuf:"bytes,9002,opt,name=current_sync_committee,json=currentSyncCommittee,proto3" json:"current_sync_committee,omitempty"` NextSyncCommittee *SyncCommittee `protobuf:"bytes,9003,opt,name=next_sync_committee,json=nextSyncCommittee,proto3" json:"next_sync_committee,omitempty"` + LatestExecutionPayloadHeader *v1.ExecutionPayloadHeaderEPBS `protobuf:"bytes,10001,opt,name=latest_execution_payload_header,json=latestExecutionPayloadHeader,proto3" json:"latest_execution_payload_header,omitempty"` NextWithdrawalIndex uint64 `protobuf:"varint,11001,opt,name=next_withdrawal_index,json=nextWithdrawalIndex,proto3" json:"next_withdrawal_index,omitempty"` NextWithdrawalValidatorIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,11002,opt,name=next_withdrawal_validator_index,json=nextWithdrawalValidatorIndex,proto3" json:"next_withdrawal_validator_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` HistoricalSummaries []*HistoricalSummary `protobuf:"bytes,11003,rep,name=historical_summaries,json=historicalSummaries,proto3" json:"historical_summaries,omitempty" ssz-max:"16777216"` @@ -2210,14 +2211,9 @@ type BeaconStateEPBS struct { PendingDeposits []*PendingDeposit `protobuf:"bytes,12007,rep,name=pending_deposits,json=pendingDeposits,proto3" json:"pending_deposits,omitempty" ssz-max:"134217728"` PendingPartialWithdrawals []*PendingPartialWithdrawal `protobuf:"bytes,12008,rep,name=pending_partial_withdrawals,json=pendingPartialWithdrawals,proto3" json:"pending_partial_withdrawals,omitempty" ssz-max:"134217728"` PendingConsolidations []*PendingConsolidation `protobuf:"bytes,12009,rep,name=pending_consolidations,json=pendingConsolidations,proto3" json:"pending_consolidations,omitempty" ssz-max:"262144"` - PreviousInclusionListProposer github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,13001,opt,name=previous_inclusion_list_proposer,json=previousInclusionListProposer,proto3" json:"previous_inclusion_list_proposer,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - PreviousInclusionListSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,13002,opt,name=previous_inclusion_list_slot,json=previousInclusionListSlot,proto3" json:"previous_inclusion_list_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - LatestInclusionListProposer github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,13003,opt,name=latest_inclusion_list_proposer,json=latestInclusionListProposer,proto3" json:"latest_inclusion_list_proposer,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - LatestInclusionListSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,13004,opt,name=latest_inclusion_list_slot,json=latestInclusionListSlot,proto3" json:"latest_inclusion_list_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - LatestBlockHash []byte `protobuf:"bytes,13005,opt,name=latest_block_hash,json=latestBlockHash,proto3" json:"latest_block_hash,omitempty" ssz-size:"32"` - LatestFullSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,13006,opt,name=latest_full_slot,json=latestFullSlot,proto3" json:"latest_full_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - ExecutionPayloadHeader *v1.ExecutionPayloadHeaderEPBS `protobuf:"bytes,13007,opt,name=execution_payload_header,json=executionPayloadHeader,proto3" json:"execution_payload_header,omitempty"` - LastWithdrawalsRoot []byte `protobuf:"bytes,13008,opt,name=last_withdrawals_root,json=lastWithdrawalsRoot,proto3" json:"last_withdrawals_root,omitempty" ssz-size:"32"` + LatestBlockHash []byte `protobuf:"bytes,13001,opt,name=latest_block_hash,json=latestBlockHash,proto3" json:"latest_block_hash,omitempty" ssz-size:"32"` + LatestFullSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,13002,opt,name=latest_full_slot,json=latestFullSlot,proto3" json:"latest_full_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` + LastWithdrawalsRoot []byte `protobuf:"bytes,13003,opt,name=last_withdrawals_root,json=lastWithdrawalsRoot,proto3" json:"last_withdrawals_root,omitempty" ssz-size:"32"` } func (x *BeaconStateEPBS) Reset() { @@ -2420,6 +2416,13 @@ func (x *BeaconStateEPBS) GetNextSyncCommittee() *SyncCommittee { return nil } +func (x *BeaconStateEPBS) GetLatestExecutionPayloadHeader() *v1.ExecutionPayloadHeaderEPBS { + if x != nil { + return x.LatestExecutionPayloadHeader + } + return nil +} + func (x *BeaconStateEPBS) GetNextWithdrawalIndex() uint64 { if x != nil { return x.NextWithdrawalIndex @@ -2504,34 +2507,6 @@ func (x *BeaconStateEPBS) GetPendingConsolidations() []*PendingConsolidation { return nil } -func (x *BeaconStateEPBS) GetPreviousInclusionListProposer() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex { - if x != nil { - return x.PreviousInclusionListProposer - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(0) -} - -func (x *BeaconStateEPBS) GetPreviousInclusionListSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { - if x != nil { - return x.PreviousInclusionListSlot - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0) -} - -func (x *BeaconStateEPBS) GetLatestInclusionListProposer() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex { - if x != nil { - return x.LatestInclusionListProposer - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(0) -} - -func (x *BeaconStateEPBS) GetLatestInclusionListSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { - if x != nil { - return x.LatestInclusionListSlot - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0) -} - func (x *BeaconStateEPBS) GetLatestBlockHash() []byte { if x != nil { return x.LatestBlockHash @@ -2546,13 +2521,6 @@ func (x *BeaconStateEPBS) GetLatestFullSlot() github_com_prysmaticlabs_prysm_v5_ return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0) } -func (x *BeaconStateEPBS) GetExecutionPayloadHeader() *v1.ExecutionPayloadHeaderEPBS { - if x != nil { - return x.ExecutionPayloadHeader - } - return nil -} - func (x *BeaconStateEPBS) GetLastWithdrawalsRoot() []byte { if x != nil { return x.LastWithdrawalsRoot @@ -3606,7 +3574,7 @@ var file_proto_prysm_v1alpha1_beacon_state_proto_rawDesc = []byte{ 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x0a, 0x92, 0xb5, 0x18, 0x06, 0x32, 0x36, 0x32, 0x31, 0x34, 0x34, 0x52, 0x15, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, - 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xd0, 0x1f, 0x0a, + 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x99, 0x1b, 0x0a, 0x0f, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x50, 0x42, 0x53, 0x12, 0x22, 0x0a, 0x0c, 0x67, 0x65, 0x6e, 0x65, 0x73, 0x69, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x67, 0x65, 0x6e, 0x65, 0x73, 0x69, 0x73, @@ -3716,178 +3684,142 @@ var file_proto_prysm_v1alpha1_beacon_state_proto_rawDesc = []byte{ 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x52, 0x11, 0x6e, 0x65, 0x78, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x12, - 0x33, 0x0a, 0x15, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, - 0x61, 0x6c, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0xf9, 0x55, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x13, 0x6e, 0x65, 0x78, 0x74, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x12, 0x97, 0x01, 0x0a, 0x1f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x77, 0x69, - 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x6f, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0xfa, 0x55, 0x20, 0x01, 0x28, 0x04, 0x42, - 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, - 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, - 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, - 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x52, 0x1c, 0x6e, 0x65, 0x78, 0x74, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, - 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x6a, - 0x0a, 0x14, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x73, 0x75, 0x6d, - 0x6d, 0x61, 0x72, 0x69, 0x65, 0x73, 0x18, 0xfb, 0x55, 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, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, - 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x42, 0x0c, 0x92, 0xb5, 0x18, 0x08, 0x31, 0x36, 0x37, - 0x37, 0x37, 0x32, 0x31, 0x36, 0x52, 0x13, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, - 0x6c, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x69, 0x65, 0x73, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, - 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x5f, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0xe1, 0x5d, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x19, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x73, 0x53, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x83, 0x01, 0x0a, - 0x1a, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, - 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, 0xe2, 0x5d, 0x20, 0x01, - 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, - 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, - 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, - 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, - 0x69, 0x76, 0x65, 0x73, 0x2e, 0x47, 0x77, 0x65, 0x69, 0x52, 0x17, 0x64, 0x65, 0x70, 0x6f, 0x73, - 0x69, 0x74, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x43, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x12, 0x7d, 0x0a, 0x17, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, - 0x63, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, 0xe3, 0x5d, + 0x76, 0x0a, 0x1f, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x18, 0x91, 0x4e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x50, 0x42, 0x53, 0x52, 0x1c, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x33, 0x0a, 0x15, 0x6e, 0x65, 0x78, 0x74, 0x5f, + 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0xf9, 0x55, 0x20, 0x01, 0x28, 0x04, 0x52, 0x13, 0x6e, 0x65, 0x78, 0x74, 0x57, 0x69, 0x74, + 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x97, 0x01, 0x0a, + 0x1f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, + 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0xfa, 0x55, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, + 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, + 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, + 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x1c, 0x6e, 0x65, 0x78, 0x74, 0x57, 0x69, + 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, + 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x6a, 0x0a, 0x14, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x69, 0x65, 0x73, 0x18, 0xfb, + 0x55, 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, 0x48, 0x69, + 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x42, + 0x0c, 0x92, 0xb5, 0x18, 0x08, 0x31, 0x36, 0x37, 0x37, 0x37, 0x32, 0x31, 0x36, 0x52, 0x13, 0x68, + 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x69, + 0x65, 0x73, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x18, 0xe1, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x19, 0x64, 0x65, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x53, 0x74, 0x61, 0x72, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x12, 0x83, 0x01, 0x0a, 0x1a, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x18, 0xe2, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, + 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, + 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, + 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, + 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x47, 0x77, 0x65, + 0x69, 0x52, 0x17, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, + 0x65, 0x54, 0x6f, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x7d, 0x0a, 0x17, 0x65, 0x78, + 0x69, 0x74, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, 0xe3, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, + 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, + 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, + 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, + 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x47, + 0x77, 0x65, 0x69, 0x52, 0x14, 0x65, 0x78, 0x69, 0x74, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, + 0x54, 0x6f, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x77, 0x0a, 0x13, 0x65, 0x61, 0x72, + 0x6c, 0x69, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, + 0x18, 0xe4, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x46, 0x82, 0xb5, 0x18, 0x42, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, + 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, + 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, + 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, + 0x11, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73, 0x74, 0x45, 0x78, 0x69, 0x74, 0x45, 0x70, 0x6f, + 0x63, 0x68, 0x12, 0x8f, 0x01, 0x0a, 0x20, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x74, 0x6f, 0x5f, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, 0xe5, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, + 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, + 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, + 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, + 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, + 0x2e, 0x47, 0x77, 0x65, 0x69, 0x52, 0x1d, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x12, 0x89, 0x01, 0x0a, 0x1c, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73, + 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0xe6, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x46, 0x82, 0xb5, + 0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, + 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, + 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, + 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x45, + 0x70, 0x6f, 0x63, 0x68, 0x52, 0x1a, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73, 0x74, 0x43, 0x6f, + 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x70, 0x6f, 0x63, 0x68, + 0x12, 0x60, 0x0a, 0x10, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x65, 0x70, 0x6f, + 0x73, 0x69, 0x74, 0x73, 0x18, 0xe7, 0x5d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 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, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x42, 0x0d, 0x92, 0xb5, 0x18, 0x09, 0x31, 0x33, 0x34, 0x32, 0x31, 0x37, 0x37, 0x32, + 0x38, 0x52, 0x0f, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x73, 0x12, 0x7f, 0x0a, 0x1b, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, + 0x73, 0x18, 0xe8, 0x5d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, + 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, + 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x57, + 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x42, 0x0d, 0x92, 0xb5, 0x18, 0x09, 0x31, + 0x33, 0x34, 0x32, 0x31, 0x37, 0x37, 0x32, 0x38, 0x52, 0x19, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, + 0x67, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, + 0x61, 0x6c, 0x73, 0x12, 0x6f, 0x0a, 0x16, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x63, + 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe9, 0x5d, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, + 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x6e, + 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x42, 0x0a, 0x92, 0xb5, 0x18, 0x06, 0x32, 0x36, 0x32, 0x31, 0x34, 0x34, 0x52, 0x15, 0x70, + 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x33, 0x0a, 0x11, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0xc9, 0x65, 0x20, 0x01, 0x28, 0x0c, + 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x70, 0x0a, 0x10, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x74, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0xca, 0x65, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, - 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x47, 0x77, 0x65, 0x69, 0x52, 0x14, 0x65, 0x78, 0x69, - 0x74, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x12, 0x77, 0x0a, 0x13, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x78, - 0x69, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0xe4, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, - 0x46, 0x82, 0xb5, 0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, - 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, - 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, - 0x73, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x11, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73, - 0x74, 0x45, 0x78, 0x69, 0x74, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x8f, 0x01, 0x0a, 0x20, 0x63, - 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x62, 0x61, 0x6c, - 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, - 0xe5, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, - 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, - 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, - 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, - 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x47, 0x77, 0x65, 0x69, 0x52, 0x1d, 0x63, - 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x6c, 0x61, - 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x89, 0x01, 0x0a, - 0x1c, 0x65, 0x61, 0x72, 0x6c, 0x69, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, - 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0xe6, 0x5d, - 0x20, 0x01, 0x28, 0x04, 0x42, 0x46, 0x82, 0xb5, 0x18, 0x42, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, - 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, - 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, - 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, - 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x1a, 0x65, 0x61, - 0x72, 0x6c, 0x69, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x60, 0x0a, 0x10, 0x70, 0x65, 0x6e, 0x64, - 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x73, 0x18, 0xe7, 0x5d, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x25, 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, 0x6e, 0x64, - 0x69, 0x6e, 0x67, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x42, 0x0d, 0x92, 0xb5, 0x18, 0x09, - 0x31, 0x33, 0x34, 0x32, 0x31, 0x37, 0x37, 0x32, 0x38, 0x52, 0x0f, 0x70, 0x65, 0x6e, 0x64, 0x69, - 0x6e, 0x67, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x73, 0x12, 0x7f, 0x0a, 0x1b, 0x70, 0x65, - 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x77, 0x69, - 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x18, 0xe8, 0x5d, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x2f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, - 0x6c, 0x42, 0x0d, 0x92, 0xb5, 0x18, 0x09, 0x31, 0x33, 0x34, 0x32, 0x31, 0x37, 0x37, 0x32, 0x38, - 0x52, 0x19, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, - 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x12, 0x6f, 0x0a, 0x16, 0x70, - 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe9, 0x5d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, - 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, - 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x0a, 0x92, 0xb5, 0x18, 0x06, 0x32, - 0x36, 0x32, 0x31, 0x34, 0x34, 0x52, 0x15, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, - 0x6e, 0x73, 0x6f, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x99, 0x01, 0x0a, - 0x20, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, - 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, - 0x72, 0x18, 0xc9, 0x65, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, - 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, - 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, - 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x1d, 0x70, 0x72, 0x65, 0x76, 0x69, - 0x6f, 0x75, 0x73, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x12, 0x87, 0x01, 0x0a, 0x1c, 0x70, 0x72, 0x65, - 0x76, 0x69, 0x6f, 0x75, 0x73, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, - 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0xca, 0x65, 0x20, 0x01, 0x28, 0x04, - 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, - 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, - 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, - 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x19, 0x70, 0x72, 0x65, 0x76, 0x69, 0x6f, 0x75, - 0x73, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x6c, - 0x6f, 0x74, 0x12, 0x95, 0x01, 0x0a, 0x1e, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, - 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x6f, - 0x70, 0x6f, 0x73, 0x65, 0x72, 0x18, 0xcb, 0x65, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, - 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, - 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, - 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, - 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, - 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x1b, 0x6c, - 0x61, 0x74, 0x65, 0x73, 0x74, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, - 0x73, 0x74, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x12, 0x83, 0x01, 0x0a, 0x1a, 0x6c, - 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, - 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0xcc, 0x65, 0x20, 0x01, 0x28, 0x04, - 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, - 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, - 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, - 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x17, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x49, - 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x6c, 0x6f, 0x74, - 0x12, 0x33, 0x0a, 0x11, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0xcd, 0x65, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, - 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x42, 0x6c, 0x6f, 0x63, - 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x70, 0x0a, 0x10, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, - 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0xce, 0x65, 0x20, 0x01, 0x28, 0x04, - 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, - 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, - 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, - 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x0e, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x46, - 0x75, 0x6c, 0x6c, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x69, 0x0a, 0x18, 0x65, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x68, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x18, 0xcf, 0x65, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, - 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x50, 0x42, 0x53, 0x52, 0x16, 0x65, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x12, 0x3b, 0x0a, 0x15, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, - 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0xd0, 0x65, 0x20, 0x01, - 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x13, 0x6c, 0x61, 0x73, 0x74, - 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, - 0x8d, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x77, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x25, 0x0a, 0x0a, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, - 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, - 0x61, 0x73, 0x68, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, - 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, - 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x31, 0x0a, 0x10, - 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x69, 0x63, 0x75, 0x6c, 0x74, 0x79, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, - 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x69, 0x66, 0x66, 0x69, 0x63, 0x75, 0x6c, 0x74, 0x79, 0x22, - 0x7f, 0x0a, 0x11, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x53, 0x75, 0x6d, - 0x6d, 0x61, 0x72, 0x79, 0x12, 0x34, 0x0a, 0x12, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x75, - 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, - 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x53, - 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x34, 0x0a, 0x12, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x5f, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x6f, 0x6f, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x6f, 0x6f, 0x74, - 0x42, 0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, - 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, - 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, - 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, - 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, - 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, - 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, - 0x41, 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, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x0e, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x3b, 0x0a, 0x15, 0x6c, + 0x61, 0x73, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x5f, + 0x72, 0x6f, 0x6f, 0x74, 0x18, 0xcb, 0x65, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, + 0x02, 0x33, 0x32, 0x52, 0x13, 0x6c, 0x61, 0x73, 0x74, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, + 0x77, 0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x8d, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x77, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, + 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, + 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x27, 0x0a, 0x0b, + 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x31, 0x0a, 0x10, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x64, + 0x69, 0x66, 0x66, 0x69, 0x63, 0x75, 0x6c, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, + 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x69, + 0x66, 0x66, 0x69, 0x63, 0x75, 0x6c, 0x74, 0x79, 0x22, 0x7f, 0x0a, 0x11, 0x48, 0x69, 0x73, 0x74, + 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x34, 0x0a, + 0x12, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x72, + 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, + 0x32, 0x52, 0x10, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x52, + 0x6f, 0x6f, 0x74, 0x12, 0x34, 0x0a, 0x12, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x75, 0x6d, + 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, + 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x73, 0x74, 0x61, 0x74, 0x65, 0x53, 0x75, + 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x6f, 0x6f, 0x74, 0x42, 0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, + 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, + 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, + 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, + 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x41, 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 ( @@ -4019,11 +3951,11 @@ var file_proto_prysm_v1alpha1_beacon_state_proto_depIdxs = []int32{ 22, // 79: ethereum.eth.v1alpha1.BeaconStateEPBS.finalized_checkpoint:type_name -> ethereum.eth.v1alpha1.Checkpoint 10, // 80: ethereum.eth.v1alpha1.BeaconStateEPBS.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee 10, // 81: ethereum.eth.v1alpha1.BeaconStateEPBS.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee - 18, // 82: ethereum.eth.v1alpha1.BeaconStateEPBS.historical_summaries:type_name -> ethereum.eth.v1alpha1.HistoricalSummary - 27, // 83: ethereum.eth.v1alpha1.BeaconStateEPBS.pending_deposits:type_name -> ethereum.eth.v1alpha1.PendingDeposit - 28, // 84: ethereum.eth.v1alpha1.BeaconStateEPBS.pending_partial_withdrawals:type_name -> ethereum.eth.v1alpha1.PendingPartialWithdrawal - 29, // 85: ethereum.eth.v1alpha1.BeaconStateEPBS.pending_consolidations:type_name -> ethereum.eth.v1alpha1.PendingConsolidation - 30, // 86: ethereum.eth.v1alpha1.BeaconStateEPBS.execution_payload_header:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderEPBS + 30, // 82: ethereum.eth.v1alpha1.BeaconStateEPBS.latest_execution_payload_header:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderEPBS + 18, // 83: ethereum.eth.v1alpha1.BeaconStateEPBS.historical_summaries:type_name -> ethereum.eth.v1alpha1.HistoricalSummary + 27, // 84: ethereum.eth.v1alpha1.BeaconStateEPBS.pending_deposits:type_name -> ethereum.eth.v1alpha1.PendingDeposit + 28, // 85: ethereum.eth.v1alpha1.BeaconStateEPBS.pending_partial_withdrawals:type_name -> ethereum.eth.v1alpha1.PendingPartialWithdrawal + 29, // 86: ethereum.eth.v1alpha1.BeaconStateEPBS.pending_consolidations:type_name -> ethereum.eth.v1alpha1.PendingConsolidation 87, // [87:87] is the sub-list for method output_type 87, // [87:87] is the sub-list for method input_type 87, // [87:87] is the sub-list for extension type_name diff --git a/proto/prysm/v1alpha1/blind_payload_envelope.pb.go b/proto/prysm/v1alpha1/blind_payload_envelope.pb.go index 82552db4409f..c41a69a08212 100755 --- a/proto/prysm/v1alpha1/blind_payload_envelope.pb.go +++ b/proto/prysm/v1alpha1/blind_payload_envelope.pb.go @@ -83,15 +83,12 @@ type BlindPayloadEnvelope struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - PayloadRoot []byte `protobuf:"bytes,1,opt,name=payload_root,json=payloadRoot,proto3" json:"payload_root,omitempty" ssz-size:"32"` - BuilderIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,2,opt,name=builder_index,json=builderIndex,proto3" json:"builder_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - BeaconBlockRoot []byte `protobuf:"bytes,3,opt,name=beacon_block_root,json=beaconBlockRoot,proto3" json:"beacon_block_root,omitempty" ssz-size:"32"` - BlobKzgCommitments [][]byte `protobuf:"bytes,4,rep,name=blob_kzg_commitments,json=blobKzgCommitments,proto3" json:"blob_kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"` - InclusionListProposerIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,5,opt,name=inclusion_list_proposer_index,json=inclusionListProposerIndex,proto3" json:"inclusion_list_proposer_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` - InclusionListSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,6,opt,name=inclusion_list_slot,json=inclusionListSlot,proto3" json:"inclusion_list_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"` - InclusionListSignature []byte `protobuf:"bytes,7,opt,name=inclusion_list_signature,json=inclusionListSignature,proto3" json:"inclusion_list_signature,omitempty" ssz-size:"96"` - PayloadWithheld bool `protobuf:"varint,8,opt,name=payload_withheld,json=payloadWithheld,proto3" json:"payload_withheld,omitempty"` - StateRoot []byte `protobuf:"bytes,9,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"` + PayloadRoot []byte `protobuf:"bytes,1,opt,name=payload_root,json=payloadRoot,proto3" json:"payload_root,omitempty" ssz-size:"32"` + BuilderIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,2,opt,name=builder_index,json=builderIndex,proto3" json:"builder_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"` + BeaconBlockRoot []byte `protobuf:"bytes,3,opt,name=beacon_block_root,json=beaconBlockRoot,proto3" json:"beacon_block_root,omitempty" ssz-size:"32"` + BlobKzgCommitments [][]byte `protobuf:"bytes,4,rep,name=blob_kzg_commitments,json=blobKzgCommitments,proto3" json:"blob_kzg_commitments,omitempty" ssz-max:"4096" ssz-size:"?,48"` + PayloadWithheld bool `protobuf:"varint,8,opt,name=payload_withheld,json=payloadWithheld,proto3" json:"payload_withheld,omitempty"` + StateRoot []byte `protobuf:"bytes,9,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"` } func (x *BlindPayloadEnvelope) Reset() { @@ -154,27 +151,6 @@ func (x *BlindPayloadEnvelope) GetBlobKzgCommitments() [][]byte { return nil } -func (x *BlindPayloadEnvelope) GetInclusionListProposerIndex() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex { - if x != nil { - return x.InclusionListProposerIndex - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex(0) -} - -func (x *BlindPayloadEnvelope) GetInclusionListSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { - if x != nil { - return x.InclusionListSlot - } - return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0) -} - -func (x *BlindPayloadEnvelope) GetInclusionListSignature() []byte { - if x != nil { - return x.InclusionListSignature - } - return nil -} - func (x *BlindPayloadEnvelope) GetPayloadWithheld() bool { if x != nil { return x.PayloadWithheld @@ -207,7 +183,7 @@ var file_proto_prysm_v1alpha1_blind_payload_envelope_proto_rawDesc = []byte{ 0x6c, 0x6f, 0x70, 0x65, 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, 0xcf, 0x05, 0x0a, 0x14, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x50, 0x61, 0x79, + 0x75, 0x72, 0x65, 0x22, 0x81, 0x03, 0x0a, 0x14, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 0x12, 0x29, 0x0a, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0b, 0x70, 0x61, 0x79, 0x6c, @@ -226,44 +202,23 @@ var file_proto_prysm_v1alpha1_blind_payload_envelope_proto_rawDesc = []byte{ 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 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, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x4b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, - 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x92, 0x01, 0x0a, 0x1d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, - 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, - 0x72, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, - 0xb5, 0x18, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, - 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, - 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, - 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, - 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x1a, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, - 0x70, 0x6f, 0x73, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x75, 0x0a, 0x13, 0x69, 0x6e, - 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, - 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, - 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, - 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, - 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, - 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x11, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x6c, 0x6f, - 0x74, 0x12, 0x40, 0x0a, 0x18, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, - 0x69, 0x73, 0x74, 0x5f, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x16, 0x69, 0x6e, 0x63, - 0x6c, 0x75, 0x73, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, - 0x75, 0x72, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x77, - 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x70, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x57, 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, 0x12, 0x25, - 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x09, 0x20, 0x01, - 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x42, 0xa4, 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, 0x19, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, - 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, - 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, - 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, - 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, - 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x5f, 0x77, 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x57, 0x69, 0x74, 0x68, 0x68, 0x65, 0x6c, 0x64, + 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x42, 0xa4, 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, 0x19, 0x42, 0x6c, 0x69, 0x6e, 0x64, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x45, 0x6e, 0x76, 0x65, 0x6c, 0x6f, 0x70, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, + 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, + 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, + 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, + 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x76, 0x31, + 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, + 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/testing/util/BUILD.bazel b/testing/util/BUILD.bazel index 16154398cf7e..22bdb1f9affb 100644 --- a/testing/util/BUILD.bazel +++ b/testing/util/BUILD.bazel @@ -19,6 +19,8 @@ go_library( "electra.go", "electra_block.go", "electra_state.go", + "epbs_block.go", + "epbs_state.go", "helpers.go", "lightclient.go", "logging.go", diff --git a/testing/util/block.go b/testing/util/block.go index bc6f21923133..e8f988662e37 100644 --- a/testing/util/block.go +++ b/testing/util/block.go @@ -42,9 +42,12 @@ type BlockGenConfig struct { FullSyncAggregate bool NumBLSChanges uint64 // Only for post Capella blocks NumWithdrawals uint64 - NumDepositRequests uint64 // Only for post Electra blocks - NumWithdrawalRequests uint64 // Only for post Electra blocks - NumConsolidationRequests uint64 // Only for post Electra blocks + NumDepositRequests uint64 // Only for post Electra blocks + NumWithdrawalRequests uint64 // Only for post Electra blocks + NumConsolidationRequests uint64 // Only for post Electra blocks + BuilderIndex primitives.ValidatorIndex // Only for post EIP-7732 blocks. + PayloadValue uint64 // Only for post EIP-7732 blocks + NumKzgCommitmens uint64 // Only for post EIP-7732 blocks } // DefaultBlockGenConfig returns the block config that utilizes the diff --git a/testing/util/epbs_block.go b/testing/util/epbs_block.go new file mode 100644 index 000000000000..b20d4818cfce --- /dev/null +++ b/testing/util/epbs_block.go @@ -0,0 +1,260 @@ +package util + +import ( + "context" + "crypto/rand" + "fmt" + + "github.com/pkg/errors" + "github.com/prysmaticlabs/go-bitfield" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/state" + fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams" + "github.com/prysmaticlabs/prysm/v5/config/params" + "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives" + "github.com/prysmaticlabs/prysm/v5/crypto/bls" + "github.com/prysmaticlabs/prysm/v5/encoding/bytesutil" + "github.com/prysmaticlabs/prysm/v5/encoding/ssz" + v1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" + ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" +) + +// GenerateFullBlockEpbs generates a fully valid Epbs block with the requested parameters. +// Use BlockGenConfig to declare the conditions you would like the block generated under. +// This function modifies the passed state as follows: +func GenerateFullBlockEpbs( + bState state.BeaconState, + privs []bls.SecretKey, + conf *BlockGenConfig, + slot primitives.Slot, +) (*ethpb.SignedBeaconBlockEpbs, error) { + ctx := context.Background() + currentSlot := bState.Slot() + if currentSlot > slot { + return nil, fmt.Errorf("current slot in state is larger than given slot. %d > %d", currentSlot, slot) + } + bState = bState.Copy() + + if conf == nil { + conf = &BlockGenConfig{} + } + + var err error + var pSlashings []*ethpb.ProposerSlashing + numToGen := conf.NumProposerSlashings + if numToGen > 0 { + pSlashings, err = generateProposerSlashings(bState, privs, numToGen) + if err != nil { + return nil, errors.Wrapf(err, "failed generating %d proposer slashings:", numToGen) + } + } + + numToGen = conf.NumAttesterSlashings + var aSlashings []*ethpb.AttesterSlashingElectra + if numToGen > 0 { + generated, err := generateAttesterSlashings(bState, privs, numToGen) + if err != nil { + return nil, errors.Wrapf(err, "failed generating %d attester slashings:", numToGen) + } + aSlashings = make([]*ethpb.AttesterSlashingElectra, len(generated)) + var ok bool + for i, s := range generated { + aSlashings[i], ok = s.(*ethpb.AttesterSlashingElectra) + if !ok { + return nil, fmt.Errorf("attester slashing has the wrong type (expected %T, got %T)", ðpb.AttesterSlashingElectra{}, s) + } + } + } + + numToGen = conf.NumAttestations + var atts []*ethpb.AttestationElectra + if numToGen > 0 { + generatedAtts, err := GenerateAttestations(bState, privs, numToGen, slot, false) + if err != nil { + return nil, errors.Wrapf(err, "failed generating %d attestations:", numToGen) + } + atts = make([]*ethpb.AttestationElectra, len(generatedAtts)) + var ok bool + for i, a := range generatedAtts { + atts[i], ok = a.(*ethpb.AttestationElectra) + if !ok { + return nil, fmt.Errorf("attestation has the wrong type (expected %T, got %T)", ðpb.AttestationElectra{}, a) + } + } + } + + numToGen = conf.NumDeposits + var newDeposits []*ethpb.Deposit + eth1Data := bState.Eth1Data() + if numToGen > 0 { + newDeposits, eth1Data, err = generateDepositsAndEth1Data(bState, numToGen) + if err != nil { + return nil, errors.Wrapf(err, "failed generating %d deposits:", numToGen) + } + } + + numToGen = conf.NumVoluntaryExits + var exits []*ethpb.SignedVoluntaryExit + if numToGen > 0 { + exits, err = generateVoluntaryExits(bState, privs, numToGen) + if err != nil { + return nil, errors.Wrapf(err, "failed generating %d attester slashings:", numToGen) + } + } + + numToGen = conf.NumTransactions + newTransactions := make([][]byte, numToGen) + for i := uint64(0); i < numToGen; i++ { + newTransactions[i] = bytesutil.Uint64ToBytesLittleEndian(i) + } + stCopy := bState.Copy() + stCopy, err = transition.ProcessSlots(context.Background(), stCopy, slot) + if err != nil { + return nil, err + } + + blockHash := indexToHash(uint64(slot)) + var syncCommitteeBits []byte + currSize := new(ethpb.SyncAggregate).SyncCommitteeBits.Len() + switch currSize { + case 512: + syncCommitteeBits = bitfield.NewBitvector512() + case 32: + syncCommitteeBits = bitfield.NewBitvector32() + default: + return nil, errors.New("invalid bit vector size") + } + newSyncAggregate := ðpb.SyncAggregate{ + SyncCommitteeBits: syncCommitteeBits, + SyncCommitteeSignature: append([]byte{0xC0}, make([]byte, 95)...), + } + + newHeader := bState.LatestBlockHeader() + prevStateRoot, err := bState.HashTreeRoot(ctx) + if err != nil { + return nil, errors.Wrap(err, "could not hash state") + } + newHeader.StateRoot = prevStateRoot[:] + parentRoot, err := newHeader.HashTreeRoot() + if err != nil { + return nil, errors.Wrap(err, "could not hash the new header") + } + + if slot == currentSlot { + slot = currentSlot + 1 + } + + reveal, err := RandaoReveal(stCopy, time.CurrentEpoch(stCopy), privs) + if err != nil { + return nil, errors.Wrap(err, "could not compute randao reveal") + } + + idx, err := helpers.BeaconProposerIndex(ctx, stCopy) + if err != nil { + return nil, errors.Wrap(err, "could not compute beacon proposer index") + } + + changes := make([]*ethpb.SignedBLSToExecutionChange, conf.NumBLSChanges) + for i := uint64(0); i < conf.NumBLSChanges; i++ { + changes[i], err = GenerateBLSToExecutionChange(bState, privs[i+1], primitives.ValidatorIndex(i)) + if err != nil { + return nil, err + } + } + parentExecution, err := stCopy.LatestExecutionPayloadHeader() + if err != nil { + return nil, err + } + parentHash, err := parentExecution.HashTreeRoot() + if err != nil { + return nil, err + } + kzgRoot, err := generateKzgCommitmentsRoot(conf.NumKzgCommitmens) + if err != nil { + return nil, err + } + newExecutionPayloadHeader := &v1.ExecutionPayloadHeaderEPBS{ + ParentBlockHash: parentHash[:], + ParentBlockRoot: parentRoot[:], + BlockHash: blockHash[:], + BuilderIndex: conf.BuilderIndex, + Slot: slot, + Value: conf.PayloadValue, + BlobKzgCommitmentsRoot: kzgRoot[:], + } + newSignedExecutionPayloadHeader, err := SignExecutionPayloadHeader(bState, privs[conf.BuilderIndex], newExecutionPayloadHeader) + if err != nil { + return nil, err + } + + block := ðpb.BeaconBlockEpbs{ + Slot: slot, + ParentRoot: parentRoot[:], + ProposerIndex: idx, + Body: ðpb.BeaconBlockBodyEpbs{ + Eth1Data: eth1Data, + RandaoReveal: reveal, + ProposerSlashings: pSlashings, + AttesterSlashings: aSlashings, + Attestations: atts, + VoluntaryExits: exits, + Deposits: newDeposits, + Graffiti: make([]byte, fieldparams.RootLength), + SyncAggregate: newSyncAggregate, + SignedExecutionPayloadHeader: newSignedExecutionPayloadHeader, + BlsToExecutionChanges: changes, + }, + } + + // The fork can change after processing the state + signature, err := BlockSignature(bState, block, privs) + if err != nil { + return nil, errors.Wrap(err, "could not compute block signature") + } + + return ðpb.SignedBeaconBlockEpbs{Block: block, Signature: signature.Marshal()}, nil +} + +// SignExecutionPayloadHeader generates a valid SignedExecutionPayloadHeader +func SignExecutionPayloadHeader( + st state.BeaconState, + priv bls.SecretKey, + message *v1.ExecutionPayloadHeaderEPBS, +) (*v1.SignedExecutionPayloadHeader, error) { + c := params.BeaconConfig() + domain, err := signing.ComputeDomain(c.DomainBeaconBuilder, c.GenesisForkVersion, st.GenesisValidatorsRoot()) + if err != nil { + return nil, err + } + sr, err := signing.ComputeSigningRoot(message, domain) + if err != nil { + return nil, err + } + signature := priv.Sign(sr[:]).Marshal() + return &v1.SignedExecutionPayloadHeader{ + Message: message, + Signature: signature, + }, nil +} +func generateKzgCommitments(n uint64) ([][]byte, error) { + kzgs := make([][]byte, n) + for i := range kzgs { + kzgs[i] = make([]byte, 48) + _, err := rand.Read(kzgs[i]) + if err != nil { + return nil, err + } + } + return kzgs, nil +} + +func generateKzgCommitmentsRoot(n uint64) ([32]byte, error) { + kzgs, err := generateKzgCommitments(n) + if err != nil { + return [32]byte{}, err + } + return ssz.KzgCommitmentsRoot(kzgs) +} diff --git a/testing/util/epbs_state.go b/testing/util/epbs_state.go new file mode 100644 index 000000000000..6700cfc439ea --- /dev/null +++ b/testing/util/epbs_state.go @@ -0,0 +1,273 @@ +package util + +import ( + "context" + "testing" + + "github.com/pkg/errors" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/state" + state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native" + "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stateutil" + fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams" + "github.com/prysmaticlabs/prysm/v5/config/params" + "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives" + "github.com/prysmaticlabs/prysm/v5/crypto/bls" + "github.com/prysmaticlabs/prysm/v5/encoding/ssz" + enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1" + ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" + "github.com/prysmaticlabs/prysm/v5/time/slots" +) + +// emptyGenesisStateEpbs returns an empty genesis state in ePBS format. +func emptyGenesisStateEpbs() (state.BeaconState, error) { + st := ðpb.BeaconStateEPBS{ + // Misc fields. + Slot: 0, + Fork: ðpb.Fork{ + PreviousVersion: params.BeaconConfig().BellatrixForkVersion, + CurrentVersion: params.BeaconConfig().DenebForkVersion, + Epoch: 0, + }, + // Validator registry fields. + Validators: []*ethpb.Validator{}, + Balances: []uint64{}, + InactivityScores: []uint64{}, + + JustificationBits: []byte{0}, + HistoricalRoots: [][]byte{}, + CurrentEpochParticipation: []byte{}, + PreviousEpochParticipation: []byte{}, + + // Eth1 data. + Eth1Data: ðpb.Eth1Data{}, + Eth1DataVotes: []*ethpb.Eth1Data{}, + Eth1DepositIndex: 0, + + LatestExecutionPayloadHeader: &enginev1.ExecutionPayloadHeaderEPBS{}, + + DepositBalanceToConsume: primitives.Gwei(0), + ExitBalanceToConsume: primitives.Gwei(0), + ConsolidationBalanceToConsume: primitives.Gwei(0), + } + return state_native.InitializeFromProtoEpbs(st) +} + +// genesisBeaconStateEpbs returns the genesis beacon state. +func genesisBeaconStateEpbs(ctx context.Context, deposits []*ethpb.Deposit, genesisTime uint64, eth1Data *ethpb.Eth1Data) (state.BeaconState, error) { + st, err := emptyGenesisStateEpbs() + if err != nil { + return nil, err + } + + // Process initial deposits. + st, err = helpers.UpdateGenesisEth1Data(st, deposits, eth1Data) + if err != nil { + return nil, err + } + + st, err = processPreGenesisDeposits(ctx, st, deposits) + if err != nil { + return nil, errors.Wrap(err, "could not process validator deposits") + } + + return buildGenesisBeaconStateEpbs(genesisTime, st, st.Eth1Data()) +} + +func buildGenesisBeaconStateEpbs(genesisTime uint64, preState state.BeaconState, eth1Data *ethpb.Eth1Data) (state.BeaconState, error) { + if eth1Data == nil { + return nil, errors.New("no eth1data provided for genesis state") + } + + randaoMixes := make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector) + for i := 0; i < len(randaoMixes); i++ { + h := make([]byte, 32) + copy(h, eth1Data.BlockHash) + randaoMixes[i] = h + } + + zeroHash := params.BeaconConfig().ZeroHash[:] + + activeIndexRoots := make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector) + for i := 0; i < len(activeIndexRoots); i++ { + activeIndexRoots[i] = zeroHash + } + + blockRoots := make([][]byte, params.BeaconConfig().SlotsPerHistoricalRoot) + for i := 0; i < len(blockRoots); i++ { + blockRoots[i] = zeroHash + } + + stateRoots := make([][]byte, params.BeaconConfig().SlotsPerHistoricalRoot) + for i := 0; i < len(stateRoots); i++ { + stateRoots[i] = zeroHash + } + + slashings := make([]uint64, params.BeaconConfig().EpochsPerSlashingsVector) + + genesisValidatorsRoot, err := stateutil.ValidatorRegistryRoot(preState.Validators()) + if err != nil { + return nil, errors.Wrapf(err, "could not hash tree root genesis validators %v", err) + } + + prevEpochParticipation, err := preState.PreviousEpochParticipation() + if err != nil { + return nil, err + } + currEpochParticipation, err := preState.CurrentEpochParticipation() + if err != nil { + return nil, err + } + scores, err := preState.InactivityScores() + if err != nil { + return nil, err + } + tab, err := helpers.TotalActiveBalance(preState) + if err != nil { + return nil, err + } + st := ðpb.BeaconStateEPBS{ + // Misc fields. + Slot: 0, + GenesisTime: genesisTime, + GenesisValidatorsRoot: genesisValidatorsRoot[:], + + Fork: ðpb.Fork{ + PreviousVersion: params.BeaconConfig().GenesisForkVersion, + CurrentVersion: params.BeaconConfig().GenesisForkVersion, + Epoch: 0, + }, + + // Validator registry fields. + Validators: preState.Validators(), + Balances: preState.Balances(), + PreviousEpochParticipation: prevEpochParticipation, + CurrentEpochParticipation: currEpochParticipation, + InactivityScores: scores, + + // Randomness and committees. + RandaoMixes: randaoMixes, + + // Finality. + PreviousJustifiedCheckpoint: ðpb.Checkpoint{ + Epoch: 0, + Root: params.BeaconConfig().ZeroHash[:], + }, + CurrentJustifiedCheckpoint: ðpb.Checkpoint{ + Epoch: 0, + Root: params.BeaconConfig().ZeroHash[:], + }, + JustificationBits: []byte{0}, + FinalizedCheckpoint: ðpb.Checkpoint{ + Epoch: 0, + Root: params.BeaconConfig().ZeroHash[:], + }, + + HistoricalRoots: [][]byte{}, + BlockRoots: blockRoots, + StateRoots: stateRoots, + Slashings: slashings, + + // Eth1 data. + Eth1Data: eth1Data, + Eth1DataVotes: []*ethpb.Eth1Data{}, + Eth1DepositIndex: preState.Eth1DepositIndex(), + + // Electra Data + DepositRequestsStartIndex: params.BeaconConfig().UnsetDepositRequestsStartIndex, + ExitBalanceToConsume: helpers.ActivationExitChurnLimit(primitives.Gwei(tab)), + EarliestConsolidationEpoch: helpers.ActivationExitEpoch(slots.ToEpoch(preState.Slot())), + ConsolidationBalanceToConsume: helpers.ConsolidationChurnLimit(primitives.Gwei(tab)), + PendingBalanceDeposits: make([]*ethpb.PendingBalanceDeposit, 0), + PendingPartialWithdrawals: make([]*ethpb.PendingPartialWithdrawal, 0), + PendingConsolidations: make([]*ethpb.PendingConsolidation, 0), + } + + var scBits [fieldparams.SyncAggregateSyncCommitteeBytesLength]byte + kzgs := make([][]byte, 0) + kzgRoot, err := ssz.KzgCommitmentsRoot(kzgs) + if err != nil { + return nil, err + } + bodyRoot, err := (ðpb.BeaconBlockBodyEpbs{ + RandaoReveal: make([]byte, 96), + Eth1Data: ðpb.Eth1Data{ + DepositRoot: make([]byte, 32), + BlockHash: make([]byte, 32), + }, + Graffiti: make([]byte, 32), + SyncAggregate: ðpb.SyncAggregate{ + SyncCommitteeBits: scBits[:], + SyncCommitteeSignature: make([]byte, 96), + }, + SignedExecutionPayloadHeader: &enginev1.SignedExecutionPayloadHeader{ + Message: &enginev1.ExecutionPayloadHeaderEPBS{ + ParentBlockHash: make([]byte, 32), + ParentBlockRoot: make([]byte, 32), + BlockHash: make([]byte, 32), + BlobKzgCommitmentsRoot: kzgRoot[:], + }, + Signature: make([]byte, 96), + }, + BlsToExecutionChanges: make([]*ethpb.SignedBLSToExecutionChange, 0), + PayloadAttestations: make([]*ethpb.PayloadAttestation, 0), + }).HashTreeRoot() + if err != nil { + return nil, errors.Wrap(err, "could not hash tree root empty block body") + } + + st.LatestBlockHeader = ðpb.BeaconBlockHeader{ + ParentRoot: zeroHash, + StateRoot: zeroHash, + BodyRoot: bodyRoot[:], + } + + var pubKeys [][]byte + vals := preState.Validators() + for i := uint64(0); i < params.BeaconConfig().SyncCommitteeSize; i++ { + j := i % uint64(len(vals)) + pubKeys = append(pubKeys, vals[j].PublicKey) + } + aggregated, err := bls.AggregatePublicKeys(pubKeys) + if err != nil { + return nil, err + } + st.CurrentSyncCommittee = ðpb.SyncCommittee{ + Pubkeys: pubKeys, + AggregatePubkey: aggregated.Marshal(), + } + st.NextSyncCommittee = ðpb.SyncCommittee{ + Pubkeys: pubKeys, + AggregatePubkey: aggregated.Marshal(), + } + + st.LatestExecutionPayloadHeader = &enginev1.ExecutionPayloadHeaderEPBS{ + ParentBlockHash: make([]byte, 32), + ParentBlockRoot: make([]byte, 32), + BlockHash: make([]byte, 32), + BlobKzgCommitmentsRoot: kzgRoot[:], + } + + return state_native.InitializeFromProtoEpbs(st) +} + +// DeterministicGenesisStateEpbs returns a genesis state in ePBS format made using the deterministic deposits. +func DeterministicGenesisStateEpbs(t testing.TB, numValidators uint64) (state.BeaconState, []bls.SecretKey) { + deposits, privKeys, err := DeterministicDepositsAndKeys(numValidators) + if err != nil { + t.Fatal(errors.Wrapf(err, "failed to get %d deposits", numValidators)) + } + eth1Data, err := DeterministicEth1Data(len(deposits)) + if err != nil { + t.Fatal(errors.Wrapf(err, "failed to get eth1data for %d deposits", numValidators)) + } + beaconState, err := genesisBeaconStateEpbs(context.Background(), deposits, uint64(0), eth1Data) + if err != nil { + t.Fatal(errors.Wrapf(err, "failed to get genesis beacon state of %d validators", numValidators)) + } + if err := setKeysToActive(beaconState); err != nil { + t.Fatal(errors.Wrapf(err, "failed to set keys to active")) + } + resetCache() + return beaconState, privKeys +} diff --git a/testing/util/helpers.go b/testing/util/helpers.go index b207813a0861..375376bcfbc8 100644 --- a/testing/util/helpers.go +++ b/testing/util/helpers.go @@ -59,6 +59,8 @@ func BlockSignature( wsb, err = blocks.NewSignedBeaconBlock(ðpb.SignedBeaconBlockDeneb{Block: b}) case *ethpb.BeaconBlockElectra: wsb, err = blocks.NewSignedBeaconBlock(ðpb.SignedBeaconBlockElectra{Block: b}) + case *ethpb.BeaconBlockEpbs: + wsb, err = blocks.NewSignedBeaconBlock(ðpb.SignedBeaconBlockEpbs{Block: b}) default: return nil, fmt.Errorf("unsupported block type %T", b) } @@ -83,6 +85,8 @@ func BlockSignature( b.StateRoot = s[:] case *ethpb.BeaconBlockElectra: b.StateRoot = s[:] + case *ethpb.BeaconBlockEpbs: + b.StateRoot = s[:] } // Temporarily increasing the beacon state slot here since BeaconProposerIndex is a @@ -101,6 +105,8 @@ func BlockSignature( blockSlot = b.Slot case *ethpb.BeaconBlockElectra: blockSlot = b.Slot + case *ethpb.BeaconBlockEpbs: + blockSlot = b.Slot } // process slots to get the right fork @@ -128,6 +134,8 @@ func BlockSignature( blockRoot, err = signing.ComputeSigningRoot(b, domain) case *ethpb.BeaconBlockElectra: blockRoot, err = signing.ComputeSigningRoot(b, domain) + case *ethpb.BeaconBlockEpbs: + blockRoot, err = signing.ComputeSigningRoot(b, domain) } if err != nil { return nil, err diff --git a/testing/util/random/epbs.go b/testing/util/random/epbs.go index 51925c36e6ea..dfbacbb31e4e 100644 --- a/testing/util/random/epbs.go +++ b/testing/util/random/epbs.go @@ -47,13 +47,13 @@ func BeaconBlockBody(t *testing.T) *ethpb.BeaconBlockBodyEpbs { {Header_1: SignedBeaconBlockHeader(t), Header_2: SignedBeaconBlockHeader(t)}, }, - AttesterSlashings: []*ethpb.AttesterSlashing{ + AttesterSlashings: []*ethpb.AttesterSlashingElectra{ { Attestation_1: IndexedAttestation(t), Attestation_2: IndexedAttestation(t), }, }, - Attestations: []*ethpb.Attestation{Attestation(t), Attestation(t), Attestation(t)}, + Attestations: []*ethpb.AttestationElectra{Attestation(t), Attestation(t), Attestation(t)}, Deposits: []*ethpb.Deposit{Deposit(t), Deposit(t), Deposit(t)}, VoluntaryExits: []*ethpb.SignedVoluntaryExit{SignedVoluntaryExit(t), SignedVoluntaryExit(t)}, SyncAggregate: ðpb.SyncAggregate{ @@ -83,8 +83,8 @@ func SignedBeaconBlockHeader(t *testing.T) *ethpb.SignedBeaconBlockHeader { } // IndexedAttestation creates a random IndexedAttestation for testing purposes. -func IndexedAttestation(t *testing.T) *ethpb.IndexedAttestation { - return ðpb.IndexedAttestation{ +func IndexedAttestation(t *testing.T) *ethpb.IndexedAttestationElectra { + return ðpb.IndexedAttestationElectra{ AttestingIndices: []uint64{randomUint64(t), randomUint64(t), randomUint64(t)}, Data: AttestationData(t), Signature: randomBytes(96, t), @@ -92,11 +92,12 @@ func IndexedAttestation(t *testing.T) *ethpb.IndexedAttestation { } // Attestation creates a random Attestation for testing purposes. -func Attestation(t *testing.T) *ethpb.Attestation { - return ðpb.Attestation{ +func Attestation(t *testing.T) *ethpb.AttestationElectra { + return ðpb.AttestationElectra{ AggregationBits: bitfield.NewBitlist(123), Data: AttestationData(t), Signature: randomBytes(96, t), + CommitteeBits: primitives.NewAttestationCommitteeBits(), } }