EIP-7549: Aggregation gRPC metods (#14115)

* impl

* tests

* generate mock

* review feedback

* gofmt
This commit is contained in:
Radosław Kapka
2024-07-01 20:40:33 +02:00
committed by GitHub
parent 657750b803
commit 2e6f1de29a
10 changed files with 1576 additions and 793 deletions

View File

@@ -250,37 +250,64 @@ func (s *Service) SubmitSignedContributionAndProof(
// SubmitSignedAggregateSelectionProof verifies given aggregate and proofs and publishes them on appropriate gossipsub topic.
func (s *Service) SubmitSignedAggregateSelectionProof(
ctx context.Context,
req *ethpb.SignedAggregateSubmitRequest,
agg ethpb.SignedAggregateAttAndProof,
) *RpcError {
ctx, span := trace.StartSpan(ctx, "coreService.SubmitSignedAggregateSelectionProof")
defer span.End()
if req.SignedAggregateAndProof == nil || req.SignedAggregateAndProof.Message == nil ||
req.SignedAggregateAndProof.Message.Aggregate == nil || req.SignedAggregateAndProof.Message.Aggregate.Data == nil {
if agg == nil {
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(req.SignedAggregateAndProof.Signature, emptySig) ||
bytes.Equal(req.SignedAggregateAndProof.Message.SelectionProof, emptySig) {
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}
}
// As a preventive measure, a beacon node shouldn't broadcast an attestation whose slot is out of range.
if err := helpers.ValidateAttestationTime(req.SignedAggregateAndProof.Message.Aggregate.Data.Slot,
s.GenesisTimeFetcher.GenesisTime(), params.BeaconConfig().MaximumGossipClockDisparityDuration()); err != nil {
if err := helpers.ValidateAttestationTime(
data.Slot,
s.GenesisTimeFetcher.GenesisTime(),
params.BeaconConfig().MaximumGossipClockDisparityDuration(),
); err != nil {
return &RpcError{Err: errors.New("attestation slot is no longer valid from current time"), Reason: BadRequest}
}
if err := s.Broadcaster.Broadcast(ctx, req.SignedAggregateAndProof); err != nil {
if err := s.Broadcaster.Broadcast(ctx, agg); err != nil {
return &RpcError{Err: &AggregateBroadcastFailedError{err: err}, Reason: Internal}
}
log.WithFields(logrus.Fields{
"slot": req.SignedAggregateAndProof.Message.Aggregate.Data.Slot,
"committeeIndex": req.SignedAggregateAndProof.Message.Aggregate.Data.CommitteeIndex,
"validatorIndex": req.SignedAggregateAndProof.Message.AggregatorIndex,
"aggregatedCount": req.SignedAggregateAndProof.Message.Aggregate.AggregationBits.Count(),
}).Debug("Broadcasting aggregated attestation and proof")
if logrus.GetLevel() >= logrus.DebugLevel {
var fields logrus.Fields
if agg.Version() >= version.Electra {
fields = logrus.Fields{
"slot": data.Slot,
"committeeCount": att.CommitteeBitsVal().Count(),
"committeeIndices": att.CommitteeBitsVal().BitIndices(),
"validatorIndex": attAndProof.GetAggregatorIndex(),
"aggregatedCount": att.GetAggregationBits().Count(),
}
} else {
fields = logrus.Fields{
"slot": data.Slot,
"committeeIndex": data.CommitteeIndex,
"validatorIndex": attAndProof.GetAggregatorIndex(),
"aggregatedCount": att.GetAggregationBits().Count(),
}
}
log.WithFields(fields).Debug("Broadcasting aggregated attestation and proof")
}
return nil
}

View File

@@ -173,10 +173,7 @@ func (s *Server) SubmitAggregateAndProofs(w http.ResponseWriter, r *http.Request
httputil.HandleError(w, "Could not convert request aggregate to consensus aggregate: "+err.Error(), http.StatusBadRequest)
return
}
rpcError := s.CoreService.SubmitSignedAggregateSelectionProof(
ctx,
&ethpbalpha.SignedAggregateSubmitRequest{SignedAggregateAndProof: consensusItem},
)
rpcError := s.CoreService.SubmitSignedAggregateSelectionProof(ctx, consensusItem)
if rpcError != nil {
_, ok := rpcError.Err.(*core.AggregateBroadcastFailedError)
if ok {

View File

@@ -6,6 +6,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/rpc/core"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/time/slots"
@@ -26,7 +27,8 @@ func (vs *Server) SubmitAggregateSelectionProof(ctx context.Context, req *ethpb.
return nil, status.Errorf(codes.Unavailable, "Syncing to latest head, not ready to respond")
}
// An optimistic validator MUST NOT participate in attestation. (i.e., sign across the DOMAIN_BEACON_ATTESTER, DOMAIN_SELECTION_PROOF or DOMAIN_AGGREGATE_AND_PROOF domains).
// An optimistic validator MUST NOT participate in attestation
// (i.e., sign across the DOMAIN_BEACON_ATTESTER, DOMAIN_SELECTION_PROOF or DOMAIN_AGGREGATE_AND_PROOF domains).
if err := vs.optimisticStatus(ctx); err != nil {
return nil, err
}
@@ -64,11 +66,11 @@ func (vs *Server) SubmitAggregateSelectionProof(ctx context.Context, req *ethpb.
return nil, status.Errorf(codes.InvalidArgument, "Validator is not an aggregator")
}
aggregatedAtts := vs.AttPool.AggregatedAttestationsBySlotIndex(ctx, req.Slot, req.CommitteeIndex)
atts := vs.AttPool.AggregatedAttestationsBySlotIndex(ctx, req.Slot, req.CommitteeIndex)
// Filter out the best aggregated attestation (ie. the one with the most aggregated bits).
if len(aggregatedAtts) == 0 {
aggregatedAtts = vs.AttPool.UnaggregatedAttestationsBySlotIndex(ctx, req.Slot, req.CommitteeIndex)
if len(aggregatedAtts) == 0 {
if len(atts) == 0 {
atts = vs.AttPool.UnaggregatedAttestationsBySlotIndex(ctx, req.Slot, req.CommitteeIndex)
if len(atts) == 0 {
return nil, status.Errorf(codes.NotFound, "Could not find attestation for slot and committee in pool")
}
}
@@ -80,31 +82,91 @@ func (vs *Server) SubmitAggregateSelectionProof(ctx context.Context, req *ethpb.
}
}
best := aggregatedAtts[0]
for _, aggregatedAtt := range aggregatedAtts[1:] {
// The aggregator should prefer an attestation that they have signed. We check this by
// looking at the attestation's committee index against the validator's committee index
// and check the aggregate bits to ensure the validator's index is set.
if aggregatedAtt.GetData().CommitteeIndex == req.CommitteeIndex &&
aggregatedAtt.GetAggregationBits().BitAt(indexInCommittee) &&
(!best.GetAggregationBits().BitAt(indexInCommittee) ||
aggregatedAtt.GetAggregationBits().Count() > best.GetAggregationBits().Count()) {
best = aggregatedAtt
}
// If the "best" still doesn't contain the validator's index, check the aggregation bits to
// choose the attestation with the most bits set.
if !best.GetAggregationBits().BitAt(indexInCommittee) &&
aggregatedAtt.GetAggregationBits().Count() > best.GetAggregationBits().Count() {
best = aggregatedAtt
}
}
a := &ethpb.AggregateAttestationAndProof{
best := bestAggregate(atts, req.CommitteeIndex, indexInCommittee)
attAndProof := &ethpb.AggregateAttestationAndProof{
Aggregate: best,
SelectionProof: req.SlotSignature,
AggregatorIndex: validatorIndex,
}
return &ethpb.AggregateSelectionResponse{AggregateAndProof: a}, nil
return &ethpb.AggregateSelectionResponse{AggregateAndProof: attAndProof}, nil
}
// SubmitAggregateSelectionProofElectra is called by a validator when its assigned to be an aggregator.
// The aggregator submits the selection proof to obtain the aggregated attestation
// object to sign over.
func (vs *Server) SubmitAggregateSelectionProofElectra(
ctx context.Context,
req *ethpb.AggregateSelectionRequest,
) (*ethpb.AggregateSelectionElectraResponse, error) {
ctx, span := trace.StartSpan(ctx, "AggregatorServer.SubmitAggregateSelectionProofElectra")
defer span.End()
span.AddAttributes(trace.Int64Attribute("slot", int64(req.Slot)))
if vs.SyncChecker.Syncing() {
return nil, status.Errorf(codes.Unavailable, "Syncing to latest head, not ready to respond")
}
// An optimistic validator MUST NOT participate in attestation
// (i.e., sign across the DOMAIN_BEACON_ATTESTER, DOMAIN_SELECTION_PROOF or DOMAIN_AGGREGATE_AND_PROOF domains).
if err := vs.optimisticStatus(ctx); err != nil {
return nil, err
}
st, err := vs.HeadFetcher.HeadStateReadOnly(ctx)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not determine head state: %v", err)
}
validatorIndex, exists := st.ValidatorIndexByPubkey(bytesutil.ToBytes48(req.PublicKey))
if !exists {
return nil, status.Error(codes.Internal, "Could not locate validator index in DB")
}
epoch := slots.ToEpoch(req.Slot)
activeValidatorIndices, err := helpers.ActiveValidatorIndices(ctx, st, epoch)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get validators: %v", err)
}
seed, err := helpers.Seed(st, epoch, params.BeaconConfig().DomainBeaconAttester)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get seed: %v", err)
}
committee, err := helpers.BeaconCommittee(ctx, activeValidatorIndices, seed, req.Slot, req.CommitteeIndex)
if err != nil {
return nil, err
}
// Check if the validator is an aggregator
isAggregator, err := helpers.IsAggregator(uint64(len(committee)), req.SlotSignature)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get aggregator status: %v", err)
}
if !isAggregator {
return nil, status.Errorf(codes.InvalidArgument, "Validator is not an aggregator")
}
atts := vs.AttPool.AggregatedAttestationsBySlotIndexElectra(ctx, req.Slot, req.CommitteeIndex)
if len(atts) == 0 {
atts = vs.AttPool.UnaggregatedAttestationsBySlotIndexElectra(ctx, req.Slot, req.CommitteeIndex)
if len(atts) == 0 {
return nil, status.Errorf(codes.NotFound, "No attestations found in pool")
}
}
var indexInCommittee uint64
for i, idx := range committee {
if idx == validatorIndex {
indexInCommittee = uint64(i)
}
}
best := bestAggregate(atts, req.CommitteeIndex, indexInCommittee)
attAndProof := &ethpb.AggregateAttestationAndProofElectra{
Aggregate: best,
SelectionProof: req.SlotSignature,
AggregatorIndex: validatorIndex,
}
return &ethpb.AggregateSelectionElectraResponse{AggregateAndProof: attAndProof}, nil
}
// SubmitSignedAggregateSelectionProof is called by a validator to broadcast a signed
@@ -113,8 +175,43 @@ func (vs *Server) SubmitSignedAggregateSelectionProof(
ctx context.Context,
req *ethpb.SignedAggregateSubmitRequest,
) (*ethpb.SignedAggregateSubmitResponse, error) {
if err := vs.CoreService.SubmitSignedAggregateSelectionProof(ctx, req); err != nil {
if err := vs.CoreService.SubmitSignedAggregateSelectionProof(ctx, req.SignedAggregateAndProof); err != nil {
return nil, status.Errorf(core.ErrorReasonToGRPC(err.Reason), "Could not submit aggregate: %v", err.Err)
}
return &ethpb.SignedAggregateSubmitResponse{}, nil
}
// SubmitSignedAggregateSelectionProofElectra is called by a validator to broadcast a signed
// aggregated and proof object.
func (vs *Server) SubmitSignedAggregateSelectionProofElectra(
ctx context.Context,
req *ethpb.SignedAggregateSubmitElectraRequest,
) (*ethpb.SignedAggregateSubmitResponse, error) {
if err := vs.CoreService.SubmitSignedAggregateSelectionProof(ctx, req.SignedAggregateAndProof); err != nil {
return nil, status.Errorf(core.ErrorReasonToGRPC(err.Reason), "Could not submit aggregate: %v", err.Err)
}
return &ethpb.SignedAggregateSubmitResponse{}, nil
}
func bestAggregate[T ethpb.Att](atts []T, committeeIndex primitives.CommitteeIndex, indexInCommittee uint64) T {
best := atts[0]
for _, a := range atts[1:] {
// The aggregator should prefer an attestation that they have signed. We check this by
// looking at the attestation's committee index against the validator's committee index
// and check the aggregate bits to ensure the validator's index is set.
if a.CommitteeBitsVal().BitAt(uint64(committeeIndex)) &&
a.GetAggregationBits().BitAt(indexInCommittee) &&
(!best.GetAggregationBits().BitAt(indexInCommittee) ||
a.GetAggregationBits().Count() > best.GetAggregationBits().Count()) {
best = a
}
// If the "best" still doesn't contain the validator's index, check the aggregation bits to
// choose the attestation with the most bits set.
if !best.GetAggregationBits().BitAt(indexInCommittee) &&
a.GetAggregationBits().Count() > best.GetAggregationBits().Count() {
best = a
}
}
return best
}

View File

@@ -451,3 +451,158 @@ func TestSubmitSignedAggregateSelectionProof_InvalidSlot(t *testing.T) {
_, err := aggregatorServer.SubmitSignedAggregateSelectionProof(context.Background(), req)
require.ErrorContains(t, "attestation slot is no longer valid from current time", err)
}
func TestSubmitSignedAggregateSelectionProofElectra_ZeroHashesSignatures(t *testing.T) {
aggregatorServer := &Server{
TimeFetcher: &mock.ChainService{Genesis: time.Now()},
}
req := &ethpb.SignedAggregateSubmitElectraRequest{
SignedAggregateAndProof: &ethpb.SignedAggregateAttestationAndProofElectra{
Signature: make([]byte, fieldparams.BLSSignatureLength),
Message: &ethpb.AggregateAttestationAndProofElectra{
Aggregate: &ethpb.AttestationElectra{
Data: &ethpb.AttestationData{},
},
},
},
}
_, err := aggregatorServer.SubmitSignedAggregateSelectionProofElectra(context.Background(), req)
require.ErrorContains(t, "signed signatures can't be zero hashes", err)
req = &ethpb.SignedAggregateSubmitElectraRequest{
SignedAggregateAndProof: &ethpb.SignedAggregateAttestationAndProofElectra{
Signature: []byte{'a'},
Message: &ethpb.AggregateAttestationAndProofElectra{
Aggregate: &ethpb.AttestationElectra{
Data: &ethpb.AttestationData{},
},
SelectionProof: make([]byte, fieldparams.BLSSignatureLength),
},
},
}
_, err = aggregatorServer.SubmitSignedAggregateSelectionProofElectra(context.Background(), req)
require.ErrorContains(t, "signed signatures can't be zero hashes", err)
}
func TestSubmitSignedAggregateSelectionProofElectra_InvalidSlot(t *testing.T) {
c := &mock.ChainService{Genesis: time.Now()}
aggregatorServer := &Server{
CoreService: &core.Service{
GenesisTimeFetcher: c,
},
}
req := &ethpb.SignedAggregateSubmitElectraRequest{
SignedAggregateAndProof: &ethpb.SignedAggregateAttestationAndProofElectra{
Signature: []byte{'a'},
Message: &ethpb.AggregateAttestationAndProofElectra{
SelectionProof: []byte{'a'},
Aggregate: &ethpb.AttestationElectra{
Data: &ethpb.AttestationData{Slot: 1000},
},
},
},
}
_, err := aggregatorServer.SubmitSignedAggregateSelectionProofElectra(context.Background(), req)
require.ErrorContains(t, "attestation slot is no longer valid from current time", err)
}
func Test_bestAggregate(t *testing.T) {
type testCase struct {
name string
atts []*ethpb.Attestation
best *ethpb.Attestation
}
var testCases []testCase
tc := testCase{
name: "single attestation",
atts: []*ethpb.Attestation{{}},
}
tc.best = tc.atts[0]
testCases = append(testCases, tc)
tc = testCase{
name: "choose attestation with most aggregation bits",
atts: []*ethpb.Attestation{
{
AggregationBits: bitfield.Bitlist{0b10001},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b11111},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b10101},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
},
}
tc.best = tc.atts[1]
testCases = append(testCases, tc)
tc = testCase{
name: "do not choose attestation with other committee index",
atts: []*ethpb.Attestation{
{
AggregationBits: bitfield.Bitlist{0b10001},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b11111},
Data: &ethpb.AttestationData{CommitteeIndex: 1},
},
{
AggregationBits: bitfield.Bitlist{0b10101},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
},
}
tc.best = tc.atts[2]
testCases = append(testCases, tc)
tc = testCase{
name: "do not choose attestation with other index in committee",
atts: []*ethpb.Attestation{
{
AggregationBits: bitfield.Bitlist{0b10001},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b11110},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b10101},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
},
}
tc.best = tc.atts[2]
testCases = append(testCases, tc)
tc = testCase{
name: "no attestation with correct index in committee - choose max att bits",
atts: []*ethpb.Attestation{
{
AggregationBits: bitfield.Bitlist{0b11000},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b11110},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
{
AggregationBits: bitfield.Bitlist{0b10110},
Data: &ethpb.AttestationData{CommitteeIndex: 0},
},
},
}
tc.best = tc.atts[1]
testCases = append(testCases, tc)
for _, tc := range testCases {
assert.Equal(t, tc.best, bestAggregate(tc.atts, 0, 0), tc.name)
}
}

View File

@@ -80,7 +80,9 @@ func (a *Attestation) Copy() Att {
// CommitteeBitsVal --
func (a *Attestation) CommitteeBitsVal() bitfield.Bitfield {
return nil
cb := primitives.NewAttestationCommitteeBits()
cb.SetBitAt(uint64(a.Data.CommitteeIndex), true)
return cb
}
// Version --

File diff suppressed because it is too large Load Diff

View File

@@ -502,6 +502,40 @@ func local_request_BeaconNodeValidator_SubmitAggregateSelectionProof_0(ctx conte
}
func request_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconNodeValidatorClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq AggregateSelectionRequest
var metadata runtime.ServerMetadata
newReader, berr := utilities.IOReaderFactory(req.Body)
if berr != nil {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr)
}
if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
}
msg, err := client.SubmitAggregateSelectionProofElectra(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
return msg, metadata, err
}
func local_request_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0(ctx context.Context, marshaler runtime.Marshaler, server BeaconNodeValidatorServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq AggregateSelectionRequest
var metadata runtime.ServerMetadata
newReader, berr := utilities.IOReaderFactory(req.Body)
if berr != nil {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr)
}
if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
}
msg, err := server.SubmitAggregateSelectionProofElectra(ctx, &protoReq)
return msg, metadata, err
}
func request_BeaconNodeValidator_SubmitSignedAggregateSelectionProof_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconNodeValidatorClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq SignedAggregateSubmitRequest
var metadata runtime.ServerMetadata
@@ -536,6 +570,40 @@ func local_request_BeaconNodeValidator_SubmitSignedAggregateSelectionProof_0(ctx
}
func request_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconNodeValidatorClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq SignedAggregateSubmitElectraRequest
var metadata runtime.ServerMetadata
newReader, berr := utilities.IOReaderFactory(req.Body)
if berr != nil {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr)
}
if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
}
msg, err := client.SubmitSignedAggregateSelectionProofElectra(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
return msg, metadata, err
}
func local_request_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0(ctx context.Context, marshaler runtime.Marshaler, server BeaconNodeValidatorServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq SignedAggregateSubmitElectraRequest
var metadata runtime.ServerMetadata
newReader, berr := utilities.IOReaderFactory(req.Body)
if berr != nil {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr)
}
if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF {
return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
}
msg, err := server.SubmitSignedAggregateSelectionProofElectra(ctx, &protoReq)
return msg, metadata, err
}
func request_BeaconNodeValidator_ProposeExit_0(ctx context.Context, marshaler runtime.Marshaler, client BeaconNodeValidatorClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
var protoReq SignedVoluntaryExit
var metadata runtime.ServerMetadata
@@ -1252,6 +1320,29 @@ func RegisterBeaconNodeValidatorHandlerServer(ctx context.Context, mux *runtime.
})
mux.Handle("POST", pattern_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
var stream runtime.ServerTransportStream
ctx = grpc.NewContextWithServerTransportStream(ctx, &stream)
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/ethereum.eth.v1alpha1.BeaconNodeValidator/SubmitAggregateSelectionProofElectra")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := local_request_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0(rctx, inboundMarshaler, server, req, pathParams)
md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer())
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("POST", pattern_BeaconNodeValidator_SubmitSignedAggregateSelectionProof_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -1275,6 +1366,29 @@ func RegisterBeaconNodeValidatorHandlerServer(ctx context.Context, mux *runtime.
})
mux.Handle("POST", pattern_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
var stream runtime.ServerTransportStream
ctx = grpc.NewContextWithServerTransportStream(ctx, &stream)
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req, "/ethereum.eth.v1alpha1.BeaconNodeValidator/SubmitSignedAggregateSelectionProofElectra")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := local_request_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0(rctx, inboundMarshaler, server, req, pathParams)
md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer())
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("POST", pattern_BeaconNodeValidator_ProposeExit_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -1863,6 +1977,26 @@ func RegisterBeaconNodeValidatorHandlerClient(ctx context.Context, mux *runtime.
})
mux.Handle("POST", pattern_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateContext(ctx, mux, req, "/ethereum.eth.v1alpha1.BeaconNodeValidator/SubmitAggregateSelectionProofElectra")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := request_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0(rctx, inboundMarshaler, client, req, pathParams)
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("POST", pattern_BeaconNodeValidator_SubmitSignedAggregateSelectionProof_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -1883,6 +2017,26 @@ func RegisterBeaconNodeValidatorHandlerClient(ctx context.Context, mux *runtime.
})
mux.Handle("POST", pattern_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
rctx, err := runtime.AnnotateContext(ctx, mux, req, "/ethereum.eth.v1alpha1.BeaconNodeValidator/SubmitSignedAggregateSelectionProofElectra")
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
resp, md, err := request_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0(rctx, inboundMarshaler, client, req, pathParams)
ctx = runtime.NewServerMetadataContext(ctx, md)
if err != nil {
runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
return
}
forward_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
})
mux.Handle("POST", pattern_BeaconNodeValidator_ProposeExit_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
ctx, cancel := context.WithCancel(req.Context())
defer cancel()
@@ -2175,8 +2329,12 @@ var (
pattern_BeaconNodeValidator_SubmitAggregateSelectionProof_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "validator", "aggregate"}, ""))
pattern_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "validator", "aggregate_electra"}, ""))
pattern_BeaconNodeValidator_SubmitSignedAggregateSelectionProof_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "validator", "aggregate"}, ""))
pattern_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "validator", "aggregate_electra"}, ""))
pattern_BeaconNodeValidator_ProposeExit_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"eth", "v1alpha1", "validator", "exit"}, ""))
pattern_BeaconNodeValidator_SubscribeCommitteeSubnets_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4}, []string{"eth", "v1alpha1", "validator", "subnet", "subscribe"}, ""))
@@ -2233,8 +2391,12 @@ var (
forward_BeaconNodeValidator_SubmitAggregateSelectionProof_0 = runtime.ForwardResponseMessage
forward_BeaconNodeValidator_SubmitAggregateSelectionProofElectra_0 = runtime.ForwardResponseMessage
forward_BeaconNodeValidator_SubmitSignedAggregateSelectionProof_0 = runtime.ForwardResponseMessage
forward_BeaconNodeValidator_SubmitSignedAggregateSelectionProofElectra_0 = runtime.ForwardResponseMessage
forward_BeaconNodeValidator_ProposeExit_0 = runtime.ForwardResponseMessage
forward_BeaconNodeValidator_SubscribeCommitteeSubnets_0 = runtime.ForwardResponseMessage

View File

@@ -197,6 +197,15 @@ service BeaconNodeValidator {
};
}
// Submit selection proof to the beacon node to aggregate all matching wire attestations with the same data root.
// the beacon node responses with an aggregate and proof object back to validator to sign over.
rpc SubmitAggregateSelectionProofElectra(AggregateSelectionRequest) returns (AggregateSelectionElectraResponse) {
option (google.api.http) = {
post: "/eth/v1alpha1/validator/aggregate_electra"
body: "*"
};
}
// Submit a signed aggregate and proof object, the beacon node will broadcast the
// signed aggregated attestation and proof object.
rpc SubmitSignedAggregateSelectionProof(SignedAggregateSubmitRequest) returns (SignedAggregateSubmitResponse) {
@@ -206,6 +215,15 @@ service BeaconNodeValidator {
};
}
// Submit a signed aggregate and proof object, the beacon node will broadcast the
// signed aggregated attestation and proof object.
rpc SubmitSignedAggregateSelectionProofElectra(SignedAggregateSubmitElectraRequest) returns (SignedAggregateSubmitResponse) {
option (google.api.http) = {
post: "/eth/v1alpha1/validator/aggregate_electra"
body: "*"
};
}
// Propose to leave the list of active validators.
//
// The beacon node is expected to validate the request and make it available for inclusion in
@@ -602,11 +620,21 @@ message AggregateSelectionResponse {
AggregateAttestationAndProof aggregate_and_proof = 1;
}
message AggregateSelectionElectraResponse {
// The aggregate and proof message without the signature.
AggregateAttestationAndProofElectra aggregate_and_proof = 1;
}
message SignedAggregateSubmitRequest {
// The signed aggregate and proof message with the signature.
SignedAggregateAttestationAndProof signed_aggregate_and_proof = 1;
}
message SignedAggregateSubmitElectraRequest {
// The signed aggregate and proof message with the signature.
SignedAggregateAttestationAndProofElectra signed_aggregate_and_proof = 1;
}
message SignedAggregateSubmitResponse {
// The 32 byte hash tree root of the aggregated attestation data.
bytes attestation_data_root = 1 [(ethereum.eth.ext.ssz_size) = "32"];

View File

@@ -423,6 +423,26 @@ func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitAggregateSelectionPro
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitAggregateSelectionProof", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitAggregateSelectionProof), varargs...)
}
// SubmitAggregateSelectionProofElectra mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProofElectra(arg0 context.Context, arg1 *eth.AggregateSelectionRequest, arg2 ...grpc.CallOption) (*eth.AggregateSelectionElectraResponse, error) {
m.ctrl.T.Helper()
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitAggregateSelectionProofElectra", varargs...)
ret0, _ := ret[0].(*eth.AggregateSelectionElectraResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// SubmitAggregateSelectionProofElectra indicates an expected call of SubmitAggregateSelectionProofElectra.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitAggregateSelectionProofElectra(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitAggregateSelectionProofElectra", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitAggregateSelectionProofElectra), varargs...)
}
// SubmitSignedAggregateSelectionProof mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProof(arg0 context.Context, arg1 *eth.SignedAggregateSubmitRequest, arg2 ...grpc.CallOption) (*eth.SignedAggregateSubmitResponse, error) {
m.ctrl.T.Helper()
@@ -443,6 +463,26 @@ func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedAggregateSelect
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSignedAggregateSelectionProof", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitSignedAggregateSelectionProof), varargs...)
}
// SubmitSignedAggregateSelectionProofElectra mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProofElectra(arg0 context.Context, arg1 *eth.SignedAggregateSubmitElectraRequest, arg2 ...grpc.CallOption) (*eth.SignedAggregateSubmitResponse, error) {
m.ctrl.T.Helper()
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitSignedAggregateSelectionProofElectra", varargs...)
ret0, _ := ret[0].(*eth.SignedAggregateSubmitResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// SubmitSignedAggregateSelectionProofElectra indicates an expected call of SubmitSignedAggregateSelectionProofElectra.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedAggregateSelectionProofElectra(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSignedAggregateSelectionProofElectra", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitSignedAggregateSelectionProofElectra), varargs...)
}
// SubmitSignedContributionAndProof mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitSignedContributionAndProof(arg0 context.Context, arg1 *eth.SignedContributionAndProof, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()

View File

@@ -325,6 +325,21 @@ func (mr *MockBeaconNodeValidatorServerMockRecorder) SubmitAggregateSelectionPro
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitAggregateSelectionProof", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).SubmitAggregateSelectionProof), arg0, arg1)
}
// SubmitAggregateSelectionProofElectra mocks base method.
func (m *MockBeaconNodeValidatorServer) SubmitAggregateSelectionProofElectra(arg0 context.Context, arg1 *eth.AggregateSelectionRequest) (*eth.AggregateSelectionElectraResponse, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SubmitAggregateSelectionProofElectra", arg0, arg1)
ret0, _ := ret[0].(*eth.AggregateSelectionElectraResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// SubmitAggregateSelectionProofElectra indicates an expected call of SubmitAggregateSelectionProofElectra.
func (mr *MockBeaconNodeValidatorServerMockRecorder) SubmitAggregateSelectionProofElectra(arg0, arg1 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitAggregateSelectionProofElectra", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).SubmitAggregateSelectionProofElectra), arg0, arg1)
}
// SubmitSignedAggregateSelectionProof mocks base method.
func (m *MockBeaconNodeValidatorServer) SubmitSignedAggregateSelectionProof(arg0 context.Context, arg1 *eth.SignedAggregateSubmitRequest) (*eth.SignedAggregateSubmitResponse, error) {
m.ctrl.T.Helper()
@@ -340,6 +355,21 @@ func (mr *MockBeaconNodeValidatorServerMockRecorder) SubmitSignedAggregateSelect
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSignedAggregateSelectionProof", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).SubmitSignedAggregateSelectionProof), arg0, arg1)
}
// SubmitSignedAggregateSelectionProofElectra mocks base method.
func (m *MockBeaconNodeValidatorServer) SubmitSignedAggregateSelectionProofElectra(arg0 context.Context, arg1 *eth.SignedAggregateSubmitElectraRequest) (*eth.SignedAggregateSubmitResponse, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SubmitSignedAggregateSelectionProofElectra", arg0, arg1)
ret0, _ := ret[0].(*eth.SignedAggregateSubmitResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// SubmitSignedAggregateSelectionProofElectra indicates an expected call of SubmitSignedAggregateSelectionProofElectra.
func (mr *MockBeaconNodeValidatorServerMockRecorder) SubmitSignedAggregateSelectionProofElectra(arg0, arg1 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSignedAggregateSelectionProofElectra", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).SubmitSignedAggregateSelectionProofElectra), arg0, arg1)
}
// SubmitSignedContributionAndProof mocks base method.
func (m *MockBeaconNodeValidatorServer) SubmitSignedContributionAndProof(arg0 context.Context, arg1 *eth.SignedContributionAndProof) (*emptypb.Empty, error) {
m.ctrl.T.Helper()