Validator Assignment RPC Beacon Node Stream (#563)

This commit is contained in:
Raul Jordan
2018-09-26 21:34:35 -05:00
committed by GitHub
parent 54a10913b2
commit e10baf7c80
25 changed files with 1096 additions and 1086 deletions

View File

@@ -6,9 +6,9 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/validator/beacon",
visibility = ["//validator:__subpackages__"],
deps = [
"//proto/beacon/p2p/v1:go_default_library",
"//proto/beacon/rpc/v1:go_default_library",
"//validator/params:go_default_library",
"//validator/utils:go_default_library",
"@com_github_ethereum_go_ethereum//event:go_default_library",
"@com_github_golang_protobuf//ptypes:go_default_library_gen",
"@com_github_sirupsen_logrus//:go_default_library",

View File

@@ -10,9 +10,9 @@ import (
"github.com/ethereum/go-ethereum/event"
"github.com/golang/protobuf/ptypes"
"github.com/golang/protobuf/ptypes/empty"
pbp2p "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
pb "github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1"
"github.com/prysmaticlabs/prysm/validator/params"
"github.com/prysmaticlabs/prysm/validator/utils"
"github.com/sirupsen/logrus"
)
@@ -27,13 +27,14 @@ type Service struct {
ctx context.Context
cancel context.CancelFunc
rpcClient rpcClientService
validatorIndex int
assignedSlot uint64
responsibility string
shardID uint64
role pb.ValidatorRole
attesterAssignmentFeed *event.Feed
proposerAssignmentFeed *event.Feed
processedAttestationFeed *event.Feed
genesisTimestamp time.Time
slotAlignmentDuration time.Duration
}
// NewBeaconValidator instantiates a service that interacts with a beacon node
@@ -47,17 +48,18 @@ func NewBeaconValidator(ctx context.Context, rpcClient rpcClientService) *Servic
attesterAssignmentFeed: new(event.Feed),
proposerAssignmentFeed: new(event.Feed),
processedAttestationFeed: new(event.Feed),
slotAlignmentDuration: time.Duration(params.DefaultConfig().SlotDuration) * time.Second,
}
}
// Start the main routine for a beacon client service.
func (s *Service) Start() {
log.Info("Starting service")
client := s.rpcClient.BeaconServiceClient()
beaconServiceClient := s.rpcClient.BeaconServiceClient()
// First thing the validator does is request the genesis block timestamp
// and the latest, canonical crystallized state from a beacon node. From here,
// a validator can determine its assigned slot by keeping an internal
// First thing the validator does is request the current validator assignments
// for the current beacon node cycle as well as the genesis timestamp
// from the beacon node. From here, a validator can keep an internal
// ticker that starts at the current slot the beacon node is in. This current slot
// value is determined by taking the time differential between the genesis block
// time and the current system time.
@@ -65,19 +67,24 @@ func (s *Service) Start() {
// Note: this does not validate the current system time against a global
// NTP server, which will be important to do in production.
// currently in a cycle we are supposed to participate in.
s.fetchGenesisAndCanonicalState(client)
s.fetchCurrentAssignmentsAndGenesisTime(beaconServiceClient)
// Then, we kick off a routine that uses the begins a ticker set in fetchGenesisAndCanonicalState
// to wait until the validator's assigned slot to perform proposals or attestations.
slotTicker := time.NewTicker(time.Second * time.Duration(params.DefaultConfig().SlotDuration))
go s.waitForAssignment(slotTicker.C, client)
// Then, we kick off a routine that uses the begins a ticker based on the beacon node's
// genesis timestamp and the validator will use this slot ticker to
// determine when it is assigned to perform proposals or attestations.
//
// We block until the current time is a multiple of params.SlotDuration
// so the validator and beacon node's internal tickers are aligned.
utils.BlockingWait(s.slotAlignmentDuration)
slotTicker := time.NewTicker(s.slotAlignmentDuration)
go s.waitForAssignment(slotTicker.C, beaconServiceClient)
// We then kick off a routine that listens for streams of cycle transitions
// coming from the beacon node. This will allow the validator client to recalculate
// when it has to perform its responsibilities appropriately using timestamps
// and the IndicesForSlots field inside the received crystallized state.
go s.listenForCrystallizedStates(client)
go s.listenForProcessedAttestations(client)
// when it has to perform its responsibilities.
go s.listenForCycleTransitions(beaconServiceClient)
go s.listenForProcessedAttestations(beaconServiceClient)
}
// Stop the main loop..
@@ -87,17 +94,11 @@ func (s *Service) Stop() error {
return nil
}
// CurrentBeaconSlot based on the seconds since genesis.
func (s *Service) CurrentBeaconSlot() uint64 {
secondsSinceGenesis := time.Since(s.genesisTimestamp).Seconds()
return uint64(math.Floor(secondsSinceGenesis / 8.0))
}
// fetchGenesisAndCanonicalState fetches both the genesis timestamp as well
// as the latest canonical crystallized state from a beacon node. This allows
// fetchCurrentAssignmentsAndGenesisTime fetches both the genesis timestamp as well
// as the current assignments for the current cycle in the beacon node. This allows
// the validator to do the following:
//
// (1) determine if it should act as an attester/proposer and at what slot
// (1) determine if it should act as an attester/proposer, at what slot,
// and what shard
//
// (2) determine the seconds since genesis by using the latest crystallized
@@ -106,8 +107,13 @@ func (s *Service) CurrentBeaconSlot() uint64 {
//
// From this, the validator client can deduce what slot interval the beacon
// node is in and determine when exactly it is time to propose or attest.
func (s *Service) fetchGenesisAndCanonicalState(client pb.BeaconServiceClient) {
res, err := client.GenesisTimeAndCanonicalState(s.ctx, &empty.Empty{})
func (s *Service) fetchCurrentAssignmentsAndGenesisTime(client pb.BeaconServiceClient) {
// Currently fetches assignments for all validators.
req := &pb.ValidatorAssignmentRequest{
AllValidators: true,
}
res, err := client.CurrentAssignmentsAndGenesisTime(s.ctx, req)
if err != nil {
// If this RPC request fails, the entire system should fatal as it is critical for
// the validator to begin this way.
@@ -123,9 +129,25 @@ func (s *Service) fetchGenesisAndCanonicalState(client pb.BeaconServiceClient) {
s.genesisTimestamp = genesisTimestamp
crystallized := res.GetLatestCrystallizedState()
if err := s.processCrystallizedState(crystallized); err != nil {
log.Fatalf("unable to process received crystallized state: %v", err)
// Loops through the received assignments to determine which one
// corresponds to this validator client based on a matching public key.
for _, assignment := range res.GetAssignments() {
// TODO(#566): Determine assignment based on public key flag.
pubKeyProto := assignment.GetPublicKey()
if isZeroAddress(pubKeyProto.GetPublicKey()) {
s.assignedSlot = s.CurrentCycleStartSlot() + assignment.GetAssignedSlot()
s.shardID = assignment.GetShardId()
s.role = assignment.GetRole()
break
}
}
if s.role == pb.ValidatorRole_PROPOSER {
log.Infof("Assigned as PROPOSER to slot %v, shardID: %v", s.assignedSlot, s.shardID)
} else {
log.Infof("Assigned as ATTESTER to slot %v, shardID: %v", s.assignedSlot, s.shardID)
}
if s.CurrentBeaconSlot() > s.assignedSlot {
log.Info("You joined a bit too late -the current slot is greater than assigned slot in the cycle, wait until next cycle to be re-assigned")
}
}
@@ -139,10 +161,9 @@ func (s *Service) waitForAssignment(ticker <-chan time.Time, client pb.BeaconSer
case <-s.ctx.Done():
return
case <-ticker:
log.WithField("slotNumber", s.CurrentBeaconSlot()).Info("New beacon node slot interval")
if s.responsibility == "proposer" && s.assignedSlot == s.CurrentBeaconSlot() {
log.WithField("slotNumber", s.CurrentBeaconSlot()).Info("New beacon node slot")
if s.role == pb.ValidatorRole_PROPOSER && s.assignedSlot == s.CurrentBeaconSlot() {
log.WithField("slotNumber", s.CurrentBeaconSlot()).Info("Assigned proposal slot number reached")
s.responsibility = ""
block, err := client.CanonicalHead(s.ctx, &empty.Empty{})
if err != nil {
log.Errorf("Could not fetch canonical head via gRPC from beacon node: %v", err)
@@ -150,9 +171,8 @@ func (s *Service) waitForAssignment(ticker <-chan time.Time, client pb.BeaconSer
}
// We forward the latest canonical block to the proposer service via a feed.
s.proposerAssignmentFeed.Send(block)
} else if s.responsibility == "attester" && s.assignedSlot == s.CurrentBeaconSlot() {
} else if s.role == pb.ValidatorRole_ATTESTER && s.assignedSlot == s.CurrentBeaconSlot() {
log.Info("Assigned attestation slot number reached")
s.responsibility = ""
block, err := client.CanonicalHead(s.ctx, &empty.Empty{})
if err != nil {
log.Errorf("Could not fetch canonical head via gRPC from beacon node: %v", err)
@@ -165,19 +185,21 @@ func (s *Service) waitForAssignment(ticker <-chan time.Time, client pb.BeaconSer
}
}
// listenForCrystallizedStates receives the latest canonical crystallized state
// from the beacon node's RPC server via gRPC streams.
// TODO(#545): Rename to listen for assignment instead, which is streamed from a beacon node
// upon every new cycle transition and will include the validator's index in the
// assignment bitfield as well as the assigned shard ID.
func (s *Service) listenForCrystallizedStates(client pb.BeaconServiceClient) {
stream, err := client.LatestCrystallizedState(s.ctx, &empty.Empty{})
// listenForCycleTransitions receives validator assignments from the
// the beacon node's RPC server when a new cycle transition occurs.
func (s *Service) listenForCycleTransitions(client pb.BeaconServiceClient) {
// Currently fetches assignments for all validators.
req := &pb.ValidatorAssignmentRequest{
AllValidators: true,
}
stream, err := client.ValidatorAssignments(s.ctx, req)
if err != nil {
log.Errorf("Could not setup crystallized beacon state streaming client: %v", err)
log.Errorf("Could not setup validator assignments streaming client: %v", err)
return
}
for {
crystallizedState, err := stream.Recv()
res, err := stream.Recv()
// If the stream is closed, we stop the loop.
if err == io.EOF {
break
@@ -187,56 +209,34 @@ func (s *Service) listenForCrystallizedStates(client pb.BeaconServiceClient) {
log.Debugf("Context has been canceled so shutting down the loop: %v", s.ctx.Err())
break
}
if err != nil {
log.Errorf("Could not receive latest crystallized beacon state from stream: %v", err)
log.Errorf("Could not receive validator assignments from stream: %v", err)
continue
}
if err := s.processCrystallizedState(crystallizedState); err != nil {
log.Error(err)
log.WithField("slotNumber", s.CurrentBeaconSlot()).Info("New cycle transition")
// Loops through the received assignments to determine which one
// corresponds to this validator client based on a matching public key.
for _, assignment := range res.GetAssignments() {
// TODO(#566): Determine assignment based on public key flag.
pubKeyProto := assignment.GetPublicKey()
if isZeroAddress(pubKeyProto.GetPublicKey()) {
s.assignedSlot = s.CurrentCycleStartSlot() + assignment.GetAssignedSlot()
s.shardID = assignment.GetShardId()
s.role = assignment.GetRole()
break
}
}
if s.role == pb.ValidatorRole_PROPOSER {
log.Infof("Assigned as PROPOSER to slot %v, shardID: %v", s.assignedSlot, s.shardID)
} else {
log.Infof("Assigned as ATTESTER to slot %v, shardID: %v", s.assignedSlot, s.shardID)
}
}
}
// processCrystallizedState uses a received crystallized state to determine
// whether a validator is a proposer/attester and the validator's assigned slot.
func (s *Service) processCrystallizedState(crystallizedState *pbp2p.CrystallizedState) error {
var activeValidatorIndices []int
dynasty := crystallizedState.GetCurrentDynasty()
for i, validator := range crystallizedState.GetValidators() {
if validator.StartDynasty <= dynasty && dynasty < validator.EndDynasty {
activeValidatorIndices = append(activeValidatorIndices, i)
}
}
isValidatorIndexSet := false
// We then iteratate over the activeValidatorIndices to determine what index
// this running validator client corresponds to.
for _, val := range activeValidatorIndices {
// TODO(#258): Check the public key instead of withdrawal address. This will use BLS.
if isZeroAddress(crystallizedState.Validators[val].WithdrawalAddress) {
s.validatorIndex = val
isValidatorIndexSet = true
break
}
}
// If validator was not found in the validator set was not set, keep listening for
// crystallized states.
if !isValidatorIndexSet {
log.Debug("Validator index not found in latest crystallized state's active validator list")
return nil
}
// The validator needs to propose the next block.
// TODO(#545): Determine this from a gRPC stream from the beacon node
// instead.
s.responsibility = "proposer"
s.assignedSlot = s.CurrentBeaconSlot() + 2
log.WithField("assignedSlot", s.assignedSlot).Info("Validator selected as proposer")
return nil
}
// listenForProcessedAttestations receives processed attestations from the
// the beacon node's RPC server via gRPC streams.
func (s *Service) listenForProcessedAttestations(client pb.BeaconServiceClient) {
@@ -284,6 +284,19 @@ func (s *Service) ProcessedAttestationFeed() *event.Feed {
return s.processedAttestationFeed
}
// CurrentBeaconSlot based on the genesis timestamp of the protocol.
func (s *Service) CurrentBeaconSlot() uint64 {
secondsSinceGenesis := time.Since(s.genesisTimestamp).Seconds()
return uint64(math.Floor(secondsSinceGenesis / params.DefaultConfig().SlotDuration))
}
// CurrentCycleStartSlot returns the slot at which the current cycle started.
func (s *Service) CurrentCycleStartSlot() uint64 {
currentSlot := s.CurrentBeaconSlot()
cycleNum := math.Floor(float64(currentSlot) / float64(params.DefaultConfig().CycleLength))
return uint64(cycleNum) * params.DefaultConfig().CycleLength
}
// isZeroAddress compares a withdrawal address to an empty byte array.
func isZeroAddress(withdrawalAddress []byte) bool {
return bytes.Equal(withdrawalAddress, []byte{})

View File

@@ -31,8 +31,6 @@ type mockClient struct {
func (fc *mockClient) BeaconServiceClient() pb.BeaconServiceClient {
mockServiceClient := internal.NewMockBeaconServiceClient(fc.ctrl)
stateStream := internal.NewMockBeaconService_LatestCrystallizedStateClient(fc.ctrl)
stateStream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{}, io.EOF)
attesterStream := internal.NewMockBeaconService_LatestAttestationClient(fc.ctrl)
attesterStream.EXPECT().Recv().Return(&pbp2p.AggregatedAttestation{}, io.EOF)
@@ -41,10 +39,6 @@ func (fc *mockClient) BeaconServiceClient() pb.BeaconServiceClient {
&empty.Empty{},
).Return(attesterStream, nil)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
&empty.Empty{},
).Return(stateStream, nil)
return mockServiceClient
}
@@ -55,28 +49,11 @@ type mockLifecycleClient struct {
func (fc *mockLifecycleClient) BeaconServiceClient() pb.BeaconServiceClient {
mockServiceClient := internal.NewMockBeaconServiceClient(fc.ctrl)
stateStream := internal.NewMockBeaconService_LatestCrystallizedStateClient(fc.ctrl)
stateStream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{}, io.EOF)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
&empty.Empty{},
).Return(stateStream, nil)
validator1 := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte("0x0"), StartDynasty: 1, EndDynasty: 10}
validator2 := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte("0x1"), StartDynasty: 1, EndDynasty: 10}
validator3 := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte{}, StartDynasty: 1, EndDynasty: 10}
crystallized := &pbp2p.CrystallizedState{
Validators: []*pbp2p.ValidatorRecord{validator1, validator2, validator3},
CurrentDynasty: 5,
}
mockServiceClient.EXPECT().GenesisTimeAndCanonicalState(
mockServiceClient.EXPECT().CurrentAssignmentsAndGenesisTime(
gomock.Any(),
gomock.Any(),
).Return(&pb.GenesisTimeAndStateResponse{
LatestCrystallizedState: crystallized,
GenesisTimestamp: ptypes.TimestampNow(),
).Return(&pb.CurrentAssignmentsResponse{
GenesisTimestamp: ptypes.TimestampNow(),
}, nil)
attesterStream := internal.NewMockBeaconService_LatestAttestationClient(fc.ctrl)
@@ -85,6 +62,14 @@ func (fc *mockLifecycleClient) BeaconServiceClient() pb.BeaconServiceClient {
&empty.Empty{},
).Return(attesterStream, nil)
attesterStream.EXPECT().Recv().Return(&pbp2p.AggregatedAttestation{}, io.EOF)
cycleStream := internal.NewMockBeaconService_ValidatorAssignmentsClient(fc.ctrl)
mockServiceClient.EXPECT().ValidatorAssignments(
gomock.Any(),
gomock.Any(),
).Return(cycleStream, nil)
cycleStream.EXPECT().Recv().Return(&pb.ValidatorAssignmentResponse{}, io.EOF)
return mockServiceClient
}
@@ -103,6 +88,7 @@ func TestLifecycle(t *testing.T) {
if b.ProcessedAttestationFeed() == nil {
t.Error("ProcessedAttestationFeed empty")
}
b.slotAlignmentDuration = time.Millisecond * 10
b.Start()
time.Sleep(time.Millisecond * 10)
testutil.AssertLogsContain(t, hook, "Starting service")
@@ -139,14 +125,14 @@ func TestWaitForAssignmentProposer(t *testing.T) {
<-exitRoutine
}()
b.responsibility = "proposer"
b.role = pb.ValidatorRole_PROPOSER
b.genesisTimestamp = time.Now()
b.assignedSlot = 0
timeChan <- time.Now()
b.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "New beacon node slot interval")
testutil.AssertLogsContain(t, hook, "New beacon node slot")
}
func TestWaitForAssignmentProposerError(t *testing.T) {
@@ -168,7 +154,7 @@ func TestWaitForAssignmentProposerError(t *testing.T) {
<-exitRoutine
}()
b.responsibility = "proposer"
b.role = pb.ValidatorRole_PROPOSER
b.genesisTimestamp = time.Now()
b.assignedSlot = 0
timeChan <- time.Now()
@@ -197,14 +183,14 @@ func TestWaitForAssignmentAttester(t *testing.T) {
<-exitRoutine
}()
b.responsibility = "attester"
b.role = pb.ValidatorRole_ATTESTER
b.genesisTimestamp = time.Now()
b.assignedSlot = 0
timeChan <- time.Now()
b.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "New beacon node slot interval")
testutil.AssertLogsContain(t, hook, "New beacon node slot")
}
func TestWaitForAssignmentAttesterError(t *testing.T) {
@@ -226,7 +212,7 @@ func TestWaitForAssignmentAttesterError(t *testing.T) {
<-exitRoutine
}()
b.responsibility = "attester"
b.role = pb.ValidatorRole_ATTESTER
b.genesisTimestamp = time.Now()
b.assignedSlot = 0
timeChan <- time.Now()
@@ -236,91 +222,6 @@ func TestWaitForAssignmentAttesterError(t *testing.T) {
testutil.AssertLogsContain(t, hook, "failed")
}
func TestListenForCrystallizedStates(t *testing.T) {
hook := logTest.NewGlobal()
ctrl := gomock.NewController(t)
defer ctrl.Finish()
b := NewBeaconValidator(context.Background(), &mockClient{ctrl})
// Creating a faulty stream will trigger error.
stream := internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl)
mockServiceClient := internal.NewMockBeaconServiceClient(ctrl)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
gomock.Any(),
).Return(stream, errors.New("stream creation failed"))
b.listenForCrystallizedStates(mockServiceClient)
testutil.AssertLogsContain(t, hook, "stream creation failed")
// Stream recv error should trigger error log.
stream = internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl)
stream.EXPECT().Recv().Return(nil, errors.New("recv error"))
stream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{}, io.EOF)
mockServiceClient = internal.NewMockBeaconServiceClient(ctrl)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
gomock.Any(),
).Return(stream, nil)
b.listenForCrystallizedStates(mockServiceClient)
testutil.AssertLogsContain(t, hook, "recv error")
// If the current validator is not found within the active validators list, log a debug message.
validator := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte("0x01"), StartDynasty: 1, EndDynasty: 10}
stream = internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl)
stream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{Validators: []*pbp2p.ValidatorRecord{validator}, CurrentDynasty: 5}, nil)
stream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{}, io.EOF)
mockServiceClient = internal.NewMockBeaconServiceClient(ctrl)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
gomock.Any(),
).Return(stream, nil)
b.listenForCrystallizedStates(mockServiceClient)
testutil.AssertLogsContain(t, hook, "Validator index not found in latest crystallized state's active validator list")
// If the validator is the last index in the shuffled validator indices, it should be assigned
// to be a proposer.
validator1 := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte("0x0"), StartDynasty: 1, EndDynasty: 10}
validator2 := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte("0x1"), StartDynasty: 1, EndDynasty: 10}
validator3 := &pbp2p.ValidatorRecord{WithdrawalAddress: []byte{}, StartDynasty: 1, EndDynasty: 10}
stream = internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl)
stream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{Validators: []*pbp2p.ValidatorRecord{validator1, validator2, validator3}, CurrentDynasty: 5}, nil)
stream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{}, io.EOF)
mockServiceClient = internal.NewMockBeaconServiceClient(ctrl)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
gomock.Any(),
).Return(stream, nil)
b.listenForCrystallizedStates(mockServiceClient)
testutil.AssertLogsContain(t, hook, "Validator selected as proposer")
// Test that the routine exits when context is closed
stream = internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl)
stream.EXPECT().Recv().Return(&pbp2p.CrystallizedState{}, nil)
mockServiceClient = internal.NewMockBeaconServiceClient(ctrl)
mockServiceClient.EXPECT().LatestCrystallizedState(
gomock.Any(),
gomock.Any(),
).Return(stream, nil)
b.cancel()
b.listenForCrystallizedStates(mockServiceClient)
testutil.AssertLogsContain(t, hook, "Context has been canceled so shutting down the loop")
}
func TestListenForProcessedAttestations(t *testing.T) {
hook := logTest.NewGlobal()
ctrl := gomock.NewController(t)

View File

@@ -1,5 +1,5 @@
// Code generated by MockGen. DO NOT EDIT.
// Source: github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1 (interfaces: BeaconServiceClient,BeaconService_LatestCrystallizedStateClient,BeaconService_LatestAttestationClient)
// Source: github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1 (interfaces: BeaconServiceClient,BeaconService_LatestAttestationClient,BeaconService_ValidatorAssignmentsClient)
package internal
@@ -56,22 +56,22 @@ func (mr *MockBeaconServiceClientMockRecorder) CanonicalHead(arg0, arg1 interfac
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CanonicalHead", reflect.TypeOf((*MockBeaconServiceClient)(nil).CanonicalHead), varargs...)
}
// GenesisTimeAndCanonicalState mocks base method
func (m *MockBeaconServiceClient) GenesisTimeAndCanonicalState(arg0 context.Context, arg1 *empty.Empty, arg2 ...grpc.CallOption) (*v10.GenesisTimeAndStateResponse, error) {
// CurrentAssignmentsAndGenesisTime mocks base method
func (m *MockBeaconServiceClient) CurrentAssignmentsAndGenesisTime(arg0 context.Context, arg1 *v10.ValidatorAssignmentRequest, arg2 ...grpc.CallOption) (*v10.CurrentAssignmentsResponse, error) {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "GenesisTimeAndCanonicalState", varargs...)
ret0, _ := ret[0].(*v10.GenesisTimeAndStateResponse)
ret := m.ctrl.Call(m, "CurrentAssignmentsAndGenesisTime", varargs...)
ret0, _ := ret[0].(*v10.CurrentAssignmentsResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// GenesisTimeAndCanonicalState indicates an expected call of GenesisTimeAndCanonicalState
func (mr *MockBeaconServiceClientMockRecorder) GenesisTimeAndCanonicalState(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
// CurrentAssignmentsAndGenesisTime indicates an expected call of CurrentAssignmentsAndGenesisTime
func (mr *MockBeaconServiceClientMockRecorder) CurrentAssignmentsAndGenesisTime(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GenesisTimeAndCanonicalState", reflect.TypeOf((*MockBeaconServiceClient)(nil).GenesisTimeAndCanonicalState), varargs...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CurrentAssignmentsAndGenesisTime", reflect.TypeOf((*MockBeaconServiceClient)(nil).CurrentAssignmentsAndGenesisTime), varargs...)
}
// LatestAttestation mocks base method
@@ -92,131 +92,22 @@ func (mr *MockBeaconServiceClientMockRecorder) LatestAttestation(arg0, arg1 inte
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LatestAttestation", reflect.TypeOf((*MockBeaconServiceClient)(nil).LatestAttestation), varargs...)
}
// LatestCrystallizedState mocks base method
func (m *MockBeaconServiceClient) LatestCrystallizedState(arg0 context.Context, arg1 *empty.Empty, arg2 ...grpc.CallOption) (v10.BeaconService_LatestCrystallizedStateClient, error) {
// ValidatorAssignments mocks base method
func (m *MockBeaconServiceClient) ValidatorAssignments(arg0 context.Context, arg1 *v10.ValidatorAssignmentRequest, arg2 ...grpc.CallOption) (v10.BeaconService_ValidatorAssignmentsClient, error) {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "LatestCrystallizedState", varargs...)
ret0, _ := ret[0].(v10.BeaconService_LatestCrystallizedStateClient)
ret := m.ctrl.Call(m, "ValidatorAssignments", varargs...)
ret0, _ := ret[0].(v10.BeaconService_ValidatorAssignmentsClient)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// LatestCrystallizedState indicates an expected call of LatestCrystallizedState
func (mr *MockBeaconServiceClientMockRecorder) LatestCrystallizedState(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
// ValidatorAssignments indicates an expected call of ValidatorAssignments
func (mr *MockBeaconServiceClientMockRecorder) ValidatorAssignments(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LatestCrystallizedState", reflect.TypeOf((*MockBeaconServiceClient)(nil).LatestCrystallizedState), varargs...)
}
// MockBeaconService_LatestCrystallizedStateClient is a mock of BeaconService_LatestCrystallizedStateClient interface
type MockBeaconService_LatestCrystallizedStateClient struct {
ctrl *gomock.Controller
recorder *MockBeaconService_LatestCrystallizedStateClientMockRecorder
}
// MockBeaconService_LatestCrystallizedStateClientMockRecorder is the mock recorder for MockBeaconService_LatestCrystallizedStateClient
type MockBeaconService_LatestCrystallizedStateClientMockRecorder struct {
mock *MockBeaconService_LatestCrystallizedStateClient
}
// NewMockBeaconService_LatestCrystallizedStateClient creates a new mock instance
func NewMockBeaconService_LatestCrystallizedStateClient(ctrl *gomock.Controller) *MockBeaconService_LatestCrystallizedStateClient {
mock := &MockBeaconService_LatestCrystallizedStateClient{ctrl: ctrl}
mock.recorder = &MockBeaconService_LatestCrystallizedStateClientMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use
func (m *MockBeaconService_LatestCrystallizedStateClient) EXPECT() *MockBeaconService_LatestCrystallizedStateClientMockRecorder {
return m.recorder
}
// CloseSend mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) CloseSend() error {
ret := m.ctrl.Call(m, "CloseSend")
ret0, _ := ret[0].(error)
return ret0
}
// CloseSend indicates an expected call of CloseSend
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) CloseSend() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloseSend", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).CloseSend))
}
// Context mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) Context() context.Context {
ret := m.ctrl.Call(m, "Context")
ret0, _ := ret[0].(context.Context)
return ret0
}
// Context indicates an expected call of Context
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) Context() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Context", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).Context))
}
// Header mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) Header() (metadata.MD, error) {
ret := m.ctrl.Call(m, "Header")
ret0, _ := ret[0].(metadata.MD)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Header indicates an expected call of Header
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) Header() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Header", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).Header))
}
// Recv mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) Recv() (*v1.CrystallizedState, error) {
ret := m.ctrl.Call(m, "Recv")
ret0, _ := ret[0].(*v1.CrystallizedState)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Recv indicates an expected call of Recv
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) Recv() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Recv", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).Recv))
}
// RecvMsg mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) RecvMsg(arg0 interface{}) error {
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) RecvMsg(arg0 interface{}) *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).RecvMsg), arg0)
}
// SendMsg mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) SendMsg(arg0 interface{}) error {
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) SendMsg(arg0 interface{}) *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).SendMsg), arg0)
}
// Trailer mocks base method
func (m *MockBeaconService_LatestCrystallizedStateClient) Trailer() metadata.MD {
ret := m.ctrl.Call(m, "Trailer")
ret0, _ := ret[0].(metadata.MD)
return ret0
}
// Trailer indicates an expected call of Trailer
func (mr *MockBeaconService_LatestCrystallizedStateClientMockRecorder) Trailer() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Trailer", reflect.TypeOf((*MockBeaconService_LatestCrystallizedStateClient)(nil).Trailer))
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorAssignments", reflect.TypeOf((*MockBeaconServiceClient)(nil).ValidatorAssignments), varargs...)
}
// MockBeaconService_LatestAttestationClient is a mock of BeaconService_LatestAttestationClient interface
@@ -327,3 +218,112 @@ func (m *MockBeaconService_LatestAttestationClient) Trailer() metadata.MD {
func (mr *MockBeaconService_LatestAttestationClientMockRecorder) Trailer() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Trailer", reflect.TypeOf((*MockBeaconService_LatestAttestationClient)(nil).Trailer))
}
// MockBeaconService_ValidatorAssignmentsClient is a mock of BeaconService_ValidatorAssignmentsClient interface
type MockBeaconService_ValidatorAssignmentsClient struct {
ctrl *gomock.Controller
recorder *MockBeaconService_ValidatorAssignmentsClientMockRecorder
}
// MockBeaconService_ValidatorAssignmentsClientMockRecorder is the mock recorder for MockBeaconService_ValidatorAssignmentsClient
type MockBeaconService_ValidatorAssignmentsClientMockRecorder struct {
mock *MockBeaconService_ValidatorAssignmentsClient
}
// NewMockBeaconService_ValidatorAssignmentsClient creates a new mock instance
func NewMockBeaconService_ValidatorAssignmentsClient(ctrl *gomock.Controller) *MockBeaconService_ValidatorAssignmentsClient {
mock := &MockBeaconService_ValidatorAssignmentsClient{ctrl: ctrl}
mock.recorder = &MockBeaconService_ValidatorAssignmentsClientMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use
func (m *MockBeaconService_ValidatorAssignmentsClient) EXPECT() *MockBeaconService_ValidatorAssignmentsClientMockRecorder {
return m.recorder
}
// CloseSend mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) CloseSend() error {
ret := m.ctrl.Call(m, "CloseSend")
ret0, _ := ret[0].(error)
return ret0
}
// CloseSend indicates an expected call of CloseSend
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) CloseSend() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloseSend", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).CloseSend))
}
// Context mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) Context() context.Context {
ret := m.ctrl.Call(m, "Context")
ret0, _ := ret[0].(context.Context)
return ret0
}
// Context indicates an expected call of Context
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) Context() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Context", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).Context))
}
// Header mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) Header() (metadata.MD, error) {
ret := m.ctrl.Call(m, "Header")
ret0, _ := ret[0].(metadata.MD)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Header indicates an expected call of Header
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) Header() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Header", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).Header))
}
// Recv mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) Recv() (*v10.ValidatorAssignmentResponse, error) {
ret := m.ctrl.Call(m, "Recv")
ret0, _ := ret[0].(*v10.ValidatorAssignmentResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Recv indicates an expected call of Recv
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) Recv() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Recv", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).Recv))
}
// RecvMsg mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) RecvMsg(arg0 interface{}) error {
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) RecvMsg(arg0 interface{}) *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).RecvMsg), arg0)
}
// SendMsg mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) SendMsg(arg0 interface{}) error {
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) SendMsg(arg0 interface{}) *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).SendMsg), arg0)
}
// Trailer mocks base method
func (m *MockBeaconService_ValidatorAssignmentsClient) Trailer() metadata.MD {
ret := m.ctrl.Call(m, "Trailer")
ret0, _ := ret[0].(metadata.MD)
return ret0
}
// Trailer indicates an expected call of Trailer
func (mr *MockBeaconService_ValidatorAssignmentsClientMockRecorder) Trailer() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Trailer", reflect.TypeOf((*MockBeaconService_ValidatorAssignmentsClient)(nil).Trailer))
}

View File

@@ -1,5 +1,5 @@
// Code generated by MockGen. DO NOT EDIT.
// Source: github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1 (interfaces: ValidatorServiceClient,ValidatorService_ValidatorAssignmentClient)
// Source: github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1 (interfaces: ValidatorServiceClient)
package internal
@@ -10,7 +10,6 @@ import (
gomock "github.com/golang/mock/gomock"
v1 "github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1"
grpc "google.golang.org/grpc"
metadata "google.golang.org/grpc/metadata"
)
// MockValidatorServiceClient is a mock of ValidatorServiceClient interface
@@ -36,24 +35,6 @@ func (m *MockValidatorServiceClient) EXPECT() *MockValidatorServiceClientMockRec
return m.recorder
}
// ValidatorAssignment mocks base method
func (m *MockValidatorServiceClient) ValidatorAssignment(arg0 context.Context, arg1 *v1.ValidatorAssignmentRequest, arg2 ...grpc.CallOption) (v1.ValidatorService_ValidatorAssignmentClient, error) {
varargs := []interface{}{arg0, arg1}
for _, a := range arg2 {
varargs = append(varargs, a)
}
ret := m.ctrl.Call(m, "ValidatorAssignment", varargs...)
ret0, _ := ret[0].(v1.ValidatorService_ValidatorAssignmentClient)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// ValidatorAssignment indicates an expected call of ValidatorAssignment
func (mr *MockValidatorServiceClientMockRecorder) ValidatorAssignment(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call {
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorAssignment", reflect.TypeOf((*MockValidatorServiceClient)(nil).ValidatorAssignment), varargs...)
}
// ValidatorIndex mocks base method
func (m *MockValidatorServiceClient) ValidatorIndex(arg0 context.Context, arg1 *v1.PublicKey, arg2 ...grpc.CallOption) (*v1.IndexResponse, error) {
varargs := []interface{}{arg0, arg1}
@@ -107,112 +88,3 @@ func (mr *MockValidatorServiceClientMockRecorder) ValidatorSlotAndResponsibility
varargs := append([]interface{}{arg0, arg1}, arg2...)
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorSlotAndResponsibility", reflect.TypeOf((*MockValidatorServiceClient)(nil).ValidatorSlotAndResponsibility), varargs...)
}
// MockValidatorService_ValidatorAssignmentClient is a mock of ValidatorService_ValidatorAssignmentClient interface
type MockValidatorService_ValidatorAssignmentClient struct {
ctrl *gomock.Controller
recorder *MockValidatorService_ValidatorAssignmentClientMockRecorder
}
// MockValidatorService_ValidatorAssignmentClientMockRecorder is the mock recorder for MockValidatorService_ValidatorAssignmentClient
type MockValidatorService_ValidatorAssignmentClientMockRecorder struct {
mock *MockValidatorService_ValidatorAssignmentClient
}
// NewMockValidatorService_ValidatorAssignmentClient creates a new mock instance
func NewMockValidatorService_ValidatorAssignmentClient(ctrl *gomock.Controller) *MockValidatorService_ValidatorAssignmentClient {
mock := &MockValidatorService_ValidatorAssignmentClient{ctrl: ctrl}
mock.recorder = &MockValidatorService_ValidatorAssignmentClientMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use
func (m *MockValidatorService_ValidatorAssignmentClient) EXPECT() *MockValidatorService_ValidatorAssignmentClientMockRecorder {
return m.recorder
}
// CloseSend mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) CloseSend() error {
ret := m.ctrl.Call(m, "CloseSend")
ret0, _ := ret[0].(error)
return ret0
}
// CloseSend indicates an expected call of CloseSend
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) CloseSend() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CloseSend", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).CloseSend))
}
// Context mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) Context() context.Context {
ret := m.ctrl.Call(m, "Context")
ret0, _ := ret[0].(context.Context)
return ret0
}
// Context indicates an expected call of Context
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) Context() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Context", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).Context))
}
// Header mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) Header() (metadata.MD, error) {
ret := m.ctrl.Call(m, "Header")
ret0, _ := ret[0].(metadata.MD)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Header indicates an expected call of Header
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) Header() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Header", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).Header))
}
// Recv mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) Recv() (*v1.ValidatorAssignmentResponse, error) {
ret := m.ctrl.Call(m, "Recv")
ret0, _ := ret[0].(*v1.ValidatorAssignmentResponse)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Recv indicates an expected call of Recv
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) Recv() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Recv", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).Recv))
}
// RecvMsg mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) RecvMsg(arg0 interface{}) error {
ret := m.ctrl.Call(m, "RecvMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// RecvMsg indicates an expected call of RecvMsg
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) RecvMsg(arg0 interface{}) *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RecvMsg", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).RecvMsg), arg0)
}
// SendMsg mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) SendMsg(arg0 interface{}) error {
ret := m.ctrl.Call(m, "SendMsg", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// SendMsg indicates an expected call of SendMsg
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) SendMsg(arg0 interface{}) *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendMsg", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).SendMsg), arg0)
}
// Trailer mocks base method
func (m *MockValidatorService_ValidatorAssignmentClient) Trailer() metadata.MD {
ret := m.ctrl.Call(m, "Trailer")
ret0, _ := ret[0].(metadata.MD)
return ret0
}
// Trailer indicates an expected call of Trailer
func (mr *MockValidatorService_ValidatorAssignmentClientMockRecorder) Trailer() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Trailer", reflect.TypeOf((*MockValidatorService_ValidatorAssignmentClient)(nil).Trailer))
}

View File

@@ -11,7 +11,8 @@ import (
func DefaultConfig() *Config {
return &Config{
CollationSizeLimit: DefaultCollationSizeLimit(),
SlotDuration: 8,
SlotDuration: 8.0,
CycleLength: 64,
}
}
@@ -23,6 +24,7 @@ func DefaultCollationSizeLimit() int64 {
// Config contains configs for node to participate in the sharded universe.
type Config struct {
CollationSizeLimit int64 // CollationSizeLimit is the maximum size the serialized blobs in a collation can take.
SlotDuration int // SlotDuration in seconds.
CollationSizeLimit int64 // CollationSizeLimit is the maximum size the serialized blobs in a collation can take.
SlotDuration float64 // SlotDuration in seconds.
CycleLength uint64
}

View File

@@ -0,0 +1,14 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = ["clock.go"],
importpath = "github.com/prysmaticlabs/prysm/validator/utils",
visibility = ["//validator:__subpackages__"],
)
go_test(
name = "go_default_test",
srcs = ["clock_test.go"],
embed = [":go_default_library"],
)

13
validator/utils/clock.go Normal file
View File

@@ -0,0 +1,13 @@
package utils
import "time"
// BlockingWait sleeps until a specific time is reached after
// a certain duration. For example, if the genesis block
// was at 12:00:00PM and the current time is 12:00:03PM,
// we want the next slot to tick at 12:00:08PM so we can use
// this helper method to achieve that purpose.
func BlockingWait(duration time.Duration) {
d := time.Until(time.Now().Add(duration).Truncate(duration))
time.Sleep(d)
}

View File

@@ -0,0 +1,7 @@
package utils
import "testing"
func TestBlockingWait(t *testing.T) {
BlockingWait(0)
}