Get duties v2 ( gRPC) (#15273)

* adding in proto for getdutiesv2

* adding in GetDutiesV2

* updating changelog and mock

* fixing tests

* breaking up function into smaller functions for gocognit

* reverting some changes so we can break it into a separate PR for validator client only

* reverted too much adding mock change back in

* removing reserved based on preston's feedback

* adding duties tests back in

* Update beacon-chain/rpc/prysm/v1alpha1/validator/duties.go

Co-authored-by: Preston Van Loon <pvanloon@offchainlabs.com>

* updating based on preston's feedback

* updating build for new files

* maybe there's some flake with how the state is used, trying with it moved

* reverting config override

* reverting all changes to get duties v1 based on preston's feedback

* Update proto/prysm/v1alpha1/validator.proto

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* addressing partial feedback

* adding small comment

* reorganizing function based on feedback

* removing unused field

* adding some more unit tests

* adding attribute for pubkeys to span

* preston's feedback

* fixing current to next

* probably safer to register current period now

* Update beacon-chain/core/helpers/beacon_committee.go

Co-authored-by: Preston Van Loon <pvanloon@offchainlabs.com>

* adding in preston's comment

---------

Co-authored-by: Preston Van Loon <pvanloon@offchainlabs.com>
Co-authored-by: Radosław Kapka <rkapka@wp.pl>
This commit is contained in:
james-prysm
2025-06-04 10:05:57 -05:00
committed by GitHub
parent 4a4532f3ba
commit 8b9c161560
11 changed files with 2533 additions and 1145 deletions

View File

@@ -277,10 +277,10 @@ type CommitteeAssignment struct {
CommitteeIndex primitives.CommitteeIndex
}
// verifyAssignmentEpoch verifies if the given epoch is valid for assignment based on the provided state.
// VerifyAssignmentEpoch verifies if the given epoch is valid for assignment based on the provided state.
// It checks if the epoch is not greater than the next epoch, and if the start slot of the epoch is greater
// than or equal to the minimum valid start slot calculated based on the state's current slot and historical roots.
func verifyAssignmentEpoch(epoch primitives.Epoch, state state.BeaconState) error {
func VerifyAssignmentEpoch(epoch primitives.Epoch, state state.BeaconState) error {
nextEpoch := time.NextEpoch(state)
if epoch > nextEpoch {
return fmt.Errorf("epoch %d can't be greater than next epoch %d", epoch, nextEpoch)
@@ -308,7 +308,7 @@ func ProposerAssignments(ctx context.Context, state state.BeaconState, epoch pri
defer span.End()
// Verify if the epoch is valid for assignment based on the provided state.
if err := verifyAssignmentEpoch(epoch, state); err != nil {
if err := VerifyAssignmentEpoch(epoch, state); err != nil {
return nil, err
}
startSlot, err := slots.EpochStart(epoch)
@@ -351,6 +351,61 @@ func ProposerAssignments(ctx context.Context, state state.BeaconState, epoch pri
return proposerAssignments, nil
}
// LiteAssignment is a lite version of CommitteeAssignment, and has committee length
// and validator committee index instead of the full committee list
type LiteAssignment struct {
AttesterSlot primitives.Slot // slot in which to attest
CommitteeIndex primitives.CommitteeIndex // position of the committee in the slot
CommitteeLength uint64 // number of members in the committee
ValidatorCommitteeIndex uint64 // validators offset inside the committee
}
// PrecomputeCommittees returns an array indexed by (slot-startSlot)
// whose elements are the beacon committees of that slot.
func PrecomputeCommittees(
ctx context.Context,
st state.BeaconState,
startSlot primitives.Slot,
) ([][][]primitives.ValidatorIndex, error) {
cfg := params.BeaconConfig()
out := make([][][]primitives.ValidatorIndex, cfg.SlotsPerEpoch)
for relativeSlot := primitives.Slot(0); relativeSlot < cfg.SlotsPerEpoch; relativeSlot++ {
slot := startSlot + relativeSlot
comms, err := BeaconCommittees(ctx, st, slot)
if err != nil {
return nil, errors.Wrapf(err, "BeaconCommittees failed at slot %d", slot)
}
out[relativeSlot] = comms
}
return out, nil
}
// AssignmentForValidator scans the cached committees once
// and returns the duty for a single validator.
func AssignmentForValidator(
bySlot [][][]primitives.ValidatorIndex,
startSlot primitives.Slot,
vIdx primitives.ValidatorIndex,
) *LiteAssignment {
for relativeSlot, committees := range bySlot {
for cIdx, committee := range committees {
for pos, member := range committee {
if member == vIdx {
return &LiteAssignment{
AttesterSlot: startSlot + primitives.Slot(relativeSlot),
CommitteeIndex: primitives.CommitteeIndex(cIdx),
CommitteeLength: uint64(len(committee)),
ValidatorCommitteeIndex: uint64(pos),
}
}
}
}
}
return nil // validator is not scheduled this epoch
}
// CommitteeAssignments calculates committee assignments for each validator during the specified epoch.
// It retrieves active validator indices, determines the number of committees per slot, and computes
// assignments for each validator based on their presence in the provided validators slice.
@@ -359,7 +414,7 @@ func CommitteeAssignments(ctx context.Context, state state.BeaconState, epoch pr
defer span.End()
// Verify if the epoch is valid for assignment based on the provided state.
if err := verifyAssignmentEpoch(epoch, state); err != nil {
if err := VerifyAssignmentEpoch(epoch, state); err != nil {
return nil, err
}
startSlot, err := slots.EpochStart(epoch)

View File

@@ -871,3 +871,48 @@ func TestBeaconCommitteesFromCache(t *testing.T) {
assert.DeepEqual(t, committees[idx], committee)
}
}
func TestPrecomputeCommittees_HappyPath(t *testing.T) {
cfg := params.BeaconConfig()
start := primitives.Slot(100)
ctx := context.Background()
st, _ := util.DeterministicGenesisState(t, 256)
got, err := helpers.PrecomputeCommittees(ctx, st, start)
require.NoError(t, err)
require.Equal(t, len(got), int(cfg.SlotsPerEpoch), "outer slice length mismatch")
for i := range got {
expSlot := start + primitives.Slot(i)
comms, err := helpers.BeaconCommittees(ctx, st, expSlot)
require.NoError(t, err)
require.DeepEqual(t, comms, got[i])
}
}
func TestAssignmentForValidator(t *testing.T) {
start := primitives.Slot(200)
bySlot := [][][]primitives.ValidatorIndex{
{{1, 2, 3}},
{{7, 8, 9}},
}
vIdx := primitives.ValidatorIndex(8)
got := helpers.AssignmentForValidator(bySlot, start, vIdx)
require.NotNil(t, got)
require.Equal(t, start+1, got.AttesterSlot)
require.Equal(t, primitives.CommitteeIndex(0), got.CommitteeIndex)
require.Equal(t, uint64(3), got.CommitteeLength)
require.Equal(t, uint64(1), got.ValidatorCommitteeIndex)
t.Run("Not Found", func(t *testing.T) {
start = primitives.Slot(300)
bySlot = [][][]primitives.ValidatorIndex{
{{4, 5, 6}},
}
got = helpers.AssignmentForValidator(bySlot, start, primitives.ValidatorIndex(99))
require.IsNil(t, got)
})
}

View File

@@ -8,6 +8,7 @@ go_library(
"blocks.go",
"construct_generic_block.go",
"duties.go",
"duties_v2.go",
"exit.go",
"log.go",
"proposer.go",
@@ -189,6 +190,7 @@ go_test(
"blocks_test.go",
"construct_generic_block_test.go",
"duties_test.go",
"duties_v2_test.go",
"exit_test.go",
"proposer_altair_test.go",
"proposer_attestations_electra_test.go",

View File

@@ -0,0 +1,268 @@
package validator
import (
"context"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
coreTime "github.com/OffchainLabs/prysm/v6/beacon-chain/core/time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
"github.com/OffchainLabs/prysm/v6/beacon-chain/rpc/core"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/time/slots"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
// GetDutiesV2 returns the duties assigned to a list of validators specified
// in the request object.
//
// Deprecated: The gRPC API will remain the default and fully supported through v8 (expected in 2026) but will be eventually removed in favor of REST API.
func (vs *Server) GetDutiesV2(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.DutiesV2Response, error) {
if vs.SyncChecker.Syncing() {
return nil, status.Error(codes.Unavailable, "Syncing to latest head, not ready to respond")
}
return vs.dutiesv2(ctx, req)
}
// Compute the validator duties from the head state's corresponding epoch
// for validators public key / indices requested.
func (vs *Server) dutiesv2(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.DutiesV2Response, error) {
currentEpoch := slots.ToEpoch(vs.TimeFetcher.CurrentSlot())
if req.Epoch > currentEpoch+1 {
return nil, status.Errorf(codes.Unavailable, "Request epoch %d can not be greater than next epoch %d", req.Epoch, currentEpoch+1)
}
// Load head state
s, err := vs.HeadFetcher.HeadState(ctx)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get head state: %v", err)
}
// Advance to start of requested epoch if necessary
s, err = vs.stateForEpoch(ctx, s, req.Epoch)
if err != nil {
return nil, err
}
// Build duties for each validator
ctx, span := trace.StartSpan(ctx, "dutiesv2.BuildResponse")
span.SetAttributes(trace.Int64Attribute("num_pubkeys", int64(len(req.PublicKeys))))
defer span.End()
// Load committee and proposer metadata
meta, err := loadDutiesMetadata(ctx, s, req.Epoch)
if err != nil {
return nil, err
}
validatorAssignments := make([]*ethpb.DutiesV2Response_Duty, 0, len(req.PublicKeys))
nextValidatorAssignments := make([]*ethpb.DutiesV2Response_Duty, 0, len(req.PublicKeys))
// start loop for assignments for current and next epochs
for _, pubKey := range req.PublicKeys {
if ctx.Err() != nil {
return nil, status.Errorf(codes.Aborted, "Could not continue fetching assignments: %v", ctx.Err())
}
idx, ok := s.ValidatorIndexByPubkey(bytesutil.ToBytes48(pubKey))
if !ok {
// Unknown validator: still append placeholder duty with UNKNOWN_STATUS
validatorAssignments = append(validatorAssignments, &ethpb.DutiesV2Response_Duty{
PublicKey: pubKey,
Status: ethpb.ValidatorStatus_UNKNOWN_STATUS,
})
nextValidatorAssignments = append(nextValidatorAssignments, &ethpb.DutiesV2Response_Duty{
PublicKey: pubKey,
Status: ethpb.ValidatorStatus_UNKNOWN_STATUS,
})
continue
}
meta.current.liteAssignment = helpers.AssignmentForValidator(meta.current.committeesBySlot, meta.current.startSlot, idx)
meta.next.liteAssignment = helpers.AssignmentForValidator(meta.next.committeesBySlot, meta.next.startSlot, idx)
assignment, nextAssignment, err := vs.buildValidatorDuty(pubKey, idx, s, req.Epoch, meta)
if err != nil {
return nil, err
}
validatorAssignments = append(validatorAssignments, assignment)
nextValidatorAssignments = append(nextValidatorAssignments, nextAssignment)
}
// Dependent roots for fork choice
currDependentRoot, err := vs.ForkchoiceFetcher.DependentRoot(currentEpoch)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get dependent root: %v", err)
}
prevDependentRoot := currDependentRoot
if currDependentRoot != [32]byte{} && currentEpoch > 0 {
prevDependentRoot, err = vs.ForkchoiceFetcher.DependentRoot(currentEpoch - 1)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get previous dependent root: %v", err)
}
}
return &ethpb.DutiesV2Response{
PreviousDutyDependentRoot: prevDependentRoot[:],
CurrentDutyDependentRoot: currDependentRoot[:],
CurrentEpochDuties: validatorAssignments,
NextEpochDuties: nextValidatorAssignments,
}, nil
}
// stateForEpoch returns a state advanced (with empty slot transitions) to the
// start slot of the requested epoch.
func (vs *Server) stateForEpoch(ctx context.Context, s state.BeaconState, reqEpoch primitives.Epoch) (state.BeaconState, error) {
epochStartSlot, err := slots.EpochStart(reqEpoch)
if err != nil {
return nil, err
}
if s.Slot() >= epochStartSlot {
return s, nil
}
headRoot, err := vs.HeadFetcher.HeadRoot(ctx)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not retrieve head root: %v", err)
}
s, err = transition.ProcessSlotsUsingNextSlotCache(ctx, s, headRoot, epochStartSlot)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not process slots up to %d: %v", epochStartSlot, err)
}
return s, nil
}
// dutiesMetadata bundles together related data needed for duty
// construction.
type dutiesMetadata struct {
current *metadata
next *metadata
}
type metadata struct {
committeesAtSlot uint64
proposalSlots map[primitives.ValidatorIndex][]primitives.Slot
startSlot primitives.Slot
committeesBySlot [][][]primitives.ValidatorIndex
liteAssignment *helpers.LiteAssignment
}
func loadDutiesMetadata(ctx context.Context, s state.BeaconState, reqEpoch primitives.Epoch) (*dutiesMetadata, error) {
meta := &dutiesMetadata{}
var err error
meta.current, err = loadMetadata(ctx, s, reqEpoch)
if err != nil {
return nil, err
}
// note: we only set the proposer slots for the current assignment and not the next epoch assignment
meta.current.proposalSlots, err = helpers.ProposerAssignments(ctx, s, reqEpoch)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not compute proposer slots: %v", err)
}
meta.next, err = loadMetadata(ctx, s, reqEpoch+1)
if err != nil {
return nil, err
}
return meta, nil
}
func loadMetadata(ctx context.Context, s state.BeaconState, reqEpoch primitives.Epoch) (*metadata, error) {
meta := &metadata{}
if err := helpers.VerifyAssignmentEpoch(reqEpoch, s); err != nil {
return nil, err
}
activeValidatorCount, err := helpers.ActiveValidatorCount(ctx, s, reqEpoch)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not get active validator count: %v", err)
}
meta.committeesAtSlot = helpers.SlotCommitteeCount(activeValidatorCount)
meta.startSlot, err = slots.EpochStart(reqEpoch)
if err != nil {
return nil, err
}
meta.committeesBySlot, err = helpers.PrecomputeCommittees(ctx, s, meta.startSlot)
if err != nil {
return nil, err
}
return meta, nil
}
// buildValidatorDuty builds both currentepoch and nextepoch V2 duty objects
// for a single validator index.
func (vs *Server) buildValidatorDuty(
pubKey []byte,
idx primitives.ValidatorIndex,
s state.BeaconState,
reqEpoch primitives.Epoch,
meta *dutiesMetadata,
) (*ethpb.DutiesV2Response_Duty, *ethpb.DutiesV2Response_Duty, error) {
assignment := &ethpb.DutiesV2Response_Duty{PublicKey: pubKey}
nextAssignment := &ethpb.DutiesV2Response_Duty{PublicKey: pubKey}
statusEnum := assignmentStatus(s, idx)
assignment.ValidatorIndex = idx
assignment.Status = statusEnum
assignment.CommitteesAtSlot = meta.current.committeesAtSlot
assignment.ProposerSlots = meta.current.proposalSlots[idx]
populateCommitteeFields(assignment, meta.current.liteAssignment)
nextAssignment.ValidatorIndex = idx
nextAssignment.Status = statusEnum
nextAssignment.CommitteesAtSlot = meta.next.committeesAtSlot
populateCommitteeFields(nextAssignment, meta.next.liteAssignment)
// Sync committee flags
if coreTime.HigherEqualThanAltairVersionAndEpoch(s, reqEpoch) {
inSync, err := helpers.IsCurrentPeriodSyncCommittee(s, idx)
if err != nil {
return nil, nil, status.Errorf(codes.Internal, "Could not determine current epoch sync committee: %v", err)
}
assignment.IsSyncCommittee = inSync
nextAssignment.IsSyncCommittee = inSync
if inSync {
if err := core.RegisterSyncSubnetCurrentPeriodProto(s, reqEpoch, pubKey, statusEnum); err != nil {
return nil, nil, status.Errorf(codes.Internal, "Could not register sync subnet current period: %v", err)
}
}
// Next epoch sync committee duty is assigned with next period sync committee only during
// sync period epoch boundary (ie. EPOCHS_PER_SYNC_COMMITTEE_PERIOD - 1). Else wise
// next epoch sync committee duty is the same as current epoch.
nextEpoch := reqEpoch + 1
currentEpoch := coreTime.CurrentEpoch(s)
n := slots.SyncCommitteePeriod(nextEpoch)
c := slots.SyncCommitteePeriod(currentEpoch)
if n > c {
nextInSync, err := helpers.IsNextPeriodSyncCommittee(s, idx)
if err != nil {
return nil, nil, status.Errorf(codes.Internal, "Could not determine next epoch sync committee: %v", err)
}
nextAssignment.IsSyncCommittee = nextInSync
if nextInSync {
go func() {
if err := core.RegisterSyncSubnetNextPeriodProto(s, reqEpoch, pubKey, statusEnum); err != nil {
log.WithError(err).Warn("Could not register sync subnet next period")
}
}()
}
}
}
return assignment, nextAssignment, nil
}
func populateCommitteeFields(duty *ethpb.DutiesV2Response_Duty, la *helpers.LiteAssignment) {
duty.CommitteeLength = la.CommitteeLength
duty.CommitteeIndex = la.CommitteeIndex
duty.ValidatorCommitteeIndex = la.ValidatorCommitteeIndex
duty.AttesterSlot = la.AttesterSlot
}

View File

@@ -0,0 +1,562 @@
package validator
import (
"context"
"testing"
"time"
mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache/depositsnapshot"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/altair"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/execution"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
mockExecution "github.com/OffchainLabs/prysm/v6/beacon-chain/execution/testing"
mockSync "github.com/OffchainLabs/prysm/v6/beacon-chain/sync/initial-sync/testing"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/testing/assert"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/OffchainLabs/prysm/v6/testing/util"
"github.com/OffchainLabs/prysm/v6/time/slots"
)
func TestGetDutiesV2_OK(t *testing.T) {
genesis := util.NewBeaconBlock()
depChainStart := params.BeaconConfig().MinGenesisActiveValidatorCount
deposits, _, err := util.DeterministicDepositsAndKeys(depChainStart)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
bs, err := transition.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
require.NoError(t, err, "Could not setup genesis bs")
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
pubKeys := make([][]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = deposits[i].Data.PublicKey
indices[i] = uint64(i)
}
chain := &mockChain.ChainService{
State: bs, Root: genesisRoot[:], Genesis: time.Now(),
}
vs := &Server{
HeadFetcher: chain,
TimeFetcher: chain,
ForkchoiceFetcher: chain,
SyncChecker: &mockSync.Sync{IsSyncing: false},
PayloadIDCache: cache.NewPayloadIDCache(),
}
// Test the first validator in registry.
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[0].Data.PublicKey},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
if res.CurrentEpochDuties[0].AttesterSlot > bs.Slot()+params.BeaconConfig().SlotsPerEpoch {
t.Errorf("Assigned slot %d can't be higher than %d",
res.CurrentEpochDuties[0].AttesterSlot, bs.Slot()+params.BeaconConfig().SlotsPerEpoch)
}
// Test the last validator in registry.
lastValidatorIndex := depChainStart - 1
req = &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[lastValidatorIndex].Data.PublicKey},
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
if res.CurrentEpochDuties[0].AttesterSlot > bs.Slot()+params.BeaconConfig().SlotsPerEpoch {
t.Errorf("Assigned slot %d can't be higher than %d",
res.CurrentEpochDuties[0].AttesterSlot, bs.Slot()+params.BeaconConfig().SlotsPerEpoch)
}
// We request for duties for all validators.
req = &ethpb.DutiesRequest{
PublicKeys: pubKeys,
Epoch: 0,
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
for i := 0; i < len(res.CurrentEpochDuties); i++ {
assert.Equal(t, primitives.ValidatorIndex(i), res.CurrentEpochDuties[i].ValidatorIndex)
}
}
func TestGetAltairDutiesV2_SyncCommitteeOK(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = primitives.Epoch(0)
params.OverrideBeaconConfig(cfg)
genesis := util.NewBeaconBlock()
deposits, _, err := util.DeterministicDepositsAndKeys(params.BeaconConfig().SyncCommitteeSize)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
bs, err := util.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
require.NoError(t, err, "Could not setup genesis bs")
h := &ethpb.BeaconBlockHeader{
StateRoot: bytesutil.PadTo([]byte{'a'}, fieldparams.RootLength),
ParentRoot: bytesutil.PadTo([]byte{'b'}, fieldparams.RootLength),
BodyRoot: bytesutil.PadTo([]byte{'c'}, fieldparams.RootLength),
}
require.NoError(t, bs.SetLatestBlockHeader(h))
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
syncCommittee, err := altair.NextSyncCommittee(context.Background(), bs)
require.NoError(t, err)
require.NoError(t, bs.SetCurrentSyncCommittee(syncCommittee))
pubKeys := make([][]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = deposits[i].Data.PublicKey
indices[i] = uint64(i)
}
require.NoError(t, bs.SetSlot(params.BeaconConfig().SlotsPerEpoch*primitives.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)-1))
require.NoError(t, helpers.UpdateSyncCommitteeCache(bs))
pubkeysAs48ByteType := make([][fieldparams.BLSPubkeyLength]byte, len(pubKeys))
for i, pk := range pubKeys {
pubkeysAs48ByteType[i] = bytesutil.ToBytes48(pk)
}
slot := uint64(params.BeaconConfig().SlotsPerEpoch) * uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod) * params.BeaconConfig().SecondsPerSlot
chain := &mockChain.ChainService{
State: bs, Root: genesisRoot[:], Genesis: time.Now().Add(time.Duration(-1*int64(slot-1)) * time.Second),
}
vs := &Server{
HeadFetcher: chain,
TimeFetcher: chain,
ForkchoiceFetcher: chain,
Eth1InfoFetcher: &mockExecution.Chain{},
SyncChecker: &mockSync.Sync{IsSyncing: false},
PayloadIDCache: cache.NewPayloadIDCache(),
}
// Test the first validator in registry.
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[0].Data.PublicKey},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
if res.CurrentEpochDuties[0].AttesterSlot > bs.Slot()+params.BeaconConfig().SlotsPerEpoch {
t.Errorf("Assigned slot %d can't be higher than %d",
res.CurrentEpochDuties[0].AttesterSlot, bs.Slot()+params.BeaconConfig().SlotsPerEpoch)
}
// Test the last validator in registry.
lastValidatorIndex := params.BeaconConfig().SyncCommitteeSize - 1
req = &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[lastValidatorIndex].Data.PublicKey},
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
if res.CurrentEpochDuties[0].AttesterSlot > bs.Slot()+params.BeaconConfig().SlotsPerEpoch {
t.Errorf("Assigned slot %d can't be higher than %d",
res.CurrentEpochDuties[0].AttesterSlot, bs.Slot()+params.BeaconConfig().SlotsPerEpoch)
}
// We request for duties for all validators.
req = &ethpb.DutiesRequest{
PublicKeys: pubKeys,
Epoch: 0,
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
for i := 0; i < len(res.CurrentEpochDuties); i++ {
require.Equal(t, primitives.ValidatorIndex(i), res.CurrentEpochDuties[i].ValidatorIndex)
}
for i := 0; i < len(res.CurrentEpochDuties); i++ {
require.Equal(t, true, res.CurrentEpochDuties[i].IsSyncCommittee)
// Current epoch and next epoch duties should be equal before the sync period epoch boundary.
require.Equal(t, res.CurrentEpochDuties[i].IsSyncCommittee, res.NextEpochDuties[i].IsSyncCommittee)
}
// Current epoch and next epoch duties should not be equal at the sync period epoch boundary.
req = &ethpb.DutiesRequest{
PublicKeys: pubKeys,
Epoch: params.BeaconConfig().EpochsPerSyncCommitteePeriod - 1,
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
for i := 0; i < len(res.CurrentEpochDuties); i++ {
require.NotEqual(t, res.CurrentEpochDuties[i].IsSyncCommittee, res.NextEpochDuties[i].IsSyncCommittee)
}
}
func TestGetBellatrixDutiesV2_SyncCommitteeOK(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = primitives.Epoch(0)
cfg.BellatrixForkEpoch = primitives.Epoch(1)
params.OverrideBeaconConfig(cfg)
genesis := util.NewBeaconBlock()
deposits, _, err := util.DeterministicDepositsAndKeys(params.BeaconConfig().SyncCommitteeSize)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
bs, err := util.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
h := &ethpb.BeaconBlockHeader{
StateRoot: bytesutil.PadTo([]byte{'a'}, fieldparams.RootLength),
ParentRoot: bytesutil.PadTo([]byte{'b'}, fieldparams.RootLength),
BodyRoot: bytesutil.PadTo([]byte{'c'}, fieldparams.RootLength),
}
require.NoError(t, bs.SetLatestBlockHeader(h))
require.NoError(t, err, "Could not setup genesis bs")
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
syncCommittee, err := altair.NextSyncCommittee(context.Background(), bs)
require.NoError(t, err)
require.NoError(t, bs.SetCurrentSyncCommittee(syncCommittee))
pubKeys := make([][]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = deposits[i].Data.PublicKey
indices[i] = uint64(i)
}
require.NoError(t, bs.SetSlot(params.BeaconConfig().SlotsPerEpoch*primitives.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)-1))
require.NoError(t, helpers.UpdateSyncCommitteeCache(bs))
bs, err = execution.UpgradeToBellatrix(bs)
require.NoError(t, err)
pubkeysAs48ByteType := make([][fieldparams.BLSPubkeyLength]byte, len(pubKeys))
for i, pk := range pubKeys {
pubkeysAs48ByteType[i] = bytesutil.ToBytes48(pk)
}
slot := uint64(params.BeaconConfig().SlotsPerEpoch) * uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod) * params.BeaconConfig().SecondsPerSlot
chain := &mockChain.ChainService{
State: bs, Root: genesisRoot[:], Genesis: time.Now().Add(time.Duration(-1*int64(slot-1)) * time.Second),
}
vs := &Server{
HeadFetcher: chain,
TimeFetcher: chain,
ForkchoiceFetcher: chain,
Eth1InfoFetcher: &mockExecution.Chain{},
SyncChecker: &mockSync.Sync{IsSyncing: false},
PayloadIDCache: cache.NewPayloadIDCache(),
}
// Test the first validator in registry.
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[0].Data.PublicKey},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
if res.CurrentEpochDuties[0].AttesterSlot > bs.Slot()+params.BeaconConfig().SlotsPerEpoch {
t.Errorf("Assigned slot %d can't be higher than %d",
res.CurrentEpochDuties[0].AttesterSlot, bs.Slot()+params.BeaconConfig().SlotsPerEpoch)
}
// Test the last validator in registry.
lastValidatorIndex := params.BeaconConfig().SyncCommitteeSize - 1
req = &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[lastValidatorIndex].Data.PublicKey},
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
if res.CurrentEpochDuties[0].AttesterSlot > bs.Slot()+params.BeaconConfig().SlotsPerEpoch {
t.Errorf("Assigned slot %d can't be higher than %d",
res.CurrentEpochDuties[0].AttesterSlot, bs.Slot()+params.BeaconConfig().SlotsPerEpoch)
}
// We request for duties for all validators.
req = &ethpb.DutiesRequest{
PublicKeys: pubKeys,
Epoch: 0,
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
for i := 0; i < len(res.CurrentEpochDuties); i++ {
assert.Equal(t, primitives.ValidatorIndex(i), res.CurrentEpochDuties[i].ValidatorIndex)
}
for i := 0; i < len(res.CurrentEpochDuties); i++ {
assert.Equal(t, true, res.CurrentEpochDuties[i].IsSyncCommittee)
// Current epoch and next epoch duties should be equal before the sync period epoch boundary.
assert.Equal(t, res.CurrentEpochDuties[i].IsSyncCommittee, res.NextEpochDuties[i].IsSyncCommittee)
}
// Current epoch and next epoch duties should not be equal at the sync period epoch boundary.
req = &ethpb.DutiesRequest{
PublicKeys: pubKeys,
Epoch: params.BeaconConfig().EpochsPerSyncCommitteePeriod - 1,
}
res, err = vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
for i := 0; i < len(res.CurrentEpochDuties); i++ {
require.NotEqual(t, res.CurrentEpochDuties[i].IsSyncCommittee, res.NextEpochDuties[i].IsSyncCommittee)
}
}
func TestGetAltairDutiesV2_UnknownPubkey(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = primitives.Epoch(0)
params.OverrideBeaconConfig(cfg)
genesis := util.NewBeaconBlock()
deposits, _, err := util.DeterministicDepositsAndKeys(params.BeaconConfig().SyncCommitteeSize)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
bs, err := util.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
require.NoError(t, err)
h := &ethpb.BeaconBlockHeader{
StateRoot: bytesutil.PadTo([]byte{'a'}, fieldparams.RootLength),
ParentRoot: bytesutil.PadTo([]byte{'b'}, fieldparams.RootLength),
BodyRoot: bytesutil.PadTo([]byte{'c'}, fieldparams.RootLength),
}
require.NoError(t, bs.SetLatestBlockHeader(h))
require.NoError(t, err, "Could not setup genesis bs")
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
require.NoError(t, bs.SetSlot(params.BeaconConfig().SlotsPerEpoch*primitives.Slot(params.BeaconConfig().EpochsPerSyncCommitteePeriod)-1))
require.NoError(t, helpers.UpdateSyncCommitteeCache(bs))
slot := uint64(params.BeaconConfig().SlotsPerEpoch) * uint64(params.BeaconConfig().EpochsPerSyncCommitteePeriod) * params.BeaconConfig().SecondsPerSlot
chain := &mockChain.ChainService{
State: bs, Root: genesisRoot[:], Genesis: time.Now().Add(time.Duration(-1*int64(slot-1)) * time.Second),
}
depositCache, err := depositsnapshot.New()
require.NoError(t, err)
vs := &Server{
HeadFetcher: chain,
ForkchoiceFetcher: chain,
TimeFetcher: chain,
Eth1InfoFetcher: &mockExecution.Chain{},
SyncChecker: &mockSync.Sync{IsSyncing: false},
DepositFetcher: depositCache,
PayloadIDCache: cache.NewPayloadIDCache(),
}
unknownPubkey := bytesutil.PadTo([]byte{'u'}, 48)
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{unknownPubkey},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err)
assert.Equal(t, false, res.CurrentEpochDuties[0].IsSyncCommittee)
assert.Equal(t, false, res.NextEpochDuties[0].IsSyncCommittee)
}
func TestGetDutiesV2_StateAdvancement(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.ElectraForkEpoch = primitives.Epoch(0)
params.OverrideBeaconConfig(cfg)
epochStart, err := slots.EpochStart(1)
require.NoError(t, err)
st, _ := util.DeterministicGenesisStateElectra(t, 1)
require.NoError(t, st.SetSlot(epochStart-1))
// Request epoch 1 which requires slot 32 processing
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{pubKey(0)},
Epoch: 1,
}
b, err := blocks.NewSignedBeaconBlock(util.HydrateSignedBeaconBlockElectra(&ethpb.SignedBeaconBlockElectra{}))
require.NoError(t, err)
b.SetSlot(epochStart)
currentSlot := epochStart - 1
// Mock chain service with state at slot 0
chain := &mockChain.ChainService{
Root: make([]byte, 32),
State: st,
Block: b,
Slot: &currentSlot,
}
vs := &Server{
HeadFetcher: chain,
TimeFetcher: chain,
ForkchoiceFetcher: chain,
SyncChecker: &mockSync.Sync{IsSyncing: false},
}
// Verify state processing occurs
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err)
require.NotNil(t, res)
}
func TestGetDutiesV2_SlotOutOfUpperBound(t *testing.T) {
chain := &mockChain.ChainService{
Genesis: time.Now(),
}
vs := &Server{
ForkchoiceFetcher: chain,
TimeFetcher: chain,
SyncChecker: &mockSync.Sync{IsSyncing: false},
}
req := &ethpb.DutiesRequest{
Epoch: primitives.Epoch(chain.CurrentSlot()/params.BeaconConfig().SlotsPerEpoch + 2),
}
_, err := vs.GetDutiesV2(context.Background(), req)
require.ErrorContains(t, "can not be greater than next epoch", err)
}
func TestGetDutiesV2_CurrentEpoch_ShouldNotFail(t *testing.T) {
genesis := util.NewBeaconBlock()
depChainStart := params.BeaconConfig().MinGenesisActiveValidatorCount
deposits, _, err := util.DeterministicDepositsAndKeys(depChainStart)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
bState, err := transition.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
require.NoError(t, err, "Could not setup genesis state")
// Set state to non-epoch start slot.
require.NoError(t, bState.SetSlot(5))
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = bytesutil.ToBytes48(deposits[i].Data.PublicKey)
indices[i] = uint64(i)
}
chain := &mockChain.ChainService{
State: bState, Root: genesisRoot[:], Genesis: time.Now(),
}
vs := &Server{
HeadFetcher: chain,
ForkchoiceFetcher: chain,
TimeFetcher: chain,
SyncChecker: &mockSync.Sync{IsSyncing: false},
PayloadIDCache: cache.NewPayloadIDCache(),
}
// Test the first validator in registry.
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{deposits[0].Data.PublicKey},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err)
assert.Equal(t, 1, len(res.CurrentEpochDuties), "Expected 1 assignment")
}
func TestGetDutiesV2_MultipleKeys_OK(t *testing.T) {
genesis := util.NewBeaconBlock()
depChainStart := uint64(64)
deposits, _, err := util.DeterministicDepositsAndKeys(depChainStart)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
bs, err := transition.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
require.NoError(t, err, "Could not setup genesis bs")
genesisRoot, err := genesis.Block.HashTreeRoot()
require.NoError(t, err, "Could not get signing root")
pubKeys := make([][fieldparams.BLSPubkeyLength]byte, len(deposits))
indices := make([]uint64, len(deposits))
for i := 0; i < len(deposits); i++ {
pubKeys[i] = bytesutil.ToBytes48(deposits[i].Data.PublicKey)
indices[i] = uint64(i)
}
chain := &mockChain.ChainService{
State: bs, Root: genesisRoot[:], Genesis: time.Now(),
}
vs := &Server{
HeadFetcher: chain,
ForkchoiceFetcher: chain,
TimeFetcher: chain,
SyncChecker: &mockSync.Sync{IsSyncing: false},
PayloadIDCache: cache.NewPayloadIDCache(),
}
pubkey0 := deposits[0].Data.PublicKey
pubkey1 := deposits[1].Data.PublicKey
// Test the first validator in registry.
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{pubkey0, pubkey1},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err, "Could not call epoch committee assignment")
assert.Equal(t, 2, len(res.CurrentEpochDuties))
assert.Equal(t, primitives.Slot(4), res.CurrentEpochDuties[0].AttesterSlot)
assert.Equal(t, primitives.Slot(4), res.CurrentEpochDuties[1].AttesterSlot)
}
func TestGetDutiesV2_NextSyncCommitteePeriod(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.AltairForkEpoch = primitives.Epoch(0)
cfg.EpochsPerSyncCommitteePeriod = 1
params.OverrideBeaconConfig(cfg)
// Configure sync committee period boundary
epochsPerPeriod := params.BeaconConfig().EpochsPerSyncCommitteePeriod
boundaryEpoch := epochsPerPeriod - 1
// Create state at last epoch of current period
deposits, _, err := util.DeterministicDepositsAndKeys(params.BeaconConfig().SyncCommitteeSize)
require.NoError(t, err)
eth1Data, err := util.DeterministicEth1Data(len(deposits))
require.NoError(t, err)
st, err := util.GenesisBeaconState(context.Background(), deposits, 0, eth1Data)
require.NoError(t, err)
syncCommittee, err := altair.NextSyncCommittee(context.Background(), st)
require.NoError(t, err)
require.NoError(t, st.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, st.SetSlot(params.BeaconConfig().SlotsPerEpoch*primitives.Slot(boundaryEpoch)))
validatorPubkey := deposits[0].Data.PublicKey
// Request duties for boundary epoch + 1
req := &ethpb.DutiesRequest{
PublicKeys: [][]byte{validatorPubkey},
Epoch: boundaryEpoch + 1,
}
genesisRoot := [32]byte{}
chain := &mockChain.ChainService{
State: st,
Root: genesisRoot[:],
}
vs := &Server{
HeadFetcher: chain,
TimeFetcher: chain,
ForkchoiceFetcher: chain,
SyncChecker: &mockSync.Sync{IsSyncing: false},
}
res, err := vs.GetDutiesV2(context.Background(), req)
require.NoError(t, err)
//Verify next epoch duties have updated sync committee status
require.NotEqual(t,
res.CurrentEpochDuties[0].IsSyncCommittee,
res.NextEpochDuties[0].IsSyncCommittee,
)
}
func TestGetDutiesV2_SyncNotReady(t *testing.T) {
vs := &Server{
SyncChecker: &mockSync.Sync{IsSyncing: true},
}
_, err := vs.GetDutiesV2(context.Background(), &ethpb.DutiesRequest{})
assert.ErrorContains(t, "Syncing to latest head", err)
}

View File

@@ -0,0 +1,3 @@
### Added
- GetDutiesV2 gRPC function, removes committee list from duties, replaced with committee length, validator committee index.

File diff suppressed because it is too large Load Diff

View File

@@ -58,6 +58,30 @@ service BeaconNodeValidator {
};
}
// GetDutiesV2 Retrieves validator duties for the requested validators.
//
// The duties consist of:
// Proposer - the validator that creates a beacon chain block.
// Attester — a validator that is part of a committee that needs to sign off
// on a beacon chain
// Sync Committee - a special group of validators that ensure lightclients and effectively sync with Ethereum
// and continues in this role for 256 epochs if selected
// The server returns a list of duties which are the actions should be
// performed by validators for a given epoch. Validator duties should be
// polled every epoch, but due to chain reorg of >MIN_SEED_LOOKAHEAD could
// occur, the validator duties could chain. For complete safety, it is
// recommended to poll at every slot to ensure validator is fully aware of any
// sudden chain reorg.
//
// GetDutiesV2 differs from GetDuties through the removal of the full committee list in favor of adding
// committee_length and validator_committee_index fields which significantly saves on marshalling
rpc GetDutiesV2(DutiesRequest) returns (DutiesV2Response) {
option deprecated = true;
option (google.api.http) = {
get : "/eth/v1alpha1/validator/dutiesV2"
};
}
// DomainData fetches the current BLS signature domain version information
// from the running beacon node's state. This information is used when
// validators sign blocks and attestations appropriately based on their duty.
@@ -741,6 +765,63 @@ message DutiesResponse {
}
}
message DutiesV2Response {
option deprecated = true;
repeated Duty current_epoch_duties = 1;
repeated Duty next_epoch_duties = 2;
bytes previous_duty_dependent_root = 3 [ (ethereum.eth.ext.ssz_size) = "32" ];
bytes current_duty_dependent_root = 4 [ (ethereum.eth.ext.ssz_size) = "32" ];
message Duty {
// The length of the committee
uint64 committee_length = 1;
// The index of the committee which the given validator has an assignment
uint64 committee_index = 2
[ (ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/"
"primitives.CommitteeIndex" ];
// The number of committees in the duty's slot.
uint64 committees_at_slot = 3;
// The validator's index in the given committee
uint64 validator_committee_index = 4;
// Slot at which a validator must attest.
uint64 attester_slot = 5 [
(ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"
];
// Slots at which a validator must propose a beacon chain block.
repeated uint64 proposer_slots = 6 [
(ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives.Slot"
];
// 48 byte BLS public key for the validator who's assigned to perform a
// duty.
bytes public_key = 7 [ (ethereum.eth.ext.ssz_size) = "48" ];
// The current status of the validator assigned to perform the duty.
ValidatorStatus status = 8;
// The index of the validator in the beacon state.
uint64 validator_index = 9
[ (ethereum.eth.ext.cast_type) =
"github.com/OffchainLabs/prysm/v6/consensus-types/"
"primitives.ValidatorIndex" ];
// Whether the validator belongs in the sync committee and has to perform
// sync committee duty.
bool is_sync_committee = 10;
}
}
message BlockRequest {
option deprecated = true;

View File

@@ -24,7 +24,6 @@ import (
type MockBeaconNodeValidatorClient struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidatorClientMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidatorClientMockRecorder is the mock recorder for MockBeaconNodeValidatorClient.
@@ -45,10 +44,10 @@ func (m *MockBeaconNodeValidatorClient) EXPECT() *MockBeaconNodeValidatorClientM
}
// AggregatedSigAndAggregationBits mocks base method.
func (m *MockBeaconNodeValidatorClient) AggregatedSigAndAggregationBits(ctx context.Context, in *eth.AggregatedSigAndAggregationBitsRequest, opts ...grpc.CallOption) (*eth.AggregatedSigAndAggregationBitsResponse, error) {
func (m *MockBeaconNodeValidatorClient) AggregatedSigAndAggregationBits(arg0 context.Context, arg1 *eth.AggregatedSigAndAggregationBitsRequest, arg2 ...grpc.CallOption) (*eth.AggregatedSigAndAggregationBitsResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "AggregatedSigAndAggregationBits", varargs...)
@@ -58,17 +57,17 @@ func (m *MockBeaconNodeValidatorClient) AggregatedSigAndAggregationBits(ctx cont
}
// AggregatedSigAndAggregationBits indicates an expected call of AggregatedSigAndAggregationBits.
func (mr *MockBeaconNodeValidatorClientMockRecorder) AggregatedSigAndAggregationBits(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) AggregatedSigAndAggregationBits(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AggregatedSigAndAggregationBits", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).AggregatedSigAndAggregationBits), varargs...)
}
// AssignValidatorToSubnet mocks base method.
func (m *MockBeaconNodeValidatorClient) AssignValidatorToSubnet(ctx context.Context, in *eth.AssignValidatorToSubnetRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) {
func (m *MockBeaconNodeValidatorClient) AssignValidatorToSubnet(arg0 context.Context, arg1 *eth.AssignValidatorToSubnetRequest, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "AssignValidatorToSubnet", varargs...)
@@ -78,17 +77,17 @@ func (m *MockBeaconNodeValidatorClient) AssignValidatorToSubnet(ctx context.Cont
}
// AssignValidatorToSubnet indicates an expected call of AssignValidatorToSubnet.
func (mr *MockBeaconNodeValidatorClientMockRecorder) AssignValidatorToSubnet(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) AssignValidatorToSubnet(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AssignValidatorToSubnet", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).AssignValidatorToSubnet), varargs...)
}
// CheckDoppelGanger mocks base method.
func (m *MockBeaconNodeValidatorClient) CheckDoppelGanger(ctx context.Context, in *eth.DoppelGangerRequest, opts ...grpc.CallOption) (*eth.DoppelGangerResponse, error) {
func (m *MockBeaconNodeValidatorClient) CheckDoppelGanger(arg0 context.Context, arg1 *eth.DoppelGangerRequest, arg2 ...grpc.CallOption) (*eth.DoppelGangerResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "CheckDoppelGanger", varargs...)
@@ -98,17 +97,17 @@ func (m *MockBeaconNodeValidatorClient) CheckDoppelGanger(ctx context.Context, i
}
// CheckDoppelGanger indicates an expected call of CheckDoppelGanger.
func (mr *MockBeaconNodeValidatorClientMockRecorder) CheckDoppelGanger(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) CheckDoppelGanger(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CheckDoppelGanger", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).CheckDoppelGanger), varargs...)
}
// DomainData mocks base method.
func (m *MockBeaconNodeValidatorClient) DomainData(ctx context.Context, in *eth.DomainRequest, opts ...grpc.CallOption) (*eth.DomainResponse, error) {
func (m *MockBeaconNodeValidatorClient) DomainData(arg0 context.Context, arg1 *eth.DomainRequest, arg2 ...grpc.CallOption) (*eth.DomainResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "DomainData", varargs...)
@@ -118,17 +117,17 @@ func (m *MockBeaconNodeValidatorClient) DomainData(ctx context.Context, in *eth.
}
// DomainData indicates an expected call of DomainData.
func (mr *MockBeaconNodeValidatorClientMockRecorder) DomainData(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) DomainData(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DomainData", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).DomainData), varargs...)
}
// GetAttestationData mocks base method.
func (m *MockBeaconNodeValidatorClient) GetAttestationData(ctx context.Context, in *eth.AttestationDataRequest, opts ...grpc.CallOption) (*eth.AttestationData, error) {
func (m *MockBeaconNodeValidatorClient) GetAttestationData(arg0 context.Context, arg1 *eth.AttestationDataRequest, arg2 ...grpc.CallOption) (*eth.AttestationData, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetAttestationData", varargs...)
@@ -138,17 +137,17 @@ func (m *MockBeaconNodeValidatorClient) GetAttestationData(ctx context.Context,
}
// GetAttestationData indicates an expected call of GetAttestationData.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetAttestationData(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetAttestationData(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetAttestationData", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetAttestationData), varargs...)
}
// GetBeaconBlock mocks base method.
func (m *MockBeaconNodeValidatorClient) GetBeaconBlock(ctx context.Context, in *eth.BlockRequest, opts ...grpc.CallOption) (*eth.GenericBeaconBlock, error) {
func (m *MockBeaconNodeValidatorClient) GetBeaconBlock(arg0 context.Context, arg1 *eth.BlockRequest, arg2 ...grpc.CallOption) (*eth.GenericBeaconBlock, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetBeaconBlock", varargs...)
@@ -158,17 +157,17 @@ func (m *MockBeaconNodeValidatorClient) GetBeaconBlock(ctx context.Context, in *
}
// GetBeaconBlock indicates an expected call of GetBeaconBlock.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetBeaconBlock(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetBeaconBlock(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetBeaconBlock", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetBeaconBlock), varargs...)
}
// GetDuties mocks base method.
func (m *MockBeaconNodeValidatorClient) GetDuties(ctx context.Context, in *eth.DutiesRequest, opts ...grpc.CallOption) (*eth.DutiesResponse, error) {
func (m *MockBeaconNodeValidatorClient) GetDuties(arg0 context.Context, arg1 *eth.DutiesRequest, arg2 ...grpc.CallOption) (*eth.DutiesResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetDuties", varargs...)
@@ -178,17 +177,37 @@ func (m *MockBeaconNodeValidatorClient) GetDuties(ctx context.Context, in *eth.D
}
// GetDuties indicates an expected call of GetDuties.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetDuties(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetDuties(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetDuties", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetDuties), varargs...)
}
// GetFeeRecipientByPubKey mocks base method.
func (m *MockBeaconNodeValidatorClient) GetFeeRecipientByPubKey(ctx context.Context, in *eth.FeeRecipientByPubKeyRequest, opts ...grpc.CallOption) (*eth.FeeRecipientByPubKeyResponse, error) {
// GetDutiesV2 mocks base method.
func (m *MockBeaconNodeValidatorClient) GetDutiesV2(arg0 context.Context, arg1 *eth.DutiesRequest, arg2 ...grpc.CallOption) (*eth.DutiesV2Response, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetDutiesV2", varargs...)
ret0, _ := ret[0].(*eth.DutiesV2Response)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// GetDutiesV2 indicates an expected call of GetDutiesV2.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetDutiesV2(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, "GetDutiesV2", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetDutiesV2), varargs...)
}
// GetFeeRecipientByPubKey mocks base method.
func (m *MockBeaconNodeValidatorClient) GetFeeRecipientByPubKey(arg0 context.Context, arg1 *eth.FeeRecipientByPubKeyRequest, arg2 ...grpc.CallOption) (*eth.FeeRecipientByPubKeyResponse, error) {
m.ctrl.T.Helper()
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetFeeRecipientByPubKey", varargs...)
@@ -198,17 +217,17 @@ func (m *MockBeaconNodeValidatorClient) GetFeeRecipientByPubKey(ctx context.Cont
}
// GetFeeRecipientByPubKey indicates an expected call of GetFeeRecipientByPubKey.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetFeeRecipientByPubKey(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetFeeRecipientByPubKey(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetFeeRecipientByPubKey", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetFeeRecipientByPubKey), varargs...)
}
// GetSyncCommitteeContribution mocks base method.
func (m *MockBeaconNodeValidatorClient) GetSyncCommitteeContribution(ctx context.Context, in *eth.SyncCommitteeContributionRequest, opts ...grpc.CallOption) (*eth.SyncCommitteeContribution, error) {
func (m *MockBeaconNodeValidatorClient) GetSyncCommitteeContribution(arg0 context.Context, arg1 *eth.SyncCommitteeContributionRequest, arg2 ...grpc.CallOption) (*eth.SyncCommitteeContribution, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetSyncCommitteeContribution", varargs...)
@@ -218,17 +237,17 @@ func (m *MockBeaconNodeValidatorClient) GetSyncCommitteeContribution(ctx context
}
// GetSyncCommitteeContribution indicates an expected call of GetSyncCommitteeContribution.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetSyncCommitteeContribution(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetSyncCommitteeContribution(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSyncCommitteeContribution", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetSyncCommitteeContribution), varargs...)
}
// GetSyncMessageBlockRoot mocks base method.
func (m *MockBeaconNodeValidatorClient) GetSyncMessageBlockRoot(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*eth.SyncMessageBlockRootResponse, error) {
func (m *MockBeaconNodeValidatorClient) GetSyncMessageBlockRoot(arg0 context.Context, arg1 *emptypb.Empty, arg2 ...grpc.CallOption) (*eth.SyncMessageBlockRootResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetSyncMessageBlockRoot", varargs...)
@@ -238,17 +257,17 @@ func (m *MockBeaconNodeValidatorClient) GetSyncMessageBlockRoot(ctx context.Cont
}
// GetSyncMessageBlockRoot indicates an expected call of GetSyncMessageBlockRoot.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetSyncMessageBlockRoot(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetSyncMessageBlockRoot(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSyncMessageBlockRoot", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetSyncMessageBlockRoot), varargs...)
}
// GetSyncSubcommitteeIndex mocks base method.
func (m *MockBeaconNodeValidatorClient) GetSyncSubcommitteeIndex(ctx context.Context, in *eth.SyncSubcommitteeIndexRequest, opts ...grpc.CallOption) (*eth.SyncSubcommitteeIndexResponse, error) {
func (m *MockBeaconNodeValidatorClient) GetSyncSubcommitteeIndex(arg0 context.Context, arg1 *eth.SyncSubcommitteeIndexRequest, arg2 ...grpc.CallOption) (*eth.SyncSubcommitteeIndexResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GetSyncSubcommitteeIndex", varargs...)
@@ -258,17 +277,17 @@ func (m *MockBeaconNodeValidatorClient) GetSyncSubcommitteeIndex(ctx context.Con
}
// GetSyncSubcommitteeIndex indicates an expected call of GetSyncSubcommitteeIndex.
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetSyncSubcommitteeIndex(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) GetSyncSubcommitteeIndex(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSyncSubcommitteeIndex", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).GetSyncSubcommitteeIndex), varargs...)
}
// MultipleValidatorStatus mocks base method.
func (m *MockBeaconNodeValidatorClient) MultipleValidatorStatus(ctx context.Context, in *eth.MultipleValidatorStatusRequest, opts ...grpc.CallOption) (*eth.MultipleValidatorStatusResponse, error) {
func (m *MockBeaconNodeValidatorClient) MultipleValidatorStatus(arg0 context.Context, arg1 *eth.MultipleValidatorStatusRequest, arg2 ...grpc.CallOption) (*eth.MultipleValidatorStatusResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "MultipleValidatorStatus", varargs...)
@@ -278,17 +297,17 @@ func (m *MockBeaconNodeValidatorClient) MultipleValidatorStatus(ctx context.Cont
}
// MultipleValidatorStatus indicates an expected call of MultipleValidatorStatus.
func (mr *MockBeaconNodeValidatorClientMockRecorder) MultipleValidatorStatus(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) MultipleValidatorStatus(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MultipleValidatorStatus", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).MultipleValidatorStatus), varargs...)
}
// PrepareBeaconProposer mocks base method.
func (m *MockBeaconNodeValidatorClient) PrepareBeaconProposer(ctx context.Context, in *eth.PrepareBeaconProposerRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) {
func (m *MockBeaconNodeValidatorClient) PrepareBeaconProposer(arg0 context.Context, arg1 *eth.PrepareBeaconProposerRequest, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "PrepareBeaconProposer", varargs...)
@@ -298,17 +317,17 @@ func (m *MockBeaconNodeValidatorClient) PrepareBeaconProposer(ctx context.Contex
}
// PrepareBeaconProposer indicates an expected call of PrepareBeaconProposer.
func (mr *MockBeaconNodeValidatorClientMockRecorder) PrepareBeaconProposer(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) PrepareBeaconProposer(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PrepareBeaconProposer", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).PrepareBeaconProposer), varargs...)
}
// ProposeAttestation mocks base method.
func (m *MockBeaconNodeValidatorClient) ProposeAttestation(ctx context.Context, in *eth.Attestation, opts ...grpc.CallOption) (*eth.AttestResponse, error) {
func (m *MockBeaconNodeValidatorClient) ProposeAttestation(arg0 context.Context, arg1 *eth.Attestation, arg2 ...grpc.CallOption) (*eth.AttestResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ProposeAttestation", varargs...)
@@ -318,17 +337,17 @@ func (m *MockBeaconNodeValidatorClient) ProposeAttestation(ctx context.Context,
}
// ProposeAttestation indicates an expected call of ProposeAttestation.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeAttestation(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeAttestation(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeAttestation", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ProposeAttestation), varargs...)
}
// ProposeAttestationElectra mocks base method.
func (m *MockBeaconNodeValidatorClient) ProposeAttestationElectra(ctx context.Context, in *eth.SingleAttestation, opts ...grpc.CallOption) (*eth.AttestResponse, error) {
func (m *MockBeaconNodeValidatorClient) ProposeAttestationElectra(arg0 context.Context, arg1 *eth.SingleAttestation, arg2 ...grpc.CallOption) (*eth.AttestResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ProposeAttestationElectra", varargs...)
@@ -338,17 +357,17 @@ func (m *MockBeaconNodeValidatorClient) ProposeAttestationElectra(ctx context.Co
}
// ProposeAttestationElectra indicates an expected call of ProposeAttestationElectra.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeAttestationElectra(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeAttestationElectra(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeAttestationElectra", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ProposeAttestationElectra), varargs...)
}
// ProposeBeaconBlock mocks base method.
func (m *MockBeaconNodeValidatorClient) ProposeBeaconBlock(ctx context.Context, in *eth.GenericSignedBeaconBlock, opts ...grpc.CallOption) (*eth.ProposeResponse, error) {
func (m *MockBeaconNodeValidatorClient) ProposeBeaconBlock(arg0 context.Context, arg1 *eth.GenericSignedBeaconBlock, arg2 ...grpc.CallOption) (*eth.ProposeResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ProposeBeaconBlock", varargs...)
@@ -358,17 +377,17 @@ func (m *MockBeaconNodeValidatorClient) ProposeBeaconBlock(ctx context.Context,
}
// ProposeBeaconBlock indicates an expected call of ProposeBeaconBlock.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeBeaconBlock(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeBeaconBlock(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeBeaconBlock", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ProposeBeaconBlock), varargs...)
}
// ProposeExit mocks base method.
func (m *MockBeaconNodeValidatorClient) ProposeExit(ctx context.Context, in *eth.SignedVoluntaryExit, opts ...grpc.CallOption) (*eth.ProposeExitResponse, error) {
func (m *MockBeaconNodeValidatorClient) ProposeExit(arg0 context.Context, arg1 *eth.SignedVoluntaryExit, arg2 ...grpc.CallOption) (*eth.ProposeExitResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ProposeExit", varargs...)
@@ -378,17 +397,17 @@ func (m *MockBeaconNodeValidatorClient) ProposeExit(ctx context.Context, in *eth
}
// ProposeExit indicates an expected call of ProposeExit.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeExit(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) ProposeExit(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProposeExit", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ProposeExit), varargs...)
}
// StreamBlocksAltair mocks base method.
func (m *MockBeaconNodeValidatorClient) StreamBlocksAltair(ctx context.Context, in *eth.StreamBlocksRequest, opts ...grpc.CallOption) (eth.BeaconNodeValidator_StreamBlocksAltairClient, error) {
func (m *MockBeaconNodeValidatorClient) StreamBlocksAltair(arg0 context.Context, arg1 *eth.StreamBlocksRequest, arg2 ...grpc.CallOption) (eth.BeaconNodeValidator_StreamBlocksAltairClient, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "StreamBlocksAltair", varargs...)
@@ -398,17 +417,17 @@ func (m *MockBeaconNodeValidatorClient) StreamBlocksAltair(ctx context.Context,
}
// StreamBlocksAltair indicates an expected call of StreamBlocksAltair.
func (mr *MockBeaconNodeValidatorClientMockRecorder) StreamBlocksAltair(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) StreamBlocksAltair(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamBlocksAltair", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).StreamBlocksAltair), varargs...)
}
// StreamSlots mocks base method.
func (m *MockBeaconNodeValidatorClient) StreamSlots(ctx context.Context, in *eth.StreamSlotsRequest, opts ...grpc.CallOption) (eth.BeaconNodeValidator_StreamSlotsClient, error) {
func (m *MockBeaconNodeValidatorClient) StreamSlots(arg0 context.Context, arg1 *eth.StreamSlotsRequest, arg2 ...grpc.CallOption) (eth.BeaconNodeValidator_StreamSlotsClient, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "StreamSlots", varargs...)
@@ -418,17 +437,17 @@ func (m *MockBeaconNodeValidatorClient) StreamSlots(ctx context.Context, in *eth
}
// StreamSlots indicates an expected call of StreamSlots.
func (mr *MockBeaconNodeValidatorClientMockRecorder) StreamSlots(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) StreamSlots(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StreamSlots", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).StreamSlots), varargs...)
}
// SubmitAggregateSelectionProof mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProof(ctx context.Context, in *eth.AggregateSelectionRequest, opts ...grpc.CallOption) (*eth.AggregateSelectionResponse, error) {
func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProof(arg0 context.Context, arg1 *eth.AggregateSelectionRequest, arg2 ...grpc.CallOption) (*eth.AggregateSelectionResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitAggregateSelectionProof", varargs...)
@@ -438,17 +457,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProof(ctx contex
}
// SubmitAggregateSelectionProof indicates an expected call of SubmitAggregateSelectionProof.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitAggregateSelectionProof(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitAggregateSelectionProof(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitAggregateSelectionProof", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitAggregateSelectionProof), varargs...)
}
// SubmitAggregateSelectionProofElectra mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProofElectra(ctx context.Context, in *eth.AggregateSelectionRequest, opts ...grpc.CallOption) (*eth.AggregateSelectionElectraResponse, error) {
func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProofElectra(arg0 context.Context, arg1 *eth.AggregateSelectionRequest, arg2 ...grpc.CallOption) (*eth.AggregateSelectionElectraResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitAggregateSelectionProofElectra", varargs...)
@@ -458,17 +477,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitAggregateSelectionProofElectra(ctx
}
// SubmitAggregateSelectionProofElectra indicates an expected call of SubmitAggregateSelectionProofElectra.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitAggregateSelectionProofElectra(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitAggregateSelectionProofElectra(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
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(ctx context.Context, in *eth.SignedAggregateSubmitRequest, opts ...grpc.CallOption) (*eth.SignedAggregateSubmitResponse, error) {
func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProof(arg0 context.Context, arg1 *eth.SignedAggregateSubmitRequest, arg2 ...grpc.CallOption) (*eth.SignedAggregateSubmitResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitSignedAggregateSelectionProof", varargs...)
@@ -478,17 +497,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProof(ctx
}
// SubmitSignedAggregateSelectionProof indicates an expected call of SubmitSignedAggregateSelectionProof.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedAggregateSelectionProof(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedAggregateSelectionProof(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSignedAggregateSelectionProof", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitSignedAggregateSelectionProof), varargs...)
}
// SubmitSignedAggregateSelectionProofElectra mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProofElectra(ctx context.Context, in *eth.SignedAggregateSubmitElectraRequest, opts ...grpc.CallOption) (*eth.SignedAggregateSubmitResponse, error) {
func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProofElectra(arg0 context.Context, arg1 *eth.SignedAggregateSubmitElectraRequest, arg2 ...grpc.CallOption) (*eth.SignedAggregateSubmitResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitSignedAggregateSelectionProofElectra", varargs...)
@@ -498,17 +517,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitSignedAggregateSelectionProofElect
}
// SubmitSignedAggregateSelectionProofElectra indicates an expected call of SubmitSignedAggregateSelectionProofElectra.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedAggregateSelectionProofElectra(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedAggregateSelectionProofElectra(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
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(ctx context.Context, in *eth.SignedContributionAndProof, opts ...grpc.CallOption) (*emptypb.Empty, error) {
func (m *MockBeaconNodeValidatorClient) SubmitSignedContributionAndProof(arg0 context.Context, arg1 *eth.SignedContributionAndProof, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitSignedContributionAndProof", varargs...)
@@ -518,17 +537,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitSignedContributionAndProof(ctx con
}
// SubmitSignedContributionAndProof indicates an expected call of SubmitSignedContributionAndProof.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedContributionAndProof(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSignedContributionAndProof(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSignedContributionAndProof", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitSignedContributionAndProof), varargs...)
}
// SubmitSyncMessage mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitSyncMessage(ctx context.Context, in *eth.SyncCommitteeMessage, opts ...grpc.CallOption) (*emptypb.Empty, error) {
func (m *MockBeaconNodeValidatorClient) SubmitSyncMessage(arg0 context.Context, arg1 *eth.SyncCommitteeMessage, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitSyncMessage", varargs...)
@@ -538,17 +557,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitSyncMessage(ctx context.Context, i
}
// SubmitSyncMessage indicates an expected call of SubmitSyncMessage.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSyncMessage(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitSyncMessage(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitSyncMessage", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitSyncMessage), varargs...)
}
// SubmitValidatorRegistrations mocks base method.
func (m *MockBeaconNodeValidatorClient) SubmitValidatorRegistrations(ctx context.Context, in *eth.SignedValidatorRegistrationsV1, opts ...grpc.CallOption) (*emptypb.Empty, error) {
func (m *MockBeaconNodeValidatorClient) SubmitValidatorRegistrations(arg0 context.Context, arg1 *eth.SignedValidatorRegistrationsV1, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubmitValidatorRegistrations", varargs...)
@@ -558,17 +577,17 @@ func (m *MockBeaconNodeValidatorClient) SubmitValidatorRegistrations(ctx context
}
// SubmitValidatorRegistrations indicates an expected call of SubmitValidatorRegistrations.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitValidatorRegistrations(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubmitValidatorRegistrations(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubmitValidatorRegistrations", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubmitValidatorRegistrations), varargs...)
}
// SubscribeCommitteeSubnets mocks base method.
func (m *MockBeaconNodeValidatorClient) SubscribeCommitteeSubnets(ctx context.Context, in *eth.CommitteeSubnetsSubscribeRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) {
func (m *MockBeaconNodeValidatorClient) SubscribeCommitteeSubnets(arg0 context.Context, arg1 *eth.CommitteeSubnetsSubscribeRequest, arg2 ...grpc.CallOption) (*emptypb.Empty, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "SubscribeCommitteeSubnets", varargs...)
@@ -578,17 +597,17 @@ func (m *MockBeaconNodeValidatorClient) SubscribeCommitteeSubnets(ctx context.Co
}
// SubscribeCommitteeSubnets indicates an expected call of SubscribeCommitteeSubnets.
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubscribeCommitteeSubnets(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) SubscribeCommitteeSubnets(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SubscribeCommitteeSubnets", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).SubscribeCommitteeSubnets), varargs...)
}
// ValidatorIndex mocks base method.
func (m *MockBeaconNodeValidatorClient) ValidatorIndex(ctx context.Context, in *eth.ValidatorIndexRequest, opts ...grpc.CallOption) (*eth.ValidatorIndexResponse, error) {
func (m *MockBeaconNodeValidatorClient) ValidatorIndex(arg0 context.Context, arg1 *eth.ValidatorIndexRequest, arg2 ...grpc.CallOption) (*eth.ValidatorIndexResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ValidatorIndex", varargs...)
@@ -598,17 +617,17 @@ func (m *MockBeaconNodeValidatorClient) ValidatorIndex(ctx context.Context, in *
}
// ValidatorIndex indicates an expected call of ValidatorIndex.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ValidatorIndex(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) ValidatorIndex(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorIndex", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ValidatorIndex), varargs...)
}
// ValidatorStatus mocks base method.
func (m *MockBeaconNodeValidatorClient) ValidatorStatus(ctx context.Context, in *eth.ValidatorStatusRequest, opts ...grpc.CallOption) (*eth.ValidatorStatusResponse, error) {
func (m *MockBeaconNodeValidatorClient) ValidatorStatus(arg0 context.Context, arg1 *eth.ValidatorStatusRequest, arg2 ...grpc.CallOption) (*eth.ValidatorStatusResponse, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ValidatorStatus", varargs...)
@@ -618,17 +637,17 @@ func (m *MockBeaconNodeValidatorClient) ValidatorStatus(ctx context.Context, in
}
// ValidatorStatus indicates an expected call of ValidatorStatus.
func (mr *MockBeaconNodeValidatorClientMockRecorder) ValidatorStatus(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) ValidatorStatus(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorStatus", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).ValidatorStatus), varargs...)
}
// WaitForActivation mocks base method.
func (m *MockBeaconNodeValidatorClient) WaitForActivation(ctx context.Context, in *eth.ValidatorActivationRequest, opts ...grpc.CallOption) (eth.BeaconNodeValidator_WaitForActivationClient, error) {
func (m *MockBeaconNodeValidatorClient) WaitForActivation(arg0 context.Context, arg1 *eth.ValidatorActivationRequest, arg2 ...grpc.CallOption) (eth.BeaconNodeValidator_WaitForActivationClient, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "WaitForActivation", varargs...)
@@ -638,17 +657,17 @@ func (m *MockBeaconNodeValidatorClient) WaitForActivation(ctx context.Context, i
}
// WaitForActivation indicates an expected call of WaitForActivation.
func (mr *MockBeaconNodeValidatorClientMockRecorder) WaitForActivation(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) WaitForActivation(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WaitForActivation", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).WaitForActivation), varargs...)
}
// WaitForChainStart mocks base method.
func (m *MockBeaconNodeValidatorClient) WaitForChainStart(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (eth.BeaconNodeValidator_WaitForChainStartClient, error) {
func (m *MockBeaconNodeValidatorClient) WaitForChainStart(arg0 context.Context, arg1 *emptypb.Empty, arg2 ...grpc.CallOption) (eth.BeaconNodeValidator_WaitForChainStartClient, error) {
m.ctrl.T.Helper()
varargs := []any{ctx, in}
for _, a := range opts {
varargs := []any{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "WaitForChainStart", varargs...)
@@ -658,9 +677,9 @@ func (m *MockBeaconNodeValidatorClient) WaitForChainStart(ctx context.Context, i
}
// WaitForChainStart indicates an expected call of WaitForChainStart.
func (mr *MockBeaconNodeValidatorClientMockRecorder) WaitForChainStart(ctx, in any, opts ...any) *gomock.Call {
func (mr *MockBeaconNodeValidatorClientMockRecorder) WaitForChainStart(arg0, arg1 any, arg2 ...any) *gomock.Call {
mr.mock.ctrl.T.Helper()
varargs := append([]any{ctx, in}, opts...)
varargs := append([]any{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WaitForChainStart", reflect.TypeOf((*MockBeaconNodeValidatorClient)(nil).WaitForChainStart), varargs...)
}
@@ -668,7 +687,6 @@ func (mr *MockBeaconNodeValidatorClientMockRecorder) WaitForChainStart(ctx, in a
type MockBeaconNodeValidator_WaitForChainStartClient struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidator_WaitForChainStartClientMockRecorder is the mock recorder for MockBeaconNodeValidator_WaitForChainStartClient.
@@ -747,31 +765,31 @@ func (mr *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder) Recv() *g
}
// RecvMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForChainStartClient) RecvMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "RecvMsg", m)
func (m *MockBeaconNodeValidator_WaitForChainStartClient) RecvMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg.
func (mr *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder) RecvMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder) RecvMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartClient)(nil).RecvMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartClient)(nil).RecvMsg), arg0)
}
// SendMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForChainStartClient) SendMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "SendMsg", m)
func (m *MockBeaconNodeValidator_WaitForChainStartClient) SendMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg.
func (mr *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder) SendMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder) SendMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartClient)(nil).SendMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartClient)(nil).SendMsg), arg0)
}
// Trailer mocks base method.
@@ -792,7 +810,6 @@ func (mr *MockBeaconNodeValidator_WaitForChainStartClientMockRecorder) Trailer()
type MockBeaconNodeValidator_WaitForActivationClient struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidator_WaitForActivationClientMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidator_WaitForActivationClientMockRecorder is the mock recorder for MockBeaconNodeValidator_WaitForActivationClient.
@@ -871,31 +888,31 @@ func (mr *MockBeaconNodeValidator_WaitForActivationClientMockRecorder) Recv() *g
}
// RecvMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForActivationClient) RecvMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "RecvMsg", m)
func (m *MockBeaconNodeValidator_WaitForActivationClient) RecvMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg.
func (mr *MockBeaconNodeValidator_WaitForActivationClientMockRecorder) RecvMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForActivationClientMockRecorder) RecvMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationClient)(nil).RecvMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationClient)(nil).RecvMsg), arg0)
}
// SendMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForActivationClient) SendMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "SendMsg", m)
func (m *MockBeaconNodeValidator_WaitForActivationClient) SendMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg.
func (mr *MockBeaconNodeValidator_WaitForActivationClientMockRecorder) SendMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForActivationClientMockRecorder) SendMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationClient)(nil).SendMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationClient)(nil).SendMsg), arg0)
}
// Trailer mocks base method.
@@ -916,7 +933,6 @@ func (mr *MockBeaconNodeValidator_WaitForActivationClientMockRecorder) Trailer()
type MockBeaconNodeValidator_StreamSlotsClient struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidator_StreamSlotsClientMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidator_StreamSlotsClientMockRecorder is the mock recorder for MockBeaconNodeValidator_StreamSlotsClient.
@@ -995,31 +1011,31 @@ func (mr *MockBeaconNodeValidator_StreamSlotsClientMockRecorder) Recv() *gomock.
}
// RecvMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_StreamSlotsClient) RecvMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "RecvMsg", m)
func (m *MockBeaconNodeValidator_StreamSlotsClient) RecvMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg.
func (mr *MockBeaconNodeValidator_StreamSlotsClientMockRecorder) RecvMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_StreamSlotsClientMockRecorder) RecvMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsClient)(nil).RecvMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsClient)(nil).RecvMsg), arg0)
}
// SendMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_StreamSlotsClient) SendMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "SendMsg", m)
func (m *MockBeaconNodeValidator_StreamSlotsClient) SendMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg.
func (mr *MockBeaconNodeValidator_StreamSlotsClientMockRecorder) SendMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_StreamSlotsClientMockRecorder) SendMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsClient)(nil).SendMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsClient)(nil).SendMsg), arg0)
}
// Trailer mocks base method.

View File

@@ -23,7 +23,6 @@ import (
type MockBeaconNodeValidatorServer struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidatorServerMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidatorServerMockRecorder is the mock recorder for MockBeaconNodeValidatorServer.
@@ -148,6 +147,21 @@ func (mr *MockBeaconNodeValidatorServerMockRecorder) GetDuties(arg0, arg1 any) *
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetDuties", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).GetDuties), arg0, arg1)
}
// GetDutiesV2 mocks base method.
func (m *MockBeaconNodeValidatorServer) GetDutiesV2(arg0 context.Context, arg1 *eth.DutiesRequest) (*eth.DutiesV2Response, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "GetDutiesV2", arg0, arg1)
ret0, _ := ret[0].(*eth.DutiesV2Response)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// GetDutiesV2 indicates an expected call of GetDutiesV2.
func (mr *MockBeaconNodeValidatorServerMockRecorder) GetDutiesV2(arg0, arg1 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetDutiesV2", reflect.TypeOf((*MockBeaconNodeValidatorServer)(nil).GetDutiesV2), arg0, arg1)
}
// GetFeeRecipientByPubKey mocks base method.
func (m *MockBeaconNodeValidatorServer) GetFeeRecipientByPubKey(arg0 context.Context, arg1 *eth.FeeRecipientByPubKeyRequest) (*eth.FeeRecipientByPubKeyResponse, error) {
m.ctrl.T.Helper()
@@ -508,7 +522,6 @@ func (mr *MockBeaconNodeValidatorServerMockRecorder) WaitForChainStart(arg0, arg
type MockBeaconNodeValidator_WaitForActivationServer struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidator_WaitForActivationServerMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidator_WaitForActivationServerMockRecorder is the mock recorder for MockBeaconNodeValidator_WaitForActivationServer.
@@ -543,17 +556,17 @@ func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) Context()
}
// RecvMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForActivationServer) RecvMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "RecvMsg", m)
func (m *MockBeaconNodeValidator_WaitForActivationServer) RecvMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg.
func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) RecvMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) RecvMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationServer)(nil).RecvMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationServer)(nil).RecvMsg), arg0)
}
// Send mocks base method.
@@ -585,17 +598,17 @@ func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) SendHeade
}
// SendMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForActivationServer) SendMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "SendMsg", m)
func (m *MockBeaconNodeValidator_WaitForActivationServer) SendMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg.
func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) SendMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) SendMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationServer)(nil).SendMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForActivationServer)(nil).SendMsg), arg0)
}
// SetHeader mocks base method.
@@ -628,7 +641,6 @@ func (mr *MockBeaconNodeValidator_WaitForActivationServerMockRecorder) SetTraile
type MockBeaconNodeValidator_WaitForChainStartServer struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidator_WaitForChainStartServerMockRecorder is the mock recorder for MockBeaconNodeValidator_WaitForChainStartServer.
@@ -663,17 +675,17 @@ func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) Context()
}
// RecvMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForChainStartServer) RecvMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "RecvMsg", m)
func (m *MockBeaconNodeValidator_WaitForChainStartServer) RecvMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg.
func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) RecvMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) RecvMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartServer)(nil).RecvMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartServer)(nil).RecvMsg), arg0)
}
// Send mocks base method.
@@ -705,17 +717,17 @@ func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) SendHeade
}
// SendMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_WaitForChainStartServer) SendMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "SendMsg", m)
func (m *MockBeaconNodeValidator_WaitForChainStartServer) SendMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg.
func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) SendMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) SendMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartServer)(nil).SendMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_WaitForChainStartServer)(nil).SendMsg), arg0)
}
// SetHeader mocks base method.
@@ -748,7 +760,6 @@ func (mr *MockBeaconNodeValidator_WaitForChainStartServerMockRecorder) SetTraile
type MockBeaconNodeValidator_StreamSlotsServer struct {
ctrl *gomock.Controller
recorder *MockBeaconNodeValidator_StreamSlotsServerMockRecorder
isgomock struct{}
}
// MockBeaconNodeValidator_StreamSlotsServerMockRecorder is the mock recorder for MockBeaconNodeValidator_StreamSlotsServer.
@@ -783,17 +794,17 @@ func (mr *MockBeaconNodeValidator_StreamSlotsServerMockRecorder) Context() *gomo
}
// RecvMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_StreamSlotsServer) RecvMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "RecvMsg", m)
func (m *MockBeaconNodeValidator_StreamSlotsServer) RecvMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg.
func (mr *MockBeaconNodeValidator_StreamSlotsServerMockRecorder) RecvMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_StreamSlotsServerMockRecorder) RecvMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsServer)(nil).RecvMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsServer)(nil).RecvMsg), arg0)
}
// Send mocks base method.
@@ -825,17 +836,17 @@ func (mr *MockBeaconNodeValidator_StreamSlotsServerMockRecorder) SendHeader(arg0
}
// SendMsg mocks base method.
func (m_2 *MockBeaconNodeValidator_StreamSlotsServer) SendMsg(m any) error {
m_2.ctrl.T.Helper()
ret := m_2.ctrl.Call(m_2, "SendMsg", m)
func (m *MockBeaconNodeValidator_StreamSlotsServer) SendMsg(arg0 any) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg.
func (mr *MockBeaconNodeValidator_StreamSlotsServerMockRecorder) SendMsg(m any) *gomock.Call {
func (mr *MockBeaconNodeValidator_StreamSlotsServerMockRecorder) SendMsg(arg0 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsServer)(nil).SendMsg), m)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconNodeValidator_StreamSlotsServer)(nil).SendMsg), arg0)
}
// SetHeader mocks base method.

View File

@@ -33,6 +33,11 @@ func (c *grpcValidatorClient) Duties(ctx context.Context, in *ethpb.DutiesReques
return toValidatorDutiesContainer(dutiesResponse)
}
func (c *grpcValidatorClient) DutiesV2(ctx context.Context, in *ethpb.DutiesRequest) (*ethpb.ValidatorDutiesContainer, error) {
// TODO: update to v2 get duties in separate PR, used to satisfy interface
return nil, errors.New("not implemented")
}
func toValidatorDutiesContainer(dutiesResponse *ethpb.DutiesResponse) (*ethpb.ValidatorDutiesContainer, error) {
currentDuties := make([]*ethpb.ValidatorDuty, len(dutiesResponse.CurrentEpochDuties))
for i, cd := range dutiesResponse.CurrentEpochDuties {