mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-10 05:47:59 -05:00
Compare commits
25 Commits
v5.1.3-rc.
...
fix-events
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
f7d0d815fc | ||
|
|
5c11e82ab7 | ||
|
|
5f75c33c50 | ||
|
|
956d9d108c | ||
|
|
c285715f9f | ||
|
|
9382ae736d | ||
|
|
f16ff45a6b | ||
|
|
8d6577be84 | ||
|
|
9de75b5376 | ||
|
|
a7ba11df37 | ||
|
|
00aeea3656 | ||
|
|
9dbf979e77 | ||
|
|
be60504512 | ||
|
|
1857496159 | ||
|
|
ccf61e1700 | ||
|
|
4edbd2f9ef | ||
|
|
5179af1438 | ||
|
|
c0f9689e30 | ||
|
|
ff8240a04f | ||
|
|
847498c648 | ||
|
|
2633684339 | ||
|
|
ab3f1963e2 | ||
|
|
b87d02eeb3 | ||
|
|
bcb4155523 | ||
|
|
77f10b9e0e |
26
CHANGELOG.md
26
CHANGELOG.md
@@ -8,17 +8,24 @@ The format is based on Keep a Changelog, and this project adheres to Semantic Ve
|
||||
|
||||
### Added
|
||||
|
||||
- Electra EIP6110: Queue deposit [pr](https://github.com/prysmaticlabs/prysm/pull/14430)
|
||||
- Electra EIP6110: Queue deposit [pr](https://github.com/prysmaticlabs/prysm/pull/14430).
|
||||
- Add Bellatrix tests for light client functions.
|
||||
- Add Discovery Rebooter Feature.
|
||||
- Added GetBlockAttestationsV2 endpoint.
|
||||
- Light client support: Consensus types for Electra
|
||||
- Light client support: Consensus types for Electra.
|
||||
- Added SubmitPoolAttesterSlashingV2 endpoint.
|
||||
- Added SubmitAggregateAndProofsRequestV2 endpoint.
|
||||
- Updated the `beacon-chain/monitor` package to Electra. [PR](https://github.com/prysmaticlabs/prysm/pull/14562)
|
||||
- Added ListAttestationsV2 endpoint.
|
||||
- Add ability to rollback node's internal state during processing.
|
||||
- Change how unsafe protobuf state is created to prevent unnecessary copies.
|
||||
- Added benchmarks for process slots for Capella, Deneb, Electra.
|
||||
- Add helper to cast bytes to string without allocating memory.
|
||||
- Added GetAggregatedAttestationV2 endpoint.
|
||||
- Added SubmitAttestationsV2 endpoint.
|
||||
- Validator REST mode Electra block support.
|
||||
- Added validator index label to `validator_statuses` metric.
|
||||
- Added Validator REST mode use of Attestation V2 endpoints and Electra attestations.
|
||||
|
||||
### Changed
|
||||
|
||||
@@ -40,10 +47,19 @@ The format is based on Keep a Changelog, and this project adheres to Semantic Ve
|
||||
- Simplified `ExitedValidatorIndices`.
|
||||
- Simplified `EjectedValidatorIndices`.
|
||||
- `engine_newPayloadV4`,`engine_getPayloadV4` are changes due to new execution request serialization decisions, [PR](https://github.com/prysmaticlabs/prysm/pull/14580)
|
||||
- Fixed various small things in state-native code.
|
||||
- Use ROBlock earlier in block syncing pipeline.
|
||||
- Changed the signature of `ProcessPayload`.
|
||||
- Only Build the Protobuf state once during serialization.
|
||||
- Capella blocks are execution.
|
||||
- Fixed panic when http request to subscribe to event stream fails.
|
||||
- Return early for blob reconstructor during capella fork.
|
||||
- Updated block endpoint from V1 to V2.
|
||||
- Rename instances of "deposit receipts" to "deposit requests".
|
||||
- Non-blocking payload attribute event handling in beacon api [pr](https://github.com/prysmaticlabs/prysm/pull/14644).
|
||||
- Updated light client protobufs. [PR](https://github.com/prysmaticlabs/prysm/pull/14650)
|
||||
- Added `Eth-Consensus-Version` header to `ListAttestationsV2` and `GetAggregateAttestationV2` endpoints.
|
||||
- Updated light client consensus types. [PR](https://github.com/prysmaticlabs/prysm/pull/14652)
|
||||
|
||||
### Deprecated
|
||||
|
||||
@@ -53,6 +69,7 @@ The format is based on Keep a Changelog, and this project adheres to Semantic Ve
|
||||
|
||||
- Removed finalized validator index cache, no longer needed.
|
||||
- Removed validator queue position log on key reload and wait for activation.
|
||||
- Removed outdated spectest exclusions for EIP-6110.
|
||||
|
||||
### Fixed
|
||||
|
||||
@@ -67,6 +84,11 @@ The format is based on Keep a Changelog, and this project adheres to Semantic Ve
|
||||
- Fix keymanager API so that get keys returns an empty response instead of a 500 error when using an unsupported keystore.
|
||||
- Small log imporvement, removing some redundant or duplicate logs
|
||||
- EIP7521 - Fixes withdrawal bug by accounting for pending partial withdrawals and deducting already withdrawn amounts from the sweep balance. [PR](https://github.com/prysmaticlabs/prysm/pull/14578)
|
||||
- unskip electra merkle spec test
|
||||
- Fix panic in validator REST mode when checking status after removing all keys
|
||||
- Fix panic on attestation interface since we call data before validation
|
||||
- corrects nil check on some interface attestation types
|
||||
- temporary solution to handling electra attesation and attester_slashing events. [pr](14655)
|
||||
|
||||
|
||||
### Security
|
||||
|
||||
@@ -93,6 +93,7 @@ func (h *EventStream) Subscribe(eventsChannel chan<- *Event) {
|
||||
EventType: EventConnectionError,
|
||||
Data: []byte(errors.Wrap(err, client.ErrConnectionIssue.Error()).Error()),
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
defer func() {
|
||||
|
||||
@@ -40,7 +40,7 @@ func TestNewEventStream(t *testing.T) {
|
||||
|
||||
func TestEventStream(t *testing.T) {
|
||||
mux := http.NewServeMux()
|
||||
mux.HandleFunc("/eth/v1/events", func(w http.ResponseWriter, r *http.Request) {
|
||||
mux.HandleFunc("/eth/v1/events", func(w http.ResponseWriter, _ *http.Request) {
|
||||
flusher, ok := w.(http.Flusher)
|
||||
require.Equal(t, true, ok)
|
||||
for i := 1; i <= 3; i++ {
|
||||
@@ -79,3 +79,23 @@ func TestEventStream(t *testing.T) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestEventStreamRequestError(t *testing.T) {
|
||||
topics := []string{"head"}
|
||||
eventsChannel := make(chan *Event, 1)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
// use valid url that will result in failed request with nil body
|
||||
stream, err := NewEventStream(ctx, http.DefaultClient, "http://badhost:1234", topics)
|
||||
require.NoError(t, err)
|
||||
|
||||
// error will happen when request is made, should be received over events channel
|
||||
go stream.Subscribe(eventsChannel)
|
||||
|
||||
event := <-eventsChannel
|
||||
if event.EventType != EventConnectionError {
|
||||
t.Errorf("Expected event type %q, got %q", EventConnectionError, event.EventType)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -26,7 +26,7 @@ type ListAttestationsResponse struct {
|
||||
}
|
||||
|
||||
type SubmitAttestationsRequest struct {
|
||||
Data []*Attestation `json:"data"`
|
||||
Data json.RawMessage `json:"data"`
|
||||
}
|
||||
|
||||
type ListVoluntaryExitsResponse struct {
|
||||
|
||||
@@ -7,7 +7,8 @@ import (
|
||||
)
|
||||
|
||||
type AggregateAttestationResponse struct {
|
||||
Data *Attestation `json:"data"`
|
||||
Version string `json:"version,omitempty"`
|
||||
Data json.RawMessage `json:"data"`
|
||||
}
|
||||
|
||||
type SubmitContributionAndProofsRequest struct {
|
||||
|
||||
@@ -6,8 +6,11 @@ import (
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/async/event"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
|
||||
@@ -69,6 +72,7 @@ func (s *Service) notifyForkchoiceUpdate(ctx context.Context, arg *fcuConfig) (*
|
||||
if arg.attributes == nil {
|
||||
arg.attributes = payloadattribute.EmptyWithVersion(headBlk.Version())
|
||||
}
|
||||
go firePayloadAttributesEvent(ctx, s.cfg.StateNotifier.StateFeed(), arg)
|
||||
payloadID, lastValidHash, err := s.cfg.ExecutionEngineCaller.ForkchoiceUpdated(ctx, fcs, arg.attributes)
|
||||
if err != nil {
|
||||
switch {
|
||||
@@ -167,6 +171,38 @@ func (s *Service) notifyForkchoiceUpdate(ctx context.Context, arg *fcuConfig) (*
|
||||
return payloadID, nil
|
||||
}
|
||||
|
||||
func firePayloadAttributesEvent(ctx context.Context, f event.SubscriberSender, cfg *fcuConfig) {
|
||||
pidx, err := helpers.BeaconProposerIndex(ctx, cfg.headState)
|
||||
if err != nil {
|
||||
log.WithError(err).
|
||||
WithField("head_root", cfg.headRoot[:]).
|
||||
Error("Could not get proposer index for PayloadAttributes event")
|
||||
return
|
||||
}
|
||||
evd := payloadattribute.EventData{
|
||||
ProposerIndex: pidx,
|
||||
ProposalSlot: cfg.headState.Slot(),
|
||||
ParentBlockRoot: cfg.headRoot[:],
|
||||
Attributer: cfg.attributes,
|
||||
HeadRoot: cfg.headRoot,
|
||||
HeadState: cfg.headState,
|
||||
HeadBlock: cfg.headBlock,
|
||||
}
|
||||
if cfg.headBlock != nil && !cfg.headBlock.IsNil() {
|
||||
headPayload, err := cfg.headBlock.Block().Body().Execution()
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not get execution payload for head block")
|
||||
return
|
||||
}
|
||||
evd.ParentBlockHash = headPayload.BlockHash()
|
||||
evd.ParentBlockNumber = headPayload.BlockNumber()
|
||||
}
|
||||
f.Send(&feed.Event{
|
||||
Type: statefeed.PayloadAttributes,
|
||||
Data: evd,
|
||||
})
|
||||
}
|
||||
|
||||
// getPayloadHash returns the payload hash given the block root.
|
||||
// if the block is before bellatrix fork epoch, it returns the zero hash.
|
||||
func (s *Service) getPayloadHash(ctx context.Context, root []byte) ([32]byte, error) {
|
||||
|
||||
@@ -92,12 +92,12 @@ func TestStore_OnAttestation_ErrorConditions(t *testing.T) {
|
||||
{
|
||||
name: "process nil attestation",
|
||||
a: nil,
|
||||
wantedErr: "attestation can't be nil",
|
||||
wantedErr: "attestation is nil",
|
||||
},
|
||||
{
|
||||
name: "process nil field (a.Data) in attestation",
|
||||
a: ðpb.Attestation{},
|
||||
wantedErr: "attestation's data can't be nil",
|
||||
wantedErr: "attestation is nil",
|
||||
},
|
||||
{
|
||||
name: "process nil field (a.Target) in attestation",
|
||||
|
||||
@@ -7,8 +7,6 @@ import (
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
coreTime "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
|
||||
@@ -620,9 +618,6 @@ func (s *Service) lateBlockTasks(ctx context.Context) {
|
||||
if !s.inRegularSync() {
|
||||
return
|
||||
}
|
||||
s.cfg.StateNotifier.StateFeed().Send(&feed.Event{
|
||||
Type: statefeed.MissedSlot,
|
||||
})
|
||||
s.headLock.RLock()
|
||||
headRoot := s.headRoot()
|
||||
headState := s.headState(ctx)
|
||||
@@ -650,6 +645,13 @@ func (s *Service) lateBlockTasks(ctx context.Context) {
|
||||
attribute := s.getPayloadAttribute(ctx, headState, s.CurrentSlot()+1, headRoot[:])
|
||||
// return early if we are not proposing next slot
|
||||
if attribute.IsEmpty() {
|
||||
fcuArgs := &fcuConfig{
|
||||
headState: headState,
|
||||
headRoot: headRoot,
|
||||
headBlock: nil,
|
||||
attributes: attribute,
|
||||
}
|
||||
go firePayloadAttributesEvent(ctx, s.cfg.StateNotifier.StateFeed(), fcuArgs)
|
||||
return
|
||||
}
|
||||
|
||||
|
||||
@@ -448,6 +448,7 @@ func TestValidateIndexedAttestation_AboveMaxLength(t *testing.T) {
|
||||
Target: ðpb.Checkpoint{
|
||||
Epoch: primitives.Epoch(i),
|
||||
},
|
||||
Source: ðpb.Checkpoint{},
|
||||
}
|
||||
}
|
||||
|
||||
@@ -489,6 +490,7 @@ func TestValidateIndexedAttestation_BadAttestationsSignatureSet(t *testing.T) {
|
||||
Target: ðpb.Checkpoint{
|
||||
Root: []byte{},
|
||||
},
|
||||
Source: ðpb.Checkpoint{},
|
||||
},
|
||||
Signature: sig.Marshal(),
|
||||
AggregationBits: list,
|
||||
|
||||
@@ -193,7 +193,7 @@ func ProcessWithdrawals(st state.BeaconState, executionData interfaces.Execution
|
||||
}
|
||||
|
||||
if st.Version() >= version.Electra {
|
||||
if err := st.DequeuePartialWithdrawals(processedPartialWithdrawalsCount); err != nil {
|
||||
if err := st.DequeuePendingPartialWithdrawals(processedPartialWithdrawalsCount); err != nil {
|
||||
return nil, fmt.Errorf("unable to dequeue partial withdrawals from state: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -560,7 +560,7 @@ func ProcessDepositRequests(ctx context.Context, beaconState state.BeaconState,
|
||||
return beaconState, nil
|
||||
}
|
||||
|
||||
// processDepositRequest processes the specific deposit receipt
|
||||
// processDepositRequest processes the specific deposit request
|
||||
// def process_deposit_request(state: BeaconState, deposit_request: DepositRequest) -> None:
|
||||
//
|
||||
// # Set deposit request start index
|
||||
@@ -590,8 +590,8 @@ func processDepositRequest(beaconState state.BeaconState, request *enginev1.Depo
|
||||
}
|
||||
if err := beaconState.AppendPendingDeposit(ðpb.PendingDeposit{
|
||||
PublicKey: bytesutil.SafeCopyBytes(request.Pubkey),
|
||||
Amount: request.Amount,
|
||||
WithdrawalCredentials: bytesutil.SafeCopyBytes(request.WithdrawalCredentials),
|
||||
Amount: request.Amount,
|
||||
Signature: bytesutil.SafeCopyBytes(request.Signature),
|
||||
Slot: beaconState.Slot(),
|
||||
}); err != nil {
|
||||
|
||||
@@ -29,7 +29,6 @@ var (
|
||||
ProcessParticipationFlagUpdates = altair.ProcessParticipationFlagUpdates
|
||||
ProcessSyncCommitteeUpdates = altair.ProcessSyncCommitteeUpdates
|
||||
AttestationsDelta = altair.AttestationsDelta
|
||||
ProcessSyncAggregate = altair.ProcessSyncAggregate
|
||||
)
|
||||
|
||||
// ProcessEpoch describes the per epoch operations that are performed on the beacon state.
|
||||
|
||||
@@ -84,11 +84,11 @@ func ProcessOperations(
|
||||
}
|
||||
st, err = ProcessDepositRequests(ctx, st, requests.Deposits)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not process deposit receipts")
|
||||
return nil, errors.Wrap(err, "could not process deposit requests")
|
||||
}
|
||||
st, err = ProcessWithdrawalRequests(ctx, st, requests.Withdrawals)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not process execution layer withdrawal requests")
|
||||
return nil, errors.Wrap(err, "could not process withdrawal requests")
|
||||
}
|
||||
if err := ProcessConsolidationRequests(ctx, st, requests.Consolidations); err != nil {
|
||||
return nil, fmt.Errorf("could not process consolidation requests: %w", err)
|
||||
|
||||
@@ -31,6 +31,8 @@ const (
|
||||
LightClientFinalityUpdate
|
||||
// LightClientOptimisticUpdate event
|
||||
LightClientOptimisticUpdate
|
||||
// PayloadAttributes events are fired upon a missed slot or new head.
|
||||
PayloadAttributes
|
||||
)
|
||||
|
||||
// BlockProcessedData is the data sent with BlockProcessed events.
|
||||
|
||||
@@ -23,11 +23,8 @@ var (
|
||||
// Access to these nil fields will result in run time panic,
|
||||
// it is recommended to run these checks as first line of defense.
|
||||
func ValidateNilAttestation(attestation ethpb.Att) error {
|
||||
if attestation == nil {
|
||||
return errors.New("attestation can't be nil")
|
||||
}
|
||||
if attestation.GetData() == nil {
|
||||
return errors.New("attestation's data can't be nil")
|
||||
if attestation == nil || attestation.IsNil() {
|
||||
return errors.New("attestation is nil")
|
||||
}
|
||||
if attestation.GetData().Source == nil {
|
||||
return errors.New("attestation's source can't be nil")
|
||||
|
||||
@@ -260,12 +260,12 @@ func TestValidateNilAttestation(t *testing.T) {
|
||||
{
|
||||
name: "nil attestation",
|
||||
attestation: nil,
|
||||
errString: "attestation can't be nil",
|
||||
errString: "attestation is nil",
|
||||
},
|
||||
{
|
||||
name: "nil attestation data",
|
||||
attestation: ðpb.Attestation{},
|
||||
errString: "attestation's data can't be nil",
|
||||
errString: "attestation is nil",
|
||||
},
|
||||
{
|
||||
name: "nil attestation source",
|
||||
|
||||
@@ -698,3 +698,45 @@ func TestProcessSlotsConditionally(t *testing.T) {
|
||||
assert.Equal(t, primitives.Slot(6), s.Slot())
|
||||
})
|
||||
}
|
||||
|
||||
func BenchmarkProcessSlots_Capella(b *testing.B) {
|
||||
st, _ := util.DeterministicGenesisStateCapella(b, params.BeaconConfig().MaxValidatorsPerCommittee)
|
||||
|
||||
var err error
|
||||
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
st, err = transition.ProcessSlots(context.Background(), st, st.Slot()+1)
|
||||
if err != nil {
|
||||
b.Fatalf("Failed to process slot %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkProcessSlots_Deneb(b *testing.B) {
|
||||
st, _ := util.DeterministicGenesisStateDeneb(b, params.BeaconConfig().MaxValidatorsPerCommittee)
|
||||
|
||||
var err error
|
||||
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
st, err = transition.ProcessSlots(context.Background(), st, st.Slot()+1)
|
||||
if err != nil {
|
||||
b.Fatalf("Failed to process slot %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkProcessSlots_Electra(b *testing.B) {
|
||||
st, _ := util.DeterministicGenesisStateElectra(b, params.BeaconConfig().MaxValidatorsPerCommittee)
|
||||
|
||||
var err error
|
||||
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
st, err = transition.ProcessSlots(context.Background(), st, st.Slot()+1)
|
||||
if err != nil {
|
||||
b.Fatalf("Failed to process slot %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -688,7 +688,7 @@ func decodeSlasherChunk(enc []byte) ([]uint16, error) {
|
||||
// Encode attestation record to bytes.
|
||||
// The output encoded attestation record consists in the signing root concatenated with the compressed attestation record.
|
||||
func encodeAttestationRecord(att *slashertypes.IndexedAttestationWrapper) ([]byte, error) {
|
||||
if att == nil || att.IndexedAttestation == nil {
|
||||
if att == nil || att.IndexedAttestation == nil || att.IndexedAttestation.IsNil() {
|
||||
return []byte{}, errors.New("nil proposal record")
|
||||
}
|
||||
|
||||
|
||||
@@ -192,20 +192,13 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
|
||||
beacon.verifyInitWaiter = verification.NewInitializerWaiter(
|
||||
beacon.clockWaiter, forkchoice.NewROForkChoice(beacon.forkChoicer), beacon.stateGen)
|
||||
|
||||
pa := peers.NewAssigner(beacon.fetchP2P().Peers(), beacon.forkChoicer)
|
||||
|
||||
beacon.BackfillOpts = append(
|
||||
beacon.BackfillOpts,
|
||||
backfill.WithVerifierWaiter(beacon.verifyInitWaiter),
|
||||
backfill.WithInitSyncWaiter(initSyncWaiter(ctx, beacon.initialSyncComplete)),
|
||||
)
|
||||
|
||||
bf, err := backfill.NewService(ctx, bfs, beacon.BlobStorage, beacon.clockWaiter, beacon.fetchP2P(), pa, beacon.BackfillOpts...)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "error initializing backfill service")
|
||||
}
|
||||
|
||||
if err := registerServices(cliCtx, beacon, synchronizer, bf, bfs); err != nil {
|
||||
if err := registerServices(cliCtx, beacon, synchronizer, bfs); err != nil {
|
||||
return nil, errors.Wrap(err, "could not register services")
|
||||
}
|
||||
|
||||
@@ -292,11 +285,6 @@ func startBaseServices(cliCtx *cli.Context, beacon *BeaconNode, depositAddress s
|
||||
return nil, errors.Wrap(err, "could not start slashing DB")
|
||||
}
|
||||
|
||||
log.Debugln("Registering P2P Service")
|
||||
if err := beacon.registerP2P(cliCtx); err != nil {
|
||||
return nil, errors.Wrap(err, "could not register P2P service")
|
||||
}
|
||||
|
||||
bfs, err := backfill.NewUpdater(ctx, beacon.db)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not create backfill updater")
|
||||
@@ -315,9 +303,15 @@ func startBaseServices(cliCtx *cli.Context, beacon *BeaconNode, depositAddress s
|
||||
return bfs, nil
|
||||
}
|
||||
|
||||
func registerServices(cliCtx *cli.Context, beacon *BeaconNode, synchronizer *startup.ClockSynchronizer, bf *backfill.Service, bfs *backfill.Store) error {
|
||||
if err := beacon.services.RegisterService(bf); err != nil {
|
||||
return errors.Wrap(err, "could not register backfill service")
|
||||
func registerServices(cliCtx *cli.Context, beacon *BeaconNode, synchronizer *startup.ClockSynchronizer, bfs *backfill.Store) error {
|
||||
log.Debugln("Registering P2P Service")
|
||||
if err := beacon.registerP2P(cliCtx); err != nil {
|
||||
return errors.Wrap(err, "could not register P2P service")
|
||||
}
|
||||
|
||||
log.Debugln("Registering Backfill Service")
|
||||
if err := beacon.RegisterBackfillService(cliCtx, bfs); err != nil {
|
||||
return errors.Wrap(err, "could not register Back Fill service")
|
||||
}
|
||||
|
||||
log.Debugln("Registering POW Chain Service")
|
||||
@@ -1136,6 +1130,16 @@ func (b *BeaconNode) registerBuilderService(cliCtx *cli.Context) error {
|
||||
return b.services.RegisterService(svc)
|
||||
}
|
||||
|
||||
func (b *BeaconNode) RegisterBackfillService(cliCtx *cli.Context, bfs *backfill.Store) error {
|
||||
pa := peers.NewAssigner(b.fetchP2P().Peers(), b.forkChoicer)
|
||||
bf, err := backfill.NewService(cliCtx.Context, bfs, b.BlobStorage, b.clockWaiter, b.fetchP2P(), pa, b.BackfillOpts...)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "error initializing backfill service")
|
||||
}
|
||||
|
||||
return b.services.RegisterService(bf)
|
||||
}
|
||||
|
||||
func hasNetworkFlag(cliCtx *cli.Context) bool {
|
||||
for _, flag := range features.NetworkFlags {
|
||||
for _, name := range flag.Names() {
|
||||
|
||||
@@ -49,12 +49,12 @@ func TestKV_Aggregated_SaveAggregatedAttestation(t *testing.T) {
|
||||
{
|
||||
name: "nil attestation",
|
||||
att: nil,
|
||||
wantErrString: "attestation can't be nil",
|
||||
wantErrString: "attestation is nil",
|
||||
},
|
||||
{
|
||||
name: "nil attestation data",
|
||||
att: ðpb.Attestation{},
|
||||
wantErrString: "attestation's data can't be nil",
|
||||
wantErrString: "attestation is nil",
|
||||
},
|
||||
{
|
||||
name: "not aggregated",
|
||||
@@ -206,7 +206,7 @@ func TestKV_Aggregated_AggregatedAttestations(t *testing.T) {
|
||||
func TestKV_Aggregated_DeleteAggregatedAttestation(t *testing.T) {
|
||||
t.Run("nil attestation", func(t *testing.T) {
|
||||
cache := NewAttCaches()
|
||||
assert.ErrorContains(t, "attestation can't be nil", cache.DeleteAggregatedAttestation(nil))
|
||||
assert.ErrorContains(t, "attestation is nil", cache.DeleteAggregatedAttestation(nil))
|
||||
att := util.HydrateAttestation(ðpb.Attestation{AggregationBits: bitfield.Bitlist{0b10101}, Data: ðpb.AttestationData{Slot: 2}})
|
||||
assert.NoError(t, cache.DeleteAggregatedAttestation(att))
|
||||
})
|
||||
@@ -288,7 +288,7 @@ func TestKV_Aggregated_HasAggregatedAttestation(t *testing.T) {
|
||||
name: "nil attestation",
|
||||
input: nil,
|
||||
want: false,
|
||||
err: errors.New("can't be nil"),
|
||||
err: errors.New("is nil"),
|
||||
},
|
||||
{
|
||||
name: "nil attestation data",
|
||||
@@ -296,7 +296,7 @@ func TestKV_Aggregated_HasAggregatedAttestation(t *testing.T) {
|
||||
AggregationBits: bitfield.Bitlist{0b1111},
|
||||
},
|
||||
want: false,
|
||||
err: errors.New("can't be nil"),
|
||||
err: errors.New("is nil"),
|
||||
},
|
||||
{
|
||||
name: "empty cache aggregated",
|
||||
|
||||
@@ -8,7 +8,7 @@ import (
|
||||
|
||||
// SaveBlockAttestation saves an block attestation in cache.
|
||||
func (c *AttCaches) SaveBlockAttestation(att ethpb.Att) error {
|
||||
if att == nil {
|
||||
if att == nil || att.IsNil() {
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -53,10 +53,9 @@ func (c *AttCaches) BlockAttestations() []ethpb.Att {
|
||||
|
||||
// DeleteBlockAttestation deletes a block attestation in cache.
|
||||
func (c *AttCaches) DeleteBlockAttestation(att ethpb.Att) error {
|
||||
if att == nil {
|
||||
if att == nil || att.IsNil() {
|
||||
return nil
|
||||
}
|
||||
|
||||
id, err := attestation.NewId(att, attestation.Data)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not create attestation ID")
|
||||
|
||||
@@ -8,7 +8,7 @@ import (
|
||||
|
||||
// SaveForkchoiceAttestation saves an forkchoice attestation in cache.
|
||||
func (c *AttCaches) SaveForkchoiceAttestation(att ethpb.Att) error {
|
||||
if att == nil {
|
||||
if att == nil || att.IsNil() {
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -50,7 +50,7 @@ func (c *AttCaches) ForkchoiceAttestations() []ethpb.Att {
|
||||
|
||||
// DeleteForkchoiceAttestation deletes a forkchoice attestation in cache.
|
||||
func (c *AttCaches) DeleteForkchoiceAttestation(att ethpb.Att) error {
|
||||
if att == nil {
|
||||
if att == nil || att.IsNil() {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@@ -14,7 +14,7 @@ import (
|
||||
|
||||
// SaveUnaggregatedAttestation saves an unaggregated attestation in cache.
|
||||
func (c *AttCaches) SaveUnaggregatedAttestation(att ethpb.Att) error {
|
||||
if att == nil {
|
||||
if att == nil || att.IsNil() {
|
||||
return nil
|
||||
}
|
||||
if helpers.IsAggregated(att) {
|
||||
@@ -130,9 +130,10 @@ func (c *AttCaches) UnaggregatedAttestationsBySlotIndexElectra(
|
||||
|
||||
// DeleteUnaggregatedAttestation deletes the unaggregated attestations in cache.
|
||||
func (c *AttCaches) DeleteUnaggregatedAttestation(att ethpb.Att) error {
|
||||
if att == nil {
|
||||
if att == nil || att.IsNil() {
|
||||
return nil
|
||||
}
|
||||
|
||||
if helpers.IsAggregated(att) {
|
||||
return errors.New("attestation is aggregated")
|
||||
}
|
||||
@@ -161,7 +162,7 @@ func (c *AttCaches) DeleteSeenUnaggregatedAttestations() (int, error) {
|
||||
|
||||
count := 0
|
||||
for r, att := range c.unAggregatedAtt {
|
||||
if att == nil || helpers.IsAggregated(att) {
|
||||
if att == nil || att.IsNil() || helpers.IsAggregated(att) {
|
||||
continue
|
||||
}
|
||||
if seen, err := c.hasSeenBit(att); err == nil && seen {
|
||||
|
||||
@@ -29,7 +29,7 @@ func MsgID(genesisValidatorsRoot []byte, pmsg *pubsubpb.Message) string {
|
||||
// never be hit.
|
||||
msg := make([]byte, 20)
|
||||
copy(msg, "invalid")
|
||||
return string(msg)
|
||||
return bytesutil.UnsafeCastToString(msg)
|
||||
}
|
||||
digest, err := ExtractGossipDigest(*pmsg.Topic)
|
||||
if err != nil {
|
||||
@@ -37,7 +37,7 @@ func MsgID(genesisValidatorsRoot []byte, pmsg *pubsubpb.Message) string {
|
||||
// never be hit.
|
||||
msg := make([]byte, 20)
|
||||
copy(msg, "invalid")
|
||||
return string(msg)
|
||||
return bytesutil.UnsafeCastToString(msg)
|
||||
}
|
||||
_, fEpoch, err := forks.RetrieveForkDataFromDigest(digest, genesisValidatorsRoot)
|
||||
if err != nil {
|
||||
@@ -45,7 +45,7 @@ func MsgID(genesisValidatorsRoot []byte, pmsg *pubsubpb.Message) string {
|
||||
// never be hit.
|
||||
msg := make([]byte, 20)
|
||||
copy(msg, "invalid")
|
||||
return string(msg)
|
||||
return bytesutil.UnsafeCastToString(msg)
|
||||
}
|
||||
if fEpoch >= params.BeaconConfig().AltairForkEpoch {
|
||||
return postAltairMsgID(pmsg, fEpoch)
|
||||
@@ -54,11 +54,11 @@ func MsgID(genesisValidatorsRoot []byte, pmsg *pubsubpb.Message) string {
|
||||
if err != nil {
|
||||
combinedData := append(params.BeaconConfig().MessageDomainInvalidSnappy[:], pmsg.Data...)
|
||||
h := hash.Hash(combinedData)
|
||||
return string(h[:20])
|
||||
return bytesutil.UnsafeCastToString(h[:20])
|
||||
}
|
||||
combinedData := append(params.BeaconConfig().MessageDomainValidSnappy[:], decodedData...)
|
||||
h := hash.Hash(combinedData)
|
||||
return string(h[:20])
|
||||
return bytesutil.UnsafeCastToString(h[:20])
|
||||
}
|
||||
|
||||
// Spec:
|
||||
@@ -93,13 +93,13 @@ func postAltairMsgID(pmsg *pubsubpb.Message, fEpoch primitives.Epoch) string {
|
||||
// should never happen
|
||||
msg := make([]byte, 20)
|
||||
copy(msg, "invalid")
|
||||
return string(msg)
|
||||
return bytesutil.UnsafeCastToString(msg)
|
||||
}
|
||||
if uint64(totalLength) > gossipPubSubSize {
|
||||
// this should never happen
|
||||
msg := make([]byte, 20)
|
||||
copy(msg, "invalid")
|
||||
return string(msg)
|
||||
return bytesutil.UnsafeCastToString(msg)
|
||||
}
|
||||
combinedData := make([]byte, 0, totalLength)
|
||||
combinedData = append(combinedData, params.BeaconConfig().MessageDomainInvalidSnappy[:]...)
|
||||
@@ -107,7 +107,7 @@ func postAltairMsgID(pmsg *pubsubpb.Message, fEpoch primitives.Epoch) string {
|
||||
combinedData = append(combinedData, topic...)
|
||||
combinedData = append(combinedData, pmsg.Data...)
|
||||
h := hash.Hash(combinedData)
|
||||
return string(h[:20])
|
||||
return bytesutil.UnsafeCastToString(h[:20])
|
||||
}
|
||||
totalLength, err := math.AddInt(
|
||||
len(params.BeaconConfig().MessageDomainValidSnappy),
|
||||
@@ -120,7 +120,7 @@ func postAltairMsgID(pmsg *pubsubpb.Message, fEpoch primitives.Epoch) string {
|
||||
// should never happen
|
||||
msg := make([]byte, 20)
|
||||
copy(msg, "invalid")
|
||||
return string(msg)
|
||||
return bytesutil.UnsafeCastToString(msg)
|
||||
}
|
||||
combinedData := make([]byte, 0, totalLength)
|
||||
combinedData = append(combinedData, params.BeaconConfig().MessageDomainValidSnappy[:]...)
|
||||
@@ -128,5 +128,5 @@ func postAltairMsgID(pmsg *pubsubpb.Message, fEpoch primitives.Epoch) string {
|
||||
combinedData = append(combinedData, topic...)
|
||||
combinedData = append(combinedData, decodedData...)
|
||||
h := hash.Hash(combinedData)
|
||||
return string(h[:20])
|
||||
return bytesutil.UnsafeCastToString(h[:20])
|
||||
}
|
||||
|
||||
@@ -381,21 +381,12 @@ func (s *Service) SubmitSignedAggregateSelectionProof(
|
||||
ctx, span := trace.StartSpan(ctx, "coreService.SubmitSignedAggregateSelectionProof")
|
||||
defer span.End()
|
||||
|
||||
if agg == nil {
|
||||
if agg == nil || agg.IsNil() {
|
||||
return &RpcError{Err: errors.New("signed aggregate request can't be nil"), Reason: BadRequest}
|
||||
}
|
||||
attAndProof := agg.AggregateAttestationAndProof()
|
||||
if attAndProof == nil {
|
||||
return &RpcError{Err: errors.New("signed aggregate request can't be nil"), Reason: BadRequest}
|
||||
}
|
||||
att := attAndProof.AggregateVal()
|
||||
if att == nil {
|
||||
return &RpcError{Err: errors.New("signed aggregate request can't be nil"), Reason: BadRequest}
|
||||
}
|
||||
data := att.GetData()
|
||||
if data == nil {
|
||||
return &RpcError{Err: errors.New("signed aggregate request can't be nil"), Reason: BadRequest}
|
||||
}
|
||||
emptySig := make([]byte, fieldparams.BLSSignatureLength)
|
||||
if bytes.Equal(agg.GetSignature(), emptySig) || bytes.Equal(attAndProof.GetSelectionProof(), emptySig) {
|
||||
return &RpcError{Err: errors.New("signed signatures can't be zero hashes"), Reason: BadRequest}
|
||||
|
||||
@@ -199,6 +199,15 @@ func (s *Service) validatorEndpoints(
|
||||
handler: server.GetAggregateAttestation,
|
||||
methods: []string{http.MethodGet},
|
||||
},
|
||||
{
|
||||
template: "/eth/v2/validator/aggregate_attestation",
|
||||
name: namespace + ".GetAggregateAttestationV2",
|
||||
middleware: []middleware.Middleware{
|
||||
middleware.AcceptHeaderHandler([]string{api.JsonMediaType}),
|
||||
},
|
||||
handler: server.GetAggregateAttestationV2,
|
||||
methods: []string{http.MethodGet},
|
||||
},
|
||||
{
|
||||
template: "/eth/v1/validator/contribution_and_proofs",
|
||||
name: namespace + ".SubmitContributionAndProofs",
|
||||
@@ -601,7 +610,7 @@ func (s *Service) beaconEndpoints(
|
||||
middleware: []middleware.Middleware{
|
||||
middleware.AcceptHeaderHandler([]string{api.JsonMediaType}),
|
||||
},
|
||||
handler: server.GetBlockAttestations,
|
||||
handler: server.GetBlockAttestationsV2,
|
||||
methods: []string{http.MethodGet},
|
||||
},
|
||||
{
|
||||
@@ -650,6 +659,16 @@ func (s *Service) beaconEndpoints(
|
||||
handler: server.SubmitAttestations,
|
||||
methods: []string{http.MethodPost},
|
||||
},
|
||||
{
|
||||
template: "/eth/v2/beacon/pool/attestations",
|
||||
name: namespace + ".SubmitAttestationsV2",
|
||||
middleware: []middleware.Middleware{
|
||||
middleware.ContentTypeHandler([]string{api.JsonMediaType}),
|
||||
middleware.AcceptHeaderHandler([]string{api.JsonMediaType}),
|
||||
},
|
||||
handler: server.SubmitAttestationsV2,
|
||||
methods: []string{http.MethodPost},
|
||||
},
|
||||
{
|
||||
template: "/eth/v1/beacon/pool/voluntary_exits",
|
||||
name: namespace + ".ListVoluntaryExits",
|
||||
|
||||
@@ -41,7 +41,7 @@ func Test_endpoints(t *testing.T) {
|
||||
"/eth/v1/beacon/deposit_snapshot": {http.MethodGet},
|
||||
"/eth/v1/beacon/blinded_blocks/{block_id}": {http.MethodGet},
|
||||
"/eth/v1/beacon/pool/attestations": {http.MethodGet, http.MethodPost},
|
||||
"/eth/v2/beacon/pool/attestations": {http.MethodGet},
|
||||
"/eth/v2/beacon/pool/attestations": {http.MethodGet, http.MethodPost},
|
||||
"/eth/v1/beacon/pool/attester_slashings": {http.MethodGet, http.MethodPost},
|
||||
"/eth/v2/beacon/pool/attester_slashings": {http.MethodGet, http.MethodPost},
|
||||
"/eth/v1/beacon/pool/proposer_slashings": {http.MethodGet, http.MethodPost},
|
||||
@@ -101,6 +101,7 @@ func Test_endpoints(t *testing.T) {
|
||||
"/eth/v1/validator/blinded_blocks/{slot}": {http.MethodGet},
|
||||
"/eth/v1/validator/attestation_data": {http.MethodGet},
|
||||
"/eth/v1/validator/aggregate_attestation": {http.MethodGet},
|
||||
"/eth/v2/validator/aggregate_attestation": {http.MethodGet},
|
||||
"/eth/v1/validator/aggregate_and_proofs": {http.MethodPost},
|
||||
"/eth/v2/validator/aggregate_and_proofs": {http.MethodPost},
|
||||
"/eth/v1/validator/beacon_committee_subscriptions": {http.MethodPost},
|
||||
|
||||
@@ -3,13 +3,14 @@ package beacon
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/api"
|
||||
"github.com/prysmaticlabs/prysm/v5/api/server"
|
||||
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
|
||||
@@ -148,6 +149,7 @@ func (s *Server) ListAttestationsV2(w http.ResponseWriter, r *http.Request) {
|
||||
return
|
||||
}
|
||||
|
||||
w.Header().Set(api.VersionHeader, version.String(headState.Version()))
|
||||
httputil.WriteJson(w, &structs.ListAttestationsResponse{
|
||||
Version: version.String(headState.Version()),
|
||||
Data: attsData,
|
||||
@@ -189,70 +191,13 @@ func (s *Server) SubmitAttestations(w http.ResponseWriter, r *http.Request) {
|
||||
httputil.HandleError(w, "Could not decode request body: "+err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
if len(req.Data) == 0 {
|
||||
httputil.HandleError(w, "No data submitted", http.StatusBadRequest)
|
||||
|
||||
attFailures, failedBroadcasts, err := s.handleAttestations(ctx, req.Data)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
var validAttestations []*eth.Attestation
|
||||
var attFailures []*server.IndexedVerificationFailure
|
||||
for i, sourceAtt := range req.Data {
|
||||
att, err := sourceAtt.ToConsensus()
|
||||
if err != nil {
|
||||
attFailures = append(attFailures, &server.IndexedVerificationFailure{
|
||||
Index: i,
|
||||
Message: "Could not convert request attestation to consensus attestation: " + err.Error(),
|
||||
})
|
||||
continue
|
||||
}
|
||||
if _, err = bls.SignatureFromBytes(att.Signature); err != nil {
|
||||
attFailures = append(attFailures, &server.IndexedVerificationFailure{
|
||||
Index: i,
|
||||
Message: "Incorrect attestation signature: " + err.Error(),
|
||||
})
|
||||
continue
|
||||
}
|
||||
|
||||
// Broadcast the unaggregated attestation on a feed to notify other services in the beacon node
|
||||
// of a received unaggregated attestation.
|
||||
// Note we can't send for aggregated att because we don't have selection proof.
|
||||
if !corehelpers.IsAggregated(att) {
|
||||
s.OperationNotifier.OperationFeed().Send(&feed.Event{
|
||||
Type: operation.UnaggregatedAttReceived,
|
||||
Data: &operation.UnAggregatedAttReceivedData{
|
||||
Attestation: att,
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
validAttestations = append(validAttestations, att)
|
||||
}
|
||||
|
||||
failedBroadcasts := make([]string, 0)
|
||||
for i, att := range validAttestations {
|
||||
// Determine subnet to broadcast attestation to
|
||||
wantedEpoch := slots.ToEpoch(att.Data.Slot)
|
||||
vals, err := s.HeadFetcher.HeadValidatorsIndices(ctx, wantedEpoch)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get head validator indices: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
subnet := corehelpers.ComputeSubnetFromCommitteeAndSlot(uint64(len(vals)), att.Data.CommitteeIndex, att.Data.Slot)
|
||||
|
||||
if err = s.Broadcaster.BroadcastAttestation(ctx, subnet, att); err != nil {
|
||||
log.WithError(err).Errorf("could not broadcast attestation at index %d", i)
|
||||
}
|
||||
|
||||
if corehelpers.IsAggregated(att) {
|
||||
if err = s.AttestationsPool.SaveAggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Error("could not save aggregated attestation")
|
||||
}
|
||||
} else {
|
||||
if err = s.AttestationsPool.SaveUnaggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Error("could not save unaggregated attestation")
|
||||
}
|
||||
}
|
||||
}
|
||||
if len(failedBroadcasts) > 0 {
|
||||
httputil.HandleError(
|
||||
w,
|
||||
@@ -272,6 +217,213 @@ func (s *Server) SubmitAttestations(w http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
}
|
||||
|
||||
// SubmitAttestationsV2 submits an attestation object to node. If the attestation passes all validation
|
||||
// constraints, node MUST publish the attestation on an appropriate subnet.
|
||||
func (s *Server) SubmitAttestationsV2(w http.ResponseWriter, r *http.Request) {
|
||||
ctx, span := trace.StartSpan(r.Context(), "beacon.SubmitAttestationsV2")
|
||||
defer span.End()
|
||||
|
||||
versionHeader := r.Header.Get(api.VersionHeader)
|
||||
if versionHeader == "" {
|
||||
httputil.HandleError(w, api.VersionHeader+" header is required", http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
v, err := version.FromString(versionHeader)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Invalid version: "+err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
var req structs.SubmitAttestationsRequest
|
||||
err = json.NewDecoder(r.Body).Decode(&req.Data)
|
||||
switch {
|
||||
case errors.Is(err, io.EOF):
|
||||
httputil.HandleError(w, "No data submitted", http.StatusBadRequest)
|
||||
return
|
||||
case err != nil:
|
||||
httputil.HandleError(w, "Could not decode request body: "+err.Error(), http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
var attFailures []*server.IndexedVerificationFailure
|
||||
var failedBroadcasts []string
|
||||
|
||||
if v >= version.Electra {
|
||||
attFailures, failedBroadcasts, err = s.handleAttestationsElectra(ctx, req.Data)
|
||||
} else {
|
||||
attFailures, failedBroadcasts, err = s.handleAttestations(ctx, req.Data)
|
||||
}
|
||||
if err != nil {
|
||||
httputil.HandleError(w, fmt.Sprintf("Failed to handle attestations: %v", err), http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
if len(failedBroadcasts) > 0 {
|
||||
httputil.HandleError(
|
||||
w,
|
||||
fmt.Sprintf("Attestations at index %s could not be broadcasted", strings.Join(failedBroadcasts, ", ")),
|
||||
http.StatusInternalServerError,
|
||||
)
|
||||
return
|
||||
}
|
||||
|
||||
if len(attFailures) > 0 {
|
||||
failuresErr := &server.IndexedVerificationFailureError{
|
||||
Code: http.StatusBadRequest,
|
||||
Message: "One or more attestations failed validation",
|
||||
Failures: attFailures,
|
||||
}
|
||||
httputil.WriteError(w, failuresErr)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Server) handleAttestationsElectra(ctx context.Context, data json.RawMessage) (attFailures []*server.IndexedVerificationFailure, failedBroadcasts []string, err error) {
|
||||
var sourceAttestations []*structs.AttestationElectra
|
||||
|
||||
if err = json.Unmarshal(data, &sourceAttestations); err != nil {
|
||||
return nil, nil, errors.Wrap(err, "failed to unmarshal attestation")
|
||||
}
|
||||
|
||||
if len(sourceAttestations) == 0 {
|
||||
return nil, nil, errors.New("no data submitted")
|
||||
}
|
||||
|
||||
var validAttestations []*eth.AttestationElectra
|
||||
for i, sourceAtt := range sourceAttestations {
|
||||
att, err := sourceAtt.ToConsensus()
|
||||
if err != nil {
|
||||
attFailures = append(attFailures, &server.IndexedVerificationFailure{
|
||||
Index: i,
|
||||
Message: "Could not convert request attestation to consensus attestation: " + err.Error(),
|
||||
})
|
||||
continue
|
||||
}
|
||||
if _, err = bls.SignatureFromBytes(att.Signature); err != nil {
|
||||
attFailures = append(attFailures, &server.IndexedVerificationFailure{
|
||||
Index: i,
|
||||
Message: "Incorrect attestation signature: " + err.Error(),
|
||||
})
|
||||
continue
|
||||
}
|
||||
validAttestations = append(validAttestations, att)
|
||||
}
|
||||
|
||||
for i, att := range validAttestations {
|
||||
// Broadcast the unaggregated attestation on a feed to notify other services in the beacon node
|
||||
// of a received unaggregated attestation.
|
||||
// Note we can't send for aggregated att because we don't have selection proof.
|
||||
if !corehelpers.IsAggregated(att) {
|
||||
s.OperationNotifier.OperationFeed().Send(&feed.Event{
|
||||
Type: operation.UnaggregatedAttReceived,
|
||||
Data: &operation.UnAggregatedAttReceivedData{
|
||||
Attestation: att,
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
wantedEpoch := slots.ToEpoch(att.Data.Slot)
|
||||
vals, err := s.HeadFetcher.HeadValidatorsIndices(ctx, wantedEpoch)
|
||||
if err != nil {
|
||||
failedBroadcasts = append(failedBroadcasts, strconv.Itoa(i))
|
||||
continue
|
||||
}
|
||||
committeeIndex, err := att.GetCommitteeIndex()
|
||||
if err != nil {
|
||||
return nil, nil, errors.Wrap(err, "failed to retrieve attestation committee index")
|
||||
}
|
||||
subnet := corehelpers.ComputeSubnetFromCommitteeAndSlot(uint64(len(vals)), committeeIndex, att.Data.Slot)
|
||||
if err = s.Broadcaster.BroadcastAttestation(ctx, subnet, att); err != nil {
|
||||
log.WithError(err).Errorf("could not broadcast attestation at index %d", i)
|
||||
failedBroadcasts = append(failedBroadcasts, strconv.Itoa(i))
|
||||
continue
|
||||
}
|
||||
|
||||
if corehelpers.IsAggregated(att) {
|
||||
if err = s.AttestationsPool.SaveAggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Error("could not save aggregated attestation")
|
||||
}
|
||||
} else {
|
||||
if err = s.AttestationsPool.SaveUnaggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Error("could not save unaggregated attestation")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return attFailures, failedBroadcasts, nil
|
||||
}
|
||||
|
||||
func (s *Server) handleAttestations(ctx context.Context, data json.RawMessage) (attFailures []*server.IndexedVerificationFailure, failedBroadcasts []string, err error) {
|
||||
var sourceAttestations []*structs.Attestation
|
||||
|
||||
if err = json.Unmarshal(data, &sourceAttestations); err != nil {
|
||||
return nil, nil, errors.Wrap(err, "failed to unmarshal attestation")
|
||||
}
|
||||
|
||||
if len(sourceAttestations) == 0 {
|
||||
return nil, nil, errors.New("no data submitted")
|
||||
}
|
||||
|
||||
var validAttestations []*eth.Attestation
|
||||
for i, sourceAtt := range sourceAttestations {
|
||||
att, err := sourceAtt.ToConsensus()
|
||||
if err != nil {
|
||||
attFailures = append(attFailures, &server.IndexedVerificationFailure{
|
||||
Index: i,
|
||||
Message: "Could not convert request attestation to consensus attestation: " + err.Error(),
|
||||
})
|
||||
continue
|
||||
}
|
||||
if _, err = bls.SignatureFromBytes(att.Signature); err != nil {
|
||||
attFailures = append(attFailures, &server.IndexedVerificationFailure{
|
||||
Index: i,
|
||||
Message: "Incorrect attestation signature: " + err.Error(),
|
||||
})
|
||||
continue
|
||||
}
|
||||
validAttestations = append(validAttestations, att)
|
||||
}
|
||||
|
||||
for i, att := range validAttestations {
|
||||
// Broadcast the unaggregated attestation on a feed to notify other services in the beacon node
|
||||
// of a received unaggregated attestation.
|
||||
// Note we can't send for aggregated att because we don't have selection proof.
|
||||
if !corehelpers.IsAggregated(att) {
|
||||
s.OperationNotifier.OperationFeed().Send(&feed.Event{
|
||||
Type: operation.UnaggregatedAttReceived,
|
||||
Data: &operation.UnAggregatedAttReceivedData{
|
||||
Attestation: att,
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
wantedEpoch := slots.ToEpoch(att.Data.Slot)
|
||||
vals, err := s.HeadFetcher.HeadValidatorsIndices(ctx, wantedEpoch)
|
||||
if err != nil {
|
||||
failedBroadcasts = append(failedBroadcasts, strconv.Itoa(i))
|
||||
continue
|
||||
}
|
||||
|
||||
subnet := corehelpers.ComputeSubnetFromCommitteeAndSlot(uint64(len(vals)), att.Data.CommitteeIndex, att.Data.Slot)
|
||||
if err = s.Broadcaster.BroadcastAttestation(ctx, subnet, att); err != nil {
|
||||
log.WithError(err).Errorf("could not broadcast attestation at index %d", i)
|
||||
failedBroadcasts = append(failedBroadcasts, strconv.Itoa(i))
|
||||
continue
|
||||
}
|
||||
|
||||
if corehelpers.IsAggregated(att) {
|
||||
if err = s.AttestationsPool.SaveAggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Error("could not save aggregated attestation")
|
||||
}
|
||||
} else {
|
||||
if err = s.AttestationsPool.SaveUnaggregatedAttestation(att); err != nil {
|
||||
log.WithError(err).Error("could not save unaggregated attestation")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return attFailures, failedBroadcasts, nil
|
||||
}
|
||||
|
||||
// ListVoluntaryExits retrieves voluntary exits known by the node but
|
||||
// not necessarily incorporated into any block.
|
||||
func (s *Server) ListVoluntaryExits(w http.ResponseWriter, r *http.Request) {
|
||||
|
||||
@@ -500,95 +500,292 @@ func TestSubmitAttestations(t *testing.T) {
|
||||
ChainInfoFetcher: chainService,
|
||||
OperationNotifier: &blockchainmock.MockOperationNotifier{},
|
||||
}
|
||||
t.Run("V1", func(t *testing.T) {
|
||||
t.Run("single", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
|
||||
t.Run("single", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(singleAtt)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(singleAtt)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
s.SubmitAttestations(writer, request)
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 1, broadcaster.NumAttestations())
|
||||
assert.Equal(t, "0x03", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetAggregationBits()))
|
||||
assert.Equal(t, "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetSignature()))
|
||||
assert.Equal(t, primitives.Slot(0), broadcaster.BroadcastAttestations[0].GetData().Slot)
|
||||
assert.Equal(t, primitives.CommitteeIndex(0), broadcaster.BroadcastAttestations[0].GetData().CommitteeIndex)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().BeaconBlockRoot))
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Source.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Source.Epoch)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Target.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Target.Epoch)
|
||||
assert.Equal(t, 1, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 1, broadcaster.NumAttestations())
|
||||
assert.Equal(t, "0x03", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetAggregationBits()))
|
||||
assert.Equal(t, "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetSignature()))
|
||||
assert.Equal(t, primitives.Slot(0), broadcaster.BroadcastAttestations[0].GetData().Slot)
|
||||
assert.Equal(t, primitives.CommitteeIndex(0), broadcaster.BroadcastAttestations[0].GetData().CommitteeIndex)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().BeaconBlockRoot))
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Source.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Source.Epoch)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Target.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Target.Epoch)
|
||||
assert.Equal(t, 1, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
})
|
||||
t.Run("multiple", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(multipleAtts)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 2, broadcaster.NumAttestations())
|
||||
assert.Equal(t, 2, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
})
|
||||
t.Run("no body", func(t *testing.T) {
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "No data submitted"))
|
||||
})
|
||||
t.Run("empty", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString("[]")
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "no data submitted"))
|
||||
})
|
||||
t.Run("invalid", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(invalidAtt)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &server.IndexedVerificationFailureError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
require.Equal(t, 1, len(e.Failures))
|
||||
assert.Equal(t, true, strings.Contains(e.Failures[0].Message, "Incorrect attestation signature"))
|
||||
})
|
||||
})
|
||||
t.Run("multiple", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
t.Run("V2", func(t *testing.T) {
|
||||
t.Run("pre-electra", func(t *testing.T) {
|
||||
t.Run("single", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(multipleAtts)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(singleAtt)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Phase0))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 2, broadcaster.NumAttestations())
|
||||
assert.Equal(t, 2, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 1, broadcaster.NumAttestations())
|
||||
assert.Equal(t, "0x03", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetAggregationBits()))
|
||||
assert.Equal(t, "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetSignature()))
|
||||
assert.Equal(t, primitives.Slot(0), broadcaster.BroadcastAttestations[0].GetData().Slot)
|
||||
assert.Equal(t, primitives.CommitteeIndex(0), broadcaster.BroadcastAttestations[0].GetData().CommitteeIndex)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().BeaconBlockRoot))
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Source.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Source.Epoch)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Target.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Target.Epoch)
|
||||
assert.Equal(t, 1, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
})
|
||||
t.Run("multiple", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(multipleAtts)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Phase0))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 2, broadcaster.NumAttestations())
|
||||
assert.Equal(t, 2, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
})
|
||||
t.Run("no body", func(t *testing.T) {
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", nil)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Phase0))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "No data submitted"))
|
||||
})
|
||||
t.Run("empty", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString("[]")
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Phase0))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "no data submitted"))
|
||||
})
|
||||
t.Run("invalid", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(invalidAtt)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Phase0))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &server.IndexedVerificationFailureError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
require.Equal(t, 1, len(e.Failures))
|
||||
assert.Equal(t, true, strings.Contains(e.Failures[0].Message, "Incorrect attestation signature"))
|
||||
})
|
||||
})
|
||||
t.Run("post-electra", func(t *testing.T) {
|
||||
t.Run("single", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(singleAttElectra)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Electra))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 1, broadcaster.NumAttestations())
|
||||
assert.Equal(t, "0x03", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetAggregationBits()))
|
||||
assert.Equal(t, "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetSignature()))
|
||||
assert.Equal(t, primitives.Slot(0), broadcaster.BroadcastAttestations[0].GetData().Slot)
|
||||
assert.Equal(t, primitives.CommitteeIndex(0), broadcaster.BroadcastAttestations[0].GetData().CommitteeIndex)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().BeaconBlockRoot))
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Source.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Source.Epoch)
|
||||
assert.Equal(t, "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2", hexutil.Encode(broadcaster.BroadcastAttestations[0].GetData().Target.Root))
|
||||
assert.Equal(t, primitives.Epoch(0), broadcaster.BroadcastAttestations[0].GetData().Target.Epoch)
|
||||
assert.Equal(t, 1, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
})
|
||||
t.Run("multiple", func(t *testing.T) {
|
||||
broadcaster := &p2pMock.MockBroadcaster{}
|
||||
s.Broadcaster = broadcaster
|
||||
s.AttestationsPool = attestations.NewPool()
|
||||
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(multipleAttsElectra)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Electra))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
assert.Equal(t, true, broadcaster.BroadcastCalled.Load())
|
||||
assert.Equal(t, 2, broadcaster.NumAttestations())
|
||||
assert.Equal(t, 2, s.AttestationsPool.UnaggregatedAttestationCount())
|
||||
})
|
||||
t.Run("no body", func(t *testing.T) {
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", nil)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Electra))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "No data submitted"))
|
||||
})
|
||||
t.Run("empty", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString("[]")
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Electra))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "no data submitted"))
|
||||
})
|
||||
t.Run("invalid", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(invalidAttElectra)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
request.Header.Set(api.VersionHeader, version.String(version.Electra))
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestationsV2(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &server.IndexedVerificationFailureError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
require.Equal(t, 1, len(e.Failures))
|
||||
assert.Equal(t, true, strings.Contains(e.Failures[0].Message, "Incorrect attestation signature"))
|
||||
})
|
||||
})
|
||||
})
|
||||
t.Run("no body", func(t *testing.T) {
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "No data submitted"))
|
||||
})
|
||||
t.Run("empty", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString("[]")
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "No data submitted"))
|
||||
})
|
||||
t.Run("invalid", func(t *testing.T) {
|
||||
var body bytes.Buffer
|
||||
_, err := body.WriteString(invalidAtt)
|
||||
require.NoError(t, err)
|
||||
request := httptest.NewRequest(http.MethodPost, "http://example.com", &body)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.SubmitAttestations(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &server.IndexedVerificationFailureError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
require.Equal(t, 1, len(e.Failures))
|
||||
assert.Equal(t, true, strings.Contains(e.Failures[0].Message, "Incorrect attestation signature"))
|
||||
})
|
||||
}
|
||||
|
||||
func TestListVoluntaryExits(t *testing.T) {
|
||||
@@ -2063,6 +2260,85 @@ var (
|
||||
}
|
||||
}
|
||||
}
|
||||
]`
|
||||
singleAttElectra = `[
|
||||
{
|
||||
"aggregation_bits": "0x03",
|
||||
"committee_bits": "0x0100000000000000",
|
||||
"signature": "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15",
|
||||
"data": {
|
||||
"slot": "0",
|
||||
"index": "0",
|
||||
"beacon_block_root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2",
|
||||
"source": {
|
||||
"epoch": "0",
|
||||
"root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2"
|
||||
},
|
||||
"target": {
|
||||
"epoch": "0",
|
||||
"root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2"
|
||||
}
|
||||
}
|
||||
}
|
||||
]`
|
||||
multipleAttsElectra = `[
|
||||
{
|
||||
"aggregation_bits": "0x03",
|
||||
"committee_bits": "0x0100000000000000",
|
||||
"signature": "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15",
|
||||
"data": {
|
||||
"slot": "0",
|
||||
"index": "0",
|
||||
"beacon_block_root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2",
|
||||
"source": {
|
||||
"epoch": "0",
|
||||
"root": "0x736f75726365726f6f7431000000000000000000000000000000000000000000"
|
||||
},
|
||||
"target": {
|
||||
"epoch": "0",
|
||||
"root": "0x746172676574726f6f7431000000000000000000000000000000000000000000"
|
||||
}
|
||||
}
|
||||
},
|
||||
{
|
||||
"aggregation_bits": "0x03",
|
||||
"committee_bits": "0x0100000000000000",
|
||||
"signature": "0x8146f4397bfd8fd057ebbcd6a67327bdc7ed5fb650533edcb6377b650dea0b6da64c14ecd60846d5c0a0cd43893d6972092500f82c9d8a955e2b58c5ed3cbe885d84008ace6bd86ba9e23652f58e2ec207cec494c916063257abf285b9b15b15",
|
||||
"data": {
|
||||
"slot": "0",
|
||||
"index": "0",
|
||||
"beacon_block_root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2",
|
||||
"source": {
|
||||
"epoch": "0",
|
||||
"root": "0x736f75726365726f6f7431000000000000000000000000000000000000000000"
|
||||
},
|
||||
"target": {
|
||||
"epoch": "0",
|
||||
"root": "0x746172676574726f6f7432000000000000000000000000000000000000000000"
|
||||
}
|
||||
}
|
||||
}
|
||||
]`
|
||||
// signature is invalid
|
||||
invalidAttElectra = `[
|
||||
{
|
||||
"aggregation_bits": "0x03",
|
||||
"committee_bits": "0x0100000000000000",
|
||||
"signature": "0x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",
|
||||
"data": {
|
||||
"slot": "0",
|
||||
"index": "0",
|
||||
"beacon_block_root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2",
|
||||
"source": {
|
||||
"epoch": "0",
|
||||
"root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2"
|
||||
},
|
||||
"target": {
|
||||
"epoch": "0",
|
||||
"root": "0xcf8e0d4e9587369b2301d0790347320302cc0943d5a1884560367e8208d920f2"
|
||||
}
|
||||
}
|
||||
}
|
||||
]`
|
||||
exit1 = `{
|
||||
"message": {
|
||||
|
||||
@@ -19,11 +19,12 @@ go_library(
|
||||
"//beacon-chain/core/feed/state:go_default_library",
|
||||
"//beacon-chain/core/helpers:go_default_library",
|
||||
"//beacon-chain/core/time:go_default_library",
|
||||
"//beacon-chain/core/transition:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/payload-attribute:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//monitoring/tracing/trace:go_default_library",
|
||||
"//network/httputil:go_default_library",
|
||||
"//proto/engine/v1:go_default_library",
|
||||
"//proto/eth/v1:go_default_library",
|
||||
"//proto/eth/v2:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
@@ -52,6 +53,7 @@ go_test(
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
"//consensus-types/payload-attribute:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//proto/eth/v1:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
|
||||
@@ -7,6 +7,7 @@ import (
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
@@ -18,11 +19,12 @@ import (
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
chaintime "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
payloadattribute "github.com/prysmaticlabs/prysm/v5/consensus-types/payload-attribute"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
|
||||
"github.com/prysmaticlabs/prysm/v5/network/httputil"
|
||||
engine "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/eth/v1"
|
||||
ethpbv2 "github.com/prysmaticlabs/prysm/v5/proto/eth/v2"
|
||||
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
@@ -31,6 +33,7 @@ import (
|
||||
)
|
||||
|
||||
const DefaultEventFeedDepth = 1000
|
||||
const payloadAttributeTimeout = 2 * time.Second
|
||||
|
||||
const (
|
||||
InvalidTopic = "__invalid__"
|
||||
@@ -89,12 +92,12 @@ var opsFeedEventTopics = map[feed.EventType]string{
|
||||
|
||||
var stateFeedEventTopics = map[feed.EventType]string{
|
||||
statefeed.NewHead: HeadTopic,
|
||||
statefeed.MissedSlot: PayloadAttributesTopic,
|
||||
statefeed.FinalizedCheckpoint: FinalizedCheckpointTopic,
|
||||
statefeed.LightClientFinalityUpdate: LightClientFinalityUpdateTopic,
|
||||
statefeed.LightClientOptimisticUpdate: LightClientOptimisticUpdateTopic,
|
||||
statefeed.Reorg: ChainReorgTopic,
|
||||
statefeed.BlockProcessed: BlockTopic,
|
||||
statefeed.PayloadAttributes: PayloadAttributesTopic,
|
||||
}
|
||||
|
||||
var topicsForStateFeed = topicsForFeed(stateFeedEventTopics)
|
||||
@@ -418,10 +421,9 @@ func topicForEvent(event *feed.Event) string {
|
||||
return ChainReorgTopic
|
||||
case *statefeed.BlockProcessedData:
|
||||
return BlockTopic
|
||||
case payloadattribute.EventData:
|
||||
return PayloadAttributesTopic
|
||||
default:
|
||||
if event.Type == statefeed.MissedSlot {
|
||||
return PayloadAttributesTopic
|
||||
}
|
||||
return InvalidTopic
|
||||
}
|
||||
}
|
||||
@@ -431,31 +433,17 @@ func (s *Server) lazyReaderForEvent(ctx context.Context, event *feed.Event, topi
|
||||
if !topics.requested(eventName) {
|
||||
return nil, errNotRequested
|
||||
}
|
||||
if eventName == PayloadAttributesTopic {
|
||||
return s.currentPayloadAttributes(ctx)
|
||||
}
|
||||
if event == nil || event.Data == nil {
|
||||
return nil, errors.New("event or event data is nil")
|
||||
}
|
||||
switch v := event.Data.(type) {
|
||||
case payloadattribute.EventData:
|
||||
return s.payloadAttributesReader(ctx, v)
|
||||
case *ethpb.EventHead:
|
||||
// The head event is a special case because, if the client requested the payload attributes topic,
|
||||
// we send two event messages in reaction; the head event and the payload attributes.
|
||||
headReader := func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.HeadEventFromV1(v))
|
||||
}
|
||||
// Don't do the expensive attr lookup unless the client requested it.
|
||||
if !topics.requested(PayloadAttributesTopic) {
|
||||
return headReader, nil
|
||||
}
|
||||
// Since payload attributes could change before the outbox is written, we need to do a blocking operation to
|
||||
// get the current payload attributes right here.
|
||||
attrReader, err := s.currentPayloadAttributes(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get payload attributes for head event")
|
||||
}
|
||||
return func() io.Reader {
|
||||
return io.MultiReader(headReader(), attrReader())
|
||||
return jsonMarshalReader(eventName, structs.HeadEventFromV1(v))
|
||||
}, nil
|
||||
case *operation.AggregatedAttReceivedData:
|
||||
return func() io.Reader {
|
||||
@@ -463,14 +451,20 @@ func (s *Server) lazyReaderForEvent(ctx context.Context, event *feed.Event, topi
|
||||
return jsonMarshalReader(eventName, att)
|
||||
}, nil
|
||||
case *operation.UnAggregatedAttReceivedData:
|
||||
att, ok := v.Attestation.(*eth.Attestation)
|
||||
if !ok {
|
||||
switch att := v.Attestation.(type) {
|
||||
case *eth.Attestation:
|
||||
return func() io.Reader {
|
||||
att := structs.AttFromConsensus(att)
|
||||
return jsonMarshalReader(eventName, att)
|
||||
}, nil
|
||||
case *eth.AttestationElectra:
|
||||
return func() io.Reader {
|
||||
att := structs.AttElectraFromConsensus(att)
|
||||
return jsonMarshalReader(eventName, att)
|
||||
}, nil
|
||||
default:
|
||||
return nil, errors.Wrapf(errUnhandledEventData, "Unexpected type %T for the .Attestation field of UnAggregatedAttReceivedData", v.Attestation)
|
||||
}
|
||||
return func() io.Reader {
|
||||
att := structs.AttFromConsensus(att)
|
||||
return jsonMarshalReader(eventName, att)
|
||||
}, nil
|
||||
case *operation.ExitReceivedData:
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.SignedExitFromConsensus(v.Exit))
|
||||
@@ -495,13 +489,18 @@ func (s *Server) lazyReaderForEvent(ctx context.Context, event *feed.Event, topi
|
||||
})
|
||||
}, nil
|
||||
case *operation.AttesterSlashingReceivedData:
|
||||
slashing, ok := v.AttesterSlashing.(*eth.AttesterSlashing)
|
||||
if !ok {
|
||||
switch slashing := v.AttesterSlashing.(type) {
|
||||
case *eth.AttesterSlashing:
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.AttesterSlashingFromConsensus(slashing))
|
||||
}, nil
|
||||
case *eth.AttesterSlashingElectra:
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.AttesterSlashingElectraFromConsensus(slashing))
|
||||
}, nil
|
||||
default:
|
||||
return nil, errors.Wrapf(errUnhandledEventData, "Unexpected type %T for the .AttesterSlashing field of AttesterSlashingReceivedData", v.AttesterSlashing)
|
||||
}
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.AttesterSlashingFromConsensus(slashing))
|
||||
}, nil
|
||||
case *operation.ProposerSlashingReceivedData:
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.ProposerSlashingFromConsensus(v.ProposerSlashing))
|
||||
@@ -556,115 +555,202 @@ func (s *Server) lazyReaderForEvent(ctx context.Context, event *feed.Event, topi
|
||||
}
|
||||
}
|
||||
|
||||
// This event stream is intended to be used by builders and relays.
|
||||
// Parent fields are based on state at N_{current_slot}, while the rest of fields are based on state of N_{current_slot + 1}
|
||||
func (s *Server) currentPayloadAttributes(ctx context.Context) (lazyReader, error) {
|
||||
headRoot, err := s.HeadFetcher.HeadRoot(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head root")
|
||||
}
|
||||
st, err := s.HeadFetcher.HeadState(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head state")
|
||||
}
|
||||
// advance the head state
|
||||
headState, err := transition.ProcessSlotsIfPossible(ctx, st, s.ChainInfoFetcher.CurrentSlot()+1)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not advance head state")
|
||||
var errUnsupportedPayloadAttribute = errors.New("cannot compute payload attributes pre-Bellatrix")
|
||||
|
||||
func (s *Server) computePayloadAttributes(ctx context.Context, ev payloadattribute.EventData) (payloadattribute.Attributer, error) {
|
||||
v := ev.HeadState.Version()
|
||||
if v < version.Bellatrix {
|
||||
return nil, errors.Wrapf(errUnsupportedPayloadAttribute, "%s is not supported", version.String(v))
|
||||
}
|
||||
|
||||
headBlock, err := s.HeadFetcher.HeadBlock(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head block")
|
||||
}
|
||||
|
||||
headPayload, err := headBlock.Block().Body().Execution()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get execution payload")
|
||||
}
|
||||
|
||||
t, err := slots.ToTime(headState.GenesisTime(), headState.Slot())
|
||||
t, err := slots.ToTime(ev.HeadState.GenesisTime(), ev.HeadState.Slot())
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head state slot time")
|
||||
}
|
||||
|
||||
prevRando, err := helpers.RandaoMix(headState, chaintime.CurrentEpoch(headState))
|
||||
timestamp := uint64(t.Unix())
|
||||
prevRando, err := helpers.RandaoMix(ev.HeadState, chaintime.CurrentEpoch(ev.HeadState))
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head state randao mix")
|
||||
}
|
||||
|
||||
proposerIndex, err := helpers.BeaconProposerIndex(ctx, headState)
|
||||
proposerIndex, err := helpers.BeaconProposerIndex(ctx, ev.HeadState)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head state proposer index")
|
||||
}
|
||||
feeRecipient := params.BeaconConfig().DefaultFeeRecipient.Bytes()
|
||||
feeRecpt := params.BeaconConfig().DefaultFeeRecipient.Bytes()
|
||||
tValidator, exists := s.TrackedValidatorsCache.Validator(proposerIndex)
|
||||
if exists {
|
||||
feeRecipient = tValidator.FeeRecipient[:]
|
||||
}
|
||||
var attributes interface{}
|
||||
switch headState.Version() {
|
||||
case version.Bellatrix:
|
||||
attributes = &structs.PayloadAttributesV1{
|
||||
Timestamp: fmt.Sprintf("%d", t.Unix()),
|
||||
PrevRandao: hexutil.Encode(prevRando),
|
||||
SuggestedFeeRecipient: hexutil.Encode(feeRecipient),
|
||||
}
|
||||
case version.Capella:
|
||||
withdrawals, _, err := headState.ExpectedWithdrawals()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head state expected withdrawals")
|
||||
}
|
||||
attributes = &structs.PayloadAttributesV2{
|
||||
Timestamp: fmt.Sprintf("%d", t.Unix()),
|
||||
PrevRandao: hexutil.Encode(prevRando),
|
||||
SuggestedFeeRecipient: hexutil.Encode(feeRecipient),
|
||||
Withdrawals: structs.WithdrawalsFromConsensus(withdrawals),
|
||||
}
|
||||
case version.Deneb, version.Electra:
|
||||
withdrawals, _, err := headState.ExpectedWithdrawals()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head state expected withdrawals")
|
||||
}
|
||||
parentRoot, err := headBlock.Block().HashTreeRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get head block root")
|
||||
}
|
||||
attributes = &structs.PayloadAttributesV3{
|
||||
Timestamp: fmt.Sprintf("%d", t.Unix()),
|
||||
PrevRandao: hexutil.Encode(prevRando),
|
||||
SuggestedFeeRecipient: hexutil.Encode(feeRecipient),
|
||||
Withdrawals: structs.WithdrawalsFromConsensus(withdrawals),
|
||||
ParentBeaconBlockRoot: hexutil.Encode(parentRoot[:]),
|
||||
}
|
||||
default:
|
||||
return nil, errors.Wrapf(err, "Payload version %s is not supported", version.String(headState.Version()))
|
||||
feeRecpt = tValidator.FeeRecipient[:]
|
||||
}
|
||||
|
||||
attributesBytes, err := json.Marshal(attributes)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "errors marshaling payload attributes to json")
|
||||
}
|
||||
eventData := structs.PayloadAttributesEventData{
|
||||
ProposerIndex: fmt.Sprintf("%d", proposerIndex),
|
||||
ProposalSlot: fmt.Sprintf("%d", headState.Slot()),
|
||||
ParentBlockNumber: fmt.Sprintf("%d", headPayload.BlockNumber()),
|
||||
ParentBlockRoot: hexutil.Encode(headRoot),
|
||||
ParentBlockHash: hexutil.Encode(headPayload.BlockHash()),
|
||||
PayloadAttributes: attributesBytes,
|
||||
}
|
||||
eventDataBytes, err := json.Marshal(eventData)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "errors marshaling payload attributes event data to json")
|
||||
}
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(PayloadAttributesTopic, &structs.PayloadAttributesEvent{
|
||||
Version: version.String(headState.Version()),
|
||||
Data: eventDataBytes,
|
||||
if v == version.Bellatrix {
|
||||
return payloadattribute.New(&engine.PayloadAttributes{
|
||||
Timestamp: timestamp,
|
||||
PrevRandao: prevRando,
|
||||
SuggestedFeeRecipient: feeRecpt,
|
||||
})
|
||||
}
|
||||
|
||||
w, _, err := ev.HeadState.ExpectedWithdrawals()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get withdrawals from head state")
|
||||
}
|
||||
if v == version.Capella {
|
||||
return payloadattribute.New(&engine.PayloadAttributesV2{
|
||||
Timestamp: timestamp,
|
||||
PrevRandao: prevRando,
|
||||
SuggestedFeeRecipient: feeRecpt,
|
||||
Withdrawals: w,
|
||||
})
|
||||
}
|
||||
|
||||
pr, err := ev.HeadBlock.Block().HashTreeRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not compute head block root")
|
||||
}
|
||||
return payloadattribute.New(&engine.PayloadAttributesV3{
|
||||
Timestamp: timestamp,
|
||||
PrevRandao: prevRando,
|
||||
SuggestedFeeRecipient: feeRecpt,
|
||||
Withdrawals: w,
|
||||
ParentBeaconBlockRoot: pr[:],
|
||||
})
|
||||
}
|
||||
|
||||
type asyncPayloadAttrData struct {
|
||||
data json.RawMessage
|
||||
version string
|
||||
err error
|
||||
}
|
||||
|
||||
func (s *Server) fillEventData(ctx context.Context, ev payloadattribute.EventData) (payloadattribute.EventData, error) {
|
||||
if ev.HeadBlock == nil || ev.HeadBlock.IsNil() {
|
||||
hb, err := s.HeadFetcher.HeadBlock(ctx)
|
||||
if err != nil {
|
||||
return ev, errors.Wrap(err, "Could not look up head block")
|
||||
}
|
||||
root, err := hb.Block().HashTreeRoot()
|
||||
if err != nil {
|
||||
return ev, errors.Wrap(err, "Could not compute head block root")
|
||||
}
|
||||
if ev.HeadRoot != root {
|
||||
return ev, errors.Wrap(err, "head root changed before payload attribute event handler execution")
|
||||
}
|
||||
ev.HeadBlock = hb
|
||||
payload, err := hb.Block().Body().Execution()
|
||||
if err != nil {
|
||||
return ev, errors.Wrap(err, "Could not get execution payload for head block")
|
||||
}
|
||||
ev.ParentBlockHash = payload.BlockHash()
|
||||
ev.ParentBlockNumber = payload.BlockNumber()
|
||||
}
|
||||
|
||||
attr := ev.Attributer
|
||||
if attr == nil || attr.IsEmpty() {
|
||||
attr, err := s.computePayloadAttributes(ctx, ev)
|
||||
if err != nil {
|
||||
return ev, errors.Wrap(err, "Could not compute payload attributes")
|
||||
}
|
||||
ev.Attributer = attr
|
||||
}
|
||||
return ev, nil
|
||||
}
|
||||
|
||||
// This event stream is intended to be used by builders and relays.
|
||||
// Parent fields are based on state at N_{current_slot}, while the rest of fields are based on state of N_{current_slot + 1}
|
||||
func (s *Server) payloadAttributesReader(ctx context.Context, ev payloadattribute.EventData) (lazyReader, error) {
|
||||
ctx, cancel := context.WithTimeout(ctx, payloadAttributeTimeout)
|
||||
edc := make(chan asyncPayloadAttrData)
|
||||
go func() {
|
||||
d := asyncPayloadAttrData{
|
||||
version: version.String(ev.HeadState.Version()),
|
||||
}
|
||||
|
||||
defer func() {
|
||||
edc <- d
|
||||
}()
|
||||
ev, err := s.fillEventData(ctx, ev)
|
||||
if err != nil {
|
||||
d.err = errors.Wrap(err, "Could not fill event data")
|
||||
return
|
||||
}
|
||||
attributesBytes, err := marshalAttributes(ev.Attributer)
|
||||
if err != nil {
|
||||
d.err = errors.Wrap(err, "errors marshaling payload attributes to json")
|
||||
return
|
||||
}
|
||||
d.data, d.err = json.Marshal(structs.PayloadAttributesEventData{
|
||||
ProposerIndex: strconv.FormatUint(uint64(ev.ProposerIndex), 10),
|
||||
ProposalSlot: strconv.FormatUint(uint64(ev.ProposalSlot), 10),
|
||||
ParentBlockNumber: strconv.FormatUint(ev.ParentBlockNumber, 10),
|
||||
ParentBlockRoot: hexutil.Encode(ev.ParentBlockRoot),
|
||||
ParentBlockHash: hexutil.Encode(ev.ParentBlockHash),
|
||||
PayloadAttributes: attributesBytes,
|
||||
})
|
||||
if d.err != nil {
|
||||
d.err = errors.Wrap(d.err, "errors marshaling payload attributes event data to json")
|
||||
}
|
||||
}()
|
||||
return func() io.Reader {
|
||||
defer cancel()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.WithError(ctx.Err()).Warn("Context canceled while waiting for payload attributes event data")
|
||||
return nil
|
||||
case ed := <-edc:
|
||||
if ed.err != nil {
|
||||
log.WithError(ed.err).Warn("Error while marshaling payload attributes event data")
|
||||
return nil
|
||||
}
|
||||
return jsonMarshalReader(PayloadAttributesTopic, &structs.PayloadAttributesEvent{
|
||||
Version: ed.version,
|
||||
Data: ed.data,
|
||||
})
|
||||
}
|
||||
}, nil
|
||||
}
|
||||
|
||||
func marshalAttributes(attr payloadattribute.Attributer) ([]byte, error) {
|
||||
v := attr.Version()
|
||||
if v < version.Bellatrix {
|
||||
return nil, errors.Wrapf(errUnsupportedPayloadAttribute, "Payload version %s is not supported", version.String(v))
|
||||
}
|
||||
|
||||
timestamp := strconv.FormatUint(attr.Timestamp(), 10)
|
||||
prevRandao := hexutil.Encode(attr.PrevRandao())
|
||||
feeRecpt := hexutil.Encode(attr.SuggestedFeeRecipient())
|
||||
if v == version.Bellatrix {
|
||||
return json.Marshal(&structs.PayloadAttributesV1{
|
||||
Timestamp: timestamp,
|
||||
PrevRandao: prevRandao,
|
||||
SuggestedFeeRecipient: feeRecpt,
|
||||
})
|
||||
}
|
||||
w, err := attr.Withdrawals()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get withdrawals from payload attributes event")
|
||||
}
|
||||
withdrawals := structs.WithdrawalsFromConsensus(w)
|
||||
if v == version.Capella {
|
||||
return json.Marshal(&structs.PayloadAttributesV2{
|
||||
Timestamp: timestamp,
|
||||
PrevRandao: prevRandao,
|
||||
SuggestedFeeRecipient: feeRecpt,
|
||||
Withdrawals: withdrawals,
|
||||
})
|
||||
}
|
||||
parentRoot, err := attr.ParentBeaconBlockRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get parent beacon block root from payload attributes event")
|
||||
}
|
||||
return json.Marshal(&structs.PayloadAttributesV3{
|
||||
Timestamp: timestamp,
|
||||
PrevRandao: prevRandao,
|
||||
SuggestedFeeRecipient: feeRecpt,
|
||||
Withdrawals: withdrawals,
|
||||
ParentBeaconBlockRoot: hexutil.Encode(parentRoot),
|
||||
})
|
||||
}
|
||||
|
||||
func newStreamingResponseController(rw http.ResponseWriter, timeout time.Duration) *streamingResponseWriterController {
|
||||
rc := http.NewResponseController(rw)
|
||||
return &streamingResponseWriterController{
|
||||
|
||||
@@ -21,6 +21,7 @@ import (
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
payloadattribute "github.com/prysmaticlabs/prysm/v5/consensus-types/payload-attribute"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/eth/v1"
|
||||
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
@@ -489,7 +490,21 @@ func TestStreamEvents_OperationsEvents(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
request := topics.testHttpRequest(testSync.ctx, t)
|
||||
w := NewStreamingResponseWriterRecorder(testSync.ctx)
|
||||
events := []*feed.Event{&feed.Event{Type: statefeed.MissedSlot}}
|
||||
events := []*feed.Event{
|
||||
&feed.Event{
|
||||
Type: statefeed.PayloadAttributes,
|
||||
Data: payloadattribute.EventData{
|
||||
ProposerIndex: 0,
|
||||
ProposalSlot: 0,
|
||||
ParentBlockNumber: 0,
|
||||
ParentBlockRoot: make([]byte, 32),
|
||||
ParentBlockHash: make([]byte, 32),
|
||||
HeadState: st,
|
||||
HeadBlock: b,
|
||||
HeadRoot: [fieldparams.RootLength]byte{},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
go func() {
|
||||
s.StreamEvents(w, request)
|
||||
|
||||
@@ -40,6 +40,7 @@ go_library(
|
||||
"//monitoring/tracing/trace:go_default_library",
|
||||
"//network/httputil:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//proto/prysm/v1alpha1/attestation/aggregation/attestations:go_default_library",
|
||||
"//runtime/version:go_default_library",
|
||||
"//time/slots:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
|
||||
@@ -82,6 +83,7 @@ go_test(
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/primitives:go_default_library",
|
||||
"//crypto/bls:go_default_library",
|
||||
"//crypto/bls/common:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//network/httputil:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
@@ -93,6 +95,7 @@ go_test(
|
||||
"//time/slots:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
|
||||
"@com_github_pkg_errors//:go_default_library",
|
||||
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
|
||||
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
|
||||
"@org_uber_go_mock//gomock:go_default_library",
|
||||
],
|
||||
|
||||
@@ -2,11 +2,13 @@ package validator
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"cmp"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"slices"
|
||||
"sort"
|
||||
"strconv"
|
||||
"time"
|
||||
@@ -32,6 +34,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
|
||||
"github.com/prysmaticlabs/prysm/v5/network/httputil"
|
||||
ethpbalpha "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1/attestation/aggregation/attestations"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
"github.com/prysmaticlabs/prysm/v5/time/slots"
|
||||
"github.com/sirupsen/logrus"
|
||||
@@ -48,71 +51,165 @@ func (s *Server) GetAggregateAttestation(w http.ResponseWriter, r *http.Request)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
_, slot, ok := shared.UintFromQuery(w, r, "slot", true)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
var match ethpbalpha.Att
|
||||
var err error
|
||||
|
||||
match, err = matchingAtt(s.AttestationsPool.AggregatedAttestations(), primitives.Slot(slot), attDataRoot)
|
||||
agg := s.aggregatedAttestation(w, primitives.Slot(slot), attDataRoot, 0)
|
||||
if agg == nil {
|
||||
return
|
||||
}
|
||||
typedAgg, ok := agg.(*ethpbalpha.Attestation)
|
||||
if !ok {
|
||||
httputil.HandleError(w, fmt.Sprintf("Attestation is not of type %T", ðpbalpha.Attestation{}), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
data, err := json.Marshal(structs.AttFromConsensus(typedAgg))
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get matching attestation: "+err.Error(), http.StatusInternalServerError)
|
||||
httputil.HandleError(w, "Could not marshal attestation: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
if match == nil {
|
||||
atts, err := s.AttestationsPool.UnaggregatedAttestations()
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get unaggregated attestations: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
match, err = matchingAtt(atts, primitives.Slot(slot), attDataRoot)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get matching attestation: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
}
|
||||
if match == nil {
|
||||
httputil.HandleError(w, "No matching attestation found", http.StatusNotFound)
|
||||
return
|
||||
}
|
||||
|
||||
response := &structs.AggregateAttestationResponse{
|
||||
Data: &structs.Attestation{
|
||||
AggregationBits: hexutil.Encode(match.GetAggregationBits()),
|
||||
Data: &structs.AttestationData{
|
||||
Slot: strconv.FormatUint(uint64(match.GetData().Slot), 10),
|
||||
CommitteeIndex: strconv.FormatUint(uint64(match.GetData().CommitteeIndex), 10),
|
||||
BeaconBlockRoot: hexutil.Encode(match.GetData().BeaconBlockRoot),
|
||||
Source: &structs.Checkpoint{
|
||||
Epoch: strconv.FormatUint(uint64(match.GetData().Source.Epoch), 10),
|
||||
Root: hexutil.Encode(match.GetData().Source.Root),
|
||||
},
|
||||
Target: &structs.Checkpoint{
|
||||
Epoch: strconv.FormatUint(uint64(match.GetData().Target.Epoch), 10),
|
||||
Root: hexutil.Encode(match.GetData().Target.Root),
|
||||
},
|
||||
},
|
||||
Signature: hexutil.Encode(match.GetSignature()),
|
||||
}}
|
||||
httputil.WriteJson(w, response)
|
||||
httputil.WriteJson(w, &structs.AggregateAttestationResponse{Data: data})
|
||||
}
|
||||
|
||||
func matchingAtt(atts []ethpbalpha.Att, slot primitives.Slot, attDataRoot []byte) (ethpbalpha.Att, error) {
|
||||
// GetAggregateAttestationV2 aggregates all attestations matching the given attestation data root and slot, returning the aggregated result.
|
||||
func (s *Server) GetAggregateAttestationV2(w http.ResponseWriter, r *http.Request) {
|
||||
ctx, span := trace.StartSpan(r.Context(), "validator.GetAggregateAttestationV2")
|
||||
defer span.End()
|
||||
|
||||
_, attDataRoot, ok := shared.HexFromQuery(w, r, "attestation_data_root", fieldparams.RootLength, true)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
_, slot, ok := shared.UintFromQuery(w, r, "slot", true)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
_, index, ok := shared.UintFromQuery(w, r, "committee_index", true)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
agg := s.aggregatedAttestation(w, primitives.Slot(slot), attDataRoot, primitives.CommitteeIndex(index))
|
||||
if agg == nil {
|
||||
return
|
||||
}
|
||||
resp := &structs.AggregateAttestationResponse{
|
||||
Version: version.String(agg.Version()),
|
||||
}
|
||||
if agg.Version() >= version.Electra {
|
||||
typedAgg, ok := agg.(*ethpbalpha.AttestationElectra)
|
||||
if !ok {
|
||||
httputil.HandleError(w, fmt.Sprintf("Attestation is not of type %T", ðpbalpha.AttestationElectra{}), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
data, err := json.Marshal(structs.AttElectraFromConsensus(typedAgg))
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not marshal attestation: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
resp.Data = data
|
||||
} else {
|
||||
typedAgg, ok := agg.(*ethpbalpha.Attestation)
|
||||
if !ok {
|
||||
httputil.HandleError(w, fmt.Sprintf("Attestation is not of type %T", ðpbalpha.Attestation{}), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
data, err := json.Marshal(structs.AttFromConsensus(typedAgg))
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not marshal attestation: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
resp.Data = data
|
||||
}
|
||||
headState, err := s.ChainInfoFetcher.HeadStateReadOnly(ctx)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get head state: "+err.Error(), http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
w.Header().Set(api.VersionHeader, version.String(headState.Version()))
|
||||
httputil.WriteJson(w, resp)
|
||||
}
|
||||
|
||||
func (s *Server) aggregatedAttestation(w http.ResponseWriter, slot primitives.Slot, attDataRoot []byte, index primitives.CommitteeIndex) ethpbalpha.Att {
|
||||
var err error
|
||||
|
||||
match, err := matchingAtts(s.AttestationsPool.AggregatedAttestations(), slot, attDataRoot, index)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get matching attestations: "+err.Error(), http.StatusInternalServerError)
|
||||
return nil
|
||||
}
|
||||
if len(match) > 0 {
|
||||
// If there are multiple matching aggregated attestations,
|
||||
// then we return the one with the most aggregation bits.
|
||||
slices.SortFunc(match, func(a, b ethpbalpha.Att) int {
|
||||
return cmp.Compare(b.GetAggregationBits().Count(), a.GetAggregationBits().Count())
|
||||
})
|
||||
return match[0]
|
||||
}
|
||||
|
||||
atts, err := s.AttestationsPool.UnaggregatedAttestations()
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get unaggregated attestations: "+err.Error(), http.StatusInternalServerError)
|
||||
return nil
|
||||
}
|
||||
match, err = matchingAtts(atts, slot, attDataRoot, index)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not get matching attestations: "+err.Error(), http.StatusInternalServerError)
|
||||
return nil
|
||||
}
|
||||
if len(match) == 0 {
|
||||
httputil.HandleError(w, "No matching attestations found", http.StatusNotFound)
|
||||
return nil
|
||||
}
|
||||
agg, err := attestations.Aggregate(match)
|
||||
if err != nil {
|
||||
httputil.HandleError(w, "Could not aggregate unaggregated attestations: "+err.Error(), http.StatusInternalServerError)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Aggregating unaggregated attestations will in theory always return just one aggregate,
|
||||
// so we can take the first one and be done with it.
|
||||
return agg[0]
|
||||
}
|
||||
|
||||
func matchingAtts(atts []ethpbalpha.Att, slot primitives.Slot, attDataRoot []byte, index primitives.CommitteeIndex) ([]ethpbalpha.Att, error) {
|
||||
if len(atts) == 0 {
|
||||
return []ethpbalpha.Att{}, nil
|
||||
}
|
||||
|
||||
postElectra := atts[0].Version() >= version.Electra
|
||||
|
||||
result := make([]ethpbalpha.Att, 0)
|
||||
for _, att := range atts {
|
||||
if att.GetData().Slot == slot {
|
||||
root, err := att.GetData().HashTreeRoot()
|
||||
if att.GetData().Slot != slot {
|
||||
continue
|
||||
}
|
||||
// We ignore the committee index from the request before Electra.
|
||||
// This is because before Electra the committee index is part of the attestation data,
|
||||
// meaning that comparing the data root is sufficient.
|
||||
// Post-Electra the committee index in the data root is always 0, so we need to
|
||||
// compare the committee index separately.
|
||||
if postElectra {
|
||||
ci, err := att.GetCommitteeIndex()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get attestation data root")
|
||||
return nil, err
|
||||
}
|
||||
if bytes.Equal(root[:], attDataRoot) {
|
||||
return att, nil
|
||||
if ci != index {
|
||||
continue
|
||||
}
|
||||
}
|
||||
root, err := att.GetData().HashTreeRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get attestation data root")
|
||||
}
|
||||
if bytes.Equal(root[:], attDataRoot) {
|
||||
result = append(result, att)
|
||||
}
|
||||
}
|
||||
return nil, nil
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// SubmitContributionAndProofs publishes multiple signed sync committee contribution and proofs.
|
||||
|
||||
@@ -14,6 +14,7 @@ import (
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/go-bitfield"
|
||||
"github.com/prysmaticlabs/prysm/v5/api"
|
||||
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
|
||||
mockChain "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
@@ -35,6 +36,7 @@ import (
|
||||
"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/crypto/bls/common"
|
||||
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v5/network/httputil"
|
||||
ethpbalpha "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
@@ -48,292 +50,340 @@ import (
|
||||
|
||||
func TestGetAggregateAttestation(t *testing.T) {
|
||||
root1 := bytesutil.PadTo([]byte("root1"), 32)
|
||||
sig1 := bytesutil.PadTo([]byte("sig1"), fieldparams.BLSSignatureLength)
|
||||
attSlot1 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{0, 1},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root1,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root1,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root1,
|
||||
},
|
||||
},
|
||||
Signature: sig1,
|
||||
}
|
||||
root21 := bytesutil.PadTo([]byte("root2_1"), 32)
|
||||
sig21 := bytesutil.PadTo([]byte("sig2_1"), fieldparams.BLSSignatureLength)
|
||||
attslot21 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{0, 1, 1},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 2,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root21,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root21,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root21,
|
||||
},
|
||||
},
|
||||
Signature: sig21,
|
||||
}
|
||||
root22 := bytesutil.PadTo([]byte("root2_2"), 32)
|
||||
sig22 := bytesutil.PadTo([]byte("sig2_2"), fieldparams.BLSSignatureLength)
|
||||
attslot22 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{0, 1, 1, 1},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 2,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root22,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root22,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root22,
|
||||
},
|
||||
},
|
||||
Signature: sig22,
|
||||
}
|
||||
root31 := bytesutil.PadTo([]byte("root3_1"), 32)
|
||||
sig31 := bls.NewAggregateSignature().Marshal()
|
||||
attslot31 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{1, 0},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 3,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root31,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root31,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root31,
|
||||
},
|
||||
},
|
||||
Signature: sig31,
|
||||
}
|
||||
root32 := bytesutil.PadTo([]byte("root3_2"), 32)
|
||||
sig32 := bls.NewAggregateSignature().Marshal()
|
||||
attslot32 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{0, 1},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 3,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root32,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root32,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root32,
|
||||
},
|
||||
},
|
||||
Signature: sig32,
|
||||
}
|
||||
root2 := bytesutil.PadTo([]byte("root2"), 32)
|
||||
key, err := bls.RandKey()
|
||||
require.NoError(t, err)
|
||||
sig := key.Sign([]byte("sig"))
|
||||
|
||||
pool := attestations.NewPool()
|
||||
err := pool.SaveAggregatedAttestations([]ethpbalpha.Att{attSlot1, attslot21, attslot22})
|
||||
assert.NoError(t, err)
|
||||
err = pool.SaveUnaggregatedAttestations([]ethpbalpha.Att{attslot31, attslot32})
|
||||
assert.NoError(t, err)
|
||||
t.Run("V1", func(t *testing.T) {
|
||||
createAttestation := func(slot primitives.Slot, aggregationBits bitfield.Bitlist, root []byte) *ethpbalpha.Attestation {
|
||||
return ðpbalpha.Attestation{
|
||||
AggregationBits: aggregationBits,
|
||||
Data: createAttestationData(slot, 1, 1, root),
|
||||
Signature: sig.Marshal(),
|
||||
}
|
||||
}
|
||||
|
||||
s := &Server{
|
||||
AttestationsPool: pool,
|
||||
}
|
||||
aggSlot1_Root1_1 := createAttestation(1, bitfield.Bitlist{0b11100}, root1)
|
||||
aggSlot1_Root1_2 := createAttestation(1, bitfield.Bitlist{0b10111}, root1)
|
||||
aggSlot1_Root2 := createAttestation(1, bitfield.Bitlist{0b11100}, root2)
|
||||
aggSlot2 := createAttestation(2, bitfield.Bitlist{0b11100}, root1)
|
||||
unaggSlot3_Root1_1 := createAttestation(3, bitfield.Bitlist{0b11000}, root1)
|
||||
unaggSlot3_Root1_2 := createAttestation(3, bitfield.Bitlist{0b10100}, root1)
|
||||
unaggSlot3_Root2 := createAttestation(3, bitfield.Bitlist{0b11000}, root2)
|
||||
unaggSlot4 := createAttestation(4, bitfield.Bitlist{0b11000}, root1)
|
||||
|
||||
t.Run("matching aggregated att", func(t *testing.T) {
|
||||
reqRoot, err := attslot22.Data.HashTreeRoot()
|
||||
compareResult := func(
|
||||
t *testing.T,
|
||||
attestation structs.Attestation,
|
||||
expectedSlot string,
|
||||
expectedAggregationBits string,
|
||||
expectedRoot []byte,
|
||||
expectedSig []byte,
|
||||
) {
|
||||
assert.Equal(t, expectedAggregationBits, attestation.AggregationBits, "Unexpected aggregation bits in attestation")
|
||||
assert.Equal(t, hexutil.Encode(expectedSig), attestation.Signature, "Signature mismatch")
|
||||
assert.Equal(t, expectedSlot, attestation.Data.Slot, "Slot mismatch in attestation data")
|
||||
assert.Equal(t, "1", attestation.Data.CommitteeIndex, "Committee index mismatch")
|
||||
assert.Equal(t, hexutil.Encode(expectedRoot), attestation.Data.BeaconBlockRoot, "Beacon block root mismatch")
|
||||
|
||||
// Source checkpoint checks
|
||||
require.NotNil(t, attestation.Data.Source, "Source checkpoint should not be nil")
|
||||
assert.Equal(t, "1", attestation.Data.Source.Epoch, "Source epoch mismatch")
|
||||
assert.Equal(t, hexutil.Encode(expectedRoot), attestation.Data.Source.Root, "Source root mismatch")
|
||||
|
||||
// Target checkpoint checks
|
||||
require.NotNil(t, attestation.Data.Target, "Target checkpoint should not be nil")
|
||||
assert.Equal(t, "1", attestation.Data.Target.Epoch, "Target epoch mismatch")
|
||||
assert.Equal(t, hexutil.Encode(expectedRoot), attestation.Data.Target.Root, "Target root mismatch")
|
||||
}
|
||||
|
||||
pool := attestations.NewPool()
|
||||
require.NoError(t, pool.SaveUnaggregatedAttestations([]ethpbalpha.Att{unaggSlot3_Root1_1, unaggSlot3_Root1_2, unaggSlot3_Root2, unaggSlot4}), "Failed to save unaggregated attestations")
|
||||
unagg, err := pool.UnaggregatedAttestations()
|
||||
require.NoError(t, err)
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=2"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
require.Equal(t, 4, len(unagg), "Expected 4 unaggregated attestations")
|
||||
require.NoError(t, pool.SaveAggregatedAttestations([]ethpbalpha.Att{aggSlot1_Root1_1, aggSlot1_Root1_2, aggSlot1_Root2, aggSlot2}), "Failed to save aggregated attestations")
|
||||
agg := pool.AggregatedAttestations()
|
||||
require.Equal(t, 4, len(agg), "Expected 4 aggregated attestations")
|
||||
s := &Server{
|
||||
AttestationsPool: pool,
|
||||
}
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
resp := &structs.AggregateAttestationResponse{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), resp))
|
||||
require.NotNil(t, resp)
|
||||
require.NotNil(t, resp.Data)
|
||||
assert.DeepEqual(t, "0x00010101", resp.Data.AggregationBits)
|
||||
assert.DeepEqual(t, hexutil.Encode(sig22), resp.Data.Signature)
|
||||
assert.Equal(t, "2", resp.Data.Data.Slot)
|
||||
assert.Equal(t, "1", resp.Data.Data.CommitteeIndex)
|
||||
assert.DeepEqual(t, hexutil.Encode(root22), resp.Data.Data.BeaconBlockRoot)
|
||||
require.NotNil(t, resp.Data.Data.Source)
|
||||
assert.Equal(t, "1", resp.Data.Data.Source.Epoch)
|
||||
assert.DeepEqual(t, hexutil.Encode(root22), resp.Data.Data.Source.Root)
|
||||
require.NotNil(t, resp.Data.Data.Target)
|
||||
assert.Equal(t, "1", resp.Data.Data.Target.Epoch)
|
||||
assert.DeepEqual(t, hexutil.Encode(root22), resp.Data.Data.Target.Root)
|
||||
t.Run("non-matching attestation request", func(t *testing.T) {
|
||||
reqRoot, err := aggSlot2.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusNotFound, writer.Code, "Expected HTTP status NotFound for non-matching request")
|
||||
})
|
||||
t.Run("1 matching aggregated attestation", func(t *testing.T) {
|
||||
reqRoot, err := aggSlot2.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=2"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.Attestation
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
|
||||
compareResult(t, attestation, "2", hexutil.Encode(aggSlot2.AggregationBits), root1, sig.Marshal())
|
||||
})
|
||||
t.Run("multiple matching aggregated attestations - return the one with most bits", func(t *testing.T) {
|
||||
reqRoot, err := aggSlot1_Root1_1.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.Attestation
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
|
||||
compareResult(t, attestation, "1", hexutil.Encode(aggSlot1_Root1_2.AggregationBits), root1, sig.Marshal())
|
||||
})
|
||||
t.Run("1 matching unaggregated attestation", func(t *testing.T) {
|
||||
reqRoot, err := unaggSlot4.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=4"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.Attestation
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
compareResult(t, attestation, "4", hexutil.Encode(unaggSlot4.AggregationBits), root1, sig.Marshal())
|
||||
})
|
||||
t.Run("multiple matching unaggregated attestations - their aggregate is returned", func(t *testing.T) {
|
||||
reqRoot, err := unaggSlot3_Root1_1.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=3"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.Attestation
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
sig1, err := bls.SignatureFromBytes(unaggSlot3_Root1_1.Signature)
|
||||
require.NoError(t, err)
|
||||
sig2, err := bls.SignatureFromBytes(unaggSlot3_Root1_2.Signature)
|
||||
require.NoError(t, err)
|
||||
expectedSig := bls.AggregateSignatures([]common.Signature{sig1, sig2})
|
||||
compareResult(t, attestation, "3", hexutil.Encode(bitfield.Bitlist{0b11100}), root1, expectedSig.Marshal())
|
||||
})
|
||||
})
|
||||
t.Run("matching unaggregated att", func(t *testing.T) {
|
||||
reqRoot, err := attslot32.Data.HashTreeRoot()
|
||||
t.Run("V2", func(t *testing.T) {
|
||||
createAttestation := func(slot primitives.Slot, aggregationBits bitfield.Bitlist, root []byte, bits uint64) *ethpbalpha.AttestationElectra {
|
||||
committeeBits := bitfield.NewBitvector64()
|
||||
committeeBits.SetBitAt(bits, true)
|
||||
|
||||
return ðpbalpha.AttestationElectra{
|
||||
CommitteeBits: committeeBits,
|
||||
AggregationBits: aggregationBits,
|
||||
Data: createAttestationData(slot, 0, 1, root),
|
||||
Signature: sig.Marshal(),
|
||||
}
|
||||
}
|
||||
|
||||
aggSlot1_Root1_1 := createAttestation(1, bitfield.Bitlist{0b11100}, root1, 1)
|
||||
aggSlot1_Root1_2 := createAttestation(1, bitfield.Bitlist{0b10111}, root1, 1)
|
||||
aggSlot1_Root2 := createAttestation(1, bitfield.Bitlist{0b11100}, root2, 1)
|
||||
aggSlot2 := createAttestation(2, bitfield.Bitlist{0b11100}, root1, 1)
|
||||
unaggSlot3_Root1_1 := createAttestation(3, bitfield.Bitlist{0b11000}, root1, 1)
|
||||
unaggSlot3_Root1_2 := createAttestation(3, bitfield.Bitlist{0b10100}, root1, 1)
|
||||
unaggSlot3_Root2 := createAttestation(3, bitfield.Bitlist{0b11000}, root2, 1)
|
||||
unaggSlot4 := createAttestation(4, bitfield.Bitlist{0b11000}, root1, 1)
|
||||
|
||||
compareResult := func(
|
||||
t *testing.T,
|
||||
attestation structs.AttestationElectra,
|
||||
expectedSlot string,
|
||||
expectedAggregationBits string,
|
||||
expectedRoot []byte,
|
||||
expectedSig []byte,
|
||||
expectedCommitteeBits string,
|
||||
) {
|
||||
assert.Equal(t, expectedAggregationBits, attestation.AggregationBits, "Unexpected aggregation bits in attestation")
|
||||
assert.Equal(t, expectedCommitteeBits, attestation.CommitteeBits)
|
||||
assert.Equal(t, hexutil.Encode(expectedSig), attestation.Signature, "Signature mismatch")
|
||||
assert.Equal(t, expectedSlot, attestation.Data.Slot, "Slot mismatch in attestation data")
|
||||
assert.Equal(t, "0", attestation.Data.CommitteeIndex, "Committee index mismatch")
|
||||
assert.Equal(t, hexutil.Encode(expectedRoot), attestation.Data.BeaconBlockRoot, "Beacon block root mismatch")
|
||||
|
||||
// Source checkpoint checks
|
||||
require.NotNil(t, attestation.Data.Source, "Source checkpoint should not be nil")
|
||||
assert.Equal(t, "1", attestation.Data.Source.Epoch, "Source epoch mismatch")
|
||||
assert.Equal(t, hexutil.Encode(expectedRoot), attestation.Data.Source.Root, "Source root mismatch")
|
||||
|
||||
// Target checkpoint checks
|
||||
require.NotNil(t, attestation.Data.Target, "Target checkpoint should not be nil")
|
||||
assert.Equal(t, "1", attestation.Data.Target.Epoch, "Target epoch mismatch")
|
||||
assert.Equal(t, hexutil.Encode(expectedRoot), attestation.Data.Target.Root, "Target root mismatch")
|
||||
}
|
||||
|
||||
pool := attestations.NewPool()
|
||||
require.NoError(t, pool.SaveUnaggregatedAttestations([]ethpbalpha.Att{unaggSlot3_Root1_1, unaggSlot3_Root1_2, unaggSlot3_Root2, unaggSlot4}), "Failed to save unaggregated attestations")
|
||||
unagg, err := pool.UnaggregatedAttestations()
|
||||
require.NoError(t, err)
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=3"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
require.Equal(t, 4, len(unagg), "Expected 4 unaggregated attestations")
|
||||
require.NoError(t, pool.SaveAggregatedAttestations([]ethpbalpha.Att{aggSlot1_Root1_1, aggSlot1_Root1_2, aggSlot1_Root2, aggSlot2}), "Failed to save aggregated attestations")
|
||||
agg := pool.AggregatedAttestations()
|
||||
require.Equal(t, 4, len(agg), "Expected 4 aggregated attestations")
|
||||
bs, err := util.NewBeaconState()
|
||||
require.NoError(t, err)
|
||||
s := &Server{
|
||||
ChainInfoFetcher: &mockChain.ChainService{State: bs},
|
||||
AttestationsPool: pool,
|
||||
}
|
||||
t.Run("non-matching attestation request", func(t *testing.T) {
|
||||
reqRoot, err := aggSlot2.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=1" + "&committee_index=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
resp := &structs.AggregateAttestationResponse{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), resp))
|
||||
require.NotNil(t, resp)
|
||||
require.NotNil(t, resp.Data)
|
||||
assert.DeepEqual(t, "0x0001", resp.Data.AggregationBits)
|
||||
assert.DeepEqual(t, hexutil.Encode(sig32), resp.Data.Signature)
|
||||
assert.Equal(t, "3", resp.Data.Data.Slot)
|
||||
assert.Equal(t, "1", resp.Data.Data.CommitteeIndex)
|
||||
assert.DeepEqual(t, hexutil.Encode(root32), resp.Data.Data.BeaconBlockRoot)
|
||||
require.NotNil(t, resp.Data.Data.Source)
|
||||
assert.Equal(t, "1", resp.Data.Data.Source.Epoch)
|
||||
assert.DeepEqual(t, hexutil.Encode(root32), resp.Data.Data.Source.Root)
|
||||
require.NotNil(t, resp.Data.Data.Target)
|
||||
assert.Equal(t, "1", resp.Data.Data.Target.Epoch)
|
||||
assert.DeepEqual(t, hexutil.Encode(root32), resp.Data.Data.Target.Root)
|
||||
})
|
||||
t.Run("no matching attestation", func(t *testing.T) {
|
||||
attDataRoot := hexutil.Encode(bytesutil.PadTo([]byte("foo"), 32))
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=2"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
s.GetAggregateAttestationV2(writer, request)
|
||||
assert.Equal(t, http.StatusNotFound, writer.Code, "Expected HTTP status NotFound for non-matching request")
|
||||
})
|
||||
t.Run("1 matching aggregated attestation", func(t *testing.T) {
|
||||
reqRoot, err := aggSlot2.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=2" + "&committee_index=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusNotFound, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusNotFound, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "No matching attestation found"))
|
||||
})
|
||||
t.Run("no attestation_data_root provided", func(t *testing.T) {
|
||||
url := "http://example.com?slot=2"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
s.GetAggregateAttestationV2(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "attestation_data_root is required"))
|
||||
})
|
||||
t.Run("invalid attestation_data_root provided", func(t *testing.T) {
|
||||
url := "http://example.com?attestation_data_root=foo&slot=2"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "attestation_data_root is invalid"))
|
||||
})
|
||||
t.Run("no slot provided", func(t *testing.T) {
|
||||
attDataRoot := hexutil.Encode(bytesutil.PadTo([]byte("foo"), 32))
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
var attestation structs.AttestationElectra
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "slot is required"))
|
||||
})
|
||||
t.Run("invalid slot provided", func(t *testing.T) {
|
||||
attDataRoot := hexutil.Encode(bytesutil.PadTo([]byte("foo"), 32))
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=foo"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
compareResult(t, attestation, "2", hexutil.Encode(aggSlot2.AggregationBits), root1, sig.Marshal(), hexutil.Encode(aggSlot2.CommitteeBits))
|
||||
})
|
||||
t.Run("multiple matching aggregated attestations - return the one with most bits", func(t *testing.T) {
|
||||
reqRoot, err := aggSlot1_Root1_1.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=1" + "&committee_index=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusBadRequest, writer.Code)
|
||||
e := &httputil.DefaultJsonError{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), e))
|
||||
assert.Equal(t, http.StatusBadRequest, e.Code)
|
||||
assert.Equal(t, true, strings.Contains(e.Message, "slot is invalid"))
|
||||
s.GetAggregateAttestationV2(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.AttestationElectra
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
|
||||
compareResult(t, attestation, "1", hexutil.Encode(aggSlot1_Root1_2.AggregationBits), root1, sig.Marshal(), hexutil.Encode(aggSlot1_Root1_1.CommitteeBits))
|
||||
})
|
||||
t.Run("1 matching unaggregated attestation", func(t *testing.T) {
|
||||
reqRoot, err := unaggSlot4.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=4" + "&committee_index=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestationV2(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.AttestationElectra
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
compareResult(t, attestation, "4", hexutil.Encode(unaggSlot4.AggregationBits), root1, sig.Marshal(), hexutil.Encode(unaggSlot4.CommitteeBits))
|
||||
})
|
||||
t.Run("multiple matching unaggregated attestations - their aggregate is returned", func(t *testing.T) {
|
||||
reqRoot, err := unaggSlot3_Root1_1.Data.HashTreeRoot()
|
||||
require.NoError(t, err, "Failed to generate attestation data hash tree root")
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=3" + "&committee_index=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
|
||||
s.GetAggregateAttestationV2(writer, request)
|
||||
require.Equal(t, http.StatusOK, writer.Code, "Expected HTTP status OK")
|
||||
|
||||
var resp structs.AggregateAttestationResponse
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp), "Failed to unmarshal response")
|
||||
require.NotNil(t, resp.Data, "Response data should not be nil")
|
||||
|
||||
var attestation structs.AttestationElectra
|
||||
require.NoError(t, json.Unmarshal(resp.Data, &attestation), "Failed to unmarshal attestation data")
|
||||
sig1, err := bls.SignatureFromBytes(unaggSlot3_Root1_1.Signature)
|
||||
require.NoError(t, err)
|
||||
sig2, err := bls.SignatureFromBytes(unaggSlot3_Root1_2.Signature)
|
||||
require.NoError(t, err)
|
||||
expectedSig := bls.AggregateSignatures([]common.Signature{sig1, sig2})
|
||||
compareResult(t, attestation, "3", hexutil.Encode(bitfield.Bitlist{0b11100}), root1, expectedSig.Marshal(), hexutil.Encode(unaggSlot3_Root1_1.CommitteeBits))
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
func TestGetAggregateAttestation_SameSlotAndRoot_ReturnMostAggregationBits(t *testing.T) {
|
||||
root := bytesutil.PadTo([]byte("root"), 32)
|
||||
sig := bytesutil.PadTo([]byte("sig"), fieldparams.BLSSignatureLength)
|
||||
att1 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{3, 0, 0, 1},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root,
|
||||
},
|
||||
func createAttestationData(
|
||||
slot primitives.Slot,
|
||||
committeeIndex primitives.CommitteeIndex,
|
||||
epoch primitives.Epoch,
|
||||
root []byte,
|
||||
) *ethpbalpha.AttestationData {
|
||||
return ðpbalpha.AttestationData{
|
||||
Slot: slot,
|
||||
CommitteeIndex: committeeIndex,
|
||||
BeaconBlockRoot: root,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: epoch,
|
||||
Root: root,
|
||||
},
|
||||
Signature: sig,
|
||||
}
|
||||
att2 := ðpbalpha.Attestation{
|
||||
AggregationBits: []byte{0, 3, 0, 1},
|
||||
Data: ðpbalpha.AttestationData{
|
||||
Slot: 1,
|
||||
CommitteeIndex: 1,
|
||||
BeaconBlockRoot: root,
|
||||
Source: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: 1,
|
||||
Root: root,
|
||||
},
|
||||
Target: ðpbalpha.Checkpoint{
|
||||
Epoch: epoch,
|
||||
Root: root,
|
||||
},
|
||||
Signature: sig,
|
||||
}
|
||||
pool := attestations.NewPool()
|
||||
err := pool.SaveAggregatedAttestations([]ethpbalpha.Att{att1, att2})
|
||||
assert.NoError(t, err)
|
||||
s := &Server{
|
||||
AttestationsPool: pool,
|
||||
}
|
||||
reqRoot, err := att1.Data.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
attDataRoot := hexutil.Encode(reqRoot[:])
|
||||
url := "http://example.com?attestation_data_root=" + attDataRoot + "&slot=1"
|
||||
request := httptest.NewRequest(http.MethodGet, url, nil)
|
||||
writer := httptest.NewRecorder()
|
||||
writer.Body = &bytes.Buffer{}
|
||||
|
||||
s.GetAggregateAttestation(writer, request)
|
||||
assert.Equal(t, http.StatusOK, writer.Code)
|
||||
resp := &structs.AggregateAttestationResponse{}
|
||||
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), resp))
|
||||
require.NotNil(t, resp)
|
||||
assert.DeepEqual(t, "0x03000001", resp.Data.AggregationBits)
|
||||
}
|
||||
|
||||
func TestSubmitContributionAndProofs(t *testing.T) {
|
||||
|
||||
@@ -287,6 +287,9 @@ func (vs *Server) validatorStatus(
|
||||
Status: ethpb.ValidatorStatus_UNKNOWN_STATUS,
|
||||
ActivationEpoch: params.BeaconConfig().FarFutureEpoch,
|
||||
}
|
||||
if len(pubKey) == 0 {
|
||||
return resp, nonExistentIndex
|
||||
}
|
||||
vStatus, idx, err := statusForPubKey(headState, pubKey)
|
||||
if err != nil && !errors.Is(err, errPubkeyDoesNotExist) {
|
||||
tracing.AnnotateError(span, err)
|
||||
|
||||
@@ -97,10 +97,7 @@ func (s *Service) filterAttestations(
|
||||
// detection (except for the genesis epoch).
|
||||
func validateAttestationIntegrity(att ethpb.IndexedAtt) bool {
|
||||
// If an attestation is malformed, we drop it.
|
||||
if att == nil ||
|
||||
att.GetData() == nil ||
|
||||
att.GetData().Source == nil ||
|
||||
att.GetData().Target == nil {
|
||||
if att == nil || att.IsNil() || att.GetData().Source == nil || att.GetData().Target == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
|
||||
@@ -316,7 +316,7 @@ type WriteOnlySyncCommittee interface {
|
||||
|
||||
type WriteOnlyWithdrawals interface {
|
||||
AppendPendingPartialWithdrawal(ppw *ethpb.PendingPartialWithdrawal) error
|
||||
DequeuePartialWithdrawals(num uint64) error
|
||||
DequeuePendingPartialWithdrawals(num uint64) error
|
||||
SetNextWithdrawalIndex(i uint64) error
|
||||
SetNextWithdrawalValidatorIndex(i primitives.ValidatorIndex) error
|
||||
}
|
||||
|
||||
@@ -29,7 +29,6 @@ type BeaconState struct {
|
||||
stateRoots customtypes.StateRoots
|
||||
stateRootsMultiValue *MultiValueStateRoots
|
||||
historicalRoots customtypes.HistoricalRoots
|
||||
historicalSummaries []*ethpb.HistoricalSummary
|
||||
eth1Data *ethpb.Eth1Data
|
||||
eth1DataVotes []*ethpb.Eth1Data
|
||||
eth1DepositIndex uint64
|
||||
@@ -55,8 +54,11 @@ type BeaconState struct {
|
||||
latestExecutionPayloadHeader *enginev1.ExecutionPayloadHeader
|
||||
latestExecutionPayloadHeaderCapella *enginev1.ExecutionPayloadHeaderCapella
|
||||
latestExecutionPayloadHeaderDeneb *enginev1.ExecutionPayloadHeaderDeneb
|
||||
nextWithdrawalIndex uint64
|
||||
nextWithdrawalValidatorIndex primitives.ValidatorIndex
|
||||
|
||||
// Capella fields
|
||||
nextWithdrawalIndex uint64
|
||||
nextWithdrawalValidatorIndex primitives.ValidatorIndex
|
||||
historicalSummaries []*ethpb.HistoricalSummary
|
||||
|
||||
// Electra fields
|
||||
depositRequestsStartIndex uint64
|
||||
@@ -90,7 +92,6 @@ type beaconStateMarshalable struct {
|
||||
BlockRoots customtypes.BlockRoots `json:"block_roots" yaml:"block_roots"`
|
||||
StateRoots customtypes.StateRoots `json:"state_roots" yaml:"state_roots"`
|
||||
HistoricalRoots customtypes.HistoricalRoots `json:"historical_roots" yaml:"historical_roots"`
|
||||
HistoricalSummaries []*ethpb.HistoricalSummary `json:"historical_summaries" yaml:"historical_summaries"`
|
||||
Eth1Data *ethpb.Eth1Data `json:"eth_1_data" yaml:"eth_1_data"`
|
||||
Eth1DataVotes []*ethpb.Eth1Data `json:"eth_1_data_votes" yaml:"eth_1_data_votes"`
|
||||
Eth1DepositIndex uint64 `json:"eth_1_deposit_index" yaml:"eth_1_deposit_index"`
|
||||
@@ -114,6 +115,7 @@ type beaconStateMarshalable struct {
|
||||
LatestExecutionPayloadHeaderDeneb *enginev1.ExecutionPayloadHeaderDeneb `json:"latest_execution_payload_header_deneb" yaml:"latest_execution_payload_header_deneb"`
|
||||
NextWithdrawalIndex uint64 `json:"next_withdrawal_index" yaml:"next_withdrawal_index"`
|
||||
NextWithdrawalValidatorIndex primitives.ValidatorIndex `json:"next_withdrawal_validator_index" yaml:"next_withdrawal_validator_index"`
|
||||
HistoricalSummaries []*ethpb.HistoricalSummary `json:"historical_summaries" yaml:"historical_summaries"`
|
||||
DepositRequestsStartIndex uint64 `json:"deposit_requests_start_index" yaml:"deposit_requests_start_index"`
|
||||
DepositBalanceToConsume primitives.Gwei `json:"deposit_balance_to_consume" yaml:"deposit_balance_to_consume"`
|
||||
ExitBalanceToConsume primitives.Gwei `json:"exit_balance_to_consume" yaml:"exit_balance_to_consume"`
|
||||
@@ -159,7 +161,6 @@ func (b *BeaconState) MarshalJSON() ([]byte, error) {
|
||||
BlockRoots: bRoots,
|
||||
StateRoots: sRoots,
|
||||
HistoricalRoots: b.historicalRoots,
|
||||
HistoricalSummaries: b.historicalSummaries,
|
||||
Eth1Data: b.eth1Data,
|
||||
Eth1DataVotes: b.eth1DataVotes,
|
||||
Eth1DepositIndex: b.eth1DepositIndex,
|
||||
@@ -183,6 +184,7 @@ func (b *BeaconState) MarshalJSON() ([]byte, error) {
|
||||
LatestExecutionPayloadHeaderDeneb: b.latestExecutionPayloadHeaderDeneb,
|
||||
NextWithdrawalIndex: b.nextWithdrawalIndex,
|
||||
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
|
||||
HistoricalSummaries: b.historicalSummaries,
|
||||
DepositRequestsStartIndex: b.depositRequestsStartIndex,
|
||||
DepositBalanceToConsume: b.depositBalanceToConsume,
|
||||
ExitBalanceToConsume: b.exitBalanceToConsume,
|
||||
|
||||
@@ -4,7 +4,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
)
|
||||
|
||||
// DepositRequestsStartIndex is used for returning the deposit receipts start index which is used for eip6110
|
||||
// DepositRequestsStartIndex is used for returning the deposit requests start index which is used for eip6110
|
||||
func (b *BeaconState) DepositRequestsStartIndex() (uint64, error) {
|
||||
if b.version < version.Electra {
|
||||
return 0, errNotSupported("DepositRequestsStartIndex", b.version)
|
||||
|
||||
@@ -1,6 +1,8 @@
|
||||
package state_native
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stateutil"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
@@ -15,6 +17,9 @@ func (b *BeaconState) AppendPendingConsolidation(val *ethpb.PendingConsolidation
|
||||
if b.version < version.Electra {
|
||||
return errNotSupported("AppendPendingConsolidation", b.version)
|
||||
}
|
||||
if val == nil {
|
||||
return errors.New("cannot append nil pending consolidation")
|
||||
}
|
||||
b.lock.Lock()
|
||||
defer b.lock.Unlock()
|
||||
|
||||
|
||||
@@ -1,6 +1,8 @@
|
||||
package state_native
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stateutil"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
@@ -15,6 +17,9 @@ func (b *BeaconState) AppendPendingDeposit(pd *ethpb.PendingDeposit) error {
|
||||
if b.version < version.Electra {
|
||||
return errNotSupported("AppendPendingDeposit", b.version)
|
||||
}
|
||||
if pd == nil {
|
||||
return errors.New("cannot append nil pending deposit")
|
||||
}
|
||||
b.lock.Lock()
|
||||
defer b.lock.Unlock()
|
||||
|
||||
|
||||
@@ -64,10 +64,10 @@ func (b *BeaconState) AppendPendingPartialWithdrawal(ppw *eth.PendingPartialWith
|
||||
return nil
|
||||
}
|
||||
|
||||
// DequeuePartialWithdrawals removes the partial withdrawals from the beginning of the partial withdrawals list.
|
||||
func (b *BeaconState) DequeuePartialWithdrawals(n uint64) error {
|
||||
// DequeuePendingPartialWithdrawals removes the partial withdrawals from the beginning of the partial withdrawals list.
|
||||
func (b *BeaconState) DequeuePendingPartialWithdrawals(n uint64) error {
|
||||
if b.version < version.Electra {
|
||||
return errNotSupported("DequeuePartialWithdrawals", b.version)
|
||||
return errNotSupported("DequeuePendingPartialWithdrawals", b.version)
|
||||
}
|
||||
|
||||
if n > uint64(len(b.pendingPartialWithdrawals)) {
|
||||
|
||||
@@ -68,7 +68,7 @@ func TestDequeuePendingWithdrawals(t *testing.T) {
|
||||
num, err := s.NumPendingPartialWithdrawals()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, uint64(3), num)
|
||||
require.NoError(t, s.DequeuePartialWithdrawals(2))
|
||||
require.NoError(t, s.DequeuePendingPartialWithdrawals(2))
|
||||
num, err = s.NumPendingPartialWithdrawals()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, uint64(1), num)
|
||||
@@ -77,13 +77,13 @@ func TestDequeuePendingWithdrawals(t *testing.T) {
|
||||
num, err = s.NumPendingPartialWithdrawals()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, uint64(1), num)
|
||||
require.ErrorContains(t, "cannot dequeue more withdrawals than are in the queue", s.DequeuePartialWithdrawals(2))
|
||||
require.ErrorContains(t, "cannot dequeue more withdrawals than are in the queue", s.DequeuePendingPartialWithdrawals(2))
|
||||
|
||||
// Removing all pending partial withdrawals should be OK.
|
||||
num, err = s.NumPendingPartialWithdrawals()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, uint64(1), num)
|
||||
require.NoError(t, s.DequeuePartialWithdrawals(1))
|
||||
require.NoError(t, s.DequeuePendingPartialWithdrawals(1))
|
||||
num, err = s.Copy().NumPendingPartialWithdrawals()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, uint64(0), num)
|
||||
@@ -91,7 +91,7 @@ func TestDequeuePendingWithdrawals(t *testing.T) {
|
||||
s, err = InitializeFromProtoDeneb(ð.BeaconStateDeneb{})
|
||||
require.NoError(t, err)
|
||||
|
||||
require.ErrorContains(t, "is not supported", s.DequeuePartialWithdrawals(0))
|
||||
require.ErrorContains(t, "is not supported", s.DequeuePendingPartialWithdrawals(0))
|
||||
}
|
||||
|
||||
func TestAppendPendingWithdrawals(t *testing.T) {
|
||||
|
||||
@@ -14,7 +14,7 @@ func (b *BeaconState) ProportionalSlashingMultiplier() (uint64, error) {
|
||||
case version.Phase0:
|
||||
return params.BeaconConfig().ProportionalSlashingMultiplier, nil
|
||||
}
|
||||
return 0, errNotSupported("ProportionalSlashingMultiplier()", b.version)
|
||||
return 0, errNotSupported("ProportionalSlashingMultiplier", b.version)
|
||||
}
|
||||
|
||||
func (b *BeaconState) InactivityPenaltyQuotient() (uint64, error) {
|
||||
@@ -26,5 +26,5 @@ func (b *BeaconState) InactivityPenaltyQuotient() (uint64, error) {
|
||||
case version.Phase0:
|
||||
return params.BeaconConfig().InactivityPenaltyQuotient, nil
|
||||
}
|
||||
return 0, errNotSupported("InactivityPenaltyQuotient()", b.version)
|
||||
return 0, errNotSupported("InactivityPenaltyQuotient", b.version)
|
||||
}
|
||||
|
||||
@@ -96,10 +96,10 @@ var denebFields = append(
|
||||
|
||||
var electraFields = append(
|
||||
altairFields,
|
||||
types.LatestExecutionPayloadHeaderDeneb,
|
||||
types.NextWithdrawalIndex,
|
||||
types.NextWithdrawalValidatorIndex,
|
||||
types.HistoricalSummaries,
|
||||
types.LatestExecutionPayloadHeaderDeneb,
|
||||
types.DepositRequestsStartIndex,
|
||||
types.DepositBalanceToConsume,
|
||||
types.ExitBalanceToConsume,
|
||||
|
||||
@@ -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/io/file"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
"github.com/prysmaticlabs/prysm/v5/time/slots"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
@@ -62,6 +63,10 @@ func (s *Service) beaconBlockSubscriber(ctx context.Context, msg proto.Message)
|
||||
// This function reconstructs the blob sidecars from the EL using the block's KZG commitments,
|
||||
// broadcasts the reconstructed blobs over P2P, and saves them into the blob storage.
|
||||
func (s *Service) reconstructAndBroadcastBlobs(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock) {
|
||||
if block.Version() < version.Deneb {
|
||||
return
|
||||
}
|
||||
|
||||
startTime, err := slots.ToTime(uint64(s.cfg.chain.GenesisTime().Unix()), block.Block().Slot())
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Failed to convert slot to time")
|
||||
|
||||
@@ -57,11 +57,10 @@ func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, ms
|
||||
}
|
||||
|
||||
aggregate := m.AggregateAttestationAndProof().AggregateVal()
|
||||
data := aggregate.GetData()
|
||||
|
||||
if err := helpers.ValidateNilAttestation(aggregate); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
data := aggregate.GetData()
|
||||
// Do not process slot 0 aggregates.
|
||||
if data.Slot == 0 {
|
||||
return pubsub.ValidationIgnore, nil
|
||||
|
||||
@@ -62,12 +62,11 @@ func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, p
|
||||
if !ok {
|
||||
return pubsub.ValidationReject, errWrongMessage
|
||||
}
|
||||
|
||||
data := att.GetData()
|
||||
|
||||
if err := helpers.ValidateNilAttestation(att); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
data := att.GetData()
|
||||
|
||||
// Do not process slot 0 attestations.
|
||||
if data.Slot == 0 {
|
||||
return pubsub.ValidationIgnore, nil
|
||||
|
||||
@@ -37,6 +37,7 @@ const (
|
||||
SyncCommitteeBranchDepth = 5 // SyncCommitteeBranchDepth defines the number of leaves in a merkle proof of a sync committee.
|
||||
SyncCommitteeBranchDepthElectra = 6 // SyncCommitteeBranchDepthElectra defines the number of leaves in a merkle proof of a sync committee.
|
||||
FinalityBranchDepth = 6 // FinalityBranchDepth defines the number of leaves in a merkle proof of the finalized checkpoint root.
|
||||
FinalityBranchDepthElectra = 7 // FinalityBranchDepthElectra defines the number of leaves in a merkle proof of the finalized checkpoint root.
|
||||
PendingDepositsLimit = 134217728 // Maximum number of pending balance deposits in the beacon state.
|
||||
PendingPartialWithdrawalsLimit = 134217728 // Maximum number of pending partial withdrawals in the beacon state.
|
||||
PendingConsolidationsLimit = 262144 // Maximum number of pending consolidations in the beacon state.
|
||||
|
||||
@@ -37,6 +37,7 @@ const (
|
||||
SyncCommitteeBranchDepth = 5 // SyncCommitteeBranchDepth defines the number of leaves in a merkle proof of a sync committee.
|
||||
SyncCommitteeBranchDepthElectra = 6 // SyncCommitteeBranchDepthElectra defines the number of leaves in a merkle proof of a sync committee.
|
||||
FinalityBranchDepth = 6 // FinalityBranchDepth defines the number of leaves in a merkle proof of the finalized checkpoint root.
|
||||
FinalityBranchDepthElectra = 7 // FinalityBranchDepthElectra defines the number of leaves in a merkle proof of the finalized checkpoint root.
|
||||
PendingDepositsLimit = 134217728 // Maximum number of pending balance deposits in the beacon state.
|
||||
PendingPartialWithdrawalsLimit = 64 // Maximum number of pending partial withdrawals in the beacon state.
|
||||
PendingConsolidationsLimit = 64 // Maximum number of pending consolidations in the beacon state.
|
||||
|
||||
@@ -5,15 +5,18 @@ import (
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
type LightClientExecutionBranch = [fieldparams.ExecutionBranchDepth][fieldparams.RootLength]byte
|
||||
type LightClientSyncCommitteeBranch = [fieldparams.SyncCommitteeBranchDepth][fieldparams.RootLength]byte
|
||||
type LightClientSyncCommitteeBranchElectra = [fieldparams.SyncCommitteeBranchDepthElectra][fieldparams.RootLength]byte
|
||||
type LightClientFinalityBranch = [fieldparams.FinalityBranchDepth][fieldparams.RootLength]byte
|
||||
type LightClientFinalityBranchElectra = [fieldparams.FinalityBranchDepthElectra][fieldparams.RootLength]byte
|
||||
|
||||
type LightClientHeader interface {
|
||||
ssz.Marshaler
|
||||
Proto() proto.Message
|
||||
Version() int
|
||||
Beacon() *pb.BeaconBlockHeader
|
||||
Execution() (ExecutionData, error)
|
||||
@@ -31,29 +34,41 @@ type LightClientBootstrap interface {
|
||||
|
||||
type LightClientUpdate interface {
|
||||
ssz.Marshaler
|
||||
Proto() proto.Message
|
||||
Version() int
|
||||
AttestedHeader() LightClientHeader
|
||||
SetAttestedHeader(header LightClientHeader) error
|
||||
NextSyncCommittee() *pb.SyncCommittee
|
||||
SetNextSyncCommittee(sc *pb.SyncCommittee)
|
||||
NextSyncCommitteeBranch() (LightClientSyncCommitteeBranch, error)
|
||||
SetNextSyncCommitteeBranch(branch [][]byte) error
|
||||
NextSyncCommitteeBranchElectra() (LightClientSyncCommitteeBranchElectra, error)
|
||||
FinalizedHeader() LightClientHeader
|
||||
FinalityBranch() LightClientFinalityBranch
|
||||
SetFinalizedHeader(header LightClientHeader) error
|
||||
FinalityBranch() (LightClientFinalityBranch, error)
|
||||
FinalityBranchElectra() (LightClientFinalityBranchElectra, error)
|
||||
SetFinalityBranch(branch [][]byte) error
|
||||
SyncAggregate() *pb.SyncAggregate
|
||||
SetSyncAggregate(sa *pb.SyncAggregate)
|
||||
SignatureSlot() primitives.Slot
|
||||
SetSignatureSlot(slot primitives.Slot)
|
||||
}
|
||||
|
||||
type LightClientFinalityUpdate interface {
|
||||
ssz.Marshaler
|
||||
Proto() proto.Message
|
||||
Version() int
|
||||
AttestedHeader() LightClientHeader
|
||||
FinalizedHeader() LightClientHeader
|
||||
FinalityBranch() LightClientFinalityBranch
|
||||
FinalityBranch() (LightClientFinalityBranch, error)
|
||||
FinalityBranchElectra() (LightClientFinalityBranchElectra, error)
|
||||
SyncAggregate() *pb.SyncAggregate
|
||||
SignatureSlot() primitives.Slot
|
||||
}
|
||||
|
||||
type LightClientOptimisticUpdate interface {
|
||||
ssz.Marshaler
|
||||
Proto() proto.Message
|
||||
Version() int
|
||||
AttestedHeader() LightClientHeader
|
||||
SyncAggregate() *pb.SyncAggregate
|
||||
|
||||
@@ -14,6 +14,7 @@ go_library(
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
"//consensus-types/interfaces:go_default_library",
|
||||
@@ -21,6 +22,7 @@ go_library(
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//runtime/version:go_default_library",
|
||||
"//time/slots:go_default_library",
|
||||
"@org_golang_google_protobuf//proto:go_default_library",
|
||||
],
|
||||
)
|
||||
|
||||
@@ -41,7 +41,7 @@ func NewWrappedBootstrapAltair(p *pb.LightClientBootstrapAltair) (interfaces.Lig
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
header, err := NewWrappedHeaderAltair(p.Header)
|
||||
header, err := NewWrappedHeader(p.Header)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -105,7 +105,7 @@ func NewWrappedBootstrapCapella(p *pb.LightClientBootstrapCapella) (interfaces.L
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
header, err := NewWrappedHeaderCapella(p.Header)
|
||||
header, err := NewWrappedHeader(p.Header)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -169,7 +169,7 @@ func NewWrappedBootstrapDeneb(p *pb.LightClientBootstrapDeneb) (interfaces.Light
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
header, err := NewWrappedHeaderDeneb(p.Header)
|
||||
header, err := NewWrappedHeader(p.Header)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -233,7 +233,7 @@ func NewWrappedBootstrapElectra(p *pb.LightClientBootstrapElectra) (interfaces.L
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
header, err := NewWrappedHeaderDeneb(p.Header)
|
||||
header, err := NewWrappedHeader(p.Header)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@@ -23,11 +23,72 @@ func NewWrappedFinalityUpdate(m proto.Message) (interfaces.LightClientFinalityUp
|
||||
return NewWrappedFinalityUpdateCapella(t)
|
||||
case *pb.LightClientFinalityUpdateDeneb:
|
||||
return NewWrappedFinalityUpdateDeneb(t)
|
||||
case *pb.LightClientFinalityUpdateElectra:
|
||||
return NewWrappedFinalityUpdateElectra(t)
|
||||
default:
|
||||
return nil, fmt.Errorf("cannot construct light client finality update from type %T", t)
|
||||
}
|
||||
}
|
||||
|
||||
func NewFinalityUpdateFromUpdate(update interfaces.LightClientUpdate) (interfaces.LightClientFinalityUpdate, error) {
|
||||
switch t := update.(type) {
|
||||
case *updateAltair:
|
||||
return &finalityUpdateAltair{
|
||||
p: &pb.LightClientFinalityUpdateAltair{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
FinalizedHeader: t.p.FinalizedHeader,
|
||||
FinalityBranch: t.p.FinalityBranch,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
finalizedHeader: t.finalizedHeader,
|
||||
finalityBranch: t.finalityBranch,
|
||||
}, nil
|
||||
case *updateCapella:
|
||||
return &finalityUpdateCapella{
|
||||
p: &pb.LightClientFinalityUpdateCapella{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
FinalizedHeader: t.p.FinalizedHeader,
|
||||
FinalityBranch: t.p.FinalityBranch,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
finalizedHeader: t.finalizedHeader,
|
||||
finalityBranch: t.finalityBranch,
|
||||
}, nil
|
||||
case *updateDeneb:
|
||||
return &finalityUpdateDeneb{
|
||||
p: &pb.LightClientFinalityUpdateDeneb{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
FinalizedHeader: t.p.FinalizedHeader,
|
||||
FinalityBranch: t.p.FinalityBranch,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
finalizedHeader: t.finalizedHeader,
|
||||
finalityBranch: t.finalityBranch,
|
||||
}, nil
|
||||
case *updateElectra:
|
||||
return &finalityUpdateElectra{
|
||||
p: &pb.LightClientFinalityUpdateElectra{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
FinalizedHeader: t.p.FinalizedHeader,
|
||||
FinalityBranch: t.p.FinalityBranch,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
finalizedHeader: t.finalizedHeader,
|
||||
finalityBranch: t.finalityBranch,
|
||||
}, nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported type %T", t)
|
||||
}
|
||||
}
|
||||
|
||||
type finalityUpdateAltair struct {
|
||||
p *pb.LightClientFinalityUpdateAltair
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
@@ -41,11 +102,11 @@ func NewWrappedFinalityUpdateAltair(p *pb.LightClientFinalityUpdateAltair) (inte
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderAltair(p.AttestedHeader)
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderAltair(p.FinalizedHeader)
|
||||
finalizedHeader, err := NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -78,6 +139,10 @@ func (u *finalityUpdateAltair) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *finalityUpdateAltair) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *finalityUpdateAltair) Version() int {
|
||||
return version.Altair
|
||||
}
|
||||
@@ -90,8 +155,12 @@ func (u *finalityUpdateAltair) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *finalityUpdateAltair) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *finalityUpdateAltair) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *finalityUpdateAltair) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return interfaces.LightClientFinalityBranchElectra{}, consensustypes.ErrNotSupported("FinalityBranchElectra", u.Version())
|
||||
}
|
||||
|
||||
func (u *finalityUpdateAltair) SyncAggregate() *pb.SyncAggregate {
|
||||
@@ -115,11 +184,11 @@ func NewWrappedFinalityUpdateCapella(p *pb.LightClientFinalityUpdateCapella) (in
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderCapella(p.AttestedHeader)
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderCapella(p.FinalizedHeader)
|
||||
finalizedHeader, err := NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -152,6 +221,10 @@ func (u *finalityUpdateCapella) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *finalityUpdateCapella) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *finalityUpdateCapella) Version() int {
|
||||
return version.Capella
|
||||
}
|
||||
@@ -164,8 +237,12 @@ func (u *finalityUpdateCapella) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *finalityUpdateCapella) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *finalityUpdateCapella) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *finalityUpdateCapella) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return interfaces.LightClientFinalityBranchElectra{}, consensustypes.ErrNotSupported("FinalityBranchElectra", u.Version())
|
||||
}
|
||||
|
||||
func (u *finalityUpdateCapella) SyncAggregate() *pb.SyncAggregate {
|
||||
@@ -189,11 +266,11 @@ func NewWrappedFinalityUpdateDeneb(p *pb.LightClientFinalityUpdateDeneb) (interf
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderDeneb(p.AttestedHeader)
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderDeneb(p.FinalizedHeader)
|
||||
finalizedHeader, err := NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -226,6 +303,10 @@ func (u *finalityUpdateDeneb) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *finalityUpdateDeneb) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *finalityUpdateDeneb) Version() int {
|
||||
return version.Deneb
|
||||
}
|
||||
@@ -238,8 +319,12 @@ func (u *finalityUpdateDeneb) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *finalityUpdateDeneb) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *finalityUpdateDeneb) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *finalityUpdateDeneb) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return interfaces.LightClientFinalityBranchElectra{}, consensustypes.ErrNotSupported("FinalityBranchElectra", u.Version())
|
||||
}
|
||||
|
||||
func (u *finalityUpdateDeneb) SyncAggregate() *pb.SyncAggregate {
|
||||
@@ -249,3 +334,86 @@ func (u *finalityUpdateDeneb) SyncAggregate() *pb.SyncAggregate {
|
||||
func (u *finalityUpdateDeneb) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
type finalityUpdateElectra struct {
|
||||
p *pb.LightClientFinalityUpdateElectra
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
finalizedHeader interfaces.LightClientHeader
|
||||
finalityBranch interfaces.LightClientFinalityBranchElectra
|
||||
}
|
||||
|
||||
var _ interfaces.LightClientFinalityUpdate = &finalityUpdateElectra{}
|
||||
|
||||
func NewWrappedFinalityUpdateElectra(p *pb.LightClientFinalityUpdateElectra) (interfaces.LightClientFinalityUpdate, error) {
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
finalityBranch, err := createBranch[interfaces.LightClientFinalityBranchElectra](
|
||||
"finality",
|
||||
p.FinalityBranch,
|
||||
fieldparams.FinalityBranchDepthElectra,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &finalityUpdateElectra{
|
||||
p: p,
|
||||
attestedHeader: attestedHeader,
|
||||
finalizedHeader: finalizedHeader,
|
||||
finalityBranch: finalityBranch,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
return u.p.MarshalSSZTo(dst)
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) MarshalSSZ() ([]byte, error) {
|
||||
return u.p.MarshalSSZ()
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return interfaces.LightClientFinalityBranch{}, consensustypes.ErrNotSupported("FinalityBranch", u.Version())
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *finalityUpdateElectra) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
@@ -4,11 +4,13 @@ import (
|
||||
"fmt"
|
||||
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
consensustypes "github.com/prysmaticlabs/prysm/v5/consensus-types"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
"github.com/prysmaticlabs/prysm/v5/time/slots"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
@@ -22,6 +24,9 @@ func NewWrappedHeader(m proto.Message) (interfaces.LightClientHeader, error) {
|
||||
case *pb.LightClientHeaderCapella:
|
||||
return NewWrappedHeaderCapella(t)
|
||||
case *pb.LightClientHeaderDeneb:
|
||||
if slots.ToEpoch(t.Beacon.Slot) >= params.BeaconConfig().ElectraForkEpoch {
|
||||
return NewWrappedHeaderElectra(t)
|
||||
}
|
||||
return NewWrappedHeaderDeneb(t)
|
||||
default:
|
||||
return nil, fmt.Errorf("cannot construct light client header from type %T", t)
|
||||
@@ -53,6 +58,10 @@ func (h *headerAltair) SizeSSZ() int {
|
||||
return h.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (h *headerAltair) Proto() proto.Message {
|
||||
return h.p
|
||||
}
|
||||
|
||||
func (h *headerAltair) Version() int {
|
||||
return version.Altair
|
||||
}
|
||||
@@ -62,11 +71,11 @@ func (h *headerAltair) Beacon() *pb.BeaconBlockHeader {
|
||||
}
|
||||
|
||||
func (h *headerAltair) Execution() (interfaces.ExecutionData, error) {
|
||||
return nil, consensustypes.ErrNotSupported("Execution", version.Altair)
|
||||
return nil, consensustypes.ErrNotSupported("Execution", h.Version())
|
||||
}
|
||||
|
||||
func (h *headerAltair) ExecutionBranch() (interfaces.LightClientExecutionBranch, error) {
|
||||
return interfaces.LightClientExecutionBranch{}, consensustypes.ErrNotSupported("ExecutionBranch", version.Altair)
|
||||
return interfaces.LightClientExecutionBranch{}, consensustypes.ErrNotSupported("ExecutionBranch", h.Version())
|
||||
}
|
||||
|
||||
type headerCapella struct {
|
||||
@@ -114,6 +123,10 @@ func (h *headerCapella) SizeSSZ() int {
|
||||
return h.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (h *headerCapella) Proto() proto.Message {
|
||||
return h.p
|
||||
}
|
||||
|
||||
func (h *headerCapella) Version() int {
|
||||
return version.Capella
|
||||
}
|
||||
@@ -175,6 +188,10 @@ func (h *headerDeneb) SizeSSZ() int {
|
||||
return h.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (h *headerDeneb) Proto() proto.Message {
|
||||
return h.p
|
||||
}
|
||||
|
||||
func (h *headerDeneb) Version() int {
|
||||
return version.Deneb
|
||||
}
|
||||
@@ -190,3 +207,68 @@ func (h *headerDeneb) Execution() (interfaces.ExecutionData, error) {
|
||||
func (h *headerDeneb) ExecutionBranch() (interfaces.LightClientExecutionBranch, error) {
|
||||
return h.executionBranch, nil
|
||||
}
|
||||
|
||||
type headerElectra struct {
|
||||
p *pb.LightClientHeaderDeneb
|
||||
execution interfaces.ExecutionData
|
||||
executionBranch interfaces.LightClientExecutionBranch
|
||||
}
|
||||
|
||||
var _ interfaces.LightClientHeader = &headerElectra{}
|
||||
|
||||
func NewWrappedHeaderElectra(p *pb.LightClientHeaderDeneb) (interfaces.LightClientHeader, error) {
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
execution, err := blocks.WrappedExecutionPayloadHeaderDeneb(p.Execution)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
branch, err := createBranch[interfaces.LightClientExecutionBranch](
|
||||
"execution",
|
||||
p.ExecutionBranch,
|
||||
fieldparams.ExecutionBranchDepth,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &headerElectra{
|
||||
p: p,
|
||||
execution: execution,
|
||||
executionBranch: branch,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (h *headerElectra) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
return h.p.MarshalSSZTo(dst)
|
||||
}
|
||||
|
||||
func (h *headerElectra) MarshalSSZ() ([]byte, error) {
|
||||
return h.p.MarshalSSZ()
|
||||
}
|
||||
|
||||
func (h *headerElectra) SizeSSZ() int {
|
||||
return h.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (h *headerElectra) Proto() proto.Message {
|
||||
return h.p
|
||||
}
|
||||
|
||||
func (h *headerElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
func (h *headerElectra) Beacon() *pb.BeaconBlockHeader {
|
||||
return h.p.Beacon
|
||||
}
|
||||
|
||||
func (h *headerElectra) Execution() (interfaces.ExecutionData, error) {
|
||||
return h.execution, nil
|
||||
}
|
||||
|
||||
func (h *headerElectra) ExecutionBranch() (interfaces.LightClientExecutionBranch, error) {
|
||||
return h.executionBranch, nil
|
||||
}
|
||||
|
||||
@@ -4,12 +4,11 @@ import (
|
||||
"fmt"
|
||||
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
||||
)
|
||||
|
||||
type branchConstraint interface {
|
||||
~interfaces.LightClientExecutionBranch | ~interfaces.LightClientSyncCommitteeBranch | ~interfaces.LightClientFinalityBranch
|
||||
[4][fieldparams.RootLength]byte | [5][fieldparams.RootLength]byte | [6][fieldparams.RootLength]byte | [7][fieldparams.RootLength]byte
|
||||
}
|
||||
|
||||
func createBranch[T branchConstraint](name string, input [][]byte, depth int) (T, error) {
|
||||
|
||||
@@ -27,12 +27,55 @@ func NewWrappedOptimisticUpdate(m proto.Message) (interfaces.LightClientOptimist
|
||||
}
|
||||
}
|
||||
|
||||
type OptimisticUpdateAltair struct {
|
||||
func NewOptimisticUpdateFromUpdate(update interfaces.LightClientUpdate) (interfaces.LightClientOptimisticUpdate, error) {
|
||||
switch t := update.(type) {
|
||||
case *updateAltair:
|
||||
return &optimisticUpdateAltair{
|
||||
p: &pb.LightClientOptimisticUpdateAltair{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
}, nil
|
||||
case *updateCapella:
|
||||
return &optimisticUpdateCapella{
|
||||
p: &pb.LightClientOptimisticUpdateCapella{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
}, nil
|
||||
case *updateDeneb:
|
||||
return &optimisticUpdateDeneb{
|
||||
p: &pb.LightClientOptimisticUpdateDeneb{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
}, nil
|
||||
case *updateElectra:
|
||||
return &optimisticUpdateDeneb{
|
||||
p: &pb.LightClientOptimisticUpdateDeneb{
|
||||
AttestedHeader: t.p.AttestedHeader,
|
||||
SyncAggregate: t.p.SyncAggregate,
|
||||
SignatureSlot: t.p.SignatureSlot,
|
||||
},
|
||||
attestedHeader: t.attestedHeader,
|
||||
}, nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported type %T", t)
|
||||
}
|
||||
}
|
||||
|
||||
type optimisticUpdateAltair struct {
|
||||
p *pb.LightClientOptimisticUpdateAltair
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
}
|
||||
|
||||
var _ interfaces.LightClientOptimisticUpdate = &OptimisticUpdateAltair{}
|
||||
var _ interfaces.LightClientOptimisticUpdate = &optimisticUpdateAltair{}
|
||||
|
||||
func NewWrappedOptimisticUpdateAltair(p *pb.LightClientOptimisticUpdateAltair) (interfaces.LightClientOptimisticUpdate, error) {
|
||||
if p == nil {
|
||||
@@ -43,46 +86,50 @@ func NewWrappedOptimisticUpdateAltair(p *pb.LightClientOptimisticUpdateAltair) (
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &OptimisticUpdateAltair{
|
||||
return &optimisticUpdateAltair{
|
||||
p: p,
|
||||
attestedHeader: attestedHeader,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
func (u *optimisticUpdateAltair) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
return u.p.MarshalSSZTo(dst)
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) MarshalSSZ() ([]byte, error) {
|
||||
func (u *optimisticUpdateAltair) MarshalSSZ() ([]byte, error) {
|
||||
return u.p.MarshalSSZ()
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) SizeSSZ() int {
|
||||
func (u *optimisticUpdateAltair) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) Version() int {
|
||||
func (u *optimisticUpdateAltair) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *optimisticUpdateAltair) Version() int {
|
||||
return version.Altair
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) AttestedHeader() interfaces.LightClientHeader {
|
||||
func (u *optimisticUpdateAltair) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) SyncAggregate() *pb.SyncAggregate {
|
||||
func (u *optimisticUpdateAltair) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateAltair) SignatureSlot() primitives.Slot {
|
||||
func (u *optimisticUpdateAltair) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
type OptimisticUpdateCapella struct {
|
||||
type optimisticUpdateCapella struct {
|
||||
p *pb.LightClientOptimisticUpdateCapella
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
}
|
||||
|
||||
var _ interfaces.LightClientOptimisticUpdate = &OptimisticUpdateCapella{}
|
||||
var _ interfaces.LightClientOptimisticUpdate = &optimisticUpdateCapella{}
|
||||
|
||||
func NewWrappedOptimisticUpdateCapella(p *pb.LightClientOptimisticUpdateCapella) (interfaces.LightClientOptimisticUpdate, error) {
|
||||
if p == nil {
|
||||
@@ -93,46 +140,50 @@ func NewWrappedOptimisticUpdateCapella(p *pb.LightClientOptimisticUpdateCapella)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &OptimisticUpdateCapella{
|
||||
return &optimisticUpdateCapella{
|
||||
p: p,
|
||||
attestedHeader: attestedHeader,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
func (u *optimisticUpdateCapella) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
return u.p.MarshalSSZTo(dst)
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) MarshalSSZ() ([]byte, error) {
|
||||
func (u *optimisticUpdateCapella) MarshalSSZ() ([]byte, error) {
|
||||
return u.p.MarshalSSZ()
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) SizeSSZ() int {
|
||||
func (u *optimisticUpdateCapella) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) Version() int {
|
||||
func (u *optimisticUpdateCapella) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *optimisticUpdateCapella) Version() int {
|
||||
return version.Capella
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) AttestedHeader() interfaces.LightClientHeader {
|
||||
func (u *optimisticUpdateCapella) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) SyncAggregate() *pb.SyncAggregate {
|
||||
func (u *optimisticUpdateCapella) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateCapella) SignatureSlot() primitives.Slot {
|
||||
func (u *optimisticUpdateCapella) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
type OptimisticUpdateDeneb struct {
|
||||
type optimisticUpdateDeneb struct {
|
||||
p *pb.LightClientOptimisticUpdateDeneb
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
}
|
||||
|
||||
var _ interfaces.LightClientOptimisticUpdate = &OptimisticUpdateDeneb{}
|
||||
var _ interfaces.LightClientOptimisticUpdate = &optimisticUpdateDeneb{}
|
||||
|
||||
func NewWrappedOptimisticUpdateDeneb(p *pb.LightClientOptimisticUpdateDeneb) (interfaces.LightClientOptimisticUpdate, error) {
|
||||
if p == nil {
|
||||
@@ -143,36 +194,40 @@ func NewWrappedOptimisticUpdateDeneb(p *pb.LightClientOptimisticUpdateDeneb) (in
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &OptimisticUpdateDeneb{
|
||||
return &optimisticUpdateDeneb{
|
||||
p: p,
|
||||
attestedHeader: attestedHeader,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
func (u *optimisticUpdateDeneb) MarshalSSZTo(dst []byte) ([]byte, error) {
|
||||
return u.p.MarshalSSZTo(dst)
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) MarshalSSZ() ([]byte, error) {
|
||||
func (u *optimisticUpdateDeneb) MarshalSSZ() ([]byte, error) {
|
||||
return u.p.MarshalSSZ()
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) SizeSSZ() int {
|
||||
func (u *optimisticUpdateDeneb) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) Version() int {
|
||||
func (u *optimisticUpdateDeneb) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *optimisticUpdateDeneb) Version() int {
|
||||
return version.Deneb
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) AttestedHeader() interfaces.LightClientHeader {
|
||||
func (u *optimisticUpdateDeneb) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) SyncAggregate() *pb.SyncAggregate {
|
||||
func (u *optimisticUpdateDeneb) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *OptimisticUpdateDeneb) SignatureSlot() primitives.Slot {
|
||||
func (u *optimisticUpdateDeneb) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
@@ -23,11 +23,17 @@ func NewWrappedUpdate(m proto.Message) (interfaces.LightClientUpdate, error) {
|
||||
return NewWrappedUpdateCapella(t)
|
||||
case *pb.LightClientUpdateDeneb:
|
||||
return NewWrappedUpdateDeneb(t)
|
||||
case *pb.LightClientUpdateElectra:
|
||||
return NewWrappedUpdateElectra(t)
|
||||
default:
|
||||
return nil, fmt.Errorf("cannot construct light client update from type %T", t)
|
||||
}
|
||||
}
|
||||
|
||||
// In addition to the proto object being wrapped, we store some fields that have to be
|
||||
// constructed from the proto, so that we don't have to reconstruct them every time
|
||||
// in getters.
|
||||
|
||||
type updateAltair struct {
|
||||
p *pb.LightClientUpdateAltair
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
@@ -42,14 +48,20 @@ func NewWrappedUpdateAltair(p *pb.LightClientUpdateAltair) (interfaces.LightClie
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderAltair(p.AttestedHeader)
|
||||
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderAltair(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
var finalizedHeader interfaces.LightClientHeader
|
||||
if p.FinalizedHeader != nil {
|
||||
finalizedHeader, err = NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
scBranch, err := createBranch[interfaces.LightClientSyncCommitteeBranch](
|
||||
"sync committee",
|
||||
p.NextSyncCommitteeBranch,
|
||||
@@ -88,6 +100,10 @@ func (u *updateAltair) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *updateAltair) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *updateAltair) Version() int {
|
||||
return version.Altair
|
||||
}
|
||||
@@ -96,14 +112,40 @@ func (u *updateAltair) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *updateAltair) SetAttestedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderAltair)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderAltair{})
|
||||
}
|
||||
u.p.AttestedHeader = proto
|
||||
u.attestedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateAltair) NextSyncCommittee() *pb.SyncCommittee {
|
||||
return u.p.NextSyncCommittee
|
||||
}
|
||||
|
||||
func (u *updateAltair) SetNextSyncCommittee(sc *pb.SyncCommittee) {
|
||||
u.p.NextSyncCommittee = sc
|
||||
}
|
||||
|
||||
func (u *updateAltair) NextSyncCommitteeBranch() (interfaces.LightClientSyncCommitteeBranch, error) {
|
||||
return u.nextSyncCommitteeBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateAltair) SetNextSyncCommitteeBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientSyncCommitteeBranch]("sync committee", branch, fieldparams.SyncCommitteeBranchDepth)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.nextSyncCommitteeBranch = b
|
||||
|
||||
u.p.NextSyncCommitteeBranch = branch
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateAltair) NextSyncCommitteeBranchElectra() (interfaces.LightClientSyncCommitteeBranchElectra, error) {
|
||||
return [6][32]byte{}, consensustypes.ErrNotSupported("NextSyncCommitteeBranchElectra", version.Altair)
|
||||
}
|
||||
@@ -112,18 +154,53 @@ func (u *updateAltair) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *updateAltair) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *updateAltair) SetFinalizedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderAltair)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderAltair{})
|
||||
}
|
||||
u.p.FinalizedHeader = proto
|
||||
u.finalizedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateAltair) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateAltair) SetFinalityBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientFinalityBranch]("finality", branch, fieldparams.FinalityBranchDepth)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.finalityBranch = b
|
||||
u.p.FinalityBranch = branch
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateAltair) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return interfaces.LightClientFinalityBranchElectra{}, consensustypes.ErrNotSupported("FinalityBranchElectra", version.Altair)
|
||||
}
|
||||
|
||||
func (u *updateAltair) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *updateAltair) SetSyncAggregate(sa *pb.SyncAggregate) {
|
||||
u.p.SyncAggregate = sa
|
||||
}
|
||||
|
||||
func (u *updateAltair) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
func (u *updateAltair) SetSignatureSlot(slot primitives.Slot) {
|
||||
u.p.SignatureSlot = slot
|
||||
}
|
||||
|
||||
// In addition to the proto object being wrapped, we store some fields that have to be
|
||||
// constructed from the proto, so that we don't have to reconstruct them every time
|
||||
// in getters.
|
||||
type updateCapella struct {
|
||||
p *pb.LightClientUpdateCapella
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
@@ -138,14 +215,20 @@ func NewWrappedUpdateCapella(p *pb.LightClientUpdateCapella) (interfaces.LightCl
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderCapella(p.AttestedHeader)
|
||||
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderCapella(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
var finalizedHeader interfaces.LightClientHeader
|
||||
if p.FinalizedHeader != nil {
|
||||
finalizedHeader, err = NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
scBranch, err := createBranch[interfaces.LightClientSyncCommitteeBranch](
|
||||
"sync committee",
|
||||
p.NextSyncCommitteeBranch,
|
||||
@@ -184,6 +267,10 @@ func (u *updateCapella) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *updateCapella) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *updateCapella) Version() int {
|
||||
return version.Capella
|
||||
}
|
||||
@@ -192,14 +279,40 @@ func (u *updateCapella) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *updateCapella) SetAttestedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderCapella)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderCapella{})
|
||||
}
|
||||
u.p.AttestedHeader = proto
|
||||
u.attestedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateCapella) NextSyncCommittee() *pb.SyncCommittee {
|
||||
return u.p.NextSyncCommittee
|
||||
}
|
||||
|
||||
func (u *updateCapella) SetNextSyncCommittee(sc *pb.SyncCommittee) {
|
||||
u.p.NextSyncCommittee = sc
|
||||
}
|
||||
|
||||
func (u *updateCapella) NextSyncCommitteeBranch() (interfaces.LightClientSyncCommitteeBranch, error) {
|
||||
return u.nextSyncCommitteeBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateCapella) SetNextSyncCommitteeBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientSyncCommitteeBranch]("sync committee", branch, fieldparams.SyncCommitteeBranchDepth)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.nextSyncCommitteeBranch = b
|
||||
|
||||
u.p.NextSyncCommitteeBranch = branch
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateCapella) NextSyncCommitteeBranchElectra() (interfaces.LightClientSyncCommitteeBranchElectra, error) {
|
||||
return [6][32]byte{}, consensustypes.ErrNotSupported("NextSyncCommitteeBranchElectra", version.Capella)
|
||||
}
|
||||
@@ -208,18 +321,53 @@ func (u *updateCapella) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *updateCapella) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *updateCapella) SetFinalizedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderCapella)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderCapella{})
|
||||
}
|
||||
u.p.FinalizedHeader = proto
|
||||
u.finalizedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateCapella) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateCapella) SetFinalityBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientFinalityBranch]("finality", branch, fieldparams.FinalityBranchDepth)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.finalityBranch = b
|
||||
u.p.FinalityBranch = branch
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateCapella) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return interfaces.LightClientFinalityBranchElectra{}, consensustypes.ErrNotSupported("FinalityBranchElectra", u.Version())
|
||||
}
|
||||
|
||||
func (u *updateCapella) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *updateCapella) SetSyncAggregate(sa *pb.SyncAggregate) {
|
||||
u.p.SyncAggregate = sa
|
||||
}
|
||||
|
||||
func (u *updateCapella) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
func (u *updateCapella) SetSignatureSlot(slot primitives.Slot) {
|
||||
u.p.SignatureSlot = slot
|
||||
}
|
||||
|
||||
// In addition to the proto object being wrapped, we store some fields that have to be
|
||||
// constructed from the proto, so that we don't have to reconstruct them every time
|
||||
// in getters.
|
||||
type updateDeneb struct {
|
||||
p *pb.LightClientUpdateDeneb
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
@@ -234,14 +382,20 @@ func NewWrappedUpdateDeneb(p *pb.LightClientUpdateDeneb) (interfaces.LightClient
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderDeneb(p.AttestedHeader)
|
||||
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderDeneb(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
var finalizedHeader interfaces.LightClientHeader
|
||||
if p.FinalizedHeader != nil {
|
||||
finalizedHeader, err = NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
scBranch, err := createBranch[interfaces.LightClientSyncCommitteeBranch](
|
||||
"sync committee",
|
||||
p.NextSyncCommitteeBranch,
|
||||
@@ -280,6 +434,10 @@ func (u *updateDeneb) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *updateDeneb) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *updateDeneb) Version() int {
|
||||
return version.Deneb
|
||||
}
|
||||
@@ -288,14 +446,40 @@ func (u *updateDeneb) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SetAttestedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderDeneb)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderDeneb{})
|
||||
}
|
||||
u.p.AttestedHeader = proto
|
||||
u.attestedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateDeneb) NextSyncCommittee() *pb.SyncCommittee {
|
||||
return u.p.NextSyncCommittee
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SetNextSyncCommittee(sc *pb.SyncCommittee) {
|
||||
u.p.NextSyncCommittee = sc
|
||||
}
|
||||
|
||||
func (u *updateDeneb) NextSyncCommitteeBranch() (interfaces.LightClientSyncCommitteeBranch, error) {
|
||||
return u.nextSyncCommitteeBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SetNextSyncCommitteeBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientSyncCommitteeBranch]("sync committee", branch, fieldparams.SyncCommitteeBranchDepth)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.nextSyncCommitteeBranch = b
|
||||
|
||||
u.p.NextSyncCommitteeBranch = branch
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateDeneb) NextSyncCommitteeBranchElectra() (interfaces.LightClientSyncCommitteeBranchElectra, error) {
|
||||
return [6][32]byte{}, consensustypes.ErrNotSupported("NextSyncCommitteeBranchElectra", version.Deneb)
|
||||
}
|
||||
@@ -304,24 +488,59 @@ func (u *updateDeneb) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *updateDeneb) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *updateDeneb) SetFinalizedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderDeneb)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderDeneb{})
|
||||
}
|
||||
u.p.FinalizedHeader = proto
|
||||
u.finalizedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateDeneb) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SetFinalityBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientFinalityBranch]("finality", branch, fieldparams.FinalityBranchDepth)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.finalityBranch = b
|
||||
u.p.FinalityBranch = branch
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateDeneb) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return interfaces.LightClientFinalityBranchElectra{}, consensustypes.ErrNotSupported("FinalityBranchElectra", u.Version())
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SetSyncAggregate(sa *pb.SyncAggregate) {
|
||||
u.p.SyncAggregate = sa
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
func (u *updateDeneb) SetSignatureSlot(slot primitives.Slot) {
|
||||
u.p.SignatureSlot = slot
|
||||
}
|
||||
|
||||
// In addition to the proto object being wrapped, we store some fields that have to be
|
||||
// constructed from the proto, so that we don't have to reconstruct them every time
|
||||
// in getters.
|
||||
type updateElectra struct {
|
||||
p *pb.LightClientUpdateElectra
|
||||
attestedHeader interfaces.LightClientHeader
|
||||
nextSyncCommitteeBranch interfaces.LightClientSyncCommitteeBranchElectra
|
||||
finalizedHeader interfaces.LightClientHeader
|
||||
finalityBranch interfaces.LightClientFinalityBranch
|
||||
finalityBranch interfaces.LightClientFinalityBranchElectra
|
||||
}
|
||||
|
||||
var _ interfaces.LightClientUpdate = &updateElectra{}
|
||||
@@ -330,14 +549,20 @@ func NewWrappedUpdateElectra(p *pb.LightClientUpdateElectra) (interfaces.LightCl
|
||||
if p == nil {
|
||||
return nil, consensustypes.ErrNilObjectWrapped
|
||||
}
|
||||
attestedHeader, err := NewWrappedHeaderDeneb(p.AttestedHeader)
|
||||
|
||||
attestedHeader, err := NewWrappedHeader(p.AttestedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalizedHeader, err := NewWrappedHeaderDeneb(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
var finalizedHeader interfaces.LightClientHeader
|
||||
if p.FinalizedHeader != nil {
|
||||
finalizedHeader, err = NewWrappedHeader(p.FinalizedHeader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
scBranch, err := createBranch[interfaces.LightClientSyncCommitteeBranchElectra](
|
||||
"sync committee",
|
||||
p.NextSyncCommitteeBranch,
|
||||
@@ -346,10 +571,11 @@ func NewWrappedUpdateElectra(p *pb.LightClientUpdateElectra) (interfaces.LightCl
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
finalityBranch, err := createBranch[interfaces.LightClientFinalityBranch](
|
||||
|
||||
finalityBranch, err := createBranch[interfaces.LightClientFinalityBranchElectra](
|
||||
"finality",
|
||||
p.FinalityBranch,
|
||||
fieldparams.FinalityBranchDepth,
|
||||
fieldparams.FinalityBranchDepthElectra,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -376,6 +602,10 @@ func (u *updateElectra) SizeSSZ() int {
|
||||
return u.p.SizeSSZ()
|
||||
}
|
||||
|
||||
func (u *updateElectra) Proto() proto.Message {
|
||||
return u.p
|
||||
}
|
||||
|
||||
func (u *updateElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
@@ -384,14 +614,40 @@ func (u *updateElectra) AttestedHeader() interfaces.LightClientHeader {
|
||||
return u.attestedHeader
|
||||
}
|
||||
|
||||
func (u *updateElectra) SetAttestedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderDeneb)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderDeneb{})
|
||||
}
|
||||
u.p.AttestedHeader = proto
|
||||
u.attestedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateElectra) NextSyncCommittee() *pb.SyncCommittee {
|
||||
return u.p.NextSyncCommittee
|
||||
}
|
||||
|
||||
func (u *updateElectra) SetNextSyncCommittee(sc *pb.SyncCommittee) {
|
||||
u.p.NextSyncCommittee = sc
|
||||
}
|
||||
|
||||
func (u *updateElectra) NextSyncCommitteeBranch() (interfaces.LightClientSyncCommitteeBranch, error) {
|
||||
return [5][32]byte{}, consensustypes.ErrNotSupported("NextSyncCommitteeBranch", version.Electra)
|
||||
}
|
||||
|
||||
func (u *updateElectra) SetNextSyncCommitteeBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientSyncCommitteeBranchElectra]("sync committee", branch, fieldparams.SyncCommitteeBranchDepthElectra)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.nextSyncCommitteeBranch = b
|
||||
|
||||
u.p.NextSyncCommitteeBranch = branch
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateElectra) NextSyncCommitteeBranchElectra() (interfaces.LightClientSyncCommitteeBranchElectra, error) {
|
||||
return u.nextSyncCommitteeBranch, nil
|
||||
}
|
||||
@@ -400,14 +656,46 @@ func (u *updateElectra) FinalizedHeader() interfaces.LightClientHeader {
|
||||
return u.finalizedHeader
|
||||
}
|
||||
|
||||
func (u *updateElectra) FinalityBranch() interfaces.LightClientFinalityBranch {
|
||||
return u.finalityBranch
|
||||
func (u *updateElectra) SetFinalizedHeader(header interfaces.LightClientHeader) error {
|
||||
proto, ok := header.Proto().(*pb.LightClientHeaderDeneb)
|
||||
if !ok {
|
||||
return fmt.Errorf("header type %T is not %T", proto, &pb.LightClientHeaderDeneb{})
|
||||
}
|
||||
u.p.FinalizedHeader = proto
|
||||
u.finalizedHeader = header
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateElectra) FinalityBranch() (interfaces.LightClientFinalityBranch, error) {
|
||||
return interfaces.LightClientFinalityBranch{}, consensustypes.ErrNotSupported("FinalityBranch", u.Version())
|
||||
}
|
||||
|
||||
func (u *updateElectra) SetFinalityBranch(branch [][]byte) error {
|
||||
b, err := createBranch[interfaces.LightClientFinalityBranchElectra]("finality", branch, fieldparams.FinalityBranchDepthElectra)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
u.finalityBranch = b
|
||||
u.p.FinalityBranch = branch
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *updateElectra) FinalityBranchElectra() (interfaces.LightClientFinalityBranchElectra, error) {
|
||||
return u.finalityBranch, nil
|
||||
}
|
||||
|
||||
func (u *updateElectra) SyncAggregate() *pb.SyncAggregate {
|
||||
return u.p.SyncAggregate
|
||||
}
|
||||
|
||||
func (u *updateElectra) SetSyncAggregate(sa *pb.SyncAggregate) {
|
||||
u.p.SyncAggregate = sa
|
||||
}
|
||||
|
||||
func (u *updateElectra) SignatureSlot() primitives.Slot {
|
||||
return u.p.SignatureSlot
|
||||
}
|
||||
|
||||
func (u *updateElectra) SetSignatureSlot(slot primitives.Slot) {
|
||||
u.p.SignatureSlot = slot
|
||||
}
|
||||
|
||||
@@ -10,8 +10,12 @@ go_library(
|
||||
importpath = "github.com/prysmaticlabs/prysm/v5/consensus-types/payload-attribute",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//beacon-chain/state:go_default_library",
|
||||
"//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",
|
||||
"//proto/engine/v1:go_default_library",
|
||||
"//runtime/version:go_default_library",
|
||||
"@com_github_pkg_errors//:go_default_library",
|
||||
|
||||
@@ -38,6 +38,16 @@ func (a *data) Withdrawals() ([]*enginev1.Withdrawal, error) {
|
||||
return a.withdrawals, nil
|
||||
}
|
||||
|
||||
func (a *data) ParentBeaconBlockRoot() ([]byte, error) {
|
||||
if len(a.parentBeaconBlockRoot) == 0 {
|
||||
return nil, errNoParentRoot
|
||||
}
|
||||
if a.version < version.Deneb {
|
||||
return nil, consensus_types.ErrNotSupported("ParentBeaconBlockRoot", a.version)
|
||||
}
|
||||
return a.parentBeaconBlockRoot, nil
|
||||
}
|
||||
|
||||
// PbV1 returns the payload attribute in version 1.
|
||||
func (a *data) PbV1() (*enginev1.PayloadAttributes, error) {
|
||||
if a == nil {
|
||||
@@ -97,6 +107,9 @@ func (a *data) PbV3() (*enginev1.PayloadAttributesV3, error) {
|
||||
|
||||
// IsEmpty returns whether the given payload attribute is empty
|
||||
func (a *data) IsEmpty() bool {
|
||||
if a == nil {
|
||||
return true
|
||||
}
|
||||
if len(a.PrevRandao()) != 0 {
|
||||
return false
|
||||
}
|
||||
|
||||
@@ -10,6 +10,7 @@ type Attributer interface {
|
||||
Timestamp() uint64
|
||||
SuggestedFeeRecipient() []byte
|
||||
Withdrawals() ([]*enginev1.Withdrawal, error)
|
||||
ParentBeaconBlockRoot() ([]byte, error)
|
||||
PbV1() (*enginev1.PayloadAttributes, error)
|
||||
PbV2() (*enginev1.PayloadAttributesV2, error)
|
||||
PbV3() (*enginev1.PayloadAttributesV3, error)
|
||||
|
||||
@@ -2,7 +2,11 @@ package payloadattribute
|
||||
|
||||
import (
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
|
||||
field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
)
|
||||
@@ -23,6 +27,7 @@ type data struct {
|
||||
var (
|
||||
errNilPayloadAttribute = errors.New("received nil payload attribute")
|
||||
errUnsupportedPayloadAttribute = errors.New("unsupported payload attribute")
|
||||
errNoParentRoot = errors.New("parent root is empty")
|
||||
)
|
||||
|
||||
// New returns a new payload attribute with the given input object.
|
||||
@@ -89,3 +94,16 @@ func initPayloadAttributeFromV3(a *enginev1.PayloadAttributesV3) (Attributer, er
|
||||
parentBeaconBlockRoot: a.ParentBeaconBlockRoot,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// EventData holds the values for a PayloadAttributes event.
|
||||
type EventData struct {
|
||||
ProposerIndex primitives.ValidatorIndex
|
||||
ProposalSlot primitives.Slot
|
||||
ParentBlockNumber uint64
|
||||
ParentBlockRoot []byte
|
||||
ParentBlockHash []byte
|
||||
Attributer Attributer
|
||||
HeadState state.BeaconState
|
||||
HeadBlock interfaces.ReadOnlySignedBeaconBlock
|
||||
HeadRoot [field_params.RootLength]byte
|
||||
}
|
||||
|
||||
@@ -3,6 +3,7 @@ package bytesutil
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"unsafe"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
)
|
||||
@@ -145,3 +146,10 @@ func ReverseByteOrder(input []byte) []byte {
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
// UnsafeCastToString casts a byte slice to a string object without performing a copy. Changes
|
||||
// to byteSlice will also modify the contents of the string, so it is the caller's responsibility
|
||||
// to ensure that the byte slice will not modified after the string is created.
|
||||
func UnsafeCastToString(byteSlice []byte) string {
|
||||
return *(*string)(unsafe.Pointer(&byteSlice)) // #nosec G103
|
||||
}
|
||||
|
||||
@@ -217,6 +217,50 @@ func TestToBytes20(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestCastToString(t *testing.T) {
|
||||
bSlice := []byte{'a', 'b', 'c'}
|
||||
bString := bytesutil.UnsafeCastToString(bSlice)
|
||||
|
||||
originalString := "abc"
|
||||
|
||||
// Mutate original slice to make sure that a copy was not performed.
|
||||
bSlice[0] = 'd'
|
||||
assert.NotEqual(t, originalString, bString)
|
||||
assert.Equal(t, "dbc", bString)
|
||||
}
|
||||
|
||||
func BenchmarkUnsafeCastToString(b *testing.B) {
|
||||
data := []byte{0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF}
|
||||
empty := []byte{}
|
||||
var nilData []byte
|
||||
|
||||
b.Run("string(b)", func(b *testing.B) {
|
||||
for i := 0; i < b.N; i++ {
|
||||
_ = string(data)
|
||||
_ = string(empty)
|
||||
_ = string(nilData)
|
||||
}
|
||||
})
|
||||
|
||||
b.Run("bytesutil.UnsafeCastToString(b)", func(b *testing.B) {
|
||||
for i := 0; i < b.N; i++ {
|
||||
_ = bytesutil.UnsafeCastToString(data)
|
||||
_ = bytesutil.UnsafeCastToString(empty)
|
||||
_ = bytesutil.UnsafeCastToString(nilData)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func FuzzUnsafeCastToString(f *testing.F) {
|
||||
f.Fuzz(func(t *testing.T, input []byte) {
|
||||
want := string(input)
|
||||
result := bytesutil.UnsafeCastToString(input)
|
||||
if result != want {
|
||||
t.Fatalf("input (%v) result (%s) did not match expected (%s)", input, result, want)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func BenchmarkToBytes32(b *testing.B) {
|
||||
x := []byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31}
|
||||
for i := 0; i < b.N; i++ {
|
||||
|
||||
@@ -141,7 +141,7 @@ func WithdrawalSliceRoot(withdrawals []*enginev1.Withdrawal, limit uint64) ([32]
|
||||
return MixInLength(bytesRoot, bytesRootBufRoot), nil
|
||||
}
|
||||
|
||||
// DepositRequestsSliceRoot computes the HTR of a slice of deposit receipts.
|
||||
// DepositRequestsSliceRoot computes the HTR of a slice of deposit requests.
|
||||
// The limit parameter is used as input to the bitwise merkleization algorithm.
|
||||
func DepositRequestsSliceRoot(depositRequests []*enginev1.DepositRequest, limit uint64) ([32]byte, error) {
|
||||
return SliceRoot(depositRequests, limit)
|
||||
|
||||
@@ -81,11 +81,11 @@ ssz_altair_objs = [
|
||||
"BeaconBlockBodyAltair",
|
||||
"BeaconStateAltair",
|
||||
"ContributionAndProof",
|
||||
"LightClientBootstrapAltair",
|
||||
"LightClientFinalityUpdateAltair",
|
||||
"LightClientHeaderAltair",
|
||||
"LightClientOptimisticUpdateAltair",
|
||||
"LightClientBootstrapAltair",
|
||||
"LightClientUpdateAltair",
|
||||
"LightClientFinalityUpdateAltair",
|
||||
"LightClientOptimisticUpdateAltair",
|
||||
"SignedBeaconBlockAltair",
|
||||
"SignedContributionAndProof",
|
||||
"SyncAggregate",
|
||||
@@ -115,11 +115,11 @@ ssz_capella_objs = [
|
||||
"BlindedBeaconBlockCapella",
|
||||
"BuilderBidCapella",
|
||||
"HistoricalSummary",
|
||||
"LightClientBootstrapCapella",
|
||||
"LightClientFinalityUpdateCapella",
|
||||
"LightClientHeaderCapella",
|
||||
"LightClientOptimisticUpdateCapella",
|
||||
"LightClientBootstrapCapella",
|
||||
"LightClientUpdateCapella",
|
||||
"LightClientFinalityUpdateCapella",
|
||||
"LightClientOptimisticUpdateCapella",
|
||||
"SignedBLSToExecutionChange",
|
||||
"SignedBeaconBlockCapella",
|
||||
"SignedBlindedBeaconBlockCapella",
|
||||
@@ -137,11 +137,11 @@ ssz_deneb_objs = [
|
||||
"BlobSidecar",
|
||||
"BlobSidecars",
|
||||
"BuilderBidDeneb",
|
||||
"LightClientBootstrapDeneb",
|
||||
"LightClientFinalityUpdateDeneb",
|
||||
"LightClientHeaderDeneb",
|
||||
"LightClientOptimisticUpdateDeneb",
|
||||
"LightClientBootstrapDeneb",
|
||||
"LightClientUpdateDeneb",
|
||||
"LightClientFinalityUpdateDeneb",
|
||||
"LightClientOptimisticUpdateDeneb",
|
||||
"SignedBeaconBlockContentsDeneb",
|
||||
"SignedBeaconBlockDeneb",
|
||||
"SignedBlindedBeaconBlockDeneb",
|
||||
@@ -151,16 +151,18 @@ ssz_electra_objs = [
|
||||
"AggregateAttestationAndProofElectra",
|
||||
"AttestationElectra",
|
||||
"AttesterSlashingElectra",
|
||||
"BeaconBlockElectra",
|
||||
"BeaconBlockBodyElectra",
|
||||
"BeaconBlockContentsElectra",
|
||||
"BeaconBlockElectra",
|
||||
"BeaconStateElectra",
|
||||
"BlindedBeaconBlockBodyElectra",
|
||||
"BlindedBeaconBlockElectra",
|
||||
"Consolidation",
|
||||
"IndexedAttestationElectra",
|
||||
"LightClientHeaderElectra",
|
||||
"LightClientBootstrapElectra",
|
||||
"LightClientUpdateElectra",
|
||||
"LightClientFinalityUpdateElectra",
|
||||
"PendingDeposit",
|
||||
"PendingDeposits",
|
||||
"PendingConsolidation",
|
||||
|
||||
@@ -25,6 +25,7 @@ type Att interface {
|
||||
CommitteeBitsVal() bitfield.Bitfield
|
||||
GetSignature() []byte
|
||||
GetCommitteeIndex() (primitives.CommitteeIndex, error)
|
||||
IsNil() bool
|
||||
}
|
||||
|
||||
// IndexedAtt defines common functionality for all indexed attestation types.
|
||||
@@ -37,6 +38,7 @@ type IndexedAtt interface {
|
||||
GetAttestingIndices() []uint64
|
||||
GetData() *AttestationData
|
||||
GetSignature() []byte
|
||||
IsNil() bool
|
||||
}
|
||||
|
||||
// SignedAggregateAttAndProof defines common functionality for all signed aggregate attestation types.
|
||||
@@ -48,6 +50,7 @@ type SignedAggregateAttAndProof interface {
|
||||
Version() int
|
||||
AggregateAttestationAndProof() AggregateAttAndProof
|
||||
GetSignature() []byte
|
||||
IsNil() bool
|
||||
}
|
||||
|
||||
// AggregateAttAndProof defines common functionality for all aggregate attestation types.
|
||||
@@ -60,6 +63,7 @@ type AggregateAttAndProof interface {
|
||||
GetAggregatorIndex() primitives.ValidatorIndex
|
||||
AggregateVal() Att
|
||||
GetSelectionProof() []byte
|
||||
IsNil() bool
|
||||
}
|
||||
|
||||
// AttSlashing defines common functionality for all attestation slashing types.
|
||||
@@ -71,6 +75,7 @@ type AttSlashing interface {
|
||||
Version() int
|
||||
FirstAttestation() IndexedAtt
|
||||
SecondAttestation() IndexedAtt
|
||||
IsNil() bool
|
||||
}
|
||||
|
||||
// Copy --
|
||||
@@ -103,20 +108,25 @@ func (a *Attestation) Version() int {
|
||||
return version.Phase0
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *Attestation) IsNil() bool {
|
||||
return a == nil || a.Data == nil
|
||||
}
|
||||
|
||||
// Clone --
|
||||
func (a *Attestation) Clone() Att {
|
||||
return a.Copy()
|
||||
}
|
||||
|
||||
// Copy --
|
||||
func (att *Attestation) Copy() *Attestation {
|
||||
if att == nil {
|
||||
func (a *Attestation) Copy() *Attestation {
|
||||
if a == nil {
|
||||
return nil
|
||||
}
|
||||
return &Attestation{
|
||||
AggregationBits: bytesutil.SafeCopyBytes(att.AggregationBits),
|
||||
Data: att.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(att.Signature),
|
||||
AggregationBits: bytesutil.SafeCopyBytes(a.AggregationBits),
|
||||
Data: a.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(a.Signature),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -140,6 +150,11 @@ func (a *PendingAttestation) Version() int {
|
||||
return version.Phase0
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *PendingAttestation) IsNil() bool {
|
||||
return a == nil || a.Data == nil
|
||||
}
|
||||
|
||||
// Clone --
|
||||
func (a *PendingAttestation) Clone() Att {
|
||||
return a.Copy()
|
||||
@@ -181,21 +196,26 @@ func (a *AttestationElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *AttestationElectra) IsNil() bool {
|
||||
return a == nil || a.Data == nil
|
||||
}
|
||||
|
||||
// Clone --
|
||||
func (a *AttestationElectra) Clone() Att {
|
||||
return a.Copy()
|
||||
}
|
||||
|
||||
// Copy --
|
||||
func (att *AttestationElectra) Copy() *AttestationElectra {
|
||||
if att == nil {
|
||||
func (a *AttestationElectra) Copy() *AttestationElectra {
|
||||
if a == nil {
|
||||
return nil
|
||||
}
|
||||
return &AttestationElectra{
|
||||
AggregationBits: bytesutil.SafeCopyBytes(att.AggregationBits),
|
||||
CommitteeBits: bytesutil.SafeCopyBytes(att.CommitteeBits),
|
||||
Data: att.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(att.Signature),
|
||||
AggregationBits: bytesutil.SafeCopyBytes(a.AggregationBits),
|
||||
CommitteeBits: bytesutil.SafeCopyBytes(a.CommitteeBits),
|
||||
Data: a.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(a.Signature),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -227,40 +247,50 @@ func (a *IndexedAttestation) Version() int {
|
||||
return version.Phase0
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *IndexedAttestation) IsNil() bool {
|
||||
return a == nil || a.Data == nil
|
||||
}
|
||||
|
||||
// Version --
|
||||
func (a *IndexedAttestationElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *IndexedAttestationElectra) IsNil() bool {
|
||||
return a == nil || a.Data == nil
|
||||
}
|
||||
|
||||
// Copy --
|
||||
func (indexedAtt *IndexedAttestation) Copy() *IndexedAttestation {
|
||||
func (a *IndexedAttestation) Copy() *IndexedAttestation {
|
||||
var indices []uint64
|
||||
if indexedAtt == nil {
|
||||
if a == nil {
|
||||
return nil
|
||||
} else if indexedAtt.AttestingIndices != nil {
|
||||
indices = make([]uint64, len(indexedAtt.AttestingIndices))
|
||||
copy(indices, indexedAtt.AttestingIndices)
|
||||
} else if a.AttestingIndices != nil {
|
||||
indices = make([]uint64, len(a.AttestingIndices))
|
||||
copy(indices, a.AttestingIndices)
|
||||
}
|
||||
return &IndexedAttestation{
|
||||
AttestingIndices: indices,
|
||||
Data: indexedAtt.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(indexedAtt.Signature),
|
||||
Data: a.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(a.Signature),
|
||||
}
|
||||
}
|
||||
|
||||
// Copy --
|
||||
func (indexedAtt *IndexedAttestationElectra) Copy() *IndexedAttestationElectra {
|
||||
func (a *IndexedAttestationElectra) Copy() *IndexedAttestationElectra {
|
||||
var indices []uint64
|
||||
if indexedAtt == nil {
|
||||
if a == nil {
|
||||
return nil
|
||||
} else if indexedAtt.AttestingIndices != nil {
|
||||
indices = make([]uint64, len(indexedAtt.AttestingIndices))
|
||||
copy(indices, indexedAtt.AttestingIndices)
|
||||
} else if a.AttestingIndices != nil {
|
||||
indices = make([]uint64, len(a.AttestingIndices))
|
||||
copy(indices, a.AttestingIndices)
|
||||
}
|
||||
return &IndexedAttestationElectra{
|
||||
AttestingIndices: indices,
|
||||
Data: indexedAtt.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(indexedAtt.Signature),
|
||||
Data: a.Data.Copy(),
|
||||
Signature: bytesutil.SafeCopyBytes(a.Signature),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -269,6 +299,13 @@ func (a *AttesterSlashing) Version() int {
|
||||
return version.Phase0
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *AttesterSlashing) IsNil() bool {
|
||||
return a == nil ||
|
||||
a.Attestation_1 == nil || a.Attestation_1.IsNil() ||
|
||||
a.Attestation_2 == nil || a.Attestation_2.IsNil()
|
||||
}
|
||||
|
||||
// FirstAttestation --
|
||||
func (a *AttesterSlashing) FirstAttestation() IndexedAtt {
|
||||
return a.Attestation_1
|
||||
@@ -284,6 +321,13 @@ func (a *AttesterSlashingElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *AttesterSlashingElectra) IsNil() bool {
|
||||
return a == nil ||
|
||||
a.Attestation_1 == nil || a.Attestation_1.IsNil() ||
|
||||
a.Attestation_2 == nil || a.Attestation_2.IsNil()
|
||||
}
|
||||
|
||||
// FirstAttestation --
|
||||
func (a *AttesterSlashingElectra) FirstAttestation() IndexedAtt {
|
||||
return a.Attestation_1
|
||||
@@ -320,6 +364,11 @@ func (a *AggregateAttestationAndProof) Version() int {
|
||||
return version.Phase0
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *AggregateAttestationAndProof) IsNil() bool {
|
||||
return a == nil || a.Aggregate == nil || a.Aggregate.IsNil()
|
||||
}
|
||||
|
||||
// AggregateVal --
|
||||
func (a *AggregateAttestationAndProof) AggregateVal() Att {
|
||||
return a.Aggregate
|
||||
@@ -330,6 +379,11 @@ func (a *AggregateAttestationAndProofElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *AggregateAttestationAndProofElectra) IsNil() bool {
|
||||
return a == nil || a.Aggregate == nil || a.Aggregate.IsNil()
|
||||
}
|
||||
|
||||
// AggregateVal --
|
||||
func (a *AggregateAttestationAndProofElectra) AggregateVal() Att {
|
||||
return a.Aggregate
|
||||
@@ -340,6 +394,11 @@ func (a *SignedAggregateAttestationAndProof) Version() int {
|
||||
return version.Phase0
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *SignedAggregateAttestationAndProof) IsNil() bool {
|
||||
return a == nil || a.Message == nil || a.Message.IsNil()
|
||||
}
|
||||
|
||||
// AggregateAttestationAndProof --
|
||||
func (a *SignedAggregateAttestationAndProof) AggregateAttestationAndProof() AggregateAttAndProof {
|
||||
return a.Message
|
||||
@@ -350,6 +409,11 @@ func (a *SignedAggregateAttestationAndProofElectra) Version() int {
|
||||
return version.Electra
|
||||
}
|
||||
|
||||
// IsNil --
|
||||
func (a *SignedAggregateAttestationAndProofElectra) IsNil() bool {
|
||||
return a == nil || a.Message == nil || a.Message.IsNil()
|
||||
}
|
||||
|
||||
// AggregateAttestationAndProof --
|
||||
func (a *SignedAggregateAttestationAndProofElectra) AggregateAttestationAndProof() AggregateAttAndProof {
|
||||
return a.Message
|
||||
|
||||
@@ -39,7 +39,7 @@ import (
|
||||
// data=attestation.data,
|
||||
// signature=attestation.signature,
|
||||
// )
|
||||
func ConvertToIndexed(ctx context.Context, attestation ethpb.Att, committees ...[]primitives.ValidatorIndex) (ethpb.IndexedAtt, error) {
|
||||
func ConvertToIndexed(_ context.Context, attestation ethpb.Att, committees ...[]primitives.ValidatorIndex) (ethpb.IndexedAtt, error) {
|
||||
attIndices, err := AttestingIndices(attestation, committees...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -185,12 +185,10 @@ func IsValidAttestationIndices(ctx context.Context, indexedAttestation ethpb.Ind
|
||||
_, span := trace.StartSpan(ctx, "attestationutil.IsValidAttestationIndices")
|
||||
defer span.End()
|
||||
|
||||
if indexedAttestation == nil ||
|
||||
indexedAttestation.GetData() == nil ||
|
||||
indexedAttestation.GetData().Target == nil ||
|
||||
indexedAttestation.GetAttestingIndices() == nil {
|
||||
if indexedAttestation == nil || indexedAttestation.IsNil() || indexedAttestation.GetData().Target == nil || indexedAttestation.GetData().Source == nil {
|
||||
return errors.New("nil or missing indexed attestation data")
|
||||
}
|
||||
|
||||
indices := indexedAttestation.GetAttestingIndices()
|
||||
if len(indices) == 0 {
|
||||
return errors.New("expected non-empty attesting indices")
|
||||
|
||||
@@ -106,10 +106,11 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
att: ð.IndexedAttestation{
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
wantedErr: "nil or missing indexed attestation data",
|
||||
wantedErr: "expected non-empty attesting indices",
|
||||
},
|
||||
{
|
||||
name: "Indices should be non-empty",
|
||||
@@ -117,6 +118,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: []uint64{},
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -128,6 +130,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: make([]uint64, params.BeaconConfig().MaxValidatorsPerCommittee+1),
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -139,6 +142,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: []uint64{3, 2, 1},
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -150,6 +154,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: []uint64{1, 2, 3},
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -160,6 +165,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: []uint64{1, 2},
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -170,6 +176,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: []uint64{1},
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -180,6 +187,7 @@ func TestIsValidAttestationIndices(t *testing.T) {
|
||||
AttestingIndices: make([]uint64, params.BeaconConfig().MaxValidatorsPerCommittee*params.BeaconConfig().MaxCommitteesPerSlot+1),
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
@@ -218,6 +226,7 @@ func BenchmarkIsValidAttestationIndices(b *testing.B) {
|
||||
AttestingIndices: indices,
|
||||
Data: ð.AttestationData{
|
||||
Target: ð.Checkpoint{},
|
||||
Source: ð.Checkpoint{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
}
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
// Code generated by fastssz. DO NOT EDIT.
|
||||
// Hash: 5ca1c2c4e61b47b1f8185b3e9c477ae280f82e1483b88d4e11fa214452da5117
|
||||
// Hash: bce36d386a65c91018c9a1edaacd2ed0f09cc4dce59fdc5f014fbd9e05bfee77
|
||||
package eth
|
||||
|
||||
import (
|
||||
@@ -4410,7 +4410,7 @@ func (l *LightClientUpdateElectra) MarshalSSZ() ([]byte, error) {
|
||||
// MarshalSSZTo ssz marshals the LightClientUpdateElectra object to a target array
|
||||
func (l *LightClientUpdateElectra) MarshalSSZTo(buf []byte) (dst []byte, err error) {
|
||||
dst = buf
|
||||
offset := int(25184)
|
||||
offset := int(25216)
|
||||
|
||||
// Offset (0) 'AttestedHeader'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
@@ -4448,11 +4448,11 @@ func (l *LightClientUpdateElectra) MarshalSSZTo(buf []byte) (dst []byte, err err
|
||||
offset += l.FinalizedHeader.SizeSSZ()
|
||||
|
||||
// Field (4) 'FinalityBranch'
|
||||
if size := len(l.FinalityBranch); size != 6 {
|
||||
err = ssz.ErrVectorLengthFn("--.FinalityBranch", size, 6)
|
||||
if size := len(l.FinalityBranch); size != 7 {
|
||||
err = ssz.ErrVectorLengthFn("--.FinalityBranch", size, 7)
|
||||
return
|
||||
}
|
||||
for ii := 0; ii < 6; ii++ {
|
||||
for ii := 0; ii < 7; ii++ {
|
||||
if size := len(l.FinalityBranch[ii]); size != 32 {
|
||||
err = ssz.ErrBytesLengthFn("--.FinalityBranch[ii]", size, 32)
|
||||
return
|
||||
@@ -4488,7 +4488,7 @@ func (l *LightClientUpdateElectra) MarshalSSZTo(buf []byte) (dst []byte, err err
|
||||
func (l *LightClientUpdateElectra) UnmarshalSSZ(buf []byte) error {
|
||||
var err error
|
||||
size := uint64(len(buf))
|
||||
if size < 25184 {
|
||||
if size < 25216 {
|
||||
return ssz.ErrSize
|
||||
}
|
||||
|
||||
@@ -4500,7 +4500,7 @@ func (l *LightClientUpdateElectra) UnmarshalSSZ(buf []byte) error {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
if o0 != 25184 {
|
||||
if o0 != 25216 {
|
||||
return ssz.ErrInvalidVariableOffset
|
||||
}
|
||||
|
||||
@@ -4527,24 +4527,24 @@ func (l *LightClientUpdateElectra) UnmarshalSSZ(buf []byte) error {
|
||||
}
|
||||
|
||||
// Field (4) 'FinalityBranch'
|
||||
l.FinalityBranch = make([][]byte, 6)
|
||||
for ii := 0; ii < 6; ii++ {
|
||||
l.FinalityBranch = make([][]byte, 7)
|
||||
for ii := 0; ii < 7; ii++ {
|
||||
if cap(l.FinalityBranch[ii]) == 0 {
|
||||
l.FinalityBranch[ii] = make([]byte, 0, len(buf[24824:25016][ii*32:(ii+1)*32]))
|
||||
l.FinalityBranch[ii] = make([]byte, 0, len(buf[24824:25048][ii*32:(ii+1)*32]))
|
||||
}
|
||||
l.FinalityBranch[ii] = append(l.FinalityBranch[ii], buf[24824:25016][ii*32:(ii+1)*32]...)
|
||||
l.FinalityBranch[ii] = append(l.FinalityBranch[ii], buf[24824:25048][ii*32:(ii+1)*32]...)
|
||||
}
|
||||
|
||||
// Field (5) 'SyncAggregate'
|
||||
if l.SyncAggregate == nil {
|
||||
l.SyncAggregate = new(SyncAggregate)
|
||||
}
|
||||
if err = l.SyncAggregate.UnmarshalSSZ(buf[25016:25176]); err != nil {
|
||||
if err = l.SyncAggregate.UnmarshalSSZ(buf[25048:25208]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Field (6) 'SignatureSlot'
|
||||
l.SignatureSlot = github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[25176:25184]))
|
||||
l.SignatureSlot = github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[25208:25216]))
|
||||
|
||||
// Field (0) 'AttestedHeader'
|
||||
{
|
||||
@@ -4572,7 +4572,7 @@ func (l *LightClientUpdateElectra) UnmarshalSSZ(buf []byte) error {
|
||||
|
||||
// SizeSSZ returns the ssz encoded size in bytes for the LightClientUpdateElectra object
|
||||
func (l *LightClientUpdateElectra) SizeSSZ() (size int) {
|
||||
size = 25184
|
||||
size = 25216
|
||||
|
||||
// Field (0) 'AttestedHeader'
|
||||
if l.AttestedHeader == nil {
|
||||
@@ -4632,8 +4632,8 @@ func (l *LightClientUpdateElectra) HashTreeRootWith(hh *ssz.Hasher) (err error)
|
||||
|
||||
// Field (4) 'FinalityBranch'
|
||||
{
|
||||
if size := len(l.FinalityBranch); size != 6 {
|
||||
err = ssz.ErrVectorLengthFn("--.FinalityBranch", size, 6)
|
||||
if size := len(l.FinalityBranch); size != 7 {
|
||||
err = ssz.ErrVectorLengthFn("--.FinalityBranch", size, 7)
|
||||
return
|
||||
}
|
||||
subIndx := hh.Index()
|
||||
@@ -4658,3 +4658,245 @@ func (l *LightClientUpdateElectra) HashTreeRootWith(hh *ssz.Hasher) (err error)
|
||||
hh.Merkleize(indx)
|
||||
return
|
||||
}
|
||||
|
||||
// MarshalSSZ ssz marshals the LightClientFinalityUpdateElectra object
|
||||
func (l *LightClientFinalityUpdateElectra) MarshalSSZ() ([]byte, error) {
|
||||
return ssz.MarshalSSZ(l)
|
||||
}
|
||||
|
||||
// MarshalSSZTo ssz marshals the LightClientFinalityUpdateElectra object to a target array
|
||||
func (l *LightClientFinalityUpdateElectra) MarshalSSZTo(buf []byte) (dst []byte, err error) {
|
||||
dst = buf
|
||||
offset := int(180)
|
||||
|
||||
// Offset (0) 'AttestedHeader'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
if l.AttestedHeader == nil {
|
||||
l.AttestedHeader = new(LightClientHeaderDeneb)
|
||||
}
|
||||
offset += l.AttestedHeader.SizeSSZ()
|
||||
|
||||
// Offset (1) 'FinalizedHeader'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
if l.FinalizedHeader == nil {
|
||||
l.FinalizedHeader = new(LightClientHeaderDeneb)
|
||||
}
|
||||
offset += l.FinalizedHeader.SizeSSZ()
|
||||
|
||||
// Offset (2) 'FinalityBranch'
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
for ii := 0; ii < len(l.FinalityBranch); ii++ {
|
||||
offset += 4
|
||||
offset += len(l.FinalityBranch[ii])
|
||||
}
|
||||
|
||||
// Field (3) 'SyncAggregate'
|
||||
if l.SyncAggregate == nil {
|
||||
l.SyncAggregate = new(SyncAggregate)
|
||||
}
|
||||
if dst, err = l.SyncAggregate.MarshalSSZTo(dst); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (4) 'SignatureSlot'
|
||||
dst = ssz.MarshalUint64(dst, uint64(l.SignatureSlot))
|
||||
|
||||
// Field (0) 'AttestedHeader'
|
||||
if dst, err = l.AttestedHeader.MarshalSSZTo(dst); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (1) 'FinalizedHeader'
|
||||
if dst, err = l.FinalizedHeader.MarshalSSZTo(dst); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (2) 'FinalityBranch'
|
||||
if size := len(l.FinalityBranch); size > 7 {
|
||||
err = ssz.ErrListTooBigFn("--.FinalityBranch", size, 7)
|
||||
return
|
||||
}
|
||||
{
|
||||
offset = 4 * len(l.FinalityBranch)
|
||||
for ii := 0; ii < len(l.FinalityBranch); ii++ {
|
||||
dst = ssz.WriteOffset(dst, offset)
|
||||
offset += len(l.FinalityBranch[ii])
|
||||
}
|
||||
}
|
||||
for ii := 0; ii < len(l.FinalityBranch); ii++ {
|
||||
if size := len(l.FinalityBranch[ii]); size > 32 {
|
||||
err = ssz.ErrBytesLengthFn("--.FinalityBranch[ii]", size, 32)
|
||||
return
|
||||
}
|
||||
dst = append(dst, l.FinalityBranch[ii]...)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// UnmarshalSSZ ssz unmarshals the LightClientFinalityUpdateElectra object
|
||||
func (l *LightClientFinalityUpdateElectra) UnmarshalSSZ(buf []byte) error {
|
||||
var err error
|
||||
size := uint64(len(buf))
|
||||
if size < 180 {
|
||||
return ssz.ErrSize
|
||||
}
|
||||
|
||||
tail := buf
|
||||
var o0, o1, o2 uint64
|
||||
|
||||
// Offset (0) 'AttestedHeader'
|
||||
if o0 = ssz.ReadOffset(buf[0:4]); o0 > size {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
if o0 != 180 {
|
||||
return ssz.ErrInvalidVariableOffset
|
||||
}
|
||||
|
||||
// Offset (1) 'FinalizedHeader'
|
||||
if o1 = ssz.ReadOffset(buf[4:8]); o1 > size || o0 > o1 {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
// Offset (2) 'FinalityBranch'
|
||||
if o2 = ssz.ReadOffset(buf[8:12]); o2 > size || o1 > o2 {
|
||||
return ssz.ErrOffset
|
||||
}
|
||||
|
||||
// Field (3) 'SyncAggregate'
|
||||
if l.SyncAggregate == nil {
|
||||
l.SyncAggregate = new(SyncAggregate)
|
||||
}
|
||||
if err = l.SyncAggregate.UnmarshalSSZ(buf[12:172]); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Field (4) 'SignatureSlot'
|
||||
l.SignatureSlot = github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(ssz.UnmarshallUint64(buf[172:180]))
|
||||
|
||||
// Field (0) 'AttestedHeader'
|
||||
{
|
||||
buf = tail[o0:o1]
|
||||
if l.AttestedHeader == nil {
|
||||
l.AttestedHeader = new(LightClientHeaderDeneb)
|
||||
}
|
||||
if err = l.AttestedHeader.UnmarshalSSZ(buf); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Field (1) 'FinalizedHeader'
|
||||
{
|
||||
buf = tail[o1:o2]
|
||||
if l.FinalizedHeader == nil {
|
||||
l.FinalizedHeader = new(LightClientHeaderDeneb)
|
||||
}
|
||||
if err = l.FinalizedHeader.UnmarshalSSZ(buf); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Field (2) 'FinalityBranch'
|
||||
{
|
||||
buf = tail[o2:]
|
||||
num, err := ssz.DecodeDynamicLength(buf, 7)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
l.FinalityBranch = make([][]byte, num)
|
||||
err = ssz.UnmarshalDynamic(buf, num, func(indx int, buf []byte) (err error) {
|
||||
if len(buf) > 32 {
|
||||
return ssz.ErrBytesLength
|
||||
}
|
||||
if cap(l.FinalityBranch[indx]) == 0 {
|
||||
l.FinalityBranch[indx] = make([]byte, 0, len(buf))
|
||||
}
|
||||
l.FinalityBranch[indx] = append(l.FinalityBranch[indx], buf...)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// SizeSSZ returns the ssz encoded size in bytes for the LightClientFinalityUpdateElectra object
|
||||
func (l *LightClientFinalityUpdateElectra) SizeSSZ() (size int) {
|
||||
size = 180
|
||||
|
||||
// Field (0) 'AttestedHeader'
|
||||
if l.AttestedHeader == nil {
|
||||
l.AttestedHeader = new(LightClientHeaderDeneb)
|
||||
}
|
||||
size += l.AttestedHeader.SizeSSZ()
|
||||
|
||||
// Field (1) 'FinalizedHeader'
|
||||
if l.FinalizedHeader == nil {
|
||||
l.FinalizedHeader = new(LightClientHeaderDeneb)
|
||||
}
|
||||
size += l.FinalizedHeader.SizeSSZ()
|
||||
|
||||
// Field (2) 'FinalityBranch'
|
||||
for ii := 0; ii < len(l.FinalityBranch); ii++ {
|
||||
size += 4
|
||||
size += len(l.FinalityBranch[ii])
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// HashTreeRoot ssz hashes the LightClientFinalityUpdateElectra object
|
||||
func (l *LightClientFinalityUpdateElectra) HashTreeRoot() ([32]byte, error) {
|
||||
return ssz.HashWithDefaultHasher(l)
|
||||
}
|
||||
|
||||
// HashTreeRootWith ssz hashes the LightClientFinalityUpdateElectra object with a hasher
|
||||
func (l *LightClientFinalityUpdateElectra) HashTreeRootWith(hh *ssz.Hasher) (err error) {
|
||||
indx := hh.Index()
|
||||
|
||||
// Field (0) 'AttestedHeader'
|
||||
if err = l.AttestedHeader.HashTreeRootWith(hh); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (1) 'FinalizedHeader'
|
||||
if err = l.FinalizedHeader.HashTreeRootWith(hh); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (2) 'FinalityBranch'
|
||||
{
|
||||
subIndx := hh.Index()
|
||||
num := uint64(len(l.FinalityBranch))
|
||||
if num > 7 {
|
||||
err = ssz.ErrIncorrectListSize
|
||||
return
|
||||
}
|
||||
for _, elem := range l.FinalityBranch {
|
||||
{
|
||||
elemIndx := hh.Index()
|
||||
byteLen := uint64(len(elem))
|
||||
if byteLen > 32 {
|
||||
err = ssz.ErrIncorrectListSize
|
||||
return
|
||||
}
|
||||
hh.AppendBytes32(elem)
|
||||
hh.MerkleizeWithMixin(elemIndx, byteLen, (32+31)/32)
|
||||
}
|
||||
}
|
||||
hh.MerkleizeWithMixin(subIndx, num, 7)
|
||||
}
|
||||
|
||||
// Field (3) 'SyncAggregate'
|
||||
if err = l.SyncAggregate.HashTreeRootWith(hh); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// Field (4) 'SignatureSlot'
|
||||
hh.PutUint64(uint64(l.SignatureSlot))
|
||||
|
||||
hh.Merkleize(indx)
|
||||
return
|
||||
}
|
||||
|
||||
365
proto/prysm/v1alpha1/light_client.pb.go
generated
365
proto/prysm/v1alpha1/light_client.pb.go
generated
@@ -743,7 +743,7 @@ type LightClientUpdateElectra struct {
|
||||
NextSyncCommittee *SyncCommittee `protobuf:"bytes,2,opt,name=next_sync_committee,json=nextSyncCommittee,proto3" json:"next_sync_committee,omitempty"`
|
||||
NextSyncCommitteeBranch [][]byte `protobuf:"bytes,3,rep,name=next_sync_committee_branch,json=nextSyncCommitteeBranch,proto3" json:"next_sync_committee_branch,omitempty" ssz-size:"6,32"`
|
||||
FinalizedHeader *LightClientHeaderDeneb `protobuf:"bytes,4,opt,name=finalized_header,json=finalizedHeader,proto3" json:"finalized_header,omitempty"`
|
||||
FinalityBranch [][]byte `protobuf:"bytes,5,rep,name=finality_branch,json=finalityBranch,proto3" json:"finality_branch,omitempty" ssz-size:"6,32"`
|
||||
FinalityBranch [][]byte `protobuf:"bytes,5,rep,name=finality_branch,json=finalityBranch,proto3" json:"finality_branch,omitempty" ssz-size:"7,32"`
|
||||
SyncAggregate *SyncAggregate `protobuf:"bytes,6,opt,name=sync_aggregate,json=syncAggregate,proto3" json:"sync_aggregate,omitempty"`
|
||||
SignatureSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,7,opt,name=signature_slot,json=signatureSlot,proto3" json:"signature_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"`
|
||||
}
|
||||
@@ -1066,6 +1066,85 @@ func (x *LightClientFinalityUpdateDeneb) GetSignatureSlot() github_com_prysmatic
|
||||
return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0)
|
||||
}
|
||||
|
||||
type LightClientFinalityUpdateElectra struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
AttestedHeader *LightClientHeaderDeneb `protobuf:"bytes,1,opt,name=attested_header,json=attestedHeader,proto3" json:"attested_header,omitempty"`
|
||||
FinalizedHeader *LightClientHeaderDeneb `protobuf:"bytes,2,opt,name=finalized_header,json=finalizedHeader,proto3" json:"finalized_header,omitempty"`
|
||||
FinalityBranch [][]byte `protobuf:"bytes,3,rep,name=finality_branch,json=finalityBranch,proto3" json:"finality_branch,omitempty" ssz-max:"7,32"`
|
||||
SyncAggregate *SyncAggregate `protobuf:"bytes,4,opt,name=sync_aggregate,json=syncAggregate,proto3" json:"sync_aggregate,omitempty"`
|
||||
SignatureSlot github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot `protobuf:"varint,5,opt,name=signature_slot,json=signatureSlot,proto3" json:"signature_slot,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"`
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) Reset() {
|
||||
*x = LightClientFinalityUpdateElectra{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[14]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*LightClientFinalityUpdateElectra) ProtoMessage() {}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[14]
|
||||
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 LightClientFinalityUpdateElectra.ProtoReflect.Descriptor instead.
|
||||
func (*LightClientFinalityUpdateElectra) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{14}
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) GetAttestedHeader() *LightClientHeaderDeneb {
|
||||
if x != nil {
|
||||
return x.AttestedHeader
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) GetFinalizedHeader() *LightClientHeaderDeneb {
|
||||
if x != nil {
|
||||
return x.FinalizedHeader
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) GetFinalityBranch() [][]byte {
|
||||
if x != nil {
|
||||
return x.FinalityBranch
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) GetSyncAggregate() *SyncAggregate {
|
||||
if x != nil {
|
||||
return x.SyncAggregate
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *LightClientFinalityUpdateElectra) GetSignatureSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot {
|
||||
if x != nil {
|
||||
return x.SignatureSlot
|
||||
}
|
||||
return github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot(0)
|
||||
}
|
||||
|
||||
type LightClientOptimisticUpdateAltair struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@@ -1079,7 +1158,7 @@ type LightClientOptimisticUpdateAltair struct {
|
||||
func (x *LightClientOptimisticUpdateAltair) Reset() {
|
||||
*x = LightClientOptimisticUpdateAltair{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[14]
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[15]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1092,7 +1171,7 @@ func (x *LightClientOptimisticUpdateAltair) String() string {
|
||||
func (*LightClientOptimisticUpdateAltair) ProtoMessage() {}
|
||||
|
||||
func (x *LightClientOptimisticUpdateAltair) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[14]
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[15]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1105,7 +1184,7 @@ func (x *LightClientOptimisticUpdateAltair) ProtoReflect() protoreflect.Message
|
||||
|
||||
// Deprecated: Use LightClientOptimisticUpdateAltair.ProtoReflect.Descriptor instead.
|
||||
func (*LightClientOptimisticUpdateAltair) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{14}
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{15}
|
||||
}
|
||||
|
||||
func (x *LightClientOptimisticUpdateAltair) GetAttestedHeader() *LightClientHeaderAltair {
|
||||
@@ -1142,7 +1221,7 @@ type LightClientOptimisticUpdateCapella struct {
|
||||
func (x *LightClientOptimisticUpdateCapella) Reset() {
|
||||
*x = LightClientOptimisticUpdateCapella{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[15]
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[16]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1155,7 +1234,7 @@ func (x *LightClientOptimisticUpdateCapella) String() string {
|
||||
func (*LightClientOptimisticUpdateCapella) ProtoMessage() {}
|
||||
|
||||
func (x *LightClientOptimisticUpdateCapella) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[15]
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[16]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1168,7 +1247,7 @@ func (x *LightClientOptimisticUpdateCapella) ProtoReflect() protoreflect.Message
|
||||
|
||||
// Deprecated: Use LightClientOptimisticUpdateCapella.ProtoReflect.Descriptor instead.
|
||||
func (*LightClientOptimisticUpdateCapella) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{15}
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{16}
|
||||
}
|
||||
|
||||
func (x *LightClientOptimisticUpdateCapella) GetAttestedHeader() *LightClientHeaderCapella {
|
||||
@@ -1205,7 +1284,7 @@ type LightClientOptimisticUpdateDeneb struct {
|
||||
func (x *LightClientOptimisticUpdateDeneb) Reset() {
|
||||
*x = LightClientOptimisticUpdateDeneb{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[16]
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[17]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1218,7 +1297,7 @@ func (x *LightClientOptimisticUpdateDeneb) String() string {
|
||||
func (*LightClientOptimisticUpdateDeneb) ProtoMessage() {}
|
||||
|
||||
func (x *LightClientOptimisticUpdateDeneb) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[16]
|
||||
mi := &file_proto_prysm_v1alpha1_light_client_proto_msgTypes[17]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1231,7 +1310,7 @@ func (x *LightClientOptimisticUpdateDeneb) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use LightClientOptimisticUpdateDeneb.ProtoReflect.Descriptor instead.
|
||||
func (*LightClientOptimisticUpdateDeneb) Descriptor() ([]byte, []int) {
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{16}
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP(), []int{17}
|
||||
}
|
||||
|
||||
func (x *LightClientOptimisticUpdateDeneb) GetAttestedHeader() *LightClientHeaderDeneb {
|
||||
@@ -1511,7 +1590,7 @@ var file_proto_prysm_v1alpha1_light_client_proto_rawDesc = []byte{
|
||||
0x72, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x52, 0x0f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65,
|
||||
0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x31, 0x0a, 0x0f, 0x66, 0x69, 0x6e, 0x61, 0x6c,
|
||||
0x69, 0x74, 0x79, 0x5f, 0x62, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0c,
|
||||
0x42, 0x08, 0x8a, 0xb5, 0x18, 0x04, 0x36, 0x2c, 0x33, 0x32, 0x52, 0x0e, 0x66, 0x69, 0x6e, 0x61,
|
||||
0x42, 0x08, 0x8a, 0xb5, 0x18, 0x04, 0x37, 0x2c, 0x33, 0x32, 0x52, 0x0e, 0x66, 0x69, 0x6e, 0x61,
|
||||
0x6c, 0x69, 0x74, 0x79, 0x42, 0x72, 0x61, 0x6e, 0x63, 0x68, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79,
|
||||
0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01,
|
||||
0x28, 0x0b, 0x32, 0x24, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74,
|
||||
@@ -1609,76 +1688,104 @@ var file_proto_prysm_v1alpha1_light_client_proto_rawDesc = []byte{
|
||||
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, 0x0d, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74,
|
||||
0x75, 0x72, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xb7, 0x02, 0x0a, 0x21, 0x4c, 0x69, 0x67, 0x68,
|
||||
0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x73, 0x74, 0x69,
|
||||
0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x12, 0x57, 0x0a,
|
||||
0x0f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
|
||||
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c,
|
||||
0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||
0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64,
|
||||
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61,
|
||||
0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24,
|
||||
0x75, 0x72, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xc2, 0x03, 0x0a, 0x20, 0x4c, 0x69, 0x67, 0x68,
|
||||
0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x55,
|
||||
0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x56, 0x0a, 0x0f,
|
||||
0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
|
||||
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69,
|
||||
0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x44,
|
||||
0x65, 0x6e, 0x65, 0x62, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x48, 0x65,
|
||||
0x61, 0x64, 0x65, 0x72, 0x12, 0x58, 0x0a, 0x10, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65,
|
||||
0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d,
|
||||
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
|
||||
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 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, 0x6c, 0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
|
||||
0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x0d, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x53, 0x6c, 0x6f,
|
||||
0x74, 0x22, 0xb9, 0x02, 0x0a, 0x22, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e,
|
||||
0x74, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x73, 0x74, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74,
|
||||
0x65, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x12, 0x58, 0x0a, 0x0f, 0x61, 0x74, 0x74, 0x65,
|
||||
0x73, 0x74, 0x65, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x2f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68,
|
||||
0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43,
|
||||
0x6c, 0x69, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x61, 0x70, 0x65, 0x6c,
|
||||
0x6c, 0x61, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x48, 0x65, 0x61, 0x64,
|
||||
0x65, 0x72, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65,
|
||||
0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 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,
|
||||
0x6c, 0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x73, 0x6c, 0x6f,
|
||||
0x74, 0x18, 0x03, 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, 0x0d,
|
||||
0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xb5, 0x02,
|
||||
0x0a, 0x20, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x70, 0x74,
|
||||
0x69, 0x6d, 0x69, 0x73, 0x74, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x6e,
|
||||
0x65, 0x62, 0x12, 0x56, 0x0a, 0x0f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x68,
|
||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x65, 0x74,
|
||||
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65,
|
||||
0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x52, 0x0f, 0x66,
|
||||
0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x31,
|
||||
0x0a, 0x0f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x62, 0x72, 0x61, 0x6e, 0x63,
|
||||
0x68, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x08, 0x92, 0xb5, 0x18, 0x04, 0x37, 0x2c, 0x33,
|
||||
0x32, 0x52, 0x0e, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x72, 0x61, 0x6e, 0x63,
|
||||
0x68, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67,
|
||||
0x61, 0x74, 0x65, 0x18, 0x04, 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, 0x6c,
|
||||
0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x5f, 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, 0x0d, 0x73,
|
||||
0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xb7, 0x02, 0x0a,
|
||||
0x21, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x70, 0x74, 0x69,
|
||||
0x6d, 0x69, 0x73, 0x74, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x74, 0x61,
|
||||
0x69, 0x72, 0x12, 0x57, 0x0a, 0x0f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x68,
|
||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x65, 0x74,
|
||||
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70,
|
||||
0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x48,
|
||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x65,
|
||||
0x73, 0x74, 0x65, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79,
|
||||
0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 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, 0x6c, 0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e, 0x61,
|
||||
0x74, 0x75, 0x72, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x0d, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
|
||||
0x65, 0x53, 0x6c, 0x6f, 0x74, 0x42, 0x99, 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, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
|
||||
0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x38, 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, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74,
|
||||
0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68,
|
||||
0x2e, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65,
|
||||
0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61,
|
||||
0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x6c, 0x74, 0x61, 0x69, 0x72, 0x52, 0x0e, 0x61, 0x74, 0x74,
|
||||
0x65, 0x73, 0x74, 0x65, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4b, 0x0a, 0x0e, 0x73,
|
||||
0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 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, 0x6c, 0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e,
|
||||
0x61, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x0d, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75,
|
||||
0x72, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x22, 0xb9, 0x02, 0x0a, 0x22, 0x4c, 0x69, 0x67, 0x68, 0x74,
|
||||
0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x73, 0x74, 0x69, 0x63,
|
||||
0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x12, 0x58, 0x0a,
|
||||
0x0f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
|
||||
0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c,
|
||||
0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||
0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x52, 0x0e, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65,
|
||||
0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f,
|
||||
0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 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, 0x6c, 0x0a, 0x0e, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72,
|
||||
0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x0d, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x53, 0x6c,
|
||||
0x6f, 0x74, 0x22, 0xb5, 0x02, 0x0a, 0x20, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c, 0x69, 0x65,
|
||||
0x6e, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6d, 0x69, 0x73, 0x74, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61,
|
||||
0x74, 0x65, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x56, 0x0a, 0x0f, 0x61, 0x74, 0x74, 0x65, 0x73,
|
||||
0x74, 0x65, 0x64, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x2d, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e,
|
||||
0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6c,
|
||||
0x69, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x52,
|
||||
0x0e, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12,
|
||||
0x4b, 0x0a, 0x0e, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74,
|
||||
0x65, 0x18, 0x02, 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, 0x6c, 0x0a, 0x0e,
|
||||
0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03,
|
||||
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, 0x0d, 0x73, 0x69, 0x67,
|
||||
0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x42, 0x99, 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, 0x4c, 0x69, 0x67, 0x68, 0x74, 0x43,
|
||||
0x6c, 0x69, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x38, 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, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
|
||||
0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
|
||||
0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02,
|
||||
0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31,
|
||||
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
@@ -1693,7 +1800,7 @@ func file_proto_prysm_v1alpha1_light_client_proto_rawDescGZIP() []byte {
|
||||
return file_proto_prysm_v1alpha1_light_client_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_proto_prysm_v1alpha1_light_client_proto_msgTypes = make([]protoimpl.MessageInfo, 17)
|
||||
var file_proto_prysm_v1alpha1_light_client_proto_msgTypes = make([]protoimpl.MessageInfo, 18)
|
||||
var file_proto_prysm_v1alpha1_light_client_proto_goTypes = []interface{}{
|
||||
(*LightClientHeaderAltair)(nil), // 0: ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
(*LightClientHeaderCapella)(nil), // 1: ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
@@ -1709,65 +1816,69 @@ var file_proto_prysm_v1alpha1_light_client_proto_goTypes = []interface{}{
|
||||
(*LightClientFinalityUpdateAltair)(nil), // 11: ethereum.eth.v1alpha1.LightClientFinalityUpdateAltair
|
||||
(*LightClientFinalityUpdateCapella)(nil), // 12: ethereum.eth.v1alpha1.LightClientFinalityUpdateCapella
|
||||
(*LightClientFinalityUpdateDeneb)(nil), // 13: ethereum.eth.v1alpha1.LightClientFinalityUpdateDeneb
|
||||
(*LightClientOptimisticUpdateAltair)(nil), // 14: ethereum.eth.v1alpha1.LightClientOptimisticUpdateAltair
|
||||
(*LightClientOptimisticUpdateCapella)(nil), // 15: ethereum.eth.v1alpha1.LightClientOptimisticUpdateCapella
|
||||
(*LightClientOptimisticUpdateDeneb)(nil), // 16: ethereum.eth.v1alpha1.LightClientOptimisticUpdateDeneb
|
||||
(*BeaconBlockHeader)(nil), // 17: ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
(*v1.ExecutionPayloadHeaderCapella)(nil), // 18: ethereum.engine.v1.ExecutionPayloadHeaderCapella
|
||||
(*v1.ExecutionPayloadHeaderDeneb)(nil), // 19: ethereum.engine.v1.ExecutionPayloadHeaderDeneb
|
||||
(*SyncCommittee)(nil), // 20: ethereum.eth.v1alpha1.SyncCommittee
|
||||
(*SyncAggregate)(nil), // 21: ethereum.eth.v1alpha1.SyncAggregate
|
||||
(*LightClientFinalityUpdateElectra)(nil), // 14: ethereum.eth.v1alpha1.LightClientFinalityUpdateElectra
|
||||
(*LightClientOptimisticUpdateAltair)(nil), // 15: ethereum.eth.v1alpha1.LightClientOptimisticUpdateAltair
|
||||
(*LightClientOptimisticUpdateCapella)(nil), // 16: ethereum.eth.v1alpha1.LightClientOptimisticUpdateCapella
|
||||
(*LightClientOptimisticUpdateDeneb)(nil), // 17: ethereum.eth.v1alpha1.LightClientOptimisticUpdateDeneb
|
||||
(*BeaconBlockHeader)(nil), // 18: ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
(*v1.ExecutionPayloadHeaderCapella)(nil), // 19: ethereum.engine.v1.ExecutionPayloadHeaderCapella
|
||||
(*v1.ExecutionPayloadHeaderDeneb)(nil), // 20: ethereum.engine.v1.ExecutionPayloadHeaderDeneb
|
||||
(*SyncCommittee)(nil), // 21: ethereum.eth.v1alpha1.SyncCommittee
|
||||
(*SyncAggregate)(nil), // 22: ethereum.eth.v1alpha1.SyncAggregate
|
||||
}
|
||||
var file_proto_prysm_v1alpha1_light_client_proto_depIdxs = []int32{
|
||||
17, // 0: ethereum.eth.v1alpha1.LightClientHeaderAltair.beacon:type_name -> ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
17, // 1: ethereum.eth.v1alpha1.LightClientHeaderCapella.beacon:type_name -> ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
18, // 2: ethereum.eth.v1alpha1.LightClientHeaderCapella.execution:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderCapella
|
||||
17, // 3: ethereum.eth.v1alpha1.LightClientHeaderDeneb.beacon:type_name -> ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
19, // 4: ethereum.eth.v1alpha1.LightClientHeaderDeneb.execution:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderDeneb
|
||||
18, // 0: ethereum.eth.v1alpha1.LightClientHeaderAltair.beacon:type_name -> ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
18, // 1: ethereum.eth.v1alpha1.LightClientHeaderCapella.beacon:type_name -> ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
19, // 2: ethereum.eth.v1alpha1.LightClientHeaderCapella.execution:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderCapella
|
||||
18, // 3: ethereum.eth.v1alpha1.LightClientHeaderDeneb.beacon:type_name -> ethereum.eth.v1alpha1.BeaconBlockHeader
|
||||
20, // 4: ethereum.eth.v1alpha1.LightClientHeaderDeneb.execution:type_name -> ethereum.engine.v1.ExecutionPayloadHeaderDeneb
|
||||
0, // 5: ethereum.eth.v1alpha1.LightClientBootstrapAltair.header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
20, // 6: ethereum.eth.v1alpha1.LightClientBootstrapAltair.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 6: ethereum.eth.v1alpha1.LightClientBootstrapAltair.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
1, // 7: ethereum.eth.v1alpha1.LightClientBootstrapCapella.header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
20, // 8: ethereum.eth.v1alpha1.LightClientBootstrapCapella.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 8: ethereum.eth.v1alpha1.LightClientBootstrapCapella.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
2, // 9: ethereum.eth.v1alpha1.LightClientBootstrapDeneb.header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
20, // 10: ethereum.eth.v1alpha1.LightClientBootstrapDeneb.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 10: ethereum.eth.v1alpha1.LightClientBootstrapDeneb.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
2, // 11: ethereum.eth.v1alpha1.LightClientBootstrapElectra.header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
20, // 12: ethereum.eth.v1alpha1.LightClientBootstrapElectra.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 12: ethereum.eth.v1alpha1.LightClientBootstrapElectra.current_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
0, // 13: ethereum.eth.v1alpha1.LightClientUpdateAltair.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
20, // 14: ethereum.eth.v1alpha1.LightClientUpdateAltair.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 14: ethereum.eth.v1alpha1.LightClientUpdateAltair.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
0, // 15: ethereum.eth.v1alpha1.LightClientUpdateAltair.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
21, // 16: ethereum.eth.v1alpha1.LightClientUpdateAltair.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
22, // 16: ethereum.eth.v1alpha1.LightClientUpdateAltair.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
1, // 17: ethereum.eth.v1alpha1.LightClientUpdateCapella.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
20, // 18: ethereum.eth.v1alpha1.LightClientUpdateCapella.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 18: ethereum.eth.v1alpha1.LightClientUpdateCapella.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
1, // 19: ethereum.eth.v1alpha1.LightClientUpdateCapella.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
21, // 20: ethereum.eth.v1alpha1.LightClientUpdateCapella.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
22, // 20: ethereum.eth.v1alpha1.LightClientUpdateCapella.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
2, // 21: ethereum.eth.v1alpha1.LightClientUpdateDeneb.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
20, // 22: ethereum.eth.v1alpha1.LightClientUpdateDeneb.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 22: ethereum.eth.v1alpha1.LightClientUpdateDeneb.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
2, // 23: ethereum.eth.v1alpha1.LightClientUpdateDeneb.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
21, // 24: ethereum.eth.v1alpha1.LightClientUpdateDeneb.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
22, // 24: ethereum.eth.v1alpha1.LightClientUpdateDeneb.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
2, // 25: ethereum.eth.v1alpha1.LightClientUpdateElectra.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
20, // 26: ethereum.eth.v1alpha1.LightClientUpdateElectra.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
21, // 26: ethereum.eth.v1alpha1.LightClientUpdateElectra.next_sync_committee:type_name -> ethereum.eth.v1alpha1.SyncCommittee
|
||||
2, // 27: ethereum.eth.v1alpha1.LightClientUpdateElectra.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
21, // 28: ethereum.eth.v1alpha1.LightClientUpdateElectra.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
22, // 28: ethereum.eth.v1alpha1.LightClientUpdateElectra.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
0, // 29: ethereum.eth.v1alpha1.LightClientFinalityUpdateAltair.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
0, // 30: ethereum.eth.v1alpha1.LightClientFinalityUpdateAltair.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
21, // 31: ethereum.eth.v1alpha1.LightClientFinalityUpdateAltair.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
22, // 31: ethereum.eth.v1alpha1.LightClientFinalityUpdateAltair.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
1, // 32: ethereum.eth.v1alpha1.LightClientFinalityUpdateCapella.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
1, // 33: ethereum.eth.v1alpha1.LightClientFinalityUpdateCapella.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
21, // 34: ethereum.eth.v1alpha1.LightClientFinalityUpdateCapella.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
22, // 34: ethereum.eth.v1alpha1.LightClientFinalityUpdateCapella.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
2, // 35: ethereum.eth.v1alpha1.LightClientFinalityUpdateDeneb.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
2, // 36: ethereum.eth.v1alpha1.LightClientFinalityUpdateDeneb.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
21, // 37: ethereum.eth.v1alpha1.LightClientFinalityUpdateDeneb.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
0, // 38: ethereum.eth.v1alpha1.LightClientOptimisticUpdateAltair.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
21, // 39: ethereum.eth.v1alpha1.LightClientOptimisticUpdateAltair.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
1, // 40: ethereum.eth.v1alpha1.LightClientOptimisticUpdateCapella.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
21, // 41: ethereum.eth.v1alpha1.LightClientOptimisticUpdateCapella.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
2, // 42: ethereum.eth.v1alpha1.LightClientOptimisticUpdateDeneb.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
21, // 43: ethereum.eth.v1alpha1.LightClientOptimisticUpdateDeneb.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
44, // [44:44] is the sub-list for method output_type
|
||||
44, // [44:44] is the sub-list for method input_type
|
||||
44, // [44:44] is the sub-list for extension type_name
|
||||
44, // [44:44] is the sub-list for extension extendee
|
||||
0, // [0:44] is the sub-list for field type_name
|
||||
22, // 37: ethereum.eth.v1alpha1.LightClientFinalityUpdateDeneb.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
2, // 38: ethereum.eth.v1alpha1.LightClientFinalityUpdateElectra.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
2, // 39: ethereum.eth.v1alpha1.LightClientFinalityUpdateElectra.finalized_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
22, // 40: ethereum.eth.v1alpha1.LightClientFinalityUpdateElectra.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
0, // 41: ethereum.eth.v1alpha1.LightClientOptimisticUpdateAltair.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderAltair
|
||||
22, // 42: ethereum.eth.v1alpha1.LightClientOptimisticUpdateAltair.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
1, // 43: ethereum.eth.v1alpha1.LightClientOptimisticUpdateCapella.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderCapella
|
||||
22, // 44: ethereum.eth.v1alpha1.LightClientOptimisticUpdateCapella.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
2, // 45: ethereum.eth.v1alpha1.LightClientOptimisticUpdateDeneb.attested_header:type_name -> ethereum.eth.v1alpha1.LightClientHeaderDeneb
|
||||
22, // 46: ethereum.eth.v1alpha1.LightClientOptimisticUpdateDeneb.sync_aggregate:type_name -> ethereum.eth.v1alpha1.SyncAggregate
|
||||
47, // [47:47] is the sub-list for method output_type
|
||||
47, // [47:47] is the sub-list for method input_type
|
||||
47, // [47:47] is the sub-list for extension type_name
|
||||
47, // [47:47] is the sub-list for extension extendee
|
||||
0, // [0:47] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_proto_prysm_v1alpha1_light_client_proto_init() }
|
||||
@@ -1947,7 +2058,7 @@ func file_proto_prysm_v1alpha1_light_client_proto_init() {
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_light_client_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*LightClientOptimisticUpdateAltair); i {
|
||||
switch v := v.(*LightClientFinalityUpdateElectra); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@@ -1959,7 +2070,7 @@ func file_proto_prysm_v1alpha1_light_client_proto_init() {
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_light_client_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*LightClientOptimisticUpdateCapella); i {
|
||||
switch v := v.(*LightClientOptimisticUpdateAltair); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@@ -1971,6 +2082,18 @@ func file_proto_prysm_v1alpha1_light_client_proto_init() {
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_light_client_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*LightClientOptimisticUpdateCapella); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_proto_prysm_v1alpha1_light_client_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*LightClientOptimisticUpdateDeneb); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
@@ -1989,7 +2112,7 @@ func file_proto_prysm_v1alpha1_light_client_proto_init() {
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_proto_prysm_v1alpha1_light_client_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 17,
|
||||
NumMessages: 18,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
|
||||
@@ -102,7 +102,7 @@ message LightClientUpdateElectra {
|
||||
SyncCommittee next_sync_committee = 2;
|
||||
repeated bytes next_sync_committee_branch = 3 [(ethereum.eth.ext.ssz_size) = "6,32"];
|
||||
LightClientHeaderDeneb finalized_header = 4;
|
||||
repeated bytes finality_branch = 5 [(ethereum.eth.ext.ssz_size) = "6,32"];
|
||||
repeated bytes finality_branch = 5 [(ethereum.eth.ext.ssz_size) = "7,32"];
|
||||
SyncAggregate sync_aggregate = 6;
|
||||
uint64 signature_slot = 7 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"];
|
||||
}
|
||||
@@ -131,6 +131,14 @@ message LightClientFinalityUpdateDeneb {
|
||||
uint64 signature_slot = 5 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"];
|
||||
}
|
||||
|
||||
message LightClientFinalityUpdateElectra {
|
||||
LightClientHeaderDeneb attested_header = 1;
|
||||
LightClientHeaderDeneb finalized_header = 2;
|
||||
repeated bytes finality_branch = 3 [(ethereum.eth.ext.ssz_max) = "7,32"];
|
||||
SyncAggregate sync_aggregate = 4;
|
||||
uint64 signature_slot = 5 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Slot"];
|
||||
}
|
||||
|
||||
message LightClientOptimisticUpdateAltair {
|
||||
LightClientHeaderAltair attested_header = 1;
|
||||
SyncAggregate sync_aggregate = 2;
|
||||
|
||||
@@ -24,179 +24,6 @@ tests/general/phase0/ssz_generic/boolean
|
||||
tests/general/phase0/ssz_generic/containers
|
||||
tests/general/phase0/ssz_generic/uints
|
||||
|
||||
# EIP6110
|
||||
tests/mainnet/eip6110/epoch_processing/effective_balance_updates
|
||||
tests/mainnet/eip6110/epoch_processing/eth1_data_reset
|
||||
tests/mainnet/eip6110/epoch_processing/historical_summaries_update
|
||||
tests/mainnet/eip6110/epoch_processing/inactivity_updates
|
||||
tests/mainnet/eip6110/epoch_processing/justification_and_finalization
|
||||
tests/mainnet/eip6110/epoch_processing/participation_flag_updates
|
||||
tests/mainnet/eip6110/epoch_processing/randao_mixes_reset
|
||||
tests/mainnet/eip6110/epoch_processing/registry_updates
|
||||
tests/mainnet/eip6110/epoch_processing/rewards_and_penalties
|
||||
tests/mainnet/eip6110/epoch_processing/slashings
|
||||
tests/mainnet/eip6110/epoch_processing/slashings_reset
|
||||
tests/mainnet/eip6110/finality/finality
|
||||
tests/mainnet/eip6110/fork_choice/ex_ante
|
||||
tests/mainnet/eip6110/fork_choice/get_head
|
||||
tests/mainnet/eip6110/fork_choice/get_proposer_head
|
||||
tests/mainnet/eip6110/fork_choice/on_block
|
||||
tests/mainnet/eip6110/fork_choice/should_override_forkchoice_update
|
||||
tests/mainnet/eip6110/operations/attestation
|
||||
tests/mainnet/eip6110/operations/attester_slashing
|
||||
tests/mainnet/eip6110/operations/block_header
|
||||
tests/mainnet/eip6110/operations/bls_to_execution_change
|
||||
tests/mainnet/eip6110/operations/deposit
|
||||
tests/mainnet/eip6110/operations/execution_payload
|
||||
tests/mainnet/eip6110/operations/proposer_slashing
|
||||
tests/mainnet/eip6110/operations/sync_aggregate
|
||||
tests/mainnet/eip6110/operations/voluntary_exit
|
||||
tests/mainnet/eip6110/operations/withdrawals
|
||||
tests/mainnet/eip6110/rewards/basic
|
||||
tests/mainnet/eip6110/rewards/leak
|
||||
tests/mainnet/eip6110/rewards/random
|
||||
tests/mainnet/eip6110/sanity/blocks
|
||||
tests/mainnet/eip6110/sanity/slots
|
||||
tests/mainnet/eip6110/ssz_static/AggregateAndProof
|
||||
tests/mainnet/eip6110/ssz_static/Attestation
|
||||
tests/mainnet/eip6110/ssz_static/AttestationData
|
||||
tests/mainnet/eip6110/ssz_static/AttesterSlashing
|
||||
tests/mainnet/eip6110/ssz_static/BLSToExecutionChange
|
||||
tests/mainnet/eip6110/ssz_static/BeaconBlock
|
||||
tests/mainnet/eip6110/ssz_static/BeaconBlockBody
|
||||
tests/mainnet/eip6110/ssz_static/BeaconBlockHeader
|
||||
tests/mainnet/eip6110/ssz_static/BeaconState
|
||||
tests/mainnet/eip6110/ssz_static/BlobIdentifier
|
||||
tests/mainnet/eip6110/ssz_static/BlobSidecar
|
||||
tests/mainnet/eip6110/ssz_static/Checkpoint
|
||||
tests/mainnet/eip6110/ssz_static/ContributionAndProof
|
||||
tests/mainnet/eip6110/ssz_static/Deposit
|
||||
tests/mainnet/eip6110/ssz_static/DepositData
|
||||
tests/mainnet/eip6110/ssz_static/DepositMessage
|
||||
tests/mainnet/eip6110/ssz_static/DepositReceipt
|
||||
tests/mainnet/eip6110/ssz_static/Eth1Block
|
||||
tests/mainnet/eip6110/ssz_static/Eth1Data
|
||||
tests/mainnet/eip6110/ssz_static/ExecutionPayload
|
||||
tests/mainnet/eip6110/ssz_static/ExecutionPayloadHeader
|
||||
tests/mainnet/eip6110/ssz_static/Fork
|
||||
tests/mainnet/eip6110/ssz_static/ForkData
|
||||
tests/mainnet/eip6110/ssz_static/HistoricalBatch
|
||||
tests/mainnet/eip6110/ssz_static/HistoricalSummary
|
||||
tests/mainnet/eip6110/ssz_static/IndexedAttestation
|
||||
tests/mainnet/eip6110/ssz_static/LightClientBootstrap
|
||||
tests/mainnet/eip6110/ssz_static/LightClientFinalityUpdate
|
||||
tests/mainnet/eip6110/ssz_static/LightClientHeader
|
||||
tests/mainnet/eip6110/ssz_static/LightClientOptimisticUpdate
|
||||
tests/mainnet/eip6110/ssz_static/LightClientUpdate
|
||||
tests/mainnet/eip6110/ssz_static/PendingAttestation
|
||||
tests/mainnet/eip6110/ssz_static/PowBlock
|
||||
tests/mainnet/eip6110/ssz_static/ProposerSlashing
|
||||
tests/mainnet/eip6110/ssz_static/SignedAggregateAndProof
|
||||
tests/mainnet/eip6110/ssz_static/SignedBLSToExecutionChange
|
||||
tests/mainnet/eip6110/ssz_static/SignedBeaconBlock
|
||||
tests/mainnet/eip6110/ssz_static/SignedBeaconBlockHeader
|
||||
tests/mainnet/eip6110/ssz_static/SignedContributionAndProof
|
||||
tests/mainnet/eip6110/ssz_static/SignedVoluntaryExit
|
||||
tests/mainnet/eip6110/ssz_static/SigningData
|
||||
tests/mainnet/eip6110/ssz_static/SyncAggregate
|
||||
tests/mainnet/eip6110/ssz_static/SyncAggregatorSelectionData
|
||||
tests/mainnet/eip6110/ssz_static/SyncCommittee
|
||||
tests/mainnet/eip6110/ssz_static/SyncCommitteeContribution
|
||||
tests/mainnet/eip6110/ssz_static/SyncCommitteeMessage
|
||||
tests/mainnet/eip6110/ssz_static/Validator
|
||||
tests/mainnet/eip6110/ssz_static/VoluntaryExit
|
||||
tests/mainnet/eip6110/ssz_static/Withdrawal
|
||||
tests/mainnet/eip6110/sync/optimistic
|
||||
tests/mainnet/eip6110/transition/core
|
||||
tests/minimal/eip6110/epoch_processing/effective_balance_updates
|
||||
tests/minimal/eip6110/epoch_processing/eth1_data_reset
|
||||
tests/minimal/eip6110/epoch_processing/historical_summaries_update
|
||||
tests/minimal/eip6110/epoch_processing/inactivity_updates
|
||||
tests/minimal/eip6110/epoch_processing/justification_and_finalization
|
||||
tests/minimal/eip6110/epoch_processing/participation_flag_updates
|
||||
tests/minimal/eip6110/epoch_processing/randao_mixes_reset
|
||||
tests/minimal/eip6110/epoch_processing/registry_updates
|
||||
tests/minimal/eip6110/epoch_processing/rewards_and_penalties
|
||||
tests/minimal/eip6110/epoch_processing/slashings
|
||||
tests/minimal/eip6110/epoch_processing/slashings_reset
|
||||
tests/minimal/eip6110/epoch_processing/sync_committee_updates
|
||||
tests/minimal/eip6110/finality/finality
|
||||
tests/minimal/eip6110/fork_choice/ex_ante
|
||||
tests/minimal/eip6110/fork_choice/get_head
|
||||
tests/minimal/eip6110/fork_choice/get_proposer_head
|
||||
tests/minimal/eip6110/fork_choice/on_block
|
||||
tests/minimal/eip6110/fork_choice/reorg
|
||||
tests/minimal/eip6110/fork_choice/should_override_forkchoice_update
|
||||
tests/minimal/eip6110/fork_choice/withholding
|
||||
tests/minimal/eip6110/genesis/initialization
|
||||
tests/minimal/eip6110/genesis/validity
|
||||
tests/minimal/eip6110/operations/attestation
|
||||
tests/minimal/eip6110/operations/attester_slashing
|
||||
tests/minimal/eip6110/operations/block_header
|
||||
tests/minimal/eip6110/operations/bls_to_execution_change
|
||||
tests/minimal/eip6110/operations/deposit
|
||||
tests/minimal/eip6110/operations/execution_payload
|
||||
tests/minimal/eip6110/operations/proposer_slashing
|
||||
tests/minimal/eip6110/operations/sync_aggregate
|
||||
tests/minimal/eip6110/operations/voluntary_exit
|
||||
tests/minimal/eip6110/operations/withdrawals
|
||||
tests/minimal/eip6110/rewards/basic
|
||||
tests/minimal/eip6110/rewards/leak
|
||||
tests/minimal/eip6110/rewards/random
|
||||
tests/minimal/eip6110/sanity/blocks
|
||||
tests/minimal/eip6110/sanity/slots
|
||||
tests/minimal/eip6110/ssz_static/AggregateAndProof
|
||||
tests/minimal/eip6110/ssz_static/Attestation
|
||||
tests/minimal/eip6110/ssz_static/AttestationData
|
||||
tests/minimal/eip6110/ssz_static/AttesterSlashing
|
||||
tests/minimal/eip6110/ssz_static/BLSToExecutionChange
|
||||
tests/minimal/eip6110/ssz_static/BeaconBlock
|
||||
tests/minimal/eip6110/ssz_static/BeaconBlockBody
|
||||
tests/minimal/eip6110/ssz_static/BeaconBlockHeader
|
||||
tests/minimal/eip6110/ssz_static/BeaconState
|
||||
tests/minimal/eip6110/ssz_static/BlobIdentifier
|
||||
tests/minimal/eip6110/ssz_static/BlobSidecar
|
||||
tests/minimal/eip6110/ssz_static/Checkpoint
|
||||
tests/minimal/eip6110/ssz_static/ContributionAndProof
|
||||
tests/minimal/eip6110/ssz_static/Deposit
|
||||
tests/minimal/eip6110/ssz_static/DepositData
|
||||
tests/minimal/eip6110/ssz_static/DepositMessage
|
||||
tests/minimal/eip6110/ssz_static/DepositReceipt
|
||||
tests/minimal/eip6110/ssz_static/Eth1Block
|
||||
tests/minimal/eip6110/ssz_static/Eth1Data
|
||||
tests/minimal/eip6110/ssz_static/ExecutionPayload
|
||||
tests/minimal/eip6110/ssz_static/ExecutionPayloadHeader
|
||||
tests/minimal/eip6110/ssz_static/Fork
|
||||
tests/minimal/eip6110/ssz_static/ForkData
|
||||
tests/minimal/eip6110/ssz_static/HistoricalBatch
|
||||
tests/minimal/eip6110/ssz_static/HistoricalSummary
|
||||
tests/minimal/eip6110/ssz_static/IndexedAttestation
|
||||
tests/minimal/eip6110/ssz_static/LightClientBootstrap
|
||||
tests/minimal/eip6110/ssz_static/LightClientFinalityUpdate
|
||||
tests/minimal/eip6110/ssz_static/LightClientHeader
|
||||
tests/minimal/eip6110/ssz_static/LightClientOptimisticUpdate
|
||||
tests/minimal/eip6110/ssz_static/LightClientUpdate
|
||||
tests/minimal/eip6110/ssz_static/PendingAttestation
|
||||
tests/minimal/eip6110/ssz_static/PowBlock
|
||||
tests/minimal/eip6110/ssz_static/ProposerSlashing
|
||||
tests/minimal/eip6110/ssz_static/SignedAggregateAndProof
|
||||
tests/minimal/eip6110/ssz_static/SignedBLSToExecutionChange
|
||||
tests/minimal/eip6110/ssz_static/SignedBeaconBlock
|
||||
tests/minimal/eip6110/ssz_static/SignedBeaconBlockHeader
|
||||
tests/minimal/eip6110/ssz_static/SignedContributionAndProof
|
||||
tests/minimal/eip6110/ssz_static/SignedVoluntaryExit
|
||||
tests/minimal/eip6110/ssz_static/SigningData
|
||||
tests/minimal/eip6110/ssz_static/SyncAggregate
|
||||
tests/minimal/eip6110/ssz_static/SyncAggregatorSelectionData
|
||||
tests/minimal/eip6110/ssz_static/SyncCommittee
|
||||
tests/minimal/eip6110/ssz_static/SyncCommitteeContribution
|
||||
tests/minimal/eip6110/ssz_static/SyncCommitteeMessage
|
||||
tests/minimal/eip6110/ssz_static/Validator
|
||||
tests/minimal/eip6110/ssz_static/VoluntaryExit
|
||||
tests/minimal/eip6110/ssz_static/Withdrawal
|
||||
tests/minimal/eip6110/sync/optimistic
|
||||
tests/minimal/eip6110/transition/core
|
||||
|
||||
# Whisk
|
||||
tests/mainnet/whisk/ssz_static/AggregateAndProof
|
||||
tests/mainnet/whisk/ssz_static/Attestation
|
||||
|
||||
@@ -7,6 +7,5 @@ import (
|
||||
)
|
||||
|
||||
func TestMainnet_Electra_MerkleProof(t *testing.T) {
|
||||
t.Skip("TODO: Electra") // These spectests are missing?
|
||||
merkle_proof.RunMerkleProofTests(t, "mainnet")
|
||||
}
|
||||
|
||||
@@ -7,6 +7,5 @@ import (
|
||||
)
|
||||
|
||||
func TestMinimal_Electra_MerkleProof(t *testing.T) {
|
||||
t.Skip("TODO: Electra") // These spectests are missing?
|
||||
merkle_proof.RunMerkleProofTests(t, "minimal")
|
||||
}
|
||||
|
||||
@@ -88,6 +88,7 @@ func (v *validator) SubmitAggregateAndProof(ctx context.Context, slot primitives
|
||||
PublicKey: pubKey[:],
|
||||
SlotSignature: slotSig,
|
||||
}
|
||||
// TODO: look at renaming SubmitAggregateSelectionProof functions as they are GET beacon API
|
||||
var agg ethpb.AggregateAttAndProof
|
||||
if postElectra {
|
||||
res, err := v.validatorClient.SubmitAggregateSelectionProofElectra(ctx, aggSelectionRequest, duty.ValidatorIndex, uint64(len(duty.Committee)))
|
||||
|
||||
@@ -97,8 +97,10 @@ go_test(
|
||||
"propose_beacon_block_blinded_bellatrix_test.go",
|
||||
"propose_beacon_block_blinded_capella_test.go",
|
||||
"propose_beacon_block_blinded_deneb_test.go",
|
||||
"propose_beacon_block_blinded_electra_test.go",
|
||||
"propose_beacon_block_capella_test.go",
|
||||
"propose_beacon_block_deneb_test.go",
|
||||
"propose_beacon_block_electra_test.go",
|
||||
"propose_beacon_block_phase0_test.go",
|
||||
"propose_beacon_block_test.go",
|
||||
"propose_exit_test.go",
|
||||
@@ -127,6 +129,7 @@ go_test(
|
||||
"//network/httputil:go_default_library",
|
||||
"//proto/engine/v1:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//runtime/version:go_default_library",
|
||||
"//testing/assert:go_default_library",
|
||||
"//testing/require:go_default_library",
|
||||
"//time/slots:go_default_library",
|
||||
|
||||
@@ -155,7 +155,12 @@ func (c *beaconApiValidatorClient) ProposeAttestation(ctx context.Context, in *e
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) ProposeAttestationElectra(ctx context.Context, in *ethpb.AttestationElectra) (*ethpb.AttestResponse, error) {
|
||||
return nil, errors.New("ProposeAttestationElectra is not implemented")
|
||||
ctx, span := trace.StartSpan(ctx, "beacon-api.ProposeAttestationElectra")
|
||||
defer span.End()
|
||||
|
||||
return wrapInMetrics[*ethpb.AttestResponse]("ProposeAttestationElectra", func() (*ethpb.AttestResponse, error) {
|
||||
return c.proposeAttestationElectra(ctx, in)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) ProposeBeaconBlock(ctx context.Context, in *ethpb.GenericSignedBeaconBlock) (*ethpb.ProposeResponse, error) {
|
||||
@@ -190,7 +195,12 @@ func (c *beaconApiValidatorClient) SubmitAggregateSelectionProof(ctx context.Con
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) SubmitAggregateSelectionProofElectra(ctx context.Context, in *ethpb.AggregateSelectionRequest, index primitives.ValidatorIndex, committeeLength uint64) (*ethpb.AggregateSelectionElectraResponse, error) {
|
||||
return nil, errors.New("SubmitAggregateSelectionProofElectra is not implemented")
|
||||
ctx, span := trace.StartSpan(ctx, "beacon-api.SubmitAggregateSelectionProofElectra")
|
||||
defer span.End()
|
||||
|
||||
return wrapInMetrics[*ethpb.AggregateSelectionElectraResponse]("SubmitAggregateSelectionProofElectra", func() (*ethpb.AggregateSelectionElectraResponse, error) {
|
||||
return c.submitAggregateSelectionProofElectra(ctx, in, index, committeeLength)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) SubmitSignedAggregateSelectionProof(ctx context.Context, in *ethpb.SignedAggregateSubmitRequest) (*ethpb.SignedAggregateSubmitResponse, error) {
|
||||
@@ -203,7 +213,12 @@ func (c *beaconApiValidatorClient) SubmitSignedAggregateSelectionProof(ctx conte
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) SubmitSignedAggregateSelectionProofElectra(ctx context.Context, in *ethpb.SignedAggregateSubmitElectraRequest) (*ethpb.SignedAggregateSubmitResponse, error) {
|
||||
return nil, errors.New("SubmitSignedAggregateSelectionProofElectra is not implemented")
|
||||
ctx, span := trace.StartSpan(ctx, "beacon-api.SubmitSignedAggregateSelectionProofElectra")
|
||||
defer span.End()
|
||||
|
||||
return wrapInMetrics[*ethpb.SignedAggregateSubmitResponse]("SubmitSignedAggregateSelectionProofElectra", func() (*ethpb.SignedAggregateSubmitResponse, error) {
|
||||
return c.submitSignedAggregateSelectionProofElectra(ctx, in)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) SubmitSignedContributionAndProof(ctx context.Context, in *ethpb.SignedContributionAndProof) (*empty.Empty, error) {
|
||||
|
||||
@@ -139,7 +139,7 @@ func TestBeaconApiValidatorClient_ProposeBeaconBlockValid(t *testing.T) {
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
map[string]string{"Eth-Consensus-Version": "phase0"},
|
||||
gomock.Any(),
|
||||
nil,
|
||||
@@ -175,7 +175,7 @@ func TestBeaconApiValidatorClient_ProposeBeaconBlockError(t *testing.T) {
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
map[string]string{"Eth-Consensus-Version": "phase0"},
|
||||
gomock.Any(),
|
||||
nil,
|
||||
|
||||
@@ -51,6 +51,14 @@ func jsonifyAttestations(attestations []*ethpb.Attestation) []*structs.Attestati
|
||||
return jsonAttestations
|
||||
}
|
||||
|
||||
func jsonifyAttestationsElectra(attestations []*ethpb.AttestationElectra) []*structs.AttestationElectra {
|
||||
jsonAttestations := make([]*structs.AttestationElectra, len(attestations))
|
||||
for index, attestation := range attestations {
|
||||
jsonAttestations[index] = jsonifyAttestationElectra(attestation)
|
||||
}
|
||||
return jsonAttestations
|
||||
}
|
||||
|
||||
func jsonifyAttesterSlashings(attesterSlashings []*ethpb.AttesterSlashing) []*structs.AttesterSlashing {
|
||||
jsonAttesterSlashings := make([]*structs.AttesterSlashing, len(attesterSlashings))
|
||||
for index, attesterSlashing := range attesterSlashings {
|
||||
@@ -164,6 +172,15 @@ func jsonifyAttestation(attestation *ethpb.Attestation) *structs.Attestation {
|
||||
}
|
||||
}
|
||||
|
||||
func jsonifyAttestationElectra(attestation *ethpb.AttestationElectra) *structs.AttestationElectra {
|
||||
return &structs.AttestationElectra{
|
||||
AggregationBits: hexutil.Encode(attestation.AggregationBits),
|
||||
Data: jsonifyAttestationData(attestation.Data),
|
||||
Signature: hexutil.Encode(attestation.Signature),
|
||||
CommitteeBits: hexutil.Encode(attestation.CommitteeBits),
|
||||
}
|
||||
}
|
||||
|
||||
func jsonifySignedAggregateAndProof(signedAggregateAndProof *ethpb.SignedAggregateAttestationAndProof) *structs.SignedAggregateAttestationAndProof {
|
||||
return &structs.SignedAggregateAttestationAndProof{
|
||||
Message: &structs.AggregateAttestationAndProof{
|
||||
@@ -175,6 +192,17 @@ func jsonifySignedAggregateAndProof(signedAggregateAndProof *ethpb.SignedAggrega
|
||||
}
|
||||
}
|
||||
|
||||
func jsonifySignedAggregateAndProofElectra(signedAggregateAndProof *ethpb.SignedAggregateAttestationAndProofElectra) *structs.SignedAggregateAttestationAndProofElectra {
|
||||
return &structs.SignedAggregateAttestationAndProofElectra{
|
||||
Message: &structs.AggregateAttestationAndProofElectra{
|
||||
AggregatorIndex: uint64ToString(signedAggregateAndProof.Message.AggregatorIndex),
|
||||
Aggregate: jsonifyAttestationElectra(signedAggregateAndProof.Message.Aggregate),
|
||||
SelectionProof: hexutil.Encode(signedAggregateAndProof.Message.SelectionProof),
|
||||
},
|
||||
Signature: hexutil.Encode(signedAggregateAndProof.Signature),
|
||||
}
|
||||
}
|
||||
|
||||
func jsonifyWithdrawals(withdrawals []*enginev1.Withdrawal) []*structs.Withdrawal {
|
||||
jsonWithdrawals := make([]*structs.Withdrawal, len(withdrawals))
|
||||
for index, withdrawal := range withdrawals {
|
||||
|
||||
@@ -197,6 +197,39 @@ func convertAttestationToProto(jsonAttestation *structs.Attestation) (*ethpb.Att
|
||||
}, nil
|
||||
}
|
||||
|
||||
func convertAttestationElectraToProto(jsonAttestation *structs.AttestationElectra) (*ethpb.AttestationElectra, error) {
|
||||
if jsonAttestation == nil {
|
||||
return nil, errors.New("json attestation is nil")
|
||||
}
|
||||
|
||||
aggregationBits, err := hexutil.Decode(jsonAttestation.AggregationBits)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to decode aggregation bits `%s`", jsonAttestation.AggregationBits)
|
||||
}
|
||||
|
||||
attestationData, err := convertAttestationDataToProto(jsonAttestation.Data)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to get attestation data")
|
||||
}
|
||||
|
||||
signature, err := hexutil.Decode(jsonAttestation.Signature)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to decode attestation signature `%s`", jsonAttestation.Signature)
|
||||
}
|
||||
|
||||
committeeBits, err := hexutil.Decode(jsonAttestation.CommitteeBits)
|
||||
if err != nil {
|
||||
return nil, errors.Wrapf(err, "failed to decode committee bits `%s`", jsonAttestation.CommitteeBits)
|
||||
}
|
||||
|
||||
return ðpb.AttestationElectra{
|
||||
AggregationBits: aggregationBits,
|
||||
Data: attestationData,
|
||||
Signature: signature,
|
||||
CommitteeBits: committeeBits,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func convertAttestationsToProto(jsonAttestations []*structs.Attestation) ([]*ethpb.Attestation, error) {
|
||||
var attestations []*ethpb.Attestation
|
||||
for index, jsonAttestation := range jsonAttestations {
|
||||
|
||||
@@ -69,8 +69,14 @@ func (c *beaconApiValidatorClient) beaconBlock(ctx context.Context, slot primiti
|
||||
blinded = produceBlockV3ResponseJson.ExecutionPayloadBlinded
|
||||
decoder = json.NewDecoder(bytes.NewReader(produceBlockV3ResponseJson.Data))
|
||||
}
|
||||
return processBlockResponse(ver, blinded, decoder)
|
||||
}
|
||||
|
||||
func processBlockResponse(ver string, isBlinded bool, decoder *json.Decoder) (*ethpb.GenericBeaconBlock, error) {
|
||||
var response *ethpb.GenericBeaconBlock
|
||||
if decoder == nil {
|
||||
return nil, errors.New("no produce block json decoder found")
|
||||
}
|
||||
switch ver {
|
||||
case version.String(version.Phase0):
|
||||
jsonPhase0Block := structs.BeaconBlock{}
|
||||
@@ -93,7 +99,7 @@ func (c *beaconApiValidatorClient) beaconBlock(ctx context.Context, slot primiti
|
||||
}
|
||||
response = genericBlock
|
||||
case version.String(version.Bellatrix):
|
||||
if blinded {
|
||||
if isBlinded {
|
||||
jsonBellatrixBlock := structs.BlindedBeaconBlockBellatrix{}
|
||||
if err := decoder.Decode(&jsonBellatrixBlock); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to decode blinded bellatrix block response json")
|
||||
@@ -115,7 +121,7 @@ func (c *beaconApiValidatorClient) beaconBlock(ctx context.Context, slot primiti
|
||||
response = genericBlock
|
||||
}
|
||||
case version.String(version.Capella):
|
||||
if blinded {
|
||||
if isBlinded {
|
||||
jsonCapellaBlock := structs.BlindedBeaconBlockCapella{}
|
||||
if err := decoder.Decode(&jsonCapellaBlock); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to decode blinded capella block response json")
|
||||
@@ -137,7 +143,7 @@ func (c *beaconApiValidatorClient) beaconBlock(ctx context.Context, slot primiti
|
||||
response = genericBlock
|
||||
}
|
||||
case version.String(version.Deneb):
|
||||
if blinded {
|
||||
if isBlinded {
|
||||
jsonDenebBlock := structs.BlindedBeaconBlockDeneb{}
|
||||
if err := decoder.Decode(&jsonDenebBlock); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to decode blinded deneb block response json")
|
||||
@@ -158,6 +164,28 @@ func (c *beaconApiValidatorClient) beaconBlock(ctx context.Context, slot primiti
|
||||
}
|
||||
response = genericBlock
|
||||
}
|
||||
case version.String(version.Electra):
|
||||
if isBlinded {
|
||||
jsonElectraBlock := structs.BlindedBeaconBlockElectra{}
|
||||
if err := decoder.Decode(&jsonElectraBlock); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to decode blinded electra block response json")
|
||||
}
|
||||
genericBlock, err := jsonElectraBlock.ToGeneric()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to get blinded electra block")
|
||||
}
|
||||
response = genericBlock
|
||||
} else {
|
||||
jsonElectraBlockContents := structs.BeaconBlockContentsElectra{}
|
||||
if err := decoder.Decode(&jsonElectraBlockContents); err != nil {
|
||||
return nil, errors.Wrap(err, "failed to decode electra block response json")
|
||||
}
|
||||
genericBlock, err := jsonElectraBlockContents.ToGeneric()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to get electra block")
|
||||
}
|
||||
response = genericBlock
|
||||
}
|
||||
default:
|
||||
return nil, errors.Errorf("unsupported consensus version `%s`", ver)
|
||||
}
|
||||
|
||||
@@ -500,6 +500,96 @@ func TestGetBeaconBlock_BlindedDenebValid(t *testing.T) {
|
||||
assert.DeepEqual(t, expectedBeaconBlock, beaconBlock)
|
||||
}
|
||||
|
||||
func TestGetBeaconBlock_ElectraValid(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
|
||||
proto := testhelpers.GenerateProtoElectraBeaconBlockContents()
|
||||
block := testhelpers.GenerateJsonElectraBeaconBlockContents()
|
||||
bytes, err := json.Marshal(block)
|
||||
require.NoError(t, err)
|
||||
|
||||
const slot = primitives.Slot(1)
|
||||
randaoReveal := []byte{2}
|
||||
graffiti := []byte{3}
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
jsonRestHandler.EXPECT().Get(
|
||||
gomock.Any(),
|
||||
fmt.Sprintf("/eth/v3/validator/blocks/%d?graffiti=%s&randao_reveal=%s", slot, hexutil.Encode(graffiti), hexutil.Encode(randaoReveal)),
|
||||
&structs.ProduceBlockV3Response{},
|
||||
).SetArg(
|
||||
2,
|
||||
structs.ProduceBlockV3Response{
|
||||
Version: "electra",
|
||||
ExecutionPayloadBlinded: false,
|
||||
Data: bytes,
|
||||
},
|
||||
).Return(
|
||||
nil,
|
||||
).Times(1)
|
||||
|
||||
validatorClient := &beaconApiValidatorClient{jsonRestHandler: jsonRestHandler}
|
||||
beaconBlock, err := validatorClient.beaconBlock(ctx, slot, randaoReveal, graffiti)
|
||||
require.NoError(t, err)
|
||||
|
||||
expectedBeaconBlock := ðpb.GenericBeaconBlock{
|
||||
Block: ðpb.GenericBeaconBlock_Electra{
|
||||
Electra: proto,
|
||||
},
|
||||
IsBlinded: false,
|
||||
}
|
||||
|
||||
assert.DeepEqual(t, expectedBeaconBlock, beaconBlock)
|
||||
}
|
||||
|
||||
func TestGetBeaconBlock_BlindedElectraValid(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
|
||||
proto := testhelpers.GenerateProtoBlindedElectraBeaconBlock()
|
||||
block := testhelpers.GenerateJsonBlindedElectraBeaconBlock()
|
||||
bytes, err := json.Marshal(block)
|
||||
require.NoError(t, err)
|
||||
|
||||
const slot = primitives.Slot(1)
|
||||
randaoReveal := []byte{2}
|
||||
graffiti := []byte{3}
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
jsonRestHandler.EXPECT().Get(
|
||||
gomock.Any(),
|
||||
fmt.Sprintf("/eth/v3/validator/blocks/%d?graffiti=%s&randao_reveal=%s", slot, hexutil.Encode(graffiti), hexutil.Encode(randaoReveal)),
|
||||
&structs.ProduceBlockV3Response{},
|
||||
).SetArg(
|
||||
2,
|
||||
structs.ProduceBlockV3Response{
|
||||
Version: "electra",
|
||||
ExecutionPayloadBlinded: true,
|
||||
Data: bytes,
|
||||
},
|
||||
).Return(
|
||||
nil,
|
||||
).Times(1)
|
||||
|
||||
validatorClient := &beaconApiValidatorClient{jsonRestHandler: jsonRestHandler}
|
||||
beaconBlock, err := validatorClient.beaconBlock(ctx, slot, randaoReveal, graffiti)
|
||||
require.NoError(t, err)
|
||||
|
||||
expectedBeaconBlock := ðpb.GenericBeaconBlock{
|
||||
Block: ðpb.GenericBeaconBlock_BlindedElectra{
|
||||
BlindedElectra: proto,
|
||||
},
|
||||
IsBlinded: true,
|
||||
}
|
||||
|
||||
assert.DeepEqual(t, expectedBeaconBlock, beaconBlock)
|
||||
}
|
||||
|
||||
func TestGetBeaconBlock_FallbackToBlindedBlock(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
|
||||
@@ -4,25 +4,73 @@ import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"net/http"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/network/httputil"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
)
|
||||
|
||||
func (c *beaconApiValidatorClient) proposeAttestation(ctx context.Context, attestation *ethpb.Attestation) (*ethpb.AttestResponse, error) {
|
||||
if err := checkNilAttestation(attestation); err != nil {
|
||||
if err := validateNilAttestation(attestation); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
marshalledAttestation, err := json.Marshal(jsonifyAttestations([]*ethpb.Attestation{attestation}))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
headers := map[string]string{"Eth-Consensus-Version": version.String(attestation.Version())}
|
||||
err = c.jsonRestHandler.Post(
|
||||
ctx,
|
||||
"/eth/v2/beacon/pool/attestations",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledAttestation),
|
||||
nil,
|
||||
)
|
||||
errJson := &httputil.DefaultJsonError{}
|
||||
if err != nil {
|
||||
// TODO: remove this when v2 becomes default
|
||||
if !errors.As(err, &errJson) {
|
||||
return nil, err
|
||||
}
|
||||
if errJson.Code != http.StatusNotFound {
|
||||
return nil, errJson
|
||||
}
|
||||
log.Debug("Endpoint /eth/v2/beacon/pool/attestations is not supported, falling back to older endpoints for submit attestation.")
|
||||
if err = c.jsonRestHandler.Post(
|
||||
ctx,
|
||||
"/eth/v1/beacon/pool/attestations",
|
||||
nil,
|
||||
bytes.NewBuffer(marshalledAttestation),
|
||||
nil,
|
||||
); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
attestationDataRoot, err := attestation.Data.HashTreeRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to compute attestation data root")
|
||||
}
|
||||
|
||||
return ðpb.AttestResponse{AttestationDataRoot: attestationDataRoot[:]}, nil
|
||||
}
|
||||
|
||||
func (c *beaconApiValidatorClient) proposeAttestationElectra(ctx context.Context, attestation *ethpb.AttestationElectra) (*ethpb.AttestResponse, error) {
|
||||
if err := validateNilAttestation(attestation); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
marshalledAttestation, err := json.Marshal(jsonifyAttestationsElectra([]*ethpb.AttestationElectra{attestation}))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
headers := map[string]string{"Eth-Consensus-Version": version.String(attestation.Version())}
|
||||
if err = c.jsonRestHandler.Post(
|
||||
ctx,
|
||||
"/eth/v1/beacon/pool/attestations",
|
||||
nil,
|
||||
"/eth/v2/beacon/pool/attestations",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledAttestation),
|
||||
nil,
|
||||
); err != nil {
|
||||
@@ -37,27 +85,27 @@ func (c *beaconApiValidatorClient) proposeAttestation(ctx context.Context, attes
|
||||
return ðpb.AttestResponse{AttestationDataRoot: attestationDataRoot[:]}, nil
|
||||
}
|
||||
|
||||
// checkNilAttestation returns error if attestation or any field of attestation is nil.
|
||||
func checkNilAttestation(attestation *ethpb.Attestation) error {
|
||||
if attestation == nil {
|
||||
return errors.New("attestation is nil")
|
||||
func validateNilAttestation(attestation ethpb.Att) error {
|
||||
if attestation == nil || attestation.IsNil() {
|
||||
return errors.New("attestation can't be nil")
|
||||
}
|
||||
|
||||
if attestation.Data == nil {
|
||||
return errors.New("attestation data is nil")
|
||||
if attestation.GetData().Source == nil {
|
||||
return errors.New("attestation's source can't be nil")
|
||||
}
|
||||
|
||||
if attestation.Data.Source == nil || attestation.Data.Target == nil {
|
||||
return errors.New("source/target in attestation data is nil")
|
||||
if attestation.GetData().Target == nil {
|
||||
return errors.New("attestation's target can't be nil")
|
||||
}
|
||||
|
||||
if len(attestation.AggregationBits) == 0 {
|
||||
return errors.New("attestation aggregation bits is empty")
|
||||
v := attestation.Version()
|
||||
if len(attestation.GetAggregationBits()) == 0 {
|
||||
return errors.New("attestation's bitfield can't be nil")
|
||||
}
|
||||
|
||||
if len(attestation.Signature) == 0 {
|
||||
return errors.New("attestation signature is empty")
|
||||
if len(attestation.GetSignature()) == 0 {
|
||||
return errors.New("attestation signature can't be nil")
|
||||
}
|
||||
if v >= version.Electra {
|
||||
if len(attestation.CommitteeBitsVal().BitIndices()) == 0 {
|
||||
return errors.New("attestation committee bits can't be nil")
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -5,9 +5,12 @@ import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"net/http"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/network/httputil"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/assert"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/validator/client/beacon-api/mock"
|
||||
@@ -48,7 +51,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
},
|
||||
{
|
||||
name: "nil attestation",
|
||||
expectedErrorMessage: "attestation is nil",
|
||||
expectedErrorMessage: "attestation can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil attestation data",
|
||||
@@ -56,7 +59,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
},
|
||||
expectedErrorMessage: "attestation data is nil",
|
||||
expectedErrorMessage: "attestation can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil source checkpoint",
|
||||
@@ -67,7 +70,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
},
|
||||
expectedErrorMessage: "source/target in attestation data is nil",
|
||||
expectedErrorMessage: "attestation's source can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil target checkpoint",
|
||||
@@ -78,7 +81,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
},
|
||||
expectedErrorMessage: "source/target in attestation data is nil",
|
||||
expectedErrorMessage: "attestation's target can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil aggregation bits",
|
||||
@@ -89,7 +92,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
},
|
||||
expectedErrorMessage: "attestation aggregation bits is empty",
|
||||
expectedErrorMessage: "attestation's bitfield can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil signature",
|
||||
@@ -100,7 +103,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
Target: ðpb.Checkpoint{},
|
||||
},
|
||||
},
|
||||
expectedErrorMessage: "attestation signature is empty",
|
||||
expectedErrorMessage: "attestation signature can't be nil",
|
||||
},
|
||||
{
|
||||
name: "bad request",
|
||||
@@ -117,7 +120,7 @@ func TestProposeAttestation(t *testing.T) {
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
|
||||
var marshalledAttestations []byte
|
||||
if checkNilAttestation(test.attestation) == nil {
|
||||
if validateNilAttestation(test.attestation) == nil {
|
||||
b, err := json.Marshal(jsonifyAttestations([]*ethpb.Attestation{test.attestation}))
|
||||
require.NoError(t, err)
|
||||
marshalledAttestations = b
|
||||
@@ -125,10 +128,11 @@ func TestProposeAttestation(t *testing.T) {
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
headers := map[string]string{"Eth-Consensus-Version": version.String(test.attestation.Version())}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/pool/attestations",
|
||||
nil,
|
||||
"/eth/v2/beacon/pool/attestations",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledAttestations),
|
||||
nil,
|
||||
).Return(
|
||||
@@ -153,3 +157,235 @@ func TestProposeAttestation(t *testing.T) {
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestProposeAttestationFallBack(t *testing.T) {
|
||||
attestation := ðpb.Attestation{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 75,
|
||||
CommitteeIndex: 76,
|
||||
BeaconBlockRoot: testhelpers.FillByteSlice(32, 38),
|
||||
Source: ðpb.Checkpoint{
|
||||
Epoch: 78,
|
||||
Root: testhelpers.FillByteSlice(32, 79),
|
||||
},
|
||||
Target: ðpb.Checkpoint{
|
||||
Epoch: 80,
|
||||
Root: testhelpers.FillByteSlice(32, 81),
|
||||
},
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
}
|
||||
|
||||
ctrl := gomock.NewController(t)
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
|
||||
var marshalledAttestations []byte
|
||||
if validateNilAttestation(attestation) == nil {
|
||||
b, err := json.Marshal(jsonifyAttestations([]*ethpb.Attestation{attestation}))
|
||||
require.NoError(t, err)
|
||||
marshalledAttestations = b
|
||||
}
|
||||
|
||||
ctx := context.Background()
|
||||
headers := map[string]string{"Eth-Consensus-Version": version.String(attestation.Version())}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v2/beacon/pool/attestations",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledAttestations),
|
||||
nil,
|
||||
).Return(
|
||||
&httputil.DefaultJsonError{
|
||||
Code: http.StatusNotFound,
|
||||
},
|
||||
).Times(1)
|
||||
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/pool/attestations",
|
||||
nil,
|
||||
bytes.NewBuffer(marshalledAttestations),
|
||||
nil,
|
||||
).Return(
|
||||
nil,
|
||||
).Times(1)
|
||||
|
||||
validatorClient := &beaconApiValidatorClient{jsonRestHandler: jsonRestHandler}
|
||||
proposeResponse, err := validatorClient.proposeAttestation(ctx, attestation)
|
||||
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, proposeResponse)
|
||||
|
||||
expectedAttestationDataRoot, err := attestation.Data.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Make sure that the attestation data root is set
|
||||
assert.DeepEqual(t, expectedAttestationDataRoot[:], proposeResponse.AttestationDataRoot)
|
||||
}
|
||||
|
||||
func TestProposeAttestationElectra(t *testing.T) {
|
||||
attestation := ðpb.AttestationElectra{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 75,
|
||||
CommitteeIndex: 76,
|
||||
BeaconBlockRoot: testhelpers.FillByteSlice(32, 38),
|
||||
Source: ðpb.Checkpoint{
|
||||
Epoch: 78,
|
||||
Root: testhelpers.FillByteSlice(32, 79),
|
||||
},
|
||||
Target: ðpb.Checkpoint{
|
||||
Epoch: 80,
|
||||
Root: testhelpers.FillByteSlice(32, 81),
|
||||
},
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
CommitteeBits: testhelpers.FillByteSlice(8, 83),
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
attestation *ethpb.AttestationElectra
|
||||
expectedErrorMessage string
|
||||
endpointError error
|
||||
endpointCall int
|
||||
}{
|
||||
{
|
||||
name: "valid",
|
||||
attestation: attestation,
|
||||
endpointCall: 1,
|
||||
},
|
||||
{
|
||||
name: "nil attestation",
|
||||
expectedErrorMessage: "attestation can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil attestation data",
|
||||
attestation: ðpb.AttestationElectra{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
CommitteeBits: testhelpers.FillByteSlice(8, 83),
|
||||
},
|
||||
expectedErrorMessage: "attestation can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil source checkpoint",
|
||||
attestation: ðpb.AttestationElectra{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Data: ðpb.AttestationData{
|
||||
Target: ðpb.Checkpoint{},
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
CommitteeBits: testhelpers.FillByteSlice(8, 83),
|
||||
},
|
||||
expectedErrorMessage: "attestation's source can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil target checkpoint",
|
||||
attestation: ðpb.AttestationElectra{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Data: ðpb.AttestationData{
|
||||
Source: ðpb.Checkpoint{},
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
CommitteeBits: testhelpers.FillByteSlice(8, 83),
|
||||
},
|
||||
expectedErrorMessage: "attestation's target can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil aggregation bits",
|
||||
attestation: ðpb.AttestationElectra{
|
||||
Data: ðpb.AttestationData{
|
||||
Source: ðpb.Checkpoint{},
|
||||
Target: ðpb.Checkpoint{},
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
CommitteeBits: testhelpers.FillByteSlice(8, 83),
|
||||
},
|
||||
expectedErrorMessage: "attestation's bitfield can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil signature",
|
||||
attestation: ðpb.AttestationElectra{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Data: ðpb.AttestationData{
|
||||
Source: ðpb.Checkpoint{},
|
||||
Target: ðpb.Checkpoint{},
|
||||
},
|
||||
CommitteeBits: testhelpers.FillByteSlice(8, 83),
|
||||
},
|
||||
expectedErrorMessage: "attestation signature can't be nil",
|
||||
},
|
||||
{
|
||||
name: "nil committee bits",
|
||||
attestation: ðpb.AttestationElectra{
|
||||
AggregationBits: testhelpers.FillByteSlice(4, 74),
|
||||
Data: ðpb.AttestationData{
|
||||
Slot: 75,
|
||||
CommitteeIndex: 76,
|
||||
BeaconBlockRoot: testhelpers.FillByteSlice(32, 38),
|
||||
Source: ðpb.Checkpoint{
|
||||
Epoch: 78,
|
||||
Root: testhelpers.FillByteSlice(32, 79),
|
||||
},
|
||||
Target: ðpb.Checkpoint{
|
||||
Epoch: 80,
|
||||
Root: testhelpers.FillByteSlice(32, 81),
|
||||
},
|
||||
},
|
||||
Signature: testhelpers.FillByteSlice(96, 82),
|
||||
},
|
||||
expectedErrorMessage: "attestation committee bits can't be nil",
|
||||
},
|
||||
{
|
||||
name: "bad request",
|
||||
attestation: attestation,
|
||||
expectedErrorMessage: "bad request",
|
||||
endpointError: errors.New("bad request"),
|
||||
endpointCall: 1,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
|
||||
var marshalledAttestations []byte
|
||||
if validateNilAttestation(test.attestation) == nil {
|
||||
b, err := json.Marshal(jsonifyAttestationsElectra([]*ethpb.AttestationElectra{test.attestation}))
|
||||
require.NoError(t, err)
|
||||
marshalledAttestations = b
|
||||
}
|
||||
|
||||
ctx := context.Background()
|
||||
headers := map[string]string{"Eth-Consensus-Version": version.String(test.attestation.Version())}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v2/beacon/pool/attestations",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledAttestations),
|
||||
nil,
|
||||
).Return(
|
||||
test.endpointError,
|
||||
).Times(test.endpointCall)
|
||||
|
||||
validatorClient := &beaconApiValidatorClient{jsonRestHandler: jsonRestHandler}
|
||||
proposeResponse, err := validatorClient.proposeAttestationElectra(ctx, test.attestation)
|
||||
if test.expectedErrorMessage != "" {
|
||||
require.ErrorContains(t, test.expectedErrorMessage, err)
|
||||
return
|
||||
}
|
||||
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, proposeResponse)
|
||||
|
||||
expectedAttestationDataRoot, err := attestation.Data.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Make sure that the attestation data root is set
|
||||
assert.DeepEqual(t, expectedAttestationDataRoot[:], proposeResponse.AttestationDataRoot)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -120,6 +120,35 @@ func (c *beaconApiValidatorClient) proposeBeaconBlock(ctx context.Context, in *e
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to marshal blinded deneb beacon block contents")
|
||||
}
|
||||
case *ethpb.GenericSignedBeaconBlock_Electra:
|
||||
consensusVersion = "electra"
|
||||
beaconBlockRoot, err = blockType.Electra.Block.HashTreeRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to compute block root for electra beacon block")
|
||||
}
|
||||
signedBlock, err := structs.SignedBeaconBlockContentsElectraFromConsensus(blockType.Electra)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to convert electra beacon block contents")
|
||||
}
|
||||
marshalledSignedBeaconBlockJson, err = json.Marshal(signedBlock)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to marshal electra beacon block contents")
|
||||
}
|
||||
case *ethpb.GenericSignedBeaconBlock_BlindedElectra:
|
||||
blinded = true
|
||||
consensusVersion = "electra"
|
||||
beaconBlockRoot, err = blockType.BlindedElectra.HashTreeRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to compute block root for blinded electra beacon block")
|
||||
}
|
||||
signedBlock, err := structs.SignedBlindedBeaconBlockElectraFromConsensus(blockType.BlindedElectra)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to convert blinded electra beacon block contents")
|
||||
}
|
||||
marshalledSignedBeaconBlockJson, err = json.Marshal(signedBlock)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to marshal blinded electra beacon block contents")
|
||||
}
|
||||
default:
|
||||
return nil, errors.Errorf("unsupported block type %T", in.Block)
|
||||
}
|
||||
@@ -127,9 +156,9 @@ func (c *beaconApiValidatorClient) proposeBeaconBlock(ctx context.Context, in *e
|
||||
var endpoint string
|
||||
|
||||
if blinded {
|
||||
endpoint = "/eth/v1/beacon/blinded_blocks"
|
||||
endpoint = "/eth/v2/beacon/blinded_blocks"
|
||||
} else {
|
||||
endpoint = "/eth/v1/beacon/blocks"
|
||||
endpoint = "/eth/v2/beacon/blocks"
|
||||
}
|
||||
|
||||
headers := map[string]string{"Eth-Consensus-Version": consensusVersion}
|
||||
|
||||
@@ -59,7 +59,7 @@ func TestProposeBeaconBlock_Altair(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "altair"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledBlock),
|
||||
nil,
|
||||
|
||||
@@ -76,7 +76,7 @@ func TestProposeBeaconBlock_Bellatrix(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "bellatrix"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledBlock),
|
||||
nil,
|
||||
|
||||
@@ -77,7 +77,7 @@ func TestProposeBeaconBlock_BlindedBellatrix(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "bellatrix"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blinded_blocks",
|
||||
"/eth/v2/beacon/blinded_blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledBlock),
|
||||
nil,
|
||||
|
||||
@@ -79,7 +79,7 @@ func TestProposeBeaconBlock_BlindedCapella(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "capella"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blinded_blocks",
|
||||
"/eth/v2/beacon/blinded_blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledBlock),
|
||||
nil,
|
||||
|
||||
@@ -15,7 +15,6 @@ import (
|
||||
)
|
||||
|
||||
func TestProposeBeaconBlock_BlindedDeneb(t *testing.T) {
|
||||
t.Skip("TODO: Fix this in the beacon-API PR")
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
@@ -32,7 +31,7 @@ func TestProposeBeaconBlock_BlindedDeneb(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "deneb"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blinded_blocks",
|
||||
"/eth/v2/beacon/blinded_blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(denebBytes),
|
||||
nil,
|
||||
|
||||
@@ -0,0 +1,50 @@
|
||||
package beacon_api
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
|
||||
rpctesting "github.com/prysmaticlabs/prysm/v5/beacon-chain/rpc/eth/shared/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/assert"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/validator/client/beacon-api/mock"
|
||||
"go.uber.org/mock/gomock"
|
||||
)
|
||||
|
||||
func TestProposeBeaconBlock_BlindedElectra(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
|
||||
var block structs.SignedBlindedBeaconBlockElectra
|
||||
err := json.Unmarshal([]byte(rpctesting.BlindedElectraBlock), &block)
|
||||
require.NoError(t, err)
|
||||
genericSignedBlock, err := block.ToGeneric()
|
||||
require.NoError(t, err)
|
||||
|
||||
electraBytes, err := json.Marshal(block)
|
||||
require.NoError(t, err)
|
||||
// Make sure that what we send in the POST body is the marshalled version of the protobuf block
|
||||
headers := map[string]string{"Eth-Consensus-Version": "electra"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v2/beacon/blinded_blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(electraBytes),
|
||||
nil,
|
||||
)
|
||||
|
||||
validatorClient := &beaconApiValidatorClient{jsonRestHandler: jsonRestHandler}
|
||||
proposeResponse, err := validatorClient.proposeBeaconBlock(context.Background(), genericSignedBlock)
|
||||
assert.NoError(t, err)
|
||||
require.NotNil(t, proposeResponse)
|
||||
|
||||
expectedBlockRoot, err := genericSignedBlock.GetBlindedElectra().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Make sure that the block root is set
|
||||
assert.DeepEqual(t, expectedBlockRoot[:], proposeResponse.BlockRoot)
|
||||
}
|
||||
@@ -76,7 +76,7 @@ func TestProposeBeaconBlock_Capella(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "capella"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledBlock),
|
||||
nil,
|
||||
|
||||
@@ -15,8 +15,6 @@ import (
|
||||
)
|
||||
|
||||
func TestProposeBeaconBlock_Deneb(t *testing.T) {
|
||||
t.Skip("TODO: Fix this in the beacon-API PR")
|
||||
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
@@ -33,7 +31,7 @@ func TestProposeBeaconBlock_Deneb(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "deneb"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(denebBytes),
|
||||
nil,
|
||||
|
||||
@@ -0,0 +1,50 @@
|
||||
package beacon_api
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
|
||||
rpctesting "github.com/prysmaticlabs/prysm/v5/beacon-chain/rpc/eth/shared/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/assert"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/validator/client/beacon-api/mock"
|
||||
"go.uber.org/mock/gomock"
|
||||
)
|
||||
|
||||
func TestProposeBeaconBlock_Electra(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
jsonRestHandler := mock.NewMockJsonRestHandler(ctrl)
|
||||
|
||||
var blockContents structs.SignedBeaconBlockContentsElectra
|
||||
err := json.Unmarshal([]byte(rpctesting.ElectraBlockContents), &blockContents)
|
||||
require.NoError(t, err)
|
||||
genericSignedBlock, err := blockContents.ToGeneric()
|
||||
require.NoError(t, err)
|
||||
|
||||
electraBytes, err := json.Marshal(blockContents)
|
||||
require.NoError(t, err)
|
||||
// Make sure that what we send in the POST body is the marshalled version of the protobuf block
|
||||
headers := map[string]string{"Eth-Consensus-Version": "electra"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v2/beacon/blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(electraBytes),
|
||||
nil,
|
||||
)
|
||||
|
||||
validatorClient := &beaconApiValidatorClient{jsonRestHandler: jsonRestHandler}
|
||||
proposeResponse, err := validatorClient.proposeBeaconBlock(context.Background(), genericSignedBlock)
|
||||
assert.NoError(t, err)
|
||||
require.NotNil(t, proposeResponse)
|
||||
|
||||
expectedBlockRoot, err := genericSignedBlock.GetElectra().Block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Make sure that the block root is set
|
||||
assert.DeepEqual(t, expectedBlockRoot[:], proposeResponse.BlockRoot)
|
||||
}
|
||||
@@ -55,7 +55,7 @@ func TestProposeBeaconBlock_Phase0(t *testing.T) {
|
||||
headers := map[string]string{"Eth-Consensus-Version": "phase0"}
|
||||
jsonRestHandler.EXPECT().Post(
|
||||
gomock.Any(),
|
||||
"/eth/v1/beacon/blocks",
|
||||
"/eth/v2/beacon/blocks",
|
||||
headers,
|
||||
bytes.NewBuffer(marshalledBlock),
|
||||
nil,
|
||||
|
||||
@@ -51,7 +51,7 @@ func TestProposeBeaconBlock_Error(t *testing.T) {
|
||||
{
|
||||
name: "phase0",
|
||||
consensusVersion: "phase0",
|
||||
endpoint: "/eth/v1/beacon/blocks",
|
||||
endpoint: "/eth/v2/beacon/blocks",
|
||||
block: ðpb.GenericSignedBeaconBlock{
|
||||
Block: generateSignedPhase0Block(),
|
||||
},
|
||||
@@ -59,7 +59,7 @@ func TestProposeBeaconBlock_Error(t *testing.T) {
|
||||
{
|
||||
name: "altair",
|
||||
consensusVersion: "altair",
|
||||
endpoint: "/eth/v1/beacon/blocks",
|
||||
endpoint: "/eth/v2/beacon/blocks",
|
||||
block: ðpb.GenericSignedBeaconBlock{
|
||||
Block: generateSignedAltairBlock(),
|
||||
},
|
||||
@@ -67,7 +67,7 @@ func TestProposeBeaconBlock_Error(t *testing.T) {
|
||||
{
|
||||
name: "bellatrix",
|
||||
consensusVersion: "bellatrix",
|
||||
endpoint: "/eth/v1/beacon/blocks",
|
||||
endpoint: "/eth/v2/beacon/blocks",
|
||||
block: ðpb.GenericSignedBeaconBlock{
|
||||
Block: generateSignedBellatrixBlock(),
|
||||
},
|
||||
@@ -75,15 +75,23 @@ func TestProposeBeaconBlock_Error(t *testing.T) {
|
||||
{
|
||||
name: "blinded bellatrix",
|
||||
consensusVersion: "bellatrix",
|
||||
endpoint: "/eth/v1/beacon/blinded_blocks",
|
||||
endpoint: "/eth/v2/beacon/blinded_blocks",
|
||||
block: ðpb.GenericSignedBeaconBlock{
|
||||
Block: generateSignedBlindedBellatrixBlock(),
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "capella",
|
||||
consensusVersion: "capella",
|
||||
endpoint: "/eth/v2/beacon/blocks",
|
||||
block: ðpb.GenericSignedBeaconBlock{
|
||||
Block: generateSignedCapellaBlock(),
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "blinded capella",
|
||||
consensusVersion: "capella",
|
||||
endpoint: "/eth/v1/beacon/blinded_blocks",
|
||||
endpoint: "/eth/v2/beacon/blinded_blocks",
|
||||
block: ðpb.GenericSignedBeaconBlock{
|
||||
Block: generateSignedBlindedCapellaBlock(),
|
||||
},
|
||||
|
||||
@@ -50,6 +50,10 @@ func (c *beaconApiValidatorClient) validatorsStatusResponse(ctx context.Context,
|
||||
[]*ethpb.ValidatorStatusResponse,
|
||||
error,
|
||||
) {
|
||||
// if no parameters are provided we should just return an empty response
|
||||
if len(inPubKeys) == 0 && len(inIndexes) == 0 {
|
||||
return [][]byte{}, []primitives.ValidatorIndex{}, []*ethpb.ValidatorStatusResponse{}, nil
|
||||
}
|
||||
// Represents the target set of keys
|
||||
stringTargetPubKeysToPubKeys := make(map[string][]byte, len(inPubKeys))
|
||||
stringTargetPubKeys := make([]string, len(inPubKeys))
|
||||
@@ -78,6 +82,7 @@ func (c *beaconApiValidatorClient) validatorsStatusResponse(ctx context.Context,
|
||||
return nil, nil, nil, errors.Wrap(err, "failed to get state validators")
|
||||
}
|
||||
|
||||
// TODO: we should remove this API call
|
||||
validatorsCountResponse, err := c.prysmChainClient.ValidatorCount(ctx, "head", nil)
|
||||
if err != nil && !errors.Is(err, iface.ErrNotSupported) {
|
||||
return nil, nil, nil, errors.Wrap(err, "failed to get total validator count")
|
||||
|
||||
@@ -215,32 +215,23 @@ func TestMultipleValidatorStatus_Nominal(t *testing.T) {
|
||||
assert.DeepEqual(t, &expectedValidatorStatusResponse, actualValidatorStatusResponse)
|
||||
}
|
||||
|
||||
func TestMultipleValidatorStatus_Error(t *testing.T) {
|
||||
func TestMultipleValidatorStatus_No_Keys(t *testing.T) {
|
||||
ctrl := gomock.NewController(t)
|
||||
defer ctrl.Finish()
|
||||
|
||||
ctx := context.Background()
|
||||
stateValidatorsProvider := mock.NewMockStateValidatorsProvider(ctrl)
|
||||
|
||||
stateValidatorsProvider.EXPECT().StateValidators(
|
||||
gomock.Any(),
|
||||
gomock.Any(),
|
||||
[]primitives.ValidatorIndex{},
|
||||
nil,
|
||||
).Return(
|
||||
&structs.GetValidatorsResponse{},
|
||||
errors.New("a specific error"),
|
||||
).Times(1)
|
||||
|
||||
validatorClient := beaconApiValidatorClient{stateValidatorsProvider: stateValidatorsProvider}
|
||||
|
||||
_, err := validatorClient.MultipleValidatorStatus(
|
||||
resp, err := validatorClient.MultipleValidatorStatus(
|
||||
ctx,
|
||||
ðpb.MultipleValidatorStatusRequest{
|
||||
PublicKeys: [][]byte{},
|
||||
},
|
||||
)
|
||||
require.ErrorContains(t, "failed to get validators status response", err)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, ðpb.MultipleValidatorStatusResponse{}, resp)
|
||||
}
|
||||
|
||||
func TestGetValidatorsStatusResponse_Nominal_SomeActiveValidators(t *testing.T) {
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user