From 3509752b32eee26cb3329be1c4395811fb8627d1 Mon Sep 17 00:00:00 2001 From: james-prysm Date: Fri, 15 Nov 2024 14:49:46 -0600 Subject: [PATCH 01/10] renaming functions and services to better reflect what they do --- beacon-chain/execution/service.go | 6 ++- beacon-chain/node/node.go | 41 ++++++++++--------- .../sync/pending_attestations_queue.go | 10 ++--- .../sync/pending_attestations_queue_test.go | 10 ++--- beacon-chain/sync/validate_aggregate_proof.go | 4 +- 5 files changed, 37 insertions(+), 34 deletions(-) diff --git a/beacon-chain/execution/service.go b/beacon-chain/execution/service.go index 41dbfdb5dcd2..e2f502eb1720 100644 --- a/beacon-chain/execution/service.go +++ b/beacon-chain/execution/service.go @@ -316,6 +316,7 @@ func (s *Service) updateConnectedETH1(state bool) { s.updateBeaconNodeStats() } +// TODO: deprecate sometime after Electra // refers to the latest eth1 block which follows the condition: eth1_timestamp + // SECONDS_PER_ETH1_BLOCK * ETH1_FOLLOW_DISTANCE <= current_unix_time func (s *Service) followedBlockHeight(ctx context.Context) (uint64, error) { @@ -460,6 +461,7 @@ func safelyHandlePanic() { } } +// TODO: deprecate sometime after Electra func (s *Service) handleETH1FollowDistance() { defer safelyHandlePanic() ctx := s.ctx @@ -498,7 +500,7 @@ func (s *Service) handleETH1FollowDistance() { } } -func (s *Service) initPOWService() { +func (s *Service) initExecutionChainService() { // Use a custom logger to only log errors logCounter := 0 errorLogger := func(err error, msg string) { @@ -584,7 +586,7 @@ func (s *Service) initPOWService() { func (s *Service) run(done <-chan struct{}) { s.runError = nil - s.initPOWService() + s.initExecutionChainService() // Do not keep storing the finalized state as it is // no longer of use. s.removeStartupState() diff --git a/beacon-chain/node/node.go b/beacon-chain/node/node.go index fce4a4e56afd..ec20e5959ce1 100644 --- a/beacon-chain/node/node.go +++ b/beacon-chain/node/node.go @@ -320,8 +320,8 @@ func registerServices(cliCtx *cli.Context, beacon *BeaconNode, synchronizer *sta return errors.Wrap(err, "could not register backfill service") } - log.Debugln("Registering POW Chain Service") - if err := beacon.registerPOWChainService(); err != nil { + log.Debugln("Registering Execution Chain Service") + if err := beacon.registerExecutionChainService(); err != nil { return errors.Wrap(err, "could not register POW chain service") } @@ -729,8 +729,8 @@ func (b *BeaconNode) registerAttestationPool() error { } func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *startup.ClockSynchronizer, syncComplete chan struct{}) error { - var web3Service *execution.Service - if err := b.services.FetchService(&web3Service); err != nil { + var executionChainService *execution.Service + if err := b.services.FetchService(&executionChainService); err != nil { return err } @@ -745,8 +745,8 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st blockchain.WithForkChoiceStore(fc), blockchain.WithDatabase(b.db), blockchain.WithDepositCache(b.depositCache), - blockchain.WithChainStartFetcher(web3Service), - blockchain.WithExecutionEngineCaller(web3Service), + blockchain.WithChainStartFetcher(executionChainService), + blockchain.WithExecutionEngineCaller(executionChainService), blockchain.WithAttestationPool(b.attestationPool), blockchain.WithExitPool(b.exitPool), blockchain.WithSlashingPool(b.slashingsPool), @@ -772,10 +772,11 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st return b.services.RegisterService(blockchainService) } -func (b *BeaconNode) registerPOWChainService() error { +func (b *BeaconNode) registerExecutionChainService() error { if b.cliCtx.Bool(testSkipPowFlag) { return b.services.RegisterService(&execution.Service{}) } + // TODO: rename POW to execution bs, err := execution.NewPowchainCollector(b.ctx) if err != nil { return err @@ -798,17 +799,17 @@ func (b *BeaconNode) registerPOWChainService() error { execution.WithJwtId(b.cliCtx.String(flags.JwtId.Name)), execution.WithVerifierWaiter(b.verifyInitWaiter), ) - web3Service, err := execution.NewService(b.ctx, opts...) + executionChainService, err := execution.NewService(b.ctx, opts...) if err != nil { - return errors.Wrap(err, "could not register proof-of-work chain web3Service") + return errors.Wrap(err, "could not register execution chain service") } - return b.services.RegisterService(web3Service) + return b.services.RegisterService(executionChainService) } func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFillStore *backfill.Store) error { - var web3Service *execution.Service - if err := b.services.FetchService(&web3Service); err != nil { + var executionChainService *execution.Service + if err := b.services.FetchService(&executionChainService); err != nil { return err } @@ -839,7 +840,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil regularsync.WithStateGen(b.stateGen), regularsync.WithSlasherAttestationsFeed(b.slasherAttestationsFeed), regularsync.WithSlasherBlockHeadersFeed(b.slasherBlockHeadersFeed), - regularsync.WithReconstructor(web3Service), + regularsync.WithReconstructor(executionChainService), regularsync.WithClockWaiter(b.clockWaiter), regularsync.WithInitialSyncComplete(initialSyncComplete), regularsync.WithStateNotifier(b), @@ -910,8 +911,8 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error { return err } - var web3Service *execution.Service - if err := b.services.FetchService(&web3Service); err != nil { + var executionChainService *execution.Service + if err := b.services.FetchService(&executionChainService); err != nil { return err } @@ -939,7 +940,7 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error { chainStartFetcher = interopService } else { depositFetcher = b.depositCache - chainStartFetcher = web3Service + chainStartFetcher = executionChainService } host := b.cliCtx.String(flags.RPCHost.Name) @@ -954,8 +955,8 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error { p2pService := b.fetchP2P() rpcService := rpc.NewService(b.ctx, &rpc.Config{ - ExecutionEngineCaller: web3Service, - ExecutionReconstructor: web3Service, + ExecutionEngineCaller: executionChainService, + ExecutionReconstructor: executionChainService, Host: host, Port: port, BeaconMonitoringHost: beaconMonitoringHost, @@ -984,8 +985,8 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error { SlashingsPool: b.slashingsPool, BLSChangesPool: b.blsToExecPool, SyncCommitteeObjectPool: b.syncCommitteePool, - ExecutionChainService: web3Service, - ExecutionChainInfoFetcher: web3Service, + ExecutionChainService: executionChainService, + ExecutionChainInfoFetcher: executionChainService, ChainStartFetcher: chainStartFetcher, MockEth1Votes: mockEth1DataVotes, SyncService: syncService, diff --git a/beacon-chain/sync/pending_attestations_queue.go b/beacon-chain/sync/pending_attestations_queue.go index 448875c203c2..4f7e00c2ce2d 100644 --- a/beacon-chain/sync/pending_attestations_queue.go +++ b/beacon-chain/sync/pending_attestations_queue.go @@ -31,7 +31,7 @@ func (s *Service) processPendingAttsQueue() { mutex := new(sync.Mutex) async.RunEvery(s.ctx, processPendingAttsPeriod, func() { mutex.Lock() - if err := s.processPendingAtts(s.ctx); err != nil { + if err := s.processPendingAttsByBlkRoot(s.ctx); err != nil { log.WithError(err).Debugf("Could not process pending attestation: %v", err) } mutex.Unlock() @@ -42,7 +42,7 @@ func (s *Service) processPendingAttsQueue() { // 1. Clean up invalid pending attestations from the queue. // 2. Check if pending attestations can be processed when the block has arrived. // 3. Request block from a random peer if unable to proceed step 2. -func (s *Service) processPendingAtts(ctx context.Context) error { +func (s *Service) processPendingAttsByBlkRoot(ctx context.Context) error { ctx, span := trace.StartSpan(ctx, "processPendingAtts") defer span.End() @@ -66,7 +66,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error { s.pendingAttsLock.RUnlock() // has the pending attestation's missing block arrived and the node processed block yet? if s.cfg.beaconDB.HasBlock(ctx, bRoot) && (s.cfg.beaconDB.HasState(ctx, bRoot) || s.cfg.beaconDB.HasStateSummary(ctx, bRoot)) { - s.processAttestations(ctx, attestations) + s.processPendingAttestations(ctx, attestations) log.WithFields(logrus.Fields{ "blockRoot": hex.EncodeToString(bytesutil.Trunc(bRoot[:])), "pendingAttsCount": len(attestations), @@ -88,7 +88,7 @@ func (s *Service) processPendingAtts(ctx context.Context) error { return s.sendBatchRootRequest(ctx, pendingRoots, randGen) } -func (s *Service) processAttestations(ctx context.Context, attestations []ethpb.SignedAggregateAttAndProof) { +func (s *Service) processPendingAttestations(ctx context.Context, attestations []ethpb.SignedAggregateAttAndProof) { for _, signedAtt := range attestations { aggregate := signedAtt.AggregateAttestationAndProof().AggregateVal() data := aggregate.GetData() @@ -102,7 +102,7 @@ func (s *Service) processAttestations(ctx context.Context, attestations []ethpb. log.WithError(err).Debug("Pending aggregated attestation failed validation") } aggValid := pubsub.ValidationAccept == valRes - if s.validateBlockInAttestation(ctx, signedAtt) && aggValid { + if s.validateBlockPresenceOrQueueAttestation(ctx, signedAtt) && aggValid { if err := s.cfg.attPool.SaveAggregatedAttestation(aggregate); err != nil { log.WithError(err).Debug("Could not save aggregate attestation") continue diff --git a/beacon-chain/sync/pending_attestations_queue_test.go b/beacon-chain/sync/pending_attestations_queue_test.go index 6a7e7077a116..8de1cd11a304 100644 --- a/beacon-chain/sync/pending_attestations_queue_test.go +++ b/beacon-chain/sync/pending_attestations_queue_test.go @@ -52,7 +52,7 @@ func TestProcessPendingAtts_NoBlockRequestBlock(t *testing.T) { a := ðpb.AggregateAttestationAndProof{Aggregate: ðpb.Attestation{Data: ðpb.AttestationData{Target: ðpb.Checkpoint{Root: make([]byte, 32)}}}} r.blkRootToPendingAtts[[32]byte{'A'}] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: a}} - require.NoError(t, r.processPendingAtts(context.Background())) + require.NoError(t, r.processPendingAttsByBlkRoot(context.Background())) require.LogsContain(t, hook, "Requesting block by root") } @@ -135,7 +135,7 @@ func TestProcessPendingAtts_HasBlockSaveUnAggregatedAtt(t *testing.T) { require.NoError(t, r.cfg.beaconDB.SaveState(context.Background(), s, root)) r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}} - require.NoError(t, r.processPendingAtts(context.Background())) + require.NoError(t, r.processPendingAttsByBlkRoot(context.Background())) atts, err := r.cfg.attPool.UnaggregatedAttestations() require.NoError(t, err) @@ -183,7 +183,7 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) { require.NoError(t, r.cfg.beaconDB.SaveState(context.Background(), s, r32)) r.blkRootToPendingAtts[r32] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: a, Signature: make([]byte, fieldparams.BLSSignatureLength)}} - require.NoError(t, r.processPendingAtts(context.Background())) + require.NoError(t, r.processPendingAttsByBlkRoot(context.Background())) assert.Equal(t, false, p1.BroadcastCalled.Load(), "Broadcasted bad aggregate") // Clear pool. @@ -252,7 +252,7 @@ func TestProcessPendingAtts_NoBroadcastWithBadSignature(t *testing.T) { go r.verifierRoutine() r.blkRootToPendingAtts[r32] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}} - require.NoError(t, r.processPendingAtts(context.Background())) + require.NoError(t, r.processPendingAttsByBlkRoot(context.Background())) assert.Equal(t, true, p1.BroadcastCalled.Load(), "Could not broadcast the good aggregate") cancel() @@ -340,7 +340,7 @@ func TestProcessPendingAtts_HasBlockSaveAggregatedAtt(t *testing.T) { require.NoError(t, r.cfg.beaconDB.SaveState(context.Background(), s, root)) r.blkRootToPendingAtts[root] = []ethpb.SignedAggregateAttAndProof{ðpb.SignedAggregateAttestationAndProof{Message: aggregateAndProof, Signature: aggreSig}} - require.NoError(t, r.processPendingAtts(context.Background())) + require.NoError(t, r.processPendingAttsByBlkRoot(context.Background())) assert.Equal(t, 1, len(r.cfg.attPool.AggregatedAttestations()), "Did not save aggregated att") assert.DeepEqual(t, att, r.cfg.attPool.AggregatedAttestations()[0], "Incorrect saved att") diff --git a/beacon-chain/sync/validate_aggregate_proof.go b/beacon-chain/sync/validate_aggregate_proof.go index 7bb8cb592c61..84a11a1cc13e 100644 --- a/beacon-chain/sync/validate_aggregate_proof.go +++ b/beacon-chain/sync/validate_aggregate_proof.go @@ -118,7 +118,7 @@ func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, ms if seen { return pubsub.ValidationIgnore, nil } - if !s.validateBlockInAttestation(ctx, m) { + if !s.validateBlockPresenceOrQueueAttestation(ctx, m) { return pubsub.ValidationIgnore, nil } @@ -223,7 +223,7 @@ func (s *Service) validateAggregatedAtt(ctx context.Context, signed ethpb.Signed return s.validateWithBatchVerifier(ctx, "aggregate", set) } -func (s *Service) validateBlockInAttestation(ctx context.Context, satt ethpb.SignedAggregateAttAndProof) bool { +func (s *Service) validateBlockPresenceOrQueueAttestation(ctx context.Context, satt ethpb.SignedAggregateAttAndProof) bool { // Verify the block being voted and the processed state is in beaconDB. The block should have passed validation if it's in the beaconDB. blockRoot := bytesutil.ToBytes32(satt.AggregateAttestationAndProof().AggregateVal().GetData().BeaconBlockRoot) if !s.hasBlockAndState(ctx, blockRoot) { From 04a62213f610f932ff86896192e8a58a3d54649b Mon Sep 17 00:00:00 2001 From: james-prysm Date: Fri, 15 Nov 2024 15:23:26 -0600 Subject: [PATCH 02/10] more renames --- beacon-chain/blockchain/service.go | 2 +- beacon-chain/execution/service.go | 16 +- beacon-chain/execution/service_test.go | 8 +- .../rpc/prysm/v1alpha1/beacon/server.go | 2 +- .../validator/proposer_deposits_test.go | 2 +- .../v1alpha1/validator/proposer_eth1data.go | 6 +- .../validator/proposer_execution_payload.go | 2 +- .../proposer_execution_payload_test.go | 2 +- .../prysm/v1alpha1/validator/proposer_test.go | 222 ++++++++---------- .../rpc/prysm/v1alpha1/validator/server.go | 3 +- .../v1alpha1/validator/server_mainnet_test.go | 12 +- .../prysm/v1alpha1/validator/server_test.go | 12 +- .../v1alpha1/validator/status_mainnet_test.go | 10 +- .../prysm/v1alpha1/validator/status_test.go | 108 ++++----- beacon-chain/rpc/service.go | 5 +- 15 files changed, 197 insertions(+), 215 deletions(-) diff --git a/beacon-chain/blockchain/service.go b/beacon-chain/blockchain/service.go index c984a2f79750..f146364483a8 100644 --- a/beacon-chain/blockchain/service.go +++ b/beacon-chain/blockchain/service.go @@ -90,7 +90,7 @@ type config struct { StateGen *stategen.State SlasherAttestationsFeed *event.Feed WeakSubjectivityCheckpt *ethpb.Checkpoint - BlockFetcher execution.POWBlockFetcher + ExecutionBlockFetcher execution.BlockFetcher FinalizedStateAtStartUp state.BeaconState ExecutionEngineCaller execution.EngineCaller SyncChecker Checker diff --git a/beacon-chain/execution/service.go b/beacon-chain/execution/service.go index e2f502eb1720..41b3835a8ff7 100644 --- a/beacon-chain/execution/service.go +++ b/beacon-chain/execution/service.go @@ -83,8 +83,8 @@ type ChainInfoFetcher interface { ExecutionClientConnectionErr() error } -// POWBlockFetcher defines a struct that can retrieve mainchain blocks. -type POWBlockFetcher interface { +// BlockFetcher defines a struct that can retrieve mainchain blocks. +type BlockFetcher interface { BlockTimeByHeight(ctx context.Context, height *big.Int) (uint64, error) BlockByTimestamp(ctx context.Context, time uint64) (*types.HeaderInfo, error) BlockHashByHeight(ctx context.Context, height *big.Int) (common.Hash, error) @@ -95,7 +95,7 @@ type POWBlockFetcher interface { type Chain interface { ChainStartFetcher ChainInfoFetcher - POWBlockFetcher + BlockFetcher } // RPCClient defines the rpc methods required to interact with the eth1 node. @@ -206,7 +206,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) { } } - eth1Data, err := s.validPowchainData(ctx) + eth1Data, err := s.validExecutionChainData(ctx) if err != nil { return nil, errors.Wrap(err, "unable to validate powchain data") } @@ -812,9 +812,9 @@ func validateDepositContainers(ctrs []*ethpb.DepositContainer) bool { return true } -// Validates the current powchain data is saved and makes sure that any +// Validates the current execution chain data is saved and makes sure that any // embedded genesis state is correctly accounted for. -func (s *Service) validPowchainData(ctx context.Context) (*ethpb.ETH1ChainData, error) { +func (s *Service) validExecutionChainData(ctx context.Context) (*ethpb.ETH1ChainData, error) { genState, err := s.cfg.beaconDB.GenesisState(ctx) if err != nil { return nil, err @@ -844,11 +844,11 @@ func (s *Service) validPowchainData(ctx context.Context) (*ethpb.ETH1ChainData, BeaconState: pbState, DepositContainers: s.cfg.depositCache.AllDepositContainers(ctx), } - trie, ok := s.depositTrie.(*depositsnapshot.DepositTree) + depositTrie, ok := s.depositTrie.(*depositsnapshot.DepositTree) if !ok { return nil, errors.New("deposit trie was not EIP4881 DepositTree") } - eth1Data.DepositSnapshot, err = trie.ToProto() + eth1Data.DepositSnapshot, err = depositTrie.ToProto() if err != nil { return nil, err } diff --git a/beacon-chain/execution/service_test.go b/beacon-chain/execution/service_test.go index e6b6e2ea075e..ae6a8f2ffbaa 100644 --- a/beacon-chain/execution/service_test.go +++ b/beacon-chain/execution/service_test.go @@ -40,7 +40,7 @@ import ( var _ ChainStartFetcher = (*Service)(nil) var _ ChainInfoFetcher = (*Service)(nil) -var _ POWBlockFetcher = (*Service)(nil) +var _ ExecutionBlockFetcher = (*Service)(nil) var _ Chain = (*Service)(nil) type goodLogger struct { @@ -580,7 +580,7 @@ func TestService_EnsureConsistentPowchainData(t *testing.T) { assert.NoError(t, genState.SetSlot(1000)) require.NoError(t, s1.cfg.beaconDB.SaveGenesisData(context.Background(), genState)) - _, err = s1.validPowchainData(context.Background()) + _, err = s1.validExecutionChainData(context.Background()) require.NoError(t, err) eth1Data, err := s1.cfg.beaconDB.ExecutionChainData(context.Background()) @@ -611,7 +611,7 @@ func TestService_InitializeCorrectly(t *testing.T) { assert.NoError(t, genState.SetSlot(1000)) require.NoError(t, s1.cfg.beaconDB.SaveGenesisData(context.Background(), genState)) - _, err = s1.validPowchainData(context.Background()) + _, err = s1.validExecutionChainData(context.Background()) require.NoError(t, err) eth1Data, err := s1.cfg.beaconDB.ExecutionChainData(context.Background()) @@ -647,7 +647,7 @@ func TestService_EnsureValidPowchainData(t *testing.T) { DepositContainers: []*ethpb.DepositContainer{{Index: 1}}, }) require.NoError(t, err) - _, err = s1.validPowchainData(context.Background()) + _, err = s1.validExecutionChainData(context.Background()) require.NoError(t, err) eth1Data, err := s1.cfg.beaconDB.ExecutionChainData(context.Background()) diff --git a/beacon-chain/rpc/prysm/v1alpha1/beacon/server.go b/beacon-chain/rpc/prysm/v1alpha1/beacon/server.go index b1d3ef1ba3fa..ecec4942db86 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/beacon/server.go +++ b/beacon-chain/rpc/prysm/v1alpha1/beacon/server.go @@ -33,7 +33,7 @@ type Server struct { CanonicalFetcher blockchain.CanonicalFetcher FinalizationFetcher blockchain.FinalizationFetcher DepositFetcher cache.DepositFetcher - BlockFetcher execution.POWBlockFetcher + ExecutionBlockFetcher execution.BlockFetcher GenesisTimeFetcher blockchain.TimeFetcher StateNotifier statefeed.Notifier BlockNotifier blockfeed.Notifier diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_deposits_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_deposits_test.go index 3b58a88f4fe1..18b7586fc890 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_deposits_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_deposits_test.go @@ -187,7 +187,7 @@ func TestProposer_PendingDeposits_Electra(t *testing.T) { bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_eth1data.go b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_eth1data.go index 92eb2960dae9..d88e3f1d5e9f 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_eth1data.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_eth1data.go @@ -58,7 +58,7 @@ func (vs *Server) eth1DataMajorityVote(ctx context.Context, beaconState state.Be return vs.HeadFetcher.HeadETH1Data(), nil } - lastBlockByLatestValidTime, err := vs.Eth1BlockFetcher.BlockByTimestamp(ctx, latestValidTime) + lastBlockByLatestValidTime, err := vs.ExecutionBlockFetcher.BlockByTimestamp(ctx, latestValidTime) if err != nil { log.WithError(err).Error("Could not get last block by latest valid time") return vs.randomETH1DataVote(ctx) @@ -73,7 +73,7 @@ func (vs *Server) eth1DataMajorityVote(ctx context.Context, beaconState state.Be } if lastBlockDepositCount >= vs.HeadFetcher.HeadETH1Data().DepositCount { - h, err := vs.Eth1BlockFetcher.BlockHashByHeight(ctx, lastBlockByLatestValidTime.Number) + h, err := vs.ExecutionBlockFetcher.BlockHashByHeight(ctx, lastBlockByLatestValidTime.Number) if err != nil { log.WithError(err).Error("Could not get hash of last block by latest valid time") return vs.randomETH1DataVote(ctx) @@ -118,7 +118,7 @@ func (vs *Server) canonicalEth1Data( if features.Get().DisableStakinContractCheck && eth1BlockHash == [32]byte{} { return canonicalEth1Data, new(big.Int).SetInt64(0), nil } - _, canonicalEth1DataHeight, err := vs.Eth1BlockFetcher.BlockExists(ctx, eth1BlockHash) + _, canonicalEth1DataHeight, err := vs.ExecutionBlockFetcher.BlockExists(ctx, eth1BlockHash) if err != nil { return nil, nil, errors.Wrap(err, "could not fetch eth1data height") } diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload.go b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload.go index 56ba2798efcc..2e957e762d81 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload.go @@ -224,7 +224,7 @@ func (vs *Server) getTerminalBlockHashIfExists(ctx context.Context, transitionTi terminalBlockHash := params.BeaconConfig().TerminalBlockHash // Terminal block hash override takes precedence over terminal total difficulty. if params.BeaconConfig().TerminalBlockHash != params.BeaconConfig().ZeroHash { - exists, _, err := vs.Eth1BlockFetcher.BlockExists(ctx, terminalBlockHash) + exists, _, err := vs.ExecutionBlockFetcher.BlockExists(ctx, terminalBlockHash) if err != nil { return nil, false, err } diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload_test.go index ac52476d166e..7e59df9b9daf 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_execution_payload_test.go @@ -369,7 +369,7 @@ func TestServer_getTerminalBlockHashIfExists(t *testing.T) { c := powtesting.New() c.HashesByHeight[0] = tt.wantTerminalBlockHash vs := &Server{ - Eth1BlockFetcher: c, + ExecutionBlockFetcher: c, ExecutionEngineCaller: &powtesting.EngineClient{ ExecutionBlock: tt.currentPowBlock, BlockByHashMap: m, diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_test.go index 72060557591d..da7d6b6df4ec 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/proposer_test.go @@ -276,7 +276,7 @@ func TestServer_GetBeaconBlock_Bellatrix(t *testing.T) { } proposerServer := getProposerServer(db, beaconState, parentRoot[:]) - proposerServer.Eth1BlockFetcher = c + proposerServer.ExecutionBlockFetcher = c ed, err := blocks.NewWrappedExecutionData(payload) require.NoError(t, err) proposerServer.ExecutionEngineCaller = &mockExecution.EngineClient{ @@ -715,7 +715,7 @@ func getProposerServer(db db.HeadAccessDatabase, headState state.BeaconState, he BlockReceiver: mockChainService, ChainStartFetcher: &mockExecution.Chain{}, Eth1InfoFetcher: &mockExecution.Chain{}, - Eth1BlockFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, FinalizationFetcher: mockChainService, ForkFetcher: mockChainService, ForkchoiceFetcher: mockChainService, @@ -1009,10 +1009,10 @@ func TestProposer_ComputeStateRoot_OK(t *testing.T) { beaconState, parentRoot, privKeys := util.DeterministicGenesisStateWithGenesisBlock(t, ctx, db, 100) proposerServer := &Server{ - ChainStartFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - Eth1BlockFetcher: &mockExecution.Chain{}, - StateGen: stategen.New(db, doublylinkedtree.New()), + ChainStartFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + StateGen: stategen.New(db, doublylinkedtree.New()), } req := util.NewBeaconBlock() req.Block.ProposerIndex = 84 @@ -1079,11 +1079,11 @@ func TestProposer_PendingDeposits_Eth1DataVoteOK(t *testing.T) { require.NoError(t, err) bs := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, - HeadFetcher: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, + HeadFetcher: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, } // It should also return the recent deposits after their follow window. @@ -1214,7 +1214,7 @@ func TestProposer_PendingDeposits_OutsideEth1FollowWindow(t *testing.T) { bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, @@ -1347,7 +1347,7 @@ func TestProposer_PendingDeposits_FollowsCorrectEth1Block(t *testing.T) { bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, @@ -1450,7 +1450,7 @@ func TestProposer_PendingDeposits_CantReturnBelowStateEth1DepositIndex(t *testin bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, @@ -1550,7 +1550,7 @@ func TestProposer_PendingDeposits_CantReturnMoreThanMax(t *testing.T) { bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, @@ -1650,7 +1650,7 @@ func TestProposer_PendingDeposits_CantReturnMoreThanDepositCount(t *testing.T) { HeadFetcher: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, } @@ -1761,7 +1761,7 @@ func TestProposer_DepositTrie_UtilizesCachedFinalizedDeposits(t *testing.T) { bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, @@ -1890,7 +1890,7 @@ func TestProposer_DepositTrie_RebuildTrie(t *testing.T) { bs := &Server{ ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, BlockReceiver: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, @@ -1998,12 +1998,11 @@ func TestProposer_Eth1Data_MajorityVote_SpansGenesis(t *testing.T) { depositCache, err := depositsnapshot.New() require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{BlockHash: headBlockHash, DepositCount: 0}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{BlockHash: headBlockHash, DepositCount: 0}}, } beaconState, err := state_native.InitializeFromProtoPhase0(ðpb.BeaconState{ @@ -2061,12 +2060,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2097,12 +2095,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2133,12 +2130,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2170,12 +2166,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2207,12 +2202,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2244,12 +2238,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2274,12 +2267,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { currentEth1Data := ðpb.Eth1Data{DepositCount: 1, BlockHash: []byte("current")} ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: currentEth1Data}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: currentEth1Data}, } ctx := context.Background() @@ -2309,12 +2301,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2339,12 +2330,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2371,12 +2361,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { // Set the deposit count in current eth1data to exceed the latest most recent block's deposit count. currentEth1Data := ðpb.Eth1Data{DepositCount: 2, BlockHash: []byte("current")} ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: currentEth1Data}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: currentEth1Data}, } ctx := context.Background() @@ -2407,12 +2396,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2444,12 +2432,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2475,12 +2462,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2509,12 +2495,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 1}}, } ctx := context.Background() @@ -2548,12 +2533,11 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) { require.NoError(t, err) ps := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - Eth1BlockFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 0}}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mock.ChainService{ETH1Data: ðpb.Eth1Data{DepositCount: 0}}, } ctx := context.Background() @@ -2755,7 +2739,7 @@ func TestProposer_Deposits_ReturnsEmptyList_IfLatestEth1DataEqGenesisEth1Block(t HeadFetcher: &mock.ChainService{State: beaconState, Root: blkRoot[:]}, ChainStartFetcher: p, Eth1InfoFetcher: p, - Eth1BlockFetcher: p, + ExecutionBlockFetcher: p, DepositFetcher: depositCache, PendingDepositsFetcher: depositCache, } @@ -3057,10 +3041,10 @@ func TestProposer_GetParentHeadState(t *testing.T) { require.NoError(t, transition.UpdateNextSlotCache(ctx, parentRoot[:], parentState)) proposerServer := &Server{ - ChainStartFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - Eth1BlockFetcher: &mockExecution.Chain{}, - StateGen: stategen.New(db, doublylinkedtree.New()), + ChainStartFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + StateGen: stategen.New(db, doublylinkedtree.New()), } t.Run("successful reorg", func(tt *testing.T) { head, err := proposerServer.getParentStateFromReorgData(ctx, 1, parentRoot, parentRoot, headRoot) diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/server.go b/beacon-chain/rpc/prysm/v1alpha1/validator/server.go index 34256c733a5c..ad5a4f2516e0 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/server.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/server.go @@ -51,7 +51,6 @@ type Server struct { GenesisFetcher blockchain.GenesisFetcher FinalizationFetcher blockchain.FinalizationFetcher TimeFetcher blockchain.TimeFetcher - BlockFetcher execution.POWBlockFetcher DepositFetcher cache.DepositFetcher ChainStartFetcher execution.ChainStartFetcher Eth1InfoFetcher execution.ChainInfoFetcher @@ -67,7 +66,7 @@ type Server struct { BlockReceiver blockchain.BlockReceiver BlobReceiver blockchain.BlobReceiver MockEth1Votes bool - Eth1BlockFetcher execution.POWBlockFetcher + ExecutionBlockFetcher execution.BlockFetcher PendingDepositsFetcher depositsnapshot.PendingDepositsFetcher OperationNotifier opfeed.Notifier StateGen stategen.StateManager diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/server_mainnet_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/server_mainnet_test.go index a49e2b3704b8..1eb13ecf67e1 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/server_mainnet_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/server_mainnet_test.go @@ -74,12 +74,12 @@ func TestWaitForActivation_ValidatorOriginallyExists(t *testing.T) { s, err := state_native.InitializeFromProtoUnsafePhase0(beaconState) require.NoError(t, err) vs := &Server{ - Ctx: context.Background(), - ChainStartFetcher: &mockExecution.Chain{}, - BlockFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: s, Root: genesisRoot[:]}, + Ctx: context.Background(), + ChainStartFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: s, Root: genesisRoot[:]}, } req := ðpb.ValidatorActivationRequest{ PublicKeys: [][]byte{pubKey1, pubKey2}, diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/server_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/server_test.go index 58dfb2ea314b..1cae29de57cc 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/server_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/server_test.go @@ -74,12 +74,12 @@ func TestWaitForActivation_ContextClosed(t *testing.T) { require.NoError(t, err) vs := &Server{ - Ctx: ctx, - ChainStartFetcher: &mockExecution.Chain{}, - BlockFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: beaconState, Root: genesisRoot[:]}, + Ctx: ctx, + ChainStartFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: beaconState, Root: genesisRoot[:]}, } req := ðpb.ValidatorActivationRequest{ PublicKeys: [][]byte{pubKey(1)}, diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/status_mainnet_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/status_mainnet_test.go index c275d7877462..b42633f4f3ea 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/status_mainnet_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/status_mainnet_test.go @@ -73,11 +73,11 @@ func TestValidatorStatus_Active(t *testing.T) { }, } vs := &Server{ - ChainStartFetcher: p, - BlockFetcher: p, - Eth1InfoFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, + ChainStartFetcher: p, + ExecutionBlockFetcher: p, + Eth1InfoFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, } req := ðpb.ValidatorStatusRequest{ PublicKey: pubkey, diff --git a/beacon-chain/rpc/prysm/v1alpha1/validator/status_test.go b/beacon-chain/rpc/prysm/v1alpha1/validator/status_test.go index fec7a0f5bbaa..5ff4d098d62e 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/validator/status_test.go +++ b/beacon-chain/rpc/prysm/v1alpha1/validator/status_test.go @@ -53,8 +53,8 @@ func TestValidatorStatus_DepositedEth1(t *testing.T) { stateObj, err := state_native.InitializeFromProtoUnsafePhase0(ðpb.BeaconState{}) require.NoError(t, err) vs := &Server{ - DepositFetcher: depositCache, - BlockFetcher: p, + DepositFetcher: depositCache, + ExecutionBlockFetcher: p, HeadFetcher: &mockChain.ChainService{ State: stateObj, }, @@ -95,8 +95,8 @@ func TestValidatorStatus_Deposited(t *testing.T) { stateObj, err := state_native.InitializeFromProtoUnsafePhase0(ðpb.BeaconState{}) require.NoError(t, err) vs := &Server{ - DepositFetcher: depositCache, - BlockFetcher: p, + DepositFetcher: depositCache, + ExecutionBlockFetcher: p, HeadFetcher: &mockChain.ChainService{ State: stateObj, }, @@ -148,8 +148,8 @@ func TestValidatorStatus_PartiallyDeposited(t *testing.T) { }) require.NoError(t, err) vs := &Server{ - DepositFetcher: depositCache, - BlockFetcher: p, + DepositFetcher: depositCache, + ExecutionBlockFetcher: p, HeadFetcher: &mockChain.ChainService{ State: stateObj, }, @@ -207,8 +207,8 @@ func TestValidatorStatus_Pending_MultipleDeposits(t *testing.T) { require.NoError(t, err) require.NoError(t, stateObj.SetSlot(params.BeaconConfig().SlotsPerEpoch)) vs := &Server{ - DepositFetcher: depositCache, - BlockFetcher: p, + DepositFetcher: depositCache, + ExecutionBlockFetcher: p, HeadFetcher: &mockChain.ChainService{ State: stateObj, }, @@ -269,11 +269,11 @@ func TestValidatorStatus_Pending(t *testing.T) { }, } vs := &Server{ - ChainStartFetcher: p, - BlockFetcher: p, - Eth1InfoFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: st, Root: genesisRoot[:]}, + ChainStartFetcher: p, + ExecutionBlockFetcher: p, + Eth1InfoFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: st, Root: genesisRoot[:]}, } req := ðpb.ValidatorStatusRequest{ PublicKey: pubKey, @@ -331,11 +331,11 @@ func TestValidatorStatus_Exiting(t *testing.T) { }, } vs := &Server{ - ChainStartFetcher: p, - BlockFetcher: p, - Eth1InfoFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, + ChainStartFetcher: p, + ExecutionBlockFetcher: p, + Eth1InfoFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, } req := ðpb.ValidatorStatusRequest{ PublicKey: pubKey, @@ -390,11 +390,11 @@ func TestValidatorStatus_Slashing(t *testing.T) { }, } vs := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - DepositFetcher: depositCache, - BlockFetcher: p, - HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + DepositFetcher: depositCache, + ExecutionBlockFetcher: p, + HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, } req := ðpb.ValidatorStatusRequest{ PublicKey: pubKey, @@ -448,11 +448,11 @@ func TestValidatorStatus_Exited(t *testing.T) { }, } vs := &Server{ - ChainStartFetcher: p, - Eth1InfoFetcher: p, - BlockFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: st, Root: genesisRoot[:]}, + ChainStartFetcher: p, + Eth1InfoFetcher: p, + ExecutionBlockFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: st, Root: genesisRoot[:]}, } req := ðpb.ValidatorStatusRequest{ PublicKey: pubKey, @@ -534,12 +534,12 @@ func TestActivationStatus_OK(t *testing.T) { assert.NoError(t, depositCache.InsertDeposit(context.Background(), dep, 0, 1, root)) vs := &Server{ - Ctx: context.Background(), - ChainStartFetcher: &mockExecution.Chain{}, - BlockFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, + Ctx: context.Background(), + ChainStartFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, } activeExists, response, err := vs.activationStatus(context.Background(), pubKeys) require.NoError(t, err) @@ -685,11 +685,11 @@ func TestValidatorStatus_CorrectActivationQueue(t *testing.T) { }, } vs := &Server{ - ChainStartFetcher: p, - BlockFetcher: p, - Eth1InfoFetcher: p, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: st, Root: genesisRoot[:]}, + ChainStartFetcher: p, + ExecutionBlockFetcher: p, + Eth1InfoFetcher: p, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: st, Root: genesisRoot[:]}, } req := ðpb.ValidatorStatusRequest{ PublicKey: pbKey, @@ -765,13 +765,13 @@ func TestMultipleValidatorStatus_Pubkeys(t *testing.T) { assert.NoError(t, depositCache.InsertDeposit(context.Background(), dep, 0, 1, root)) vs := &Server{ - Ctx: context.Background(), - ChainStartFetcher: &mockExecution.Chain{}, - BlockFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - DepositFetcher: depositCache, - HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, - SyncChecker: &mockSync.Sync{IsSyncing: false}, + Ctx: context.Background(), + ChainStartFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + DepositFetcher: depositCache, + HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, + SyncChecker: &mockSync.Sync{IsSyncing: false}, } want := []*ethpb.ValidatorStatusResponse{ @@ -860,12 +860,12 @@ func TestMultipleValidatorStatus_Indices(t *testing.T) { require.NoError(t, err, "Could not get signing root") vs := &Server{ - Ctx: context.Background(), - ChainStartFetcher: &mockExecution.Chain{}, - BlockFetcher: &mockExecution.Chain{}, - Eth1InfoFetcher: &mockExecution.Chain{}, - HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, - SyncChecker: &mockSync.Sync{IsSyncing: false}, + Ctx: context.Background(), + ChainStartFetcher: &mockExecution.Chain{}, + ExecutionBlockFetcher: &mockExecution.Chain{}, + Eth1InfoFetcher: &mockExecution.Chain{}, + HeadFetcher: &mockChain.ChainService{State: stateObj, Root: genesisRoot[:]}, + SyncChecker: &mockSync.Sync{IsSyncing: false}, } want := []*ethpb.ValidatorStatusResponse{ @@ -931,8 +931,8 @@ func TestValidatorStatus_Invalid(t *testing.T) { stateObj, err := state_native.InitializeFromProtoUnsafePhase0(ðpb.BeaconState{}) require.NoError(t, err) vs := &Server{ - DepositFetcher: depositCache, - BlockFetcher: p, + DepositFetcher: depositCache, + ExecutionBlockFetcher: p, HeadFetcher: &mockChain.ChainService{ State: stateObj, }, diff --git a/beacon-chain/rpc/service.go b/beacon-chain/rpc/service.go index 4a56c0d4162e..1ef9ae4302b0 100644 --- a/beacon-chain/rpc/service.go +++ b/beacon-chain/rpc/service.go @@ -240,7 +240,6 @@ func NewService(ctx context.Context, cfg *Config) *Service { GenesisFetcher: s.cfg.GenesisFetcher, FinalizationFetcher: s.cfg.FinalizationFetcher, TimeFetcher: s.cfg.GenesisTimeFetcher, - BlockFetcher: s.cfg.ExecutionChainService, DepositFetcher: s.cfg.DepositFetcher, ChainStartFetcher: s.cfg.ChainStartFetcher, Eth1InfoFetcher: s.cfg.ExecutionChainService, @@ -253,7 +252,7 @@ func NewService(ctx context.Context, cfg *Config) *Service { BlockReceiver: s.cfg.BlockReceiver, BlobReceiver: s.cfg.BlobReceiver, MockEth1Votes: s.cfg.MockEth1Votes, - Eth1BlockFetcher: s.cfg.ExecutionChainService, + ExecutionBlockFetcher: s.cfg.ExecutionChainService, PendingDepositsFetcher: s.cfg.PendingDepositFetcher, SlashingsPool: s.cfg.SlashingsPool, StateGen: s.cfg.StateGen, @@ -294,7 +293,7 @@ func NewService(ctx context.Context, cfg *Config) *Service { CanonicalFetcher: s.cfg.CanonicalFetcher, ChainStartFetcher: s.cfg.ChainStartFetcher, DepositFetcher: s.cfg.DepositFetcher, - BlockFetcher: s.cfg.ExecutionChainService, + ExecutionBlockFetcher: s.cfg.ExecutionChainService, GenesisTimeFetcher: s.cfg.GenesisTimeFetcher, StateNotifier: s.cfg.StateNotifier, BlockNotifier: s.cfg.BlockNotifier, From 14233c730c73fb2ff076fb02a53ca0050081b7ab Mon Sep 17 00:00:00 2001 From: james-prysm Date: Fri, 15 Nov 2024 15:35:42 -0600 Subject: [PATCH 03/10] missed linting --- beacon-chain/execution/service_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon-chain/execution/service_test.go b/beacon-chain/execution/service_test.go index ae6a8f2ffbaa..8ec5e42b2572 100644 --- a/beacon-chain/execution/service_test.go +++ b/beacon-chain/execution/service_test.go @@ -40,7 +40,7 @@ import ( var _ ChainStartFetcher = (*Service)(nil) var _ ChainInfoFetcher = (*Service)(nil) -var _ ExecutionBlockFetcher = (*Service)(nil) +var _ BlockFetcher = (*Service)(nil) var _ Chain = (*Service)(nil) type goodLogger struct { From c5139a674e0702bf0ba2d6becbca15ac908abd3f Mon Sep 17 00:00:00 2001 From: james-prysm Date: Mon, 18 Nov 2024 09:28:56 -0600 Subject: [PATCH 04/10] fixing missed error update --- beacon-chain/node/node.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon-chain/node/node.go b/beacon-chain/node/node.go index ec20e5959ce1..aa2f42169b26 100644 --- a/beacon-chain/node/node.go +++ b/beacon-chain/node/node.go @@ -322,7 +322,7 @@ func registerServices(cliCtx *cli.Context, beacon *BeaconNode, synchronizer *sta log.Debugln("Registering Execution Chain Service") if err := beacon.registerExecutionChainService(); err != nil { - return errors.Wrap(err, "could not register POW chain service") + return errors.Wrap(err, "could not register execution chain service") } log.Debugln("Registering Attestation Pool Service") From 1c5665574991ea53ed47cd847b35ae2d24121395 Mon Sep 17 00:00:00 2001 From: james-prysm Date: Mon, 18 Nov 2024 10:37:58 -0600 Subject: [PATCH 05/10] small update to fix redundancy --- beacon-chain/core/helpers/beacon_committee.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/beacon-chain/core/helpers/beacon_committee.go b/beacon-chain/core/helpers/beacon_committee.go index 2e70cf5f0fd4..89ffaad25b0f 100644 --- a/beacon-chain/core/helpers/beacon_committee.go +++ b/beacon-chain/core/helpers/beacon_committee.go @@ -400,15 +400,12 @@ func UpdateCommitteeCache(ctx context.Context, state state.ReadOnlyBeaconState, return sortedIndices[i] < sortedIndices[j] }) - if err := committeeCache.AddCommitteeShuffledList(ctx, &cache.Committees{ + return committeeCache.AddCommitteeShuffledList(ctx, &cache.Committees{ ShuffledIndices: shuffledIndices, CommitteeCount: uint64(params.BeaconConfig().SlotsPerEpoch.Mul(count)), Seed: seed, SortedIndices: sortedIndices, - }); err != nil { - return err - } - return nil + }) } // UpdateProposerIndicesInCache updates proposer indices entry of the committee cache. From f5f3c5f0a9679b2dfcd642564abc840b0f09d9b8 Mon Sep 17 00:00:00 2001 From: james-prysm <90280386+james-prysm@users.noreply.github.com> Date: Mon, 18 Nov 2024 14:36:25 -0600 Subject: [PATCH 06/10] Update beacon-chain/sync/pending_attestations_queue.go Co-authored-by: Sammy Rosso <15244892+saolyn@users.noreply.github.com> --- beacon-chain/sync/pending_attestations_queue.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon-chain/sync/pending_attestations_queue.go b/beacon-chain/sync/pending_attestations_queue.go index 4f7e00c2ce2d..57ff0bfe1338 100644 --- a/beacon-chain/sync/pending_attestations_queue.go +++ b/beacon-chain/sync/pending_attestations_queue.go @@ -43,7 +43,7 @@ func (s *Service) processPendingAttsQueue() { // 2. Check if pending attestations can be processed when the block has arrived. // 3. Request block from a random peer if unable to proceed step 2. func (s *Service) processPendingAttsByBlkRoot(ctx context.Context) error { - ctx, span := trace.StartSpan(ctx, "processPendingAtts") + ctx, span := trace.StartSpan(ctx, "processPendingAttsByBlkRoot") defer span.End() // Before a node processes pending attestations queue, it verifies From cefd271afe7ab046f99b7dce1ad07517a62dc1d9 Mon Sep 17 00:00:00 2001 From: james-prysm Date: Mon, 18 Nov 2024 15:26:50 -0600 Subject: [PATCH 07/10] feedback --- beacon-chain/sync/pending_attestations_queue.go | 2 +- beacon-chain/sync/validate_aggregate_proof.go | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/beacon-chain/sync/pending_attestations_queue.go b/beacon-chain/sync/pending_attestations_queue.go index 57ff0bfe1338..525dfeaca77e 100644 --- a/beacon-chain/sync/pending_attestations_queue.go +++ b/beacon-chain/sync/pending_attestations_queue.go @@ -102,7 +102,7 @@ func (s *Service) processPendingAttestations(ctx context.Context, attestations [ log.WithError(err).Debug("Pending aggregated attestation failed validation") } aggValid := pubsub.ValidationAccept == valRes - if s.validateBlockPresenceOrQueueAttestation(ctx, signedAtt) && aggValid { + if s.checkBlockOrQueueAtt(ctx, signedAtt) && aggValid { if err := s.cfg.attPool.SaveAggregatedAttestation(aggregate); err != nil { log.WithError(err).Debug("Could not save aggregate attestation") continue diff --git a/beacon-chain/sync/validate_aggregate_proof.go b/beacon-chain/sync/validate_aggregate_proof.go index 2ebbc852932a..ff2cae1e1281 100644 --- a/beacon-chain/sync/validate_aggregate_proof.go +++ b/beacon-chain/sync/validate_aggregate_proof.go @@ -117,7 +117,7 @@ func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, ms if seen { return pubsub.ValidationIgnore, nil } - if !s.validateBlockPresenceOrQueueAttestation(ctx, m) { + if !s.checkBlockOrQueueAtt(ctx, m) { return pubsub.ValidationIgnore, nil } @@ -222,7 +222,8 @@ func (s *Service) validateAggregatedAtt(ctx context.Context, signed ethpb.Signed return s.validateWithBatchVerifier(ctx, "aggregate", set) } -func (s *Service) validateBlockPresenceOrQueueAttestation(ctx context.Context, satt ethpb.SignedAggregateAttAndProof) bool { +// checkBlockOrQueueAtt validates the block root from the pending attestation, if it's not there it queues the pending attestation +func (s *Service) checkBlockOrQueueAtt(ctx context.Context, satt ethpb.SignedAggregateAttAndProof) bool { // Verify the block being voted and the processed state is in beaconDB. The block should have passed validation if it's in the beaconDB. blockRoot := bytesutil.ToBytes32(satt.AggregateAttestationAndProof().AggregateVal().GetData().BeaconBlockRoot) if !s.hasBlockAndState(ctx, blockRoot) { From 495788d99bb5627b0423966f43bd3bc5855c06b0 Mon Sep 17 00:00:00 2001 From: james-prysm Date: Tue, 19 Nov 2024 17:19:02 -0600 Subject: [PATCH 08/10] renames in block queue --- beacon-chain/sync/pending_blocks_queue.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/beacon-chain/sync/pending_blocks_queue.go b/beacon-chain/sync/pending_blocks_queue.go index 5e639e042832..e2b51efc740c 100644 --- a/beacon-chain/sync/pending_blocks_queue.go +++ b/beacon-chain/sync/pending_blocks_queue.go @@ -59,9 +59,9 @@ func (s *Service) processPendingBlocks(ctx context.Context) error { // Remove old blocks from our expiration cache. s.deleteExpiredBlocksFromCache() - // Validate pending slots before processing. - if err := s.validatePendingSlots(); err != nil { - return errors.Wrap(err, "could not validate pending slots") + // remove expired pending blocks that are before the finalized checkpoint. + if err := s.removeExpiredPendingBlocksBySlot(); err != nil { + return errors.Wrap(err, "could not remove expired pending blocks before finalized checkpoint") } // Sort slots for ordered processing. @@ -82,14 +82,14 @@ func (s *Service) processPendingBlocks(ctx context.Context) error { ctx, span := startInnerSpan(ctx, slot) // Get blocks in the pending queue for the current slot. - blocksInCache := s.getBlocksInQueue(slot) - if len(blocksInCache) == 0 { + blocksInQueue := s.getBlocksInQueue(slot) + if len(blocksInQueue) == 0 { span.End() continue } // Process each block in the queue. - for _, b := range blocksInCache { + for _, b := range blocksInQueue { if err := blocks.BeaconBlockIsNil(b); err != nil { continue } @@ -355,10 +355,10 @@ func (s *Service) sortedPendingSlots() []primitives.Slot { return ss } -// validatePendingSlots validates the pending blocks +// removeExpiredPendingBlocksBySlot validates the pending blocks // by their slot. If they are before the current finalized // checkpoint, these blocks are removed from the queue. -func (s *Service) validatePendingSlots() error { +func (s *Service) removeExpiredPendingBlocksBySlot() error { s.pendingQueueLock.Lock() defer s.pendingQueueLock.Unlock() oldBlockRoots := make(map[[32]byte]bool) From 1b95d133fdaaa9935ac468eda5ef890239f9a083 Mon Sep 17 00:00:00 2001 From: james-prysm Date: Tue, 19 Nov 2024 17:40:23 -0600 Subject: [PATCH 09/10] adding comment for not clear process --- beacon-chain/sync/pending_blocks_queue.go | 1 + 1 file changed, 1 insertion(+) diff --git a/beacon-chain/sync/pending_blocks_queue.go b/beacon-chain/sync/pending_blocks_queue.go index e2b51efc740c..24e550dcab84 100644 --- a/beacon-chain/sync/pending_blocks_queue.go +++ b/beacon-chain/sync/pending_blocks_queue.go @@ -154,6 +154,7 @@ func (s *Service) processPendingBlocks(ctx context.Context) error { } span.End() } + // request missing parent blocks return s.sendBatchRootRequest(ctx, parentRoots, randGen) } From 1c5ccb6572a55c7a89246db3ccd60706892de634 Mon Sep 17 00:00:00 2001 From: james-prysm Date: Thu, 21 Nov 2024 08:59:45 -0600 Subject: [PATCH 10/10] fixing comment --- beacon-chain/sync/initial-sync/blocks_fetcher.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/beacon-chain/sync/initial-sync/blocks_fetcher.go b/beacon-chain/sync/initial-sync/blocks_fetcher.go index 0667983fedf0..2bc23382ac75 100644 --- a/beacon-chain/sync/initial-sync/blocks_fetcher.go +++ b/beacon-chain/sync/initial-sync/blocks_fetcher.go @@ -164,8 +164,8 @@ func newBlocksFetcher(ctx context.Context, cfg *blocksFetcherConfig) *blocksFetc } } -// This specifies the block batch limit the initial sync fetcher will use. In the event the user has provided -// and excessive number, this is automatically lowered. +// This specifies the block batch limit the initial sync fetcher will use. +// In the event the user has provided an excessive number, this is automatically lowered. func maxBatchLimit() int { currLimit := flags.Get().BlockBatchLimit maxLimit := params.BeaconConfig().MaxRequestBlocks