mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-10 05:47:59 -05:00
Compare commits
4 Commits
v6.0.5-rc.
...
focil
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
5f781a7df6 | ||
|
|
929bbb8209 | ||
|
|
cd0821d026 | ||
|
|
8b53887891 |
@@ -133,6 +133,66 @@ func SignedBLSChangesFromConsensus(src []*eth.SignedBLSToExecutionChange) []*Sig
|
||||
return changes
|
||||
}
|
||||
|
||||
func SignedInclusionListFromConsensus(src *eth.SignedInclusionList) *SignedInclusionList {
|
||||
transactions := make([]string, len(src.Message.Transactions))
|
||||
for i, transaction := range src.Message.Transactions {
|
||||
transactions[i] = hexutil.Encode(transaction)
|
||||
}
|
||||
|
||||
return &SignedInclusionList{
|
||||
Message: &InclusionList{
|
||||
Slot: fmt.Sprintf("%d", src.Message.Slot),
|
||||
ValidatorIndex: fmt.Sprintf("%d", src.Message.ValidatorIndex),
|
||||
InclusionListCommitteeRoot: hexutil.Encode(src.Message.InclusionListCommitteeRoot),
|
||||
Transactions: transactions,
|
||||
},
|
||||
Signature: hexutil.Encode(src.Signature),
|
||||
}
|
||||
}
|
||||
|
||||
func (s *SignedInclusionList) ToConsensus() (*eth.SignedInclusionList, error) {
|
||||
message, err := s.Message.ToConsensus()
|
||||
if err != nil {
|
||||
return nil, server.NewDecodeError(err, "Message")
|
||||
}
|
||||
signature, err := bytesutil.DecodeHexWithLength(s.Signature, fieldparams.BLSSignatureLength)
|
||||
if err != nil {
|
||||
return nil, server.NewDecodeError(err, "Signature")
|
||||
}
|
||||
return ð.SignedInclusionList{
|
||||
Message: message,
|
||||
Signature: signature,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *InclusionList) ToConsensus() (*eth.InclusionList, error) {
|
||||
slot, err := strconv.ParseUint(s.Slot, 10, 64)
|
||||
if err != nil {
|
||||
return nil, server.NewDecodeError(err, "Slot")
|
||||
}
|
||||
validatorIndex, err := strconv.ParseUint(s.ValidatorIndex, 10, 64)
|
||||
if err != nil {
|
||||
return nil, server.NewDecodeError(err, "ValidatorIndex")
|
||||
}
|
||||
inclusionListCommitteeRoot, err := bytesutil.DecodeHexWithLength(s.InclusionListCommitteeRoot, fieldparams.RootLength)
|
||||
if err != nil {
|
||||
return nil, server.NewDecodeError(err, "InclusionListCommitteeRoot")
|
||||
}
|
||||
transactions := make([][]byte, len(s.Transactions))
|
||||
for i, transaction := range s.Transactions {
|
||||
transactions[i], err = bytesutil.DecodeHexWithMaxLength(transaction, fieldparams.MaxBytesPerTxLength)
|
||||
if err != nil {
|
||||
return nil, server.NewDecodeError(err, fmt.Sprintf("Transactions[%d]", i))
|
||||
}
|
||||
}
|
||||
return ð.InclusionList{
|
||||
Slot: primitives.Slot(slot),
|
||||
ValidatorIndex: primitives.ValidatorIndex(validatorIndex),
|
||||
InclusionListCommitteeRoot: inclusionListCommitteeRoot,
|
||||
Transactions: transactions,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *Fork) ToConsensus() (*eth.Fork, error) {
|
||||
previousVersion, err := bytesutil.DecodeHexWithLength(s.PreviousVersion, 4)
|
||||
if err != nil {
|
||||
|
||||
@@ -103,6 +103,11 @@ type BlobSidecarEvent struct {
|
||||
VersionedHash string `json:"versioned_hash"`
|
||||
}
|
||||
|
||||
type InclusionListEvent struct {
|
||||
Version string `json:"version"`
|
||||
Data *SignedInclusionList `json:"data"`
|
||||
}
|
||||
|
||||
type LightClientFinalityUpdateEvent struct {
|
||||
Version string `json:"version"`
|
||||
Data *LightClientFinalityUpdate `json:"data"`
|
||||
|
||||
@@ -262,3 +262,15 @@ type PendingConsolidation struct {
|
||||
SourceIndex string `json:"source_index"`
|
||||
TargetIndex string `json:"target_index"`
|
||||
}
|
||||
|
||||
type SignedInclusionList struct {
|
||||
Message *InclusionList `json:"message"`
|
||||
Signature string `json:"signature"`
|
||||
}
|
||||
|
||||
type InclusionList struct {
|
||||
Slot string `json:"slot"`
|
||||
ValidatorIndex string `json:"validator_index"`
|
||||
InclusionListCommitteeRoot string `json:"inclusion_list_committee_root"`
|
||||
Transactions []string `json:"transactions"`
|
||||
}
|
||||
|
||||
@@ -12,6 +12,7 @@ go_library(
|
||||
"forkchoice_update_execution.go",
|
||||
"head.go",
|
||||
"head_sync_committee_info.go",
|
||||
"inclusion_list.go",
|
||||
"init_sync_process_block.go",
|
||||
"log.go",
|
||||
"merge_ascii_art.go",
|
||||
|
||||
@@ -42,6 +42,7 @@ type ForkchoiceFetcher interface {
|
||||
CachedHeadRoot() [32]byte
|
||||
GetProposerHead() [32]byte
|
||||
SetForkChoiceGenesisTime(time.Time)
|
||||
GetAttesterHead() [32]byte
|
||||
UpdateHead(context.Context, primitives.Slot)
|
||||
HighestReceivedBlockSlot() primitives.Slot
|
||||
ReceivedBlocksLastEpoch() (uint64, error)
|
||||
|
||||
@@ -27,6 +27,13 @@ func (s *Service) GetProposerHead() [32]byte {
|
||||
return s.cfg.ForkChoiceStore.GetProposerHead()
|
||||
}
|
||||
|
||||
// GetAttesterHead returns the corresponding value from forkchoice
|
||||
func (s *Service) GetAttesterHead() [32]byte {
|
||||
s.cfg.ForkChoiceStore.RLock()
|
||||
defer s.cfg.ForkChoiceStore.RUnlock()
|
||||
return s.cfg.ForkChoiceStore.GetAttesterHead()
|
||||
}
|
||||
|
||||
// SetForkChoiceGenesisTime sets the genesis time in Forkchoice
|
||||
func (s *Service) SetForkChoiceGenesisTime(timestamp time.Time) {
|
||||
s.cfg.ForkChoiceStore.Lock()
|
||||
|
||||
@@ -267,12 +267,25 @@ func (s *Service) notifyNewPayload(ctx context.Context, preStateVersion int,
|
||||
return false, errors.New("nil execution requests")
|
||||
}
|
||||
}
|
||||
lastValidHash, err = s.cfg.ExecutionEngineCaller.NewPayload(ctx, payload, versionedHashes, parentRoot, requests)
|
||||
|
||||
var txs [][]byte
|
||||
// Post-FOCIL, only consider the inclusion list constraint if it matches the current slot.
|
||||
if slots.ToEpoch(s.CurrentSlot()) >= params.BeaconConfig().Eip7805ForkEpoch && s.CurrentSlot() == blk.Block().Slot() {
|
||||
txs = s.inclusionListCache.Get(blk.Block().Slot() - 1)
|
||||
}
|
||||
lastValidHash, err = s.cfg.ExecutionEngineCaller.NewPayload(ctx, payload, versionedHashes, parentRoot, requests, txs)
|
||||
|
||||
switch {
|
||||
case err == nil:
|
||||
newPayloadValidNodeCount.Inc()
|
||||
return true, nil
|
||||
case errors.Is(err, execution.ErrBadInclusionListPayloadStatus):
|
||||
log.WithFields(logrus.Fields{
|
||||
"slot": blk.Block().Slot(),
|
||||
"parentRoot": fmt.Sprintf("%#x", parentRoot),
|
||||
}).Info("Called new payload but inclusion list didn't satisfy")
|
||||
blk.Block().MarkInclusionListNotSatisfied() // Cache the block root that fails to satisfy the inclusion list constraint.
|
||||
return true, nil
|
||||
case errors.Is(err, execution.ErrAcceptedSyncingPayloadStatus):
|
||||
newPayloadOptimisticNodeCount.Inc()
|
||||
log.WithFields(logrus.Fields{
|
||||
|
||||
72
beacon-chain/blockchain/inclusion_list.go
Normal file
72
beacon-chain/blockchain/inclusion_list.go
Normal file
@@ -0,0 +1,72 @@
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/time/slots"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const updateInclusionListBlockInterval = time.Second
|
||||
|
||||
// Routine that updates block building with inclusion lists one second before the slot starts.
|
||||
func (s *Service) updateBlockWithInclusionListRoutine() {
|
||||
if err := s.waitForSync(); err != nil {
|
||||
log.WithError(err).Error("Failed to wait for initial sync")
|
||||
return
|
||||
}
|
||||
|
||||
interval := time.Second*time.Duration(params.BeaconConfig().SecondsPerSlot) - updateInclusionListBlockInterval
|
||||
ticker := slots.NewSlotTickerWithIntervals(s.genesisTime, []time.Duration{interval})
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
return
|
||||
case <-ticker.C():
|
||||
s.updateBlockWithInclusionList(context.Background())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Updates block building with inclusion lists, the current payload ID, and the new upload ID.
|
||||
func (s *Service) updateBlockWithInclusionList(ctx context.Context) {
|
||||
currentSlot := s.CurrentSlot()
|
||||
|
||||
// Skip update if not in or past the FOCIL fork epoch.
|
||||
if slots.ToEpoch(currentSlot) < params.BeaconConfig().Eip7805ForkEpoch {
|
||||
return
|
||||
}
|
||||
|
||||
s.cfg.ForkChoiceStore.RLock()
|
||||
defer s.cfg.ForkChoiceStore.RUnlock()
|
||||
|
||||
headRoot := s.headRoot()
|
||||
id, found := s.cfg.PayloadIDCache.PayloadID(currentSlot+1, headRoot)
|
||||
if !found {
|
||||
return
|
||||
}
|
||||
|
||||
txs := s.inclusionListCache.Get(currentSlot)
|
||||
if len(txs) == 0 {
|
||||
log.WithField("slot", currentSlot).Warn("No inclusion list transactions found to update block")
|
||||
return
|
||||
}
|
||||
|
||||
newID, err := s.cfg.ExecutionEngineCaller.UpdatePayloadWithInclusionList(ctx, id, txs)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Failed to update block with inclusion list")
|
||||
return
|
||||
}
|
||||
|
||||
log.WithFields(logrus.Fields{
|
||||
"slot": currentSlot,
|
||||
"headRoot": fmt.Sprintf("%x", headRoot),
|
||||
"txs": len(txs),
|
||||
}).Info("Updated block with inclusion list")
|
||||
|
||||
s.cfg.PayloadIDCache.Set(currentSlot+1, headRoot, *newID)
|
||||
}
|
||||
@@ -275,3 +275,10 @@ func WithStartWaitingDataColumnSidecars(c chan bool) Option {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithInclusionListCache(c *cache.InclusionLists) Option {
|
||||
return func(s *Service) error {
|
||||
s.inclusionListCache = c
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
@@ -134,9 +134,6 @@ func (s *Service) processLightClientUpdates(cfg *postBlockProcessConfig) {
|
||||
if err := s.processLightClientUpdate(cfg); err != nil {
|
||||
log.WithError(err).Error("Failed to process light client update")
|
||||
}
|
||||
if err := s.processLightClientBootstrap(cfg); err != nil {
|
||||
log.WithError(err).Error("Failed to process light client bootstrap")
|
||||
}
|
||||
if err := s.processLightClientOptimisticUpdate(cfg.ctx, cfg.roblock, cfg.postState); err != nil {
|
||||
log.WithError(err).Error("Failed to process light client optimistic update")
|
||||
}
|
||||
@@ -192,20 +189,6 @@ func (s *Service) processLightClientUpdate(cfg *postBlockProcessConfig) error {
|
||||
return s.lcStore.SaveLightClientUpdate(cfg.ctx, period, update)
|
||||
}
|
||||
|
||||
// processLightClientBootstrap saves a light client bootstrap for this block
|
||||
// when feature flag is enabled.
|
||||
func (s *Service) processLightClientBootstrap(cfg *postBlockProcessConfig) error {
|
||||
blockRoot := cfg.roblock.Root()
|
||||
bootstrap, err := lightclient.NewLightClientBootstrapFromBeaconState(cfg.ctx, s.CurrentSlot(), cfg.postState, cfg.roblock)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "could not create light client bootstrap")
|
||||
}
|
||||
if err := s.lcStore.SaveLightClientBootstrap(cfg.ctx, blockRoot, bootstrap); err != nil {
|
||||
return errors.Wrapf(err, "could not save light client bootstrap")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Service) processLightClientFinalityUpdate(
|
||||
ctx context.Context,
|
||||
signed interfaces.ReadOnlySignedBeaconBlock,
|
||||
|
||||
@@ -2815,53 +2815,6 @@ func TestProcessLightClientUpdate(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestProcessLightClientBootstrap(t *testing.T) {
|
||||
featCfg := &features.Flags{}
|
||||
featCfg.EnableLightClient = true
|
||||
reset := features.InitWithReset(featCfg)
|
||||
defer reset()
|
||||
|
||||
s, tr := minimalTestService(t, WithLCStore())
|
||||
ctx := tr.ctx
|
||||
|
||||
for testVersion := version.Altair; testVersion <= version.Electra; testVersion++ {
|
||||
t.Run(version.String(testVersion), func(t *testing.T) {
|
||||
l := util.NewTestLightClient(t, testVersion)
|
||||
|
||||
s.genesisTime = time.Unix(time.Now().Unix()-(int64(params.BeaconConfig().VersionToForkEpochMap()[testVersion])*int64(params.BeaconConfig().SlotsPerEpoch)*int64(params.BeaconConfig().SecondsPerSlot)), 0)
|
||||
|
||||
currentBlockRoot, err := l.Block.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
roblock, err := consensusblocks.NewROBlockWithRoot(l.Block, currentBlockRoot)
|
||||
require.NoError(t, err)
|
||||
|
||||
err = s.cfg.BeaconDB.SaveBlock(ctx, roblock)
|
||||
require.NoError(t, err)
|
||||
err = s.cfg.BeaconDB.SaveState(ctx, l.State, currentBlockRoot)
|
||||
require.NoError(t, err)
|
||||
|
||||
cfg := &postBlockProcessConfig{
|
||||
ctx: ctx,
|
||||
roblock: roblock,
|
||||
postState: l.State,
|
||||
isValidPayload: true,
|
||||
}
|
||||
|
||||
require.NoError(t, s.processLightClientBootstrap(cfg))
|
||||
|
||||
// Check that the light client bootstrap is saved
|
||||
b, err := s.lcStore.LightClientBootstrap(ctx, currentBlockRoot)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, b)
|
||||
|
||||
stateRoot, err := l.State.HashTreeRoot(ctx)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, stateRoot, [32]byte(b.Header().Beacon().StateRoot))
|
||||
require.Equal(t, b.Version(), testVersion)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type testIsAvailableParams struct {
|
||||
options []Option
|
||||
blobKzgCommitmentsCount uint64
|
||||
|
||||
@@ -108,6 +108,14 @@ func (s *Service) spawnProcessAttestationsRoutine() {
|
||||
s.cfg.ForkChoiceStore.Unlock()
|
||||
|
||||
s.UpdateHead(s.ctx, slotInterval.Slot)
|
||||
|
||||
// Prune inclusion list that's more than 1 epoch old.
|
||||
// Mean at the second 0 of slot 100, we prune the inclusion list of slot 98.
|
||||
cachedSlot := primitives.Slot(0)
|
||||
if slotInterval.Slot > 2 {
|
||||
cachedSlot = slotInterval.Slot - 2
|
||||
}
|
||||
s.inclusionListCache.Delete(cachedSlot)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -300,15 +300,30 @@ func (s *Service) reportPostBlockProcessing(
|
||||
|
||||
func (s *Service) executePostFinalizationTasks(ctx context.Context, finalizedState state.BeaconState) {
|
||||
finalized := s.cfg.ForkChoiceStore.FinalizedCheckpoint()
|
||||
|
||||
// Send finalization event
|
||||
go func() {
|
||||
s.sendNewFinalizedEvent(ctx, finalizedState)
|
||||
}()
|
||||
|
||||
// Insert finalized deposits into finalized deposit trie
|
||||
depCtx, cancel := context.WithTimeout(context.Background(), depositDeadline)
|
||||
go func() {
|
||||
s.insertFinalizedDepositsAndPrune(depCtx, finalized.Root)
|
||||
cancel()
|
||||
}()
|
||||
|
||||
if features.Get().EnableLightClient {
|
||||
// Save a light client bootstrap for the finalized checkpoint
|
||||
go func() {
|
||||
err := s.lcStore.SaveLightClientBootstrap(s.ctx, finalized.Root)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not save light client bootstrap by block root")
|
||||
} else {
|
||||
log.Debugf("Saved light client bootstrap for finalized root %#x", finalized.Root)
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
// ReceiveBlockBatch processes the whole block batch at once, assuming the block batch is linear ,transitioning
|
||||
|
||||
@@ -9,7 +9,9 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
|
||||
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
|
||||
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/das"
|
||||
forkchoicetypes "github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice/types"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/operations/voluntaryexits"
|
||||
"github.com/OffchainLabs/prysm/v6/config/features"
|
||||
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
|
||||
@@ -19,6 +21,7 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
|
||||
ethpbv1 "github.com/OffchainLabs/prysm/v6/proto/eth/v1"
|
||||
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
|
||||
"github.com/OffchainLabs/prysm/v6/runtime/version"
|
||||
"github.com/OffchainLabs/prysm/v6/testing/assert"
|
||||
"github.com/OffchainLabs/prysm/v6/testing/require"
|
||||
"github.com/OffchainLabs/prysm/v6/testing/util"
|
||||
@@ -564,3 +567,48 @@ func Test_executePostFinalizationTasks(t *testing.T) {
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestProcessLightClientBootstrap(t *testing.T) {
|
||||
featCfg := &features.Flags{}
|
||||
featCfg.EnableLightClient = true
|
||||
reset := features.InitWithReset(featCfg)
|
||||
defer reset()
|
||||
|
||||
s, tr := minimalTestService(t, WithLCStore())
|
||||
ctx := tr.ctx
|
||||
|
||||
for testVersion := version.Altair; testVersion <= version.Electra; testVersion++ {
|
||||
t.Run(version.String(testVersion), func(t *testing.T) {
|
||||
l := util.NewTestLightClient(t, testVersion)
|
||||
|
||||
require.NoError(t, s.cfg.BeaconDB.SaveBlock(ctx, l.FinalizedBlock))
|
||||
finalizedBlockRoot, err := l.FinalizedBlock.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, s.cfg.BeaconDB.SaveState(ctx, l.FinalizedState, finalizedBlockRoot))
|
||||
|
||||
cp := l.AttestedState.FinalizedCheckpoint()
|
||||
require.DeepSSZEqual(t, finalizedBlockRoot, [32]byte(cp.Root))
|
||||
|
||||
require.NoError(t, s.cfg.ForkChoiceStore.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Epoch: cp.Epoch, Root: [32]byte(cp.Root)}))
|
||||
|
||||
sss, err := s.cfg.BeaconDB.State(ctx, finalizedBlockRoot)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, sss)
|
||||
|
||||
s.executePostFinalizationTasks(s.ctx, l.FinalizedState)
|
||||
|
||||
// wait for the goroutine to finish processing
|
||||
time.Sleep(1 * time.Second)
|
||||
|
||||
// Check that the light client bootstrap is saved
|
||||
b, err := s.lcStore.LightClientBootstrap(ctx, [32]byte(cp.Root))
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, b)
|
||||
|
||||
btst, err := lightClient.NewLightClientBootstrapFromBeaconState(ctx, l.FinalizedState.Slot(), l.FinalizedState, l.FinalizedBlock)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, btst, b)
|
||||
require.Equal(t, b.Version(), testVersion)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -70,6 +70,7 @@ type Service struct {
|
||||
lcStore *lightClient.Store
|
||||
startWaitingDataColumnSidecars chan bool // for testing purposes only
|
||||
syncCommitteeHeadState *cache.SyncCommitteeHeadStateCache
|
||||
inclusionListCache *cache.InclusionLists
|
||||
}
|
||||
|
||||
// config options for the service.
|
||||
@@ -222,6 +223,7 @@ func (s *Service) Start() {
|
||||
}
|
||||
s.spawnProcessAttestationsRoutine()
|
||||
go s.runLateBlockTasks()
|
||||
go s.updateBlockWithInclusionListRoutine()
|
||||
}
|
||||
|
||||
// Stop the blockchain service's main event loop and associated goroutines.
|
||||
|
||||
@@ -87,7 +87,7 @@ func (mb *mockBroadcaster) BroadcastLightClientFinalityUpdate(_ context.Context,
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mb *mockBroadcaster) BroadcastDataColumn(_ [fieldparams.RootLength]byte, _ uint64, _ *ethpb.DataColumnSidecar, _ ...chan<- bool) error {
|
||||
func (mb *mockBroadcaster) BroadcastDataColumn(_ [fieldparams.RootLength]byte, _ uint64, _ *ethpb.DataColumnSidecar) error {
|
||||
mb.broadcastCalled = true
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -640,6 +640,16 @@ func (s *ChainService) GetProposerHead() [32]byte {
|
||||
return [32]byte{}
|
||||
}
|
||||
|
||||
// GetAttesterHead mocks the same method in the chain service
|
||||
func (s *ChainService) GetAttesterHead() [32]byte {
|
||||
if s.ForkChoiceStore != nil {
|
||||
return s.ForkChoiceStore.GetAttesterHead()
|
||||
}
|
||||
var rootArr [32]byte
|
||||
copy(rootArr[:], s.Root)
|
||||
return rootArr
|
||||
}
|
||||
|
||||
// SetForkChoiceGenesisTime mocks the same method in the chain service
|
||||
func (s *ChainService) SetForkChoiceGenesisTime(timestamp time.Time) {
|
||||
if s.ForkChoiceStore != nil {
|
||||
|
||||
@@ -68,7 +68,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) {
|
||||
log.WithError(err).Error("Failed to check builder status")
|
||||
} else {
|
||||
log.WithField("endpoint", s.c.NodeURL()).Info("Builder has been configured")
|
||||
log.Warn("Outsourcing block construction to external builders adds non-trivial delay to block propagation time. " +
|
||||
log.Warn("Outsourcing block construction to external builders adds non-trivial delay to block propagation time. " +
|
||||
"Builder-constructed blocks or fallback blocks may get orphaned. Use at your own risk!")
|
||||
}
|
||||
}
|
||||
|
||||
2
beacon-chain/cache/BUILD.bazel
vendored
2
beacon-chain/cache/BUILD.bazel
vendored
@@ -15,6 +15,7 @@ go_library(
|
||||
"common.go",
|
||||
"doc.go",
|
||||
"error.go",
|
||||
"inclusion_list.go",
|
||||
"interfaces.go",
|
||||
"payload_id.go",
|
||||
"proposer_indices.go",
|
||||
@@ -75,6 +76,7 @@ go_test(
|
||||
"checkpoint_state_test.go",
|
||||
"committee_fuzz_test.go",
|
||||
"committee_test.go",
|
||||
"inclusion_list_test.go",
|
||||
"payload_id_test.go",
|
||||
"private_access_test.go",
|
||||
"proposer_indices_test.go",
|
||||
|
||||
105
beacon-chain/cache/inclusion_list.go
vendored
Normal file
105
beacon-chain/cache/inclusion_list.go
vendored
Normal file
@@ -0,0 +1,105 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"crypto/sha256"
|
||||
"sync"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
|
||||
)
|
||||
|
||||
type InclusionLists struct {
|
||||
mu sync.RWMutex
|
||||
ils map[primitives.Slot]map[primitives.ValidatorIndex]struct {
|
||||
txs [][]byte
|
||||
seenTwice bool
|
||||
isBeforeFreezeDeadline bool
|
||||
}
|
||||
}
|
||||
|
||||
// NewInclusionLists initializes a new InclusionLists instance.
|
||||
func NewInclusionLists() *InclusionLists {
|
||||
return &InclusionLists{
|
||||
ils: make(map[primitives.Slot]map[primitives.ValidatorIndex]struct {
|
||||
txs [][]byte
|
||||
seenTwice bool
|
||||
isBeforeFreezeDeadline bool
|
||||
}),
|
||||
}
|
||||
}
|
||||
|
||||
// Add adds a set of transactions for a specific slot and validator index.
|
||||
func (i *InclusionLists) Add(slot primitives.Slot, validatorIndex primitives.ValidatorIndex, txs [][]byte, isBeforeFreezeDeadline bool) {
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
if _, ok := i.ils[slot]; !ok {
|
||||
i.ils[slot] = make(map[primitives.ValidatorIndex]struct {
|
||||
txs [][]byte
|
||||
seenTwice bool
|
||||
isBeforeFreezeDeadline bool
|
||||
})
|
||||
}
|
||||
|
||||
entry := i.ils[slot][validatorIndex]
|
||||
if entry.seenTwice {
|
||||
return // No need to modify if already marked as seen twice.
|
||||
}
|
||||
|
||||
if entry.txs == nil {
|
||||
entry.txs = txs
|
||||
entry.isBeforeFreezeDeadline = isBeforeFreezeDeadline
|
||||
} else {
|
||||
entry.seenTwice = true
|
||||
entry.txs = nil // Clear transactions to save space if seen twice.
|
||||
}
|
||||
i.ils[slot][validatorIndex] = entry
|
||||
}
|
||||
|
||||
// Get retrieves unique transactions for a specific slot.
|
||||
func (i *InclusionLists) Get(slot primitives.Slot) [][]byte {
|
||||
i.mu.RLock()
|
||||
defer i.mu.RUnlock()
|
||||
|
||||
ils, exists := i.ils[slot]
|
||||
if !exists {
|
||||
return [][]byte{}
|
||||
}
|
||||
|
||||
var uniqueTxs [][]byte
|
||||
seen := make(map[[32]byte]struct{})
|
||||
for _, entry := range ils {
|
||||
if !entry.isBeforeFreezeDeadline {
|
||||
continue
|
||||
}
|
||||
for _, tx := range entry.txs {
|
||||
hash := sha256.Sum256(tx)
|
||||
if _, duplicate := seen[hash]; !duplicate {
|
||||
uniqueTxs = append(uniqueTxs, tx)
|
||||
seen[hash] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
return uniqueTxs
|
||||
}
|
||||
|
||||
// Delete removes all inclusion lists for a specific slot.
|
||||
func (i *InclusionLists) Delete(slot primitives.Slot) {
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
delete(i.ils, slot)
|
||||
}
|
||||
|
||||
// SeenTwice checks if a validator's transactions were marked as seen twice for a specific slot.
|
||||
func (i *InclusionLists) SeenTwice(slot primitives.Slot, idx primitives.ValidatorIndex) bool {
|
||||
i.mu.RLock()
|
||||
defer i.mu.RUnlock()
|
||||
|
||||
ils, exists := i.ils[slot]
|
||||
if !exists {
|
||||
return false
|
||||
}
|
||||
|
||||
entry, exists := ils[idx]
|
||||
return exists && entry.seenTwice
|
||||
}
|
||||
81
beacon-chain/cache/inclusion_list_test.go
vendored
Normal file
81
beacon-chain/cache/inclusion_list_test.go
vendored
Normal file
@@ -0,0 +1,81 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestInclusionLists(t *testing.T) {
|
||||
il := NewInclusionLists()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
actions func()
|
||||
expectedGet [][]byte
|
||||
expectedTwice bool
|
||||
}{
|
||||
{
|
||||
name: "Add single validator with unique transactions",
|
||||
actions: func() {
|
||||
il.Add(1, 1, [][]byte{[]byte("tx1"), []byte("tx2")}, true)
|
||||
},
|
||||
expectedGet: [][]byte{[]byte("tx1"), []byte("tx2")},
|
||||
expectedTwice: false,
|
||||
},
|
||||
{
|
||||
name: "Add duplicate transactions for second validator",
|
||||
actions: func() {
|
||||
il.Add(1, 2, [][]byte{[]byte("tx1"), []byte("tx3")}, true)
|
||||
},
|
||||
expectedGet: [][]byte{[]byte("tx1"), []byte("tx2"), []byte("tx3")},
|
||||
expectedTwice: false,
|
||||
},
|
||||
{
|
||||
name: "Mark validator as seen twice",
|
||||
actions: func() {
|
||||
il.Add(1, 1, [][]byte{[]byte("tx4")}, true)
|
||||
},
|
||||
expectedGet: [][]byte{[]byte("tx1"), []byte("tx3")},
|
||||
expectedTwice: true,
|
||||
},
|
||||
{
|
||||
name: "Delete a slot",
|
||||
actions: func() {
|
||||
il.Delete(1)
|
||||
},
|
||||
expectedGet: nil,
|
||||
expectedTwice: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
tt.actions()
|
||||
|
||||
// Check Get results
|
||||
got := il.Get(1)
|
||||
if !compareTransactions(got, tt.expectedGet) {
|
||||
t.Errorf("unexpected Get result: got %v, want %v", got, tt.expectedGet)
|
||||
}
|
||||
|
||||
// Check SeenTwice result for validator 1
|
||||
gotTwice := il.SeenTwice(1, 1)
|
||||
if gotTwice != tt.expectedTwice {
|
||||
t.Errorf("unexpected SeenTwice result: got %v, want %v", gotTwice, tt.expectedTwice)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// compareTransactions compares two slices of byte slices for equality.
|
||||
func compareTransactions(a, b [][]byte) bool {
|
||||
if len(a) != len(b) {
|
||||
return false
|
||||
}
|
||||
for i := range a {
|
||||
if !bytes.Equal(a[i], b[i]) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
@@ -46,6 +46,9 @@ const (
|
||||
|
||||
// DataColumnReceived is sent after a data column has been seen after gossip validation rules.
|
||||
DataColumnReceived = 12
|
||||
|
||||
// InclusionListReceived is sent after an inclusion list is received from gossip or rpc
|
||||
InclusionListReceived = 13
|
||||
)
|
||||
|
||||
// UnAggregatedAttReceivedData is the data sent with UnaggregatedAttReceived events.
|
||||
@@ -82,6 +85,11 @@ type BlobSidecarReceivedData struct {
|
||||
Blob *blocks.VerifiedROBlob
|
||||
}
|
||||
|
||||
// InclusionListReceivedData is the data sent with InclusionListReceived events.
|
||||
type InclusionListReceivedData struct {
|
||||
SignedInclusionList *ethpb.SignedInclusionList
|
||||
}
|
||||
|
||||
// ProposerSlashingReceivedData is the data sent with ProposerSlashingReceived events.
|
||||
type ProposerSlashingReceivedData struct {
|
||||
ProposerSlashing *ethpb.ProposerSlashing
|
||||
|
||||
@@ -7,6 +7,7 @@ go_library(
|
||||
"beacon_committee.go",
|
||||
"block.go",
|
||||
"genesis.go",
|
||||
"inclusion_list.go",
|
||||
"legacy.go",
|
||||
"metrics.go",
|
||||
"randao.go",
|
||||
@@ -21,6 +22,7 @@ go_library(
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/core/signing:go_default_library",
|
||||
"//beacon-chain/core/time:go_default_library",
|
||||
"//beacon-chain/forkchoice/types:go_default_library",
|
||||
"//beacon-chain/state:go_default_library",
|
||||
|
||||
@@ -272,9 +272,10 @@ func BeaconCommittee(
|
||||
|
||||
// CommitteeAssignment represents committee list, committee index, and to be attested slot for a given epoch.
|
||||
type CommitteeAssignment struct {
|
||||
Committee []primitives.ValidatorIndex
|
||||
AttesterSlot primitives.Slot
|
||||
CommitteeIndex primitives.CommitteeIndex
|
||||
Committee []primitives.ValidatorIndex
|
||||
AttesterSlot primitives.Slot
|
||||
CommitteeIndex primitives.CommitteeIndex
|
||||
InclusionListCommitteeSlot primitives.Slot
|
||||
}
|
||||
|
||||
// VerifyAssignmentEpoch verifies if the given epoch is valid for assignment based on the provided state.
|
||||
@@ -445,6 +446,22 @@ func CommitteeAssignments(ctx context.Context, state state.BeaconState, epoch pr
|
||||
assignments[vIndex].CommitteeIndex = primitives.CommitteeIndex(j)
|
||||
}
|
||||
}
|
||||
if slots.ToEpoch(slot) >= params.BeaconConfig().Eip7805ForkEpoch {
|
||||
// Retrieve inclusion list committee assignments for the slot and update the assignments map.
|
||||
indices, err := GetInclusionListCommittee(ctx, state, slot)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get inclusion list committee")
|
||||
}
|
||||
for _, vIndex := range indices {
|
||||
if _, exists := vals[vIndex]; !exists {
|
||||
continue
|
||||
}
|
||||
if _, exists := assignments[vIndex]; !exists {
|
||||
assignments[vIndex] = &CommitteeAssignment{}
|
||||
}
|
||||
assignments[vIndex].InclusionListCommitteeSlot = slot
|
||||
}
|
||||
}
|
||||
}
|
||||
return assignments, nil
|
||||
}
|
||||
|
||||
107
beacon-chain/core/helpers/inclusion_list.go
Normal file
107
beacon-chain/core/helpers/inclusion_list.go
Normal file
@@ -0,0 +1,107 @@
|
||||
package helpers
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/signing"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
|
||||
"github.com/OffchainLabs/prysm/v6/crypto/bls"
|
||||
eth "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
|
||||
"github.com/OffchainLabs/prysm/v6/time/slots"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
var (
|
||||
errNilIl = errors.New("nil inclusion list")
|
||||
errNilCommitteeRoot = errors.New("nil inclusion list committee root")
|
||||
errNilSignature = errors.New("nil signature")
|
||||
errIncorrectState = errors.New("incorrect state version")
|
||||
)
|
||||
|
||||
// ValidateNilSignedInclusionList validates that a SignedInclusionList is not nil and contains a signature.
|
||||
func ValidateNilSignedInclusionList(il *eth.SignedInclusionList) error {
|
||||
if il == nil {
|
||||
return errNilIl
|
||||
}
|
||||
if il.Signature == nil {
|
||||
return errNilSignature
|
||||
}
|
||||
return ValidateNilInclusionList(il.Message)
|
||||
}
|
||||
|
||||
// ValidateNilInclusionList validates that an InclusionList is not nil and contains a committee root.
|
||||
func ValidateNilInclusionList(il *eth.InclusionList) error {
|
||||
if il == nil {
|
||||
return errNilIl
|
||||
}
|
||||
if il.InclusionListCommitteeRoot == nil {
|
||||
return errNilCommitteeRoot
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetInclusionListCommittee retrieves the validator indices assigned to the inclusion list committee
|
||||
// for a given slot. Returns an error if the state or slot does not meet the required constraints.
|
||||
func GetInclusionListCommittee(ctx context.Context, state state.ReadOnlyBeaconState, slot primitives.Slot) ([]primitives.ValidatorIndex, error) {
|
||||
if slots.ToEpoch(slot) < params.BeaconConfig().Eip7805ForkEpoch {
|
||||
return nil, errIncorrectState
|
||||
}
|
||||
epoch := slots.ToEpoch(slot)
|
||||
seed, err := Seed(state, epoch, params.BeaconConfig().DomainInclusionListCommittee)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get seed")
|
||||
}
|
||||
indices, err := ActiveValidatorIndices(ctx, state, epoch)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
start := uint64(slot%params.BeaconConfig().SlotsPerEpoch) * params.BeaconConfig().InclusionListCommitteeSize
|
||||
end := start + params.BeaconConfig().InclusionListCommitteeSize
|
||||
|
||||
shuffledIndices := make([]primitives.ValidatorIndex, len(indices))
|
||||
copy(shuffledIndices, indices)
|
||||
shuffledList, err := UnshuffleList(shuffledIndices, seed)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return shuffledList[start:end], nil
|
||||
}
|
||||
|
||||
// ValidateInclusionListSignature verifies the signature on a SignedInclusionList against the public key
|
||||
// of the validator specified in the inclusion list.
|
||||
func ValidateInclusionListSignature(ctx context.Context, st state.ReadOnlyBeaconState, il *eth.SignedInclusionList) error {
|
||||
if err := ValidateNilSignedInclusionList(il); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
val, err := st.ValidatorAtIndex(il.Message.ValidatorIndex)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pub, err := bls.PublicKeyFromBytes(val.PublicKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
sig, err := bls.SignatureFromBytes(il.Signature)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
currentEpoch := slots.ToEpoch(st.Slot())
|
||||
domain, err := signing.Domain(st.Fork(), currentEpoch, params.BeaconConfig().DomainInclusionListCommittee, st.GenesisValidatorsRoot())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
root, err := signing.ComputeSigningRoot(il.Message, domain)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !sig.Verify(pub, root[:]) {
|
||||
return signing.ErrSigFailedToVerify
|
||||
}
|
||||
return nil
|
||||
}
|
||||
@@ -42,10 +42,31 @@ func (s *Store) LightClientBootstrap(ctx context.Context, blockRoot [32]byte) (i
|
||||
return bootstrap, nil
|
||||
}
|
||||
|
||||
func (s *Store) SaveLightClientBootstrap(ctx context.Context, blockRoot [32]byte, bootstrap interfaces.LightClientBootstrap) error {
|
||||
func (s *Store) SaveLightClientBootstrap(ctx context.Context, blockRoot [32]byte) error {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
blk, err := s.beaconDB.Block(ctx, blockRoot)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "failed to fetch block for root %x", blockRoot)
|
||||
}
|
||||
if blk == nil {
|
||||
return errors.Errorf("failed to fetch block for root %x", blockRoot)
|
||||
}
|
||||
|
||||
state, err := s.beaconDB.State(ctx, blockRoot)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "failed to fetch state for block root %x", blockRoot)
|
||||
}
|
||||
if state == nil {
|
||||
return errors.Errorf("failed to fetch state for block root %x", blockRoot)
|
||||
}
|
||||
|
||||
bootstrap, err := NewLightClientBootstrapFromBeaconState(ctx, state.Slot(), state, blk)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "failed to create light client bootstrap for block root %x", blockRoot)
|
||||
}
|
||||
|
||||
// Save the light client bootstrap to the database
|
||||
if err := s.beaconDB.SaveLightClientBootstrap(ctx, blockRoot[:], bootstrap); err != nil {
|
||||
return err
|
||||
|
||||
@@ -7,6 +7,7 @@ go_library(
|
||||
"block_reader.go",
|
||||
"deposit.go",
|
||||
"engine_client.go",
|
||||
"engine_client_focil.go",
|
||||
"errors.go",
|
||||
"log.go",
|
||||
"log_processing.go",
|
||||
@@ -62,6 +63,7 @@ go_library(
|
||||
"//time/slots:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//accounts/abi/bind:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//beacon/engine:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
|
||||
|
||||
@@ -129,19 +129,27 @@ type Reconstructor interface {
|
||||
// EngineCaller defines a client that can interact with an Ethereum
|
||||
// execution node's engine service via JSON-RPC.
|
||||
type EngineCaller interface {
|
||||
NewPayload(ctx context.Context, payload interfaces.ExecutionData, versionedHashes []common.Hash, parentBlockRoot *common.Hash, executionRequests *pb.ExecutionRequests) ([]byte, error)
|
||||
NewPayload(ctx context.Context, payload interfaces.ExecutionData, versionedHashes []common.Hash, parentBlockRoot *common.Hash, executionRequests *pb.ExecutionRequests, ilTxs [][]byte) ([]byte, error)
|
||||
ForkchoiceUpdated(
|
||||
ctx context.Context, state *pb.ForkchoiceState, attrs payloadattribute.Attributer,
|
||||
) (*pb.PayloadIDBytes, []byte, error)
|
||||
GetPayload(ctx context.Context, payloadId [8]byte, slot primitives.Slot) (*blocks.GetPayloadResponse, error)
|
||||
ExecutionBlockByHash(ctx context.Context, hash common.Hash, withTxs bool) (*pb.ExecutionBlock, error)
|
||||
GetTerminalBlockHash(ctx context.Context, transitionTime uint64) ([]byte, bool, error)
|
||||
GetInclusionList(ctx context.Context, parentHash [32]byte) ([][]byte, error)
|
||||
UpdatePayloadWithInclusionList(ctx context.Context, payloadID primitives.PayloadID, txs [][]byte) (*primitives.PayloadID, error)
|
||||
}
|
||||
|
||||
var ErrEmptyBlockHash = errors.New("Block hash is empty 0x0000...")
|
||||
|
||||
// NewPayload request calls the engine_newPayloadVX method via JSON-RPC.
|
||||
func (s *Service) NewPayload(ctx context.Context, payload interfaces.ExecutionData, versionedHashes []common.Hash, parentBlockRoot *common.Hash, executionRequests *pb.ExecutionRequests) ([]byte, error) {
|
||||
func (s *Service) NewPayload(
|
||||
ctx context.Context,
|
||||
payload interfaces.ExecutionData,
|
||||
versionedHashes []common.Hash,
|
||||
parentBlockRoot *common.Hash,
|
||||
executionRequests *pb.ExecutionRequests,
|
||||
ilTxs [][]byte) ([]byte, error) {
|
||||
ctx, span := trace.StartSpan(ctx, "powchain.engine-api-client.NewPayload")
|
||||
defer span.End()
|
||||
start := time.Now()
|
||||
@@ -171,7 +179,7 @@ func (s *Service) NewPayload(ctx context.Context, payload interfaces.ExecutionDa
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
} else {
|
||||
} else if ilTxs == nil {
|
||||
flattenedRequests, err := pb.EncodeExecutionRequests(executionRequests)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to encode execution requests")
|
||||
@@ -180,6 +188,19 @@ func (s *Service) NewPayload(ctx context.Context, payload interfaces.ExecutionDa
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
} else {
|
||||
flattenedRequests, err := pb.EncodeExecutionRequests(executionRequests)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to encode execution requests")
|
||||
}
|
||||
hexIlTxs := make([]hexutil.Bytes, len(ilTxs))
|
||||
for i, tx := range ilTxs {
|
||||
hexIlTxs[i] = tx
|
||||
}
|
||||
err = s.rpcClient.CallContext(ctx, result, NewPayloadMethodV5, payloadPb, versionedHashes, parentBlockRoot, flattenedRequests, hexIlTxs)
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
}
|
||||
default:
|
||||
return nil, errors.New("unknown execution data type")
|
||||
@@ -196,6 +217,8 @@ func (s *Service) NewPayload(ctx context.Context, payload interfaces.ExecutionDa
|
||||
return result.LatestValidHash, ErrInvalidPayloadStatus
|
||||
case pb.PayloadStatus_VALID:
|
||||
return result.LatestValidHash, nil
|
||||
case pb.PayloadStatus_INCLUSION_LIST_NOT_SATISFIED:
|
||||
return result.LatestValidHash, ErrBadInclusionListPayloadStatus
|
||||
default:
|
||||
return nil, ErrUnknownPayloadStatus
|
||||
}
|
||||
|
||||
79
beacon-chain/execution/engine_client_focil.go
Normal file
79
beacon-chain/execution/engine_client_focil.go
Normal file
@@ -0,0 +1,79 @@
|
||||
package execution
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
|
||||
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
|
||||
"github.com/ethereum/go-ethereum/beacon/engine"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
)
|
||||
|
||||
const (
|
||||
NewPayloadMethodV5 = "engine_newPayloadV5" // Do we really need this?
|
||||
GetInclusionListV1 = "engine_getInclusionListV1"
|
||||
UpdatePayloadWithInclusionListV1 = "engine_updatePayloadWithInclusionListV1"
|
||||
)
|
||||
|
||||
// GetInclusionList fetches the inclusion list for a given parent hash by invoking the execution engine RPC.
|
||||
// It uses a context with a timeout defined by the Beacon configuration.
|
||||
// Implements: https://github.com/ethereum/execution-apis/pull/609
|
||||
func (s *Service) GetInclusionList(ctx context.Context, parentHash [32]byte) ([][]byte, error) {
|
||||
ctx, span := trace.StartSpan(ctx, "execution.GetInclusionList")
|
||||
defer span.End()
|
||||
|
||||
start := time.Now()
|
||||
defer func() {
|
||||
getInclusionListLatency.Observe(float64(time.Since(start).Milliseconds()))
|
||||
}()
|
||||
|
||||
timeout := time.Duration(params.BeaconConfig().ExecutionEngineTimeoutValue) * time.Second
|
||||
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(timeout))
|
||||
defer cancel()
|
||||
|
||||
var result []hexutil.Bytes
|
||||
err := s.rpcClient.CallContext(ctx, &result, GetInclusionListV1, common.Hash(parentHash))
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
|
||||
bytesResult := make([][]byte, len(result))
|
||||
for i, b := range result {
|
||||
bytesResult[i] = b
|
||||
}
|
||||
|
||||
return bytesResult, nil
|
||||
}
|
||||
|
||||
// UpdatePayloadWithInclusionList updates a payload with a provided inclusion list of transactions.
|
||||
// It uses a context with a timeout defined by the Beacon configuration and returns the new payload ID.
|
||||
// Implements: https://github.com/ethereum/execution-apis/pull/609
|
||||
func (s *Service) UpdatePayloadWithInclusionList(ctx context.Context, payloadID primitives.PayloadID, txs [][]byte) (*primitives.PayloadID, error) {
|
||||
ctx, span := trace.StartSpan(ctx, "execution.UpdatePayloadWithInclusionList")
|
||||
defer span.End()
|
||||
|
||||
start := time.Now()
|
||||
defer func() {
|
||||
updatePayloadWithInclusionListLatency.Observe(float64(time.Since(start).Milliseconds()))
|
||||
}()
|
||||
|
||||
timeout := time.Duration(params.BeaconConfig().ExecutionEngineTimeoutValue) * time.Second
|
||||
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(timeout))
|
||||
defer cancel()
|
||||
|
||||
hexTxs := make([]hexutil.Bytes, len(txs))
|
||||
for i, tx := range txs {
|
||||
hexTxs[i] = tx
|
||||
}
|
||||
|
||||
result := &engine.PayloadID{}
|
||||
err := s.rpcClient.CallContext(ctx, result, UpdatePayloadWithInclusionListV1, engine.PayloadID(payloadID), hexTxs)
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
|
||||
return (*primitives.PayloadID)(result), nil
|
||||
}
|
||||
@@ -127,7 +127,7 @@ func TestClient_IPC(t *testing.T) {
|
||||
require.Equal(t, true, ok)
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayload(req)
|
||||
require.NoError(t, err)
|
||||
latestValidHash, err := srv.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
latestValidHash, err := srv.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, bytesutil.ToBytes32(want.LatestValidHash), bytesutil.ToBytes32(latestValidHash))
|
||||
})
|
||||
@@ -138,7 +138,7 @@ func TestClient_IPC(t *testing.T) {
|
||||
require.Equal(t, true, ok)
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadCapella(req)
|
||||
require.NoError(t, err)
|
||||
latestValidHash, err := srv.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
latestValidHash, err := srv.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, bytesutil.ToBytes32(want.LatestValidHash), bytesutil.ToBytes32(latestValidHash))
|
||||
})
|
||||
@@ -603,7 +603,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayload(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -617,7 +617,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadCapella(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -631,7 +631,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadDeneb(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil, nil)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -670,7 +670,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
},
|
||||
}
|
||||
client := newPayloadV4Setup(t, want, execPayload, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests, nil)
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -684,7 +684,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayload(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrAcceptedSyncingPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -698,7 +698,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadCapella(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrAcceptedSyncingPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -712,7 +712,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadDeneb(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil, nil)
|
||||
require.ErrorIs(t, ErrAcceptedSyncingPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -751,7 +751,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
},
|
||||
}
|
||||
client := newPayloadV4Setup(t, want, execPayload, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests, nil)
|
||||
require.ErrorIs(t, ErrAcceptedSyncingPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -765,7 +765,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayload(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrInvalidBlockHashPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -779,7 +779,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadCapella(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrInvalidBlockHashPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -793,7 +793,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadDeneb(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil, nil)
|
||||
require.ErrorIs(t, ErrInvalidBlockHashPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -831,7 +831,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
},
|
||||
}
|
||||
client := newPayloadV4Setup(t, want, execPayload, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests, nil)
|
||||
require.ErrorIs(t, ErrInvalidBlockHashPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
@@ -845,7 +845,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayload(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrInvalidPayloadStatus, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -859,7 +859,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadCapella(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrInvalidPayloadStatus, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -873,7 +873,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayloadDeneb(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, nil, nil)
|
||||
require.ErrorIs(t, ErrInvalidPayloadStatus, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -912,7 +912,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
},
|
||||
}
|
||||
client := newPayloadV4Setup(t, want, execPayload, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{'a'}, requests, nil)
|
||||
require.ErrorIs(t, ErrInvalidPayloadStatus, err)
|
||||
require.DeepEqual(t, want.LatestValidHash, resp)
|
||||
})
|
||||
@@ -926,7 +926,7 @@ func TestClient_HTTP(t *testing.T) {
|
||||
// We call the RPC method via HTTP and expect a proper result.
|
||||
wrappedPayload, err := blocks.WrappedExecutionPayload(execPayload)
|
||||
require.NoError(t, err)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil)
|
||||
resp, err := client.NewPayload(ctx, wrappedPayload, []common.Hash{}, &common.Hash{}, nil, nil)
|
||||
require.ErrorIs(t, ErrUnknownPayloadStatus, err)
|
||||
require.DeepEqual(t, []uint8(nil), resp)
|
||||
})
|
||||
|
||||
@@ -34,5 +34,6 @@ var (
|
||||
// ErrRequestTooLarge when the request is too large
|
||||
ErrRequestTooLarge = errors.New("request too large")
|
||||
// ErrUnsupportedVersion represents a case where a payload is requested for a block type that doesn't have a known mapping.
|
||||
ErrUnsupportedVersion = errors.New("unknown ExecutionPayload schema for block version")
|
||||
ErrUnsupportedVersion = errors.New("unknown ExecutionPayload schema for block version")
|
||||
ErrBadInclusionListPayloadStatus = errors.New("payload did not satisfy inclusion list")
|
||||
)
|
||||
|
||||
@@ -71,4 +71,18 @@ var (
|
||||
Name: "execution_payload_bodies_count",
|
||||
Help: "The number of requested payload bodies is too large",
|
||||
})
|
||||
getInclusionListLatency = promauto.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Name: "get_inclusion_list_v1_latency_milliseconds",
|
||||
Help: "Captures RPC latency for getInclusionListV1 in milliseconds",
|
||||
Buckets: []float64{25, 50, 100, 200, 500, 1000, 2000, 4000},
|
||||
},
|
||||
)
|
||||
updatePayloadWithInclusionListLatency = promauto.NewHistogram(
|
||||
prometheus.HistogramOpts{
|
||||
Name: "update_payload_inclusion_list_v1_latency_milliseconds",
|
||||
Help: "Captures RPC latency for updatePayloadWithInclusionListV1 in milliseconds",
|
||||
Buckets: []float64{25, 50, 100, 200, 500, 1000, 2000, 4000},
|
||||
},
|
||||
)
|
||||
)
|
||||
|
||||
@@ -43,8 +43,7 @@ type EngineClient struct {
|
||||
ErrorDataColumnSidecars error
|
||||
}
|
||||
|
||||
// NewPayload --
|
||||
func (e *EngineClient) NewPayload(_ context.Context, _ interfaces.ExecutionData, _ []common.Hash, _ *common.Hash, _ *pb.ExecutionRequests) ([]byte, error) {
|
||||
func (e *EngineClient) NewPayload(ctx context.Context, payload interfaces.ExecutionData, versionedHashes []common.Hash, parentBlockRoot *common.Hash, executionRequests *pb.ExecutionRequests, ilTxs [][]byte) ([]byte, error) {
|
||||
return e.NewPayloadResp, e.ErrNewPayload
|
||||
}
|
||||
|
||||
@@ -171,3 +170,11 @@ func (e *EngineClient) GetTerminalBlockHash(ctx context.Context, transitionTime
|
||||
blk = parentBlk
|
||||
}
|
||||
}
|
||||
|
||||
func (e *EngineClient) GetInclusionList(ctx context.Context, parentHash [32]byte) ([][]byte, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (e *EngineClient) UpdatePayloadWithInclusionList(ctx context.Context, payloadID primitives.PayloadID, txs [][]byte) (*primitives.PayloadID, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
@@ -3,6 +3,7 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = [
|
||||
"attester_head.go",
|
||||
"doc.go",
|
||||
"errors.go",
|
||||
"forkchoice.go",
|
||||
|
||||
18
beacon-chain/forkchoice/doubly-linked-tree/attester_head.go
Normal file
18
beacon-chain/forkchoice/doubly-linked-tree/attester_head.go
Normal file
@@ -0,0 +1,18 @@
|
||||
package doublylinkedtree
|
||||
|
||||
// GetAttesterHead returns the attester head root given inclusion list satisfaction.
|
||||
func (f *ForkChoice) GetAttesterHead() [32]byte {
|
||||
head := f.store.headNode
|
||||
if head == nil {
|
||||
return [32]byte{}
|
||||
}
|
||||
|
||||
parent := head.parent
|
||||
if parent == nil {
|
||||
return head.root
|
||||
}
|
||||
if head.notSatisfyingInclusionList {
|
||||
return parent.root
|
||||
}
|
||||
return head.root
|
||||
}
|
||||
@@ -147,11 +147,6 @@ func (f *ForkChoice) GetProposerHead() [32]byte {
|
||||
return head.root
|
||||
}
|
||||
|
||||
// Only orphan a block if the parent LMD vote is strong
|
||||
if parent.weight*100 < f.store.committeeWeight*params.BeaconConfig().ReorgParentWeightThreshold {
|
||||
return head.root
|
||||
}
|
||||
|
||||
// Only reorg if we are proposing early
|
||||
sss, err := slots.SinceSlotStart(currentSlot, f.store.genesisTime, time.Now())
|
||||
if err != nil {
|
||||
@@ -161,5 +156,23 @@ func (f *ForkChoice) GetProposerHead() [32]byte {
|
||||
if sss >= orphanLateBlockProposingEarly*time.Second {
|
||||
return head.root
|
||||
}
|
||||
|
||||
// Newly added in EIP-7805
|
||||
// reorg_prerequisites = all([shuffling_stable, ffg_competitive, finalization_ok,
|
||||
// proposing_on_time, single_slot_reorg, head_weak, parent_strong])
|
||||
//
|
||||
// # Check that the head block is in the unsatisfied inclusion list blocks
|
||||
// inclusion_list_not_satisfied = head_root in store.unsatisfied_inclusion_list_blocks # [New in EIP-7805]
|
||||
//
|
||||
// if reorg_prerequisites and (head_late or inclusion_list_not_satisfied):
|
||||
// return parent_root
|
||||
// else:
|
||||
// return head_root
|
||||
|
||||
// Only orphan a block if the parent LMD vote is strong and satisfies inclusion list
|
||||
if parent.weight*100 < f.store.committeeWeight*params.BeaconConfig().ReorgParentWeightThreshold && !head.notSatisfyingInclusionList {
|
||||
return head.root
|
||||
}
|
||||
|
||||
return parent.root
|
||||
}
|
||||
|
||||
@@ -90,16 +90,17 @@ func (s *Store) insert(ctx context.Context,
|
||||
|
||||
parent := s.nodeByRoot[parentRoot]
|
||||
n := &Node{
|
||||
slot: slot,
|
||||
root: root,
|
||||
parent: parent,
|
||||
justifiedEpoch: justifiedEpoch,
|
||||
unrealizedJustifiedEpoch: justifiedEpoch,
|
||||
finalizedEpoch: finalizedEpoch,
|
||||
unrealizedFinalizedEpoch: finalizedEpoch,
|
||||
optimistic: true,
|
||||
payloadHash: payloadHash,
|
||||
timestamp: time.Now(),
|
||||
slot: slot,
|
||||
root: root,
|
||||
parent: parent,
|
||||
justifiedEpoch: justifiedEpoch,
|
||||
unrealizedJustifiedEpoch: justifiedEpoch,
|
||||
finalizedEpoch: finalizedEpoch,
|
||||
unrealizedFinalizedEpoch: finalizedEpoch,
|
||||
optimistic: true,
|
||||
payloadHash: payloadHash,
|
||||
timestamp: time.Now(),
|
||||
notSatisfyingInclusionList: roblock.Block().NotSatisfyingInclusionList(),
|
||||
}
|
||||
|
||||
// Set the node's target checkpoint
|
||||
|
||||
@@ -47,21 +47,22 @@ type Store struct {
|
||||
// Node defines the individual block which includes its block parent, ancestor and how much weight accounted for it.
|
||||
// This is used as an array based stateful DAG for efficient fork choice look up.
|
||||
type Node struct {
|
||||
slot primitives.Slot // slot of the block converted to the node.
|
||||
root [fieldparams.RootLength]byte // root of the block converted to the node.
|
||||
payloadHash [fieldparams.RootLength]byte // payloadHash of the block converted to the node.
|
||||
parent *Node // parent index of this node.
|
||||
target *Node // target checkpoint for
|
||||
children []*Node // the list of direct children of this Node
|
||||
justifiedEpoch primitives.Epoch // justifiedEpoch of this node.
|
||||
unrealizedJustifiedEpoch primitives.Epoch // the epoch that would be justified if the block would be advanced to the next epoch.
|
||||
finalizedEpoch primitives.Epoch // finalizedEpoch of this node.
|
||||
unrealizedFinalizedEpoch primitives.Epoch // the epoch that would be finalized if the block would be advanced to the next epoch.
|
||||
balance uint64 // the balance that voted for this node directly
|
||||
weight uint64 // weight of this node: the total balance including children
|
||||
bestDescendant *Node // bestDescendant node of this node.
|
||||
optimistic bool // whether the block has been fully validated or not
|
||||
timestamp time.Time // The timestamp when the node was inserted.
|
||||
slot primitives.Slot // slot of the block converted to the node.
|
||||
root [fieldparams.RootLength]byte // root of the block converted to the node.
|
||||
payloadHash [fieldparams.RootLength]byte // payloadHash of the block converted to the node.
|
||||
parent *Node // parent index of this node.
|
||||
target *Node // target checkpoint for
|
||||
children []*Node // the list of direct children of this Node
|
||||
justifiedEpoch primitives.Epoch // justifiedEpoch of this node.
|
||||
unrealizedJustifiedEpoch primitives.Epoch // the epoch that would be justified if the block would be advanced to the next epoch.
|
||||
finalizedEpoch primitives.Epoch // finalizedEpoch of this node.
|
||||
unrealizedFinalizedEpoch primitives.Epoch // the epoch that would be finalized if the block would be advanced to the next epoch.
|
||||
balance uint64 // the balance that voted for this node directly
|
||||
weight uint64 // weight of this node: the total balance including children
|
||||
bestDescendant *Node // bestDescendant node of this node.
|
||||
optimistic bool // whether the block has been fully validated or not
|
||||
notSatisfyingInclusionList bool // whether the node is not satisfying the inclusion list
|
||||
timestamp time.Time // The timestamp when the node was inserted.
|
||||
}
|
||||
|
||||
// Vote defines an individual validator's vote.
|
||||
|
||||
@@ -38,6 +38,7 @@ type RLocker interface {
|
||||
type HeadRetriever interface {
|
||||
Head(context.Context) ([32]byte, error)
|
||||
GetProposerHead() [32]byte
|
||||
GetAttesterHead() [32]byte
|
||||
CachedHeadRoot() [32]byte
|
||||
}
|
||||
|
||||
|
||||
@@ -129,6 +129,7 @@ type BeaconNode struct {
|
||||
syncChecker *initialsync.SyncChecker
|
||||
slasherEnabled bool
|
||||
lcStore *lightclient.Store
|
||||
inclusionLists *cache.InclusionLists
|
||||
}
|
||||
|
||||
// New creates a new node instance, sets up configuration options, and registers
|
||||
@@ -161,6 +162,7 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
|
||||
blsToExecPool: blstoexec.NewPool(),
|
||||
trackedValidatorsCache: cache.NewTrackedValidatorsCache(),
|
||||
payloadIDCache: cache.NewPayloadIDCache(),
|
||||
inclusionLists: cache.NewInclusionLists(),
|
||||
slasherBlockHeadersFeed: new(event.Feed),
|
||||
slasherAttestationsFeed: new(event.Feed),
|
||||
serviceFlagOpts: &serviceFlagOpts{},
|
||||
@@ -802,6 +804,7 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
|
||||
blockchain.WithCustodyInfo(b.custodyInfo),
|
||||
blockchain.WithSlasherEnabled(b.slasherEnabled),
|
||||
blockchain.WithLightClientStore(b.lcStore),
|
||||
blockchain.WithInclusionListCache(b.inclusionLists),
|
||||
)
|
||||
|
||||
blockchainService, err := blockchain.NewService(b.ctx, opts...)
|
||||
@@ -891,6 +894,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
|
||||
regularsync.WithSlasherEnabled(b.slasherEnabled),
|
||||
regularsync.WithLightClientStore(b.lcStore),
|
||||
regularsync.WithBatchVerifierLimit(b.cliCtx.Int(flags.BatchVerifierLimit.Name)),
|
||||
regularsync.WithInclusionListsCache(b.inclusionLists),
|
||||
)
|
||||
return b.services.RegisterService(rs)
|
||||
}
|
||||
@@ -1038,6 +1042,7 @@ func (b *BeaconNode) registerRPCService(router *http.ServeMux) error {
|
||||
TrackedValidatorsCache: b.trackedValidatorsCache,
|
||||
PayloadIDCache: b.payloadIDCache,
|
||||
LCStore: b.lcStore,
|
||||
InclusionListsCache: b.inclusionLists,
|
||||
})
|
||||
|
||||
return b.services.RegisterService(rpcService)
|
||||
|
||||
@@ -24,6 +24,8 @@ import (
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
const minimumPeersPerSubnetForBroadcast = 1
|
||||
|
||||
// ErrMessageNotMapped occurs on a Broadcast attempt when a message has not been defined in the
|
||||
// GossipTypeMapping.
|
||||
var ErrMessageNotMapped = errors.New("message type is not mapped to a PubSub topic")
|
||||
@@ -124,15 +126,13 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6
|
||||
if err := func() error {
|
||||
s.subnetLocker(subnet).Lock()
|
||||
defer s.subnetLocker(subnet).Unlock()
|
||||
ok, err := s.FindPeersWithSubnet(ctx, attestationToTopic(subnet, forkDigest), subnet, 1)
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
if err := s.FindAndDialPeersWithSubnets(ctx, AttestationSubnetTopicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil {
|
||||
return errors.Wrap(err, "find peers with subnets")
|
||||
}
|
||||
if ok {
|
||||
savedAttestationBroadcasts.Inc()
|
||||
return nil
|
||||
}
|
||||
return errors.New("failed to find peers for subnet")
|
||||
|
||||
savedAttestationBroadcasts.Inc()
|
||||
return nil
|
||||
}(); err != nil {
|
||||
log.WithError(err).Error("Failed to find peers")
|
||||
tracing.AnnotateError(span, err)
|
||||
@@ -183,15 +183,12 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs
|
||||
if err := func() error {
|
||||
s.subnetLocker(wrappedSubIdx).Lock()
|
||||
defer s.subnetLocker(wrappedSubIdx).Unlock()
|
||||
ok, err := s.FindPeersWithSubnet(ctx, syncCommitteeToTopic(subnet, forkDigest), subnet, 1)
|
||||
if err != nil {
|
||||
return err
|
||||
if err := s.FindAndDialPeersWithSubnets(ctx, SyncCommitteeSubnetTopicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil {
|
||||
return errors.Wrap(err, "find peers with subnets")
|
||||
}
|
||||
if ok {
|
||||
savedSyncCommitteeBroadcasts.Inc()
|
||||
return nil
|
||||
}
|
||||
return errors.New("failed to find peers for subnet")
|
||||
|
||||
savedSyncCommitteeBroadcasts.Inc()
|
||||
return nil
|
||||
}(); err != nil {
|
||||
log.WithError(err).Error("Failed to find peers")
|
||||
tracing.AnnotateError(span, err)
|
||||
@@ -250,15 +247,13 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob
|
||||
if err := func() error {
|
||||
s.subnetLocker(wrappedSubIdx).Lock()
|
||||
defer s.subnetLocker(wrappedSubIdx).Unlock()
|
||||
ok, err := s.FindPeersWithSubnet(ctx, blobSubnetToTopic(subnet, forkDigest), subnet, 1)
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
if err := s.FindAndDialPeersWithSubnets(ctx, BlobSubnetTopicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil {
|
||||
return errors.Wrap(err, "find peers with subnets")
|
||||
}
|
||||
if ok {
|
||||
blobSidecarBroadcasts.Inc()
|
||||
return nil
|
||||
}
|
||||
return errors.New("failed to find peers for subnet")
|
||||
|
||||
blobSidecarBroadcasts.Inc()
|
||||
return nil
|
||||
}(); err != nil {
|
||||
log.WithError(err).Error("Failed to find peers")
|
||||
tracing.AnnotateError(span, err)
|
||||
@@ -329,7 +324,6 @@ func (s *Service) BroadcastDataColumn(
|
||||
root [fieldparams.RootLength]byte,
|
||||
dataColumnSubnet uint64,
|
||||
dataColumnSidecar *ethpb.DataColumnSidecar,
|
||||
peersCheckedChans ...chan<- bool, // Used for testing purposes to signal when peers are checked.
|
||||
) error {
|
||||
// Add tracing to the function.
|
||||
ctx, span := trace.StartSpan(s.ctx, "p2p.BroadcastDataColumn")
|
||||
@@ -349,7 +343,7 @@ func (s *Service) BroadcastDataColumn(
|
||||
}
|
||||
|
||||
// Non-blocking broadcast, with attempts to discover a column subnet peer if none available.
|
||||
go s.internalBroadcastDataColumn(ctx, root, dataColumnSubnet, dataColumnSidecar, forkDigest, peersCheckedChans)
|
||||
go s.internalBroadcastDataColumn(ctx, root, dataColumnSubnet, dataColumnSidecar, forkDigest)
|
||||
|
||||
return nil
|
||||
}
|
||||
@@ -360,7 +354,6 @@ func (s *Service) internalBroadcastDataColumn(
|
||||
columnSubnet uint64,
|
||||
dataColumnSidecar *ethpb.DataColumnSidecar,
|
||||
forkDigest [fieldparams.VersionLength]byte,
|
||||
peersCheckedChans []chan<- bool, // Used for testing purposes to signal when peers are checked.
|
||||
) {
|
||||
// Add tracing to the function.
|
||||
_, span := trace.StartSpan(ctx, "p2p.internalBroadcastDataColumn")
|
||||
@@ -382,7 +375,7 @@ func (s *Service) internalBroadcastDataColumn(
|
||||
wrappedSubIdx := columnSubnet + dataColumnSubnetVal
|
||||
|
||||
// Find peers if needed.
|
||||
if err := s.findPeersIfNeeded(ctx, wrappedSubIdx, topic, columnSubnet, peersCheckedChans); err != nil {
|
||||
if err := s.findPeersIfNeeded(ctx, wrappedSubIdx, DataColumnSubnetTopicFormat, forkDigest, columnSubnet); err != nil {
|
||||
log.WithError(err).Error("Failed to find peers for data column subnet")
|
||||
tracing.AnnotateError(span, err)
|
||||
}
|
||||
@@ -416,35 +409,19 @@ func (s *Service) internalBroadcastDataColumn(
|
||||
func (s *Service) findPeersIfNeeded(
|
||||
ctx context.Context,
|
||||
wrappedSubIdx uint64,
|
||||
topic string,
|
||||
topicFormat string,
|
||||
forkDigest [fieldparams.VersionLength]byte,
|
||||
subnet uint64,
|
||||
peersCheckedChans []chan<- bool, // Used for testing purposes to signal when peers are checked.
|
||||
) error {
|
||||
// Sending a data column sidecar to only one peer is not ideal,
|
||||
// but it ensures at least one peer receives it.
|
||||
s.subnetLocker(wrappedSubIdx).Lock()
|
||||
defer s.subnetLocker(wrappedSubIdx).Unlock()
|
||||
|
||||
// Sending a data column sidecar to only one peer is not ideal,
|
||||
// but it ensures at least one peer receives it.
|
||||
const peerCount = 1
|
||||
|
||||
if s.hasPeerWithSubnet(topic) {
|
||||
// Exit early if we already have peers with this subnet.
|
||||
return nil
|
||||
}
|
||||
|
||||
// Used for testing purposes.
|
||||
if len(peersCheckedChans) > 0 {
|
||||
peersCheckedChans[0] <- true
|
||||
}
|
||||
|
||||
// No peers found, attempt to find peers with this subnet.
|
||||
ok, err := s.FindPeersWithSubnet(ctx, topic, subnet, peerCount)
|
||||
if err != nil {
|
||||
if err := s.FindAndDialPeersWithSubnets(ctx, topicFormat, forkDigest, minimumPeersPerSubnetForBroadcast, map[uint64]bool{subnet: true}); err != nil {
|
||||
return errors.Wrap(err, "find peers with subnet")
|
||||
}
|
||||
if !ok {
|
||||
return errors.Errorf("failed to find peers for topic %s with subnet %d", topic, subnet)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -216,9 +216,10 @@ func TestService_BroadcastAttestation(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
|
||||
const port = uint(2000)
|
||||
|
||||
// Setup bootnode.
|
||||
cfg := &Config{PingInterval: testPingInterval}
|
||||
port := 2000
|
||||
cfg.UDPPort = uint(port)
|
||||
_, pkey := createAddrAndPrivKey(t)
|
||||
ipAddr := net.ParseIP("127.0.0.1")
|
||||
@@ -245,7 +246,7 @@ func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
|
||||
PingInterval: testPingInterval,
|
||||
}
|
||||
// Setup 2 different hosts
|
||||
for i := 1; i <= 2; i++ {
|
||||
for i := uint(1); i <= 2; i++ {
|
||||
h, pkey, ipAddr := createHost(t, port+i)
|
||||
cfg.UDPPort = uint(port + i)
|
||||
cfg.TCPPort = uint(port + i)
|
||||
@@ -687,7 +688,7 @@ func TestService_BroadcastDataColumn(t *testing.T) {
|
||||
// Create a host.
|
||||
_, pkey, ipAddr := createHost(t, port)
|
||||
|
||||
p := &Service{
|
||||
service := &Service{
|
||||
ctx: t.Context(),
|
||||
host: p1.BHost,
|
||||
pubsub: p1.PubSub(),
|
||||
@@ -701,56 +702,44 @@ func TestService_BroadcastDataColumn(t *testing.T) {
|
||||
}
|
||||
|
||||
// Create a listener.
|
||||
listener, err := p.startDiscoveryV5(ipAddr, pkey)
|
||||
listener, err := service.startDiscoveryV5(ipAddr, pkey)
|
||||
require.NoError(t, err)
|
||||
|
||||
p.dv5Listener = listener
|
||||
service.dv5Listener = listener
|
||||
|
||||
digest, err := p.currentForkDigest()
|
||||
digest, err := service.currentForkDigest()
|
||||
require.NoError(t, err)
|
||||
|
||||
subnet := peerdas.ComputeSubnetForDataColumnSidecar(columnIndex)
|
||||
topic := fmt.Sprintf(topicFormat, digest, subnet)
|
||||
topic := fmt.Sprintf(topicFormat, digest, subnet) + service.Encoding().ProtocolSuffix()
|
||||
|
||||
roSidecars, _ := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: columnIndex}})
|
||||
sidecar := roSidecars[0].DataColumnSidecar
|
||||
|
||||
// Async listen for the pubsub, must be before the broadcast.
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(1)
|
||||
|
||||
peersChecked := make(chan bool, 0)
|
||||
|
||||
go func(tt *testing.T) {
|
||||
defer wg.Done()
|
||||
|
||||
ctx, cancel := context.WithTimeout(t.Context(), 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
// Wait for the peers to be checked.
|
||||
<-peersChecked
|
||||
|
||||
// External peer subscribes to the topic.
|
||||
topic += p.Encoding().ProtocolSuffix()
|
||||
sub, err := p2.SubscribeToTopic(topic)
|
||||
require.NoError(tt, err)
|
||||
|
||||
msg, err := sub.Next(ctx)
|
||||
require.NoError(tt, err)
|
||||
|
||||
var result ethpb.DataColumnSidecar
|
||||
require.NoError(tt, p.Encoding().DecodeGossip(msg.Data, &result))
|
||||
require.DeepEqual(tt, &result, sidecar)
|
||||
}(t)
|
||||
|
||||
var emptyRoot [fieldparams.RootLength]byte
|
||||
|
||||
// Attempt to broadcast nil object should fail.
|
||||
err = p.BroadcastDataColumn(emptyRoot, subnet, nil)
|
||||
var emptyRoot [fieldparams.RootLength]byte
|
||||
err = service.BroadcastDataColumn(emptyRoot, subnet, nil)
|
||||
require.ErrorContains(t, "attempted to broadcast nil", err)
|
||||
|
||||
// Broadcast to peers and wait.
|
||||
err = p.BroadcastDataColumn(emptyRoot, subnet, sidecar, peersChecked)
|
||||
// Subscribe to the topic.
|
||||
sub, err := p2.SubscribeToTopic(topic)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, false, util.WaitTimeout(&wg, 1*time.Minute), "Failed to receive pubsub within 1s")
|
||||
|
||||
// libp2p fails without this delay
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
|
||||
// Broadcast to peers and wait.
|
||||
err = service.BroadcastDataColumn(emptyRoot, subnet, sidecar)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Receive the message.
|
||||
ctx, cancel := context.WithTimeout(t.Context(), 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
msg, err := sub.Next(ctx)
|
||||
require.NoError(t, err)
|
||||
|
||||
var result ethpb.DataColumnSidecar
|
||||
require.NoError(t, service.Encoding().DecodeGossip(msg.Data, &result))
|
||||
require.DeepEqual(t, &result, sidecar)
|
||||
}
|
||||
|
||||
@@ -19,8 +19,7 @@ const (
|
||||
// Burst limit for inbound dials.
|
||||
ipBurst = 8
|
||||
|
||||
// High watermark buffer signifies the buffer till which
|
||||
// we will handle inbound requests.
|
||||
// High watermark buffer signifies the buffer till which we will handle inbound requests.
|
||||
highWatermarkBuffer = 20
|
||||
)
|
||||
|
||||
@@ -53,7 +52,7 @@ func (s *Service) InterceptAccept(n network.ConnMultiaddrs) (allow bool) {
|
||||
"reason": "exceeded dial limit"}).Trace("Not accepting inbound dial from ip address")
|
||||
return false
|
||||
}
|
||||
if s.isPeerAtLimit(true /* inbound */) {
|
||||
if s.isPeerAtLimit(inbound) {
|
||||
log.WithFields(logrus.Fields{"peer": n.RemoteMultiaddr(),
|
||||
"reason": "at peer limit"}).Trace("Not accepting inbound dial")
|
||||
return false
|
||||
|
||||
@@ -2,7 +2,9 @@ package p2p
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/ecdsa"
|
||||
"math"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
@@ -23,45 +25,56 @@ import (
|
||||
ma "github.com/multiformats/go-multiaddr"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/go-bitfield"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type ListenerRebooter interface {
|
||||
Listener
|
||||
RebootListener() error
|
||||
}
|
||||
type (
|
||||
// ListenerRebooter is an interface that extends the Listener interface
|
||||
// with the `RebootListener` method.
|
||||
ListenerRebooter interface {
|
||||
Listener
|
||||
RebootListener() error
|
||||
}
|
||||
|
||||
// Listener defines the discovery V5 network interface that is used
|
||||
// to communicate with other peers.
|
||||
type Listener interface {
|
||||
Self() *enode.Node
|
||||
Close()
|
||||
Lookup(enode.ID) []*enode.Node
|
||||
Resolve(*enode.Node) *enode.Node
|
||||
RandomNodes() enode.Iterator
|
||||
Ping(*enode.Node) error
|
||||
RequestENR(*enode.Node) (*enode.Node, error)
|
||||
LocalNode() *enode.LocalNode
|
||||
}
|
||||
// Listener defines the discovery V5 network interface that is used
|
||||
// to communicate with other peers.
|
||||
Listener interface {
|
||||
Self() *enode.Node
|
||||
Close()
|
||||
Lookup(enode.ID) []*enode.Node
|
||||
Resolve(*enode.Node) *enode.Node
|
||||
RandomNodes() enode.Iterator
|
||||
Ping(*enode.Node) error
|
||||
RequestENR(*enode.Node) (*enode.Node, error)
|
||||
LocalNode() *enode.LocalNode
|
||||
}
|
||||
|
||||
const (
|
||||
udp4 = iota
|
||||
udp6
|
||||
quicProtocol uint16
|
||||
|
||||
listenerWrapper struct {
|
||||
mu sync.RWMutex
|
||||
listener *discover.UDPv5
|
||||
listenerCreator func() (*discover.UDPv5, error)
|
||||
}
|
||||
|
||||
connectivityDirection int
|
||||
udpVersion int
|
||||
)
|
||||
|
||||
const quickProtocolEnrKey = "quic"
|
||||
|
||||
type quicProtocol uint16
|
||||
const (
|
||||
udp4 udpVersion = iota
|
||||
udp6
|
||||
)
|
||||
|
||||
const (
|
||||
inbound connectivityDirection = iota
|
||||
all
|
||||
)
|
||||
|
||||
// quicProtocol is the "quic" key, which holds the QUIC port of the node.
|
||||
func (quicProtocol) ENRKey() string { return quickProtocolEnrKey }
|
||||
|
||||
type listenerWrapper struct {
|
||||
mu sync.RWMutex
|
||||
listener *discover.UDPv5
|
||||
listenerCreator func() (*discover.UDPv5, error)
|
||||
}
|
||||
|
||||
func newListener(listenerCreator func() (*discover.UDPv5, error)) (*listenerWrapper, error) {
|
||||
rawListener, err := listenerCreator()
|
||||
if err != nil {
|
||||
@@ -276,29 +289,10 @@ func (s *Service) RefreshPersistentSubnets() {
|
||||
// listen for new nodes watches for new nodes in the network and adds them to the peerstore.
|
||||
func (s *Service) listenForNewNodes() {
|
||||
const (
|
||||
minLogInterval = 1 * time.Minute
|
||||
thresholdLimit = 5
|
||||
searchPeriod = 20 * time.Second
|
||||
)
|
||||
|
||||
peersSummary := func(threshold uint) (uint, uint) {
|
||||
// Retrieve how many active peers we have.
|
||||
activePeers := s.Peers().Active()
|
||||
activePeerCount := uint(len(activePeers))
|
||||
|
||||
// Compute how many peers we are missing to reach the threshold.
|
||||
if activePeerCount >= threshold {
|
||||
return activePeerCount, 0
|
||||
}
|
||||
|
||||
missingPeerCount := threshold - activePeerCount
|
||||
|
||||
return activePeerCount, missingPeerCount
|
||||
}
|
||||
|
||||
var lastLogTime time.Time
|
||||
|
||||
iterator := s.dv5Listener.RandomNodes()
|
||||
defer iterator.Close()
|
||||
connectivityTicker := time.NewTicker(1 * time.Minute)
|
||||
thresholdCount := 0
|
||||
|
||||
@@ -330,74 +324,148 @@ func (s *Service) listenForNewNodes() {
|
||||
continue
|
||||
}
|
||||
|
||||
iterator = s.dv5Listener.RandomNodes()
|
||||
thresholdCount = 0
|
||||
}
|
||||
default:
|
||||
if s.isPeerAtLimit(false /* inbound */) {
|
||||
// Pause the main loop for a period to stop looking
|
||||
// for new peers.
|
||||
if s.isPeerAtLimit(all) {
|
||||
// Pause the main loop for a period to stop looking for new peers.
|
||||
log.Trace("Not looking for peers, at peer limit")
|
||||
time.Sleep(pollingPeriod)
|
||||
continue
|
||||
}
|
||||
|
||||
// Compute the number of new peers we want to dial.
|
||||
activePeerCount, missingPeerCount := peersSummary(s.cfg.MaxPeers)
|
||||
|
||||
fields := logrus.Fields{
|
||||
"currentPeerCount": activePeerCount,
|
||||
"targetPeerCount": s.cfg.MaxPeers,
|
||||
// Return early if the discovery listener isn't set.
|
||||
if s.dv5Listener == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if missingPeerCount == 0 {
|
||||
log.Trace("Not looking for peers, at peer limit")
|
||||
time.Sleep(pollingPeriod)
|
||||
continue
|
||||
}
|
||||
func() {
|
||||
ctx, cancel := context.WithTimeout(s.ctx, searchPeriod)
|
||||
defer cancel()
|
||||
|
||||
if time.Since(lastLogTime) > minLogInterval {
|
||||
lastLogTime = time.Now()
|
||||
log.WithFields(fields).Debug("Searching for new active peers")
|
||||
}
|
||||
|
||||
// Restrict dials if limit is applied.
|
||||
if flags.MaxDialIsActive() {
|
||||
maxConcurrentDials := uint(flags.Get().MaxConcurrentDials)
|
||||
missingPeerCount = min(missingPeerCount, maxConcurrentDials)
|
||||
}
|
||||
|
||||
// Search for new peers.
|
||||
wantedNodes := searchForPeers(iterator, batchPeriod, missingPeerCount, s.filterPeer)
|
||||
|
||||
wg := new(sync.WaitGroup)
|
||||
for i := 0; i < len(wantedNodes); i++ {
|
||||
node := wantedNodes[i]
|
||||
peerInfo, _, err := convertToAddrInfo(node)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not convert to peer info")
|
||||
continue
|
||||
if err := s.findAndDialPeers(ctx); err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
log.WithError(err).Error("Failed to find and dial peers")
|
||||
}
|
||||
|
||||
if peerInfo == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Make sure that peer is not dialed too often, for each connection attempt there's a backoff period.
|
||||
s.Peers().RandomizeBackOff(peerInfo.ID)
|
||||
wg.Add(1)
|
||||
go func(info *peer.AddrInfo) {
|
||||
if err := s.connectWithPeer(s.ctx, *info); err != nil {
|
||||
log.WithError(err).Tracef("Could not connect with peer %s", info.String())
|
||||
}
|
||||
wg.Done()
|
||||
}(peerInfo)
|
||||
}
|
||||
wg.Wait()
|
||||
}()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// FindAndDialPeersWithSubnets ensures that our node is connected to enough peers.
|
||||
// If, the threshold is met, then this function immediately returns.
|
||||
// Otherwise, it searches for new peers and dials them.
|
||||
// If `ctx“ is canceled while searching for peers, search is stopped, but new found peers are still dialed.
|
||||
// In this case, the function returns an error.
|
||||
func (s *Service) findAndDialPeers(ctx context.Context) error {
|
||||
// Restrict dials if limit is applied.
|
||||
maxConcurrentDials := math.MaxInt
|
||||
if flags.MaxDialIsActive() {
|
||||
maxConcurrentDials = flags.Get().MaxConcurrentDials
|
||||
}
|
||||
|
||||
missingPeerCount := s.missingPeerCount(s.cfg.MaxPeers)
|
||||
for missingPeerCount > 0 {
|
||||
// Stop the search/dialing loop if the context is canceled.
|
||||
if err := ctx.Err(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
peersToDial, err := func() ([]*enode.Node, error) {
|
||||
ctx, cancel := context.WithTimeout(ctx, batchPeriod)
|
||||
defer cancel()
|
||||
|
||||
peersToDial, err := s.findPeers(ctx, missingPeerCount)
|
||||
if err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
return nil, errors.Wrap(err, "find peers")
|
||||
}
|
||||
|
||||
return peersToDial, nil
|
||||
}()
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
dialedPeerCount := s.dialPeers(s.ctx, maxConcurrentDials, peersToDial)
|
||||
|
||||
if dialedPeerCount > missingPeerCount {
|
||||
missingPeerCount = 0
|
||||
continue
|
||||
}
|
||||
|
||||
missingPeerCount -= dialedPeerCount
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// findAndDialPeers finds new peers until `targetPeerCount` is reached, `batchPeriod` is over,
|
||||
// the peers iterator is exhausted or the context is canceled.
|
||||
func (s *Service) findPeers(ctx context.Context, missingPeerCount uint) ([]*enode.Node, error) {
|
||||
// Create an discovery iterator to find new peers.
|
||||
iterator := s.dv5Listener.RandomNodes()
|
||||
|
||||
// `iterator.Next` can block indefinitely. `iterator.Close` unblocks it.
|
||||
// So it is important to close the iterator when the context is done to ensure
|
||||
// that the search does not hang indefinitely.
|
||||
go func() {
|
||||
<-ctx.Done()
|
||||
iterator.Close()
|
||||
}()
|
||||
|
||||
// Crawl the network for peers subscribed to the defective subnets.
|
||||
nodeByNodeID := make(map[enode.ID]*enode.Node)
|
||||
for missingPeerCount > 0 && iterator.Next() {
|
||||
if ctx.Err() != nil {
|
||||
peersToDial := make([]*enode.Node, 0, len(nodeByNodeID))
|
||||
for _, node := range nodeByNodeID {
|
||||
peersToDial = append(peersToDial, node)
|
||||
}
|
||||
|
||||
return peersToDial, ctx.Err()
|
||||
}
|
||||
|
||||
// Skip peer not matching the filter.
|
||||
node := iterator.Node()
|
||||
if !s.filterPeer(node) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Remove duplicates, keeping the node with higher seq.
|
||||
existing, ok := nodeByNodeID[node.ID()]
|
||||
if ok && existing.Seq() > node.Seq() {
|
||||
continue
|
||||
}
|
||||
nodeByNodeID[node.ID()] = node
|
||||
|
||||
// We found a new peer. Decrease the missing peer count.
|
||||
missingPeerCount--
|
||||
}
|
||||
|
||||
// Convert the map to a slice.
|
||||
peersToDial := make([]*enode.Node, 0, len(nodeByNodeID))
|
||||
for _, node := range nodeByNodeID {
|
||||
peersToDial = append(peersToDial, node)
|
||||
}
|
||||
|
||||
return peersToDial, nil
|
||||
}
|
||||
|
||||
// missingPeerCount computes how many peers we are missing to reach the target peer count.
|
||||
func (s *Service) missingPeerCount(targetCount uint) uint {
|
||||
// Retrieve how many active peers we have.
|
||||
activePeers := s.Peers().Active()
|
||||
activePeerCount := uint(len(activePeers))
|
||||
|
||||
// Compute how many peers we are missing to reach the threshold.
|
||||
missingPeerCount := uint(0)
|
||||
if targetCount > activePeerCount {
|
||||
missingPeerCount = targetCount - activePeerCount
|
||||
}
|
||||
|
||||
return missingPeerCount
|
||||
}
|
||||
|
||||
func (s *Service) createListener(
|
||||
ipAddr net.IP,
|
||||
privKey *ecdsa.PrivateKey,
|
||||
@@ -562,8 +630,7 @@ func (s *Service) startDiscoveryV5(
|
||||
// 2. Peer hasn't been marked as 'bad'.
|
||||
// 3. Peer is not currently active or connected.
|
||||
// 4. Peer is ready to receive incoming connections.
|
||||
// 5. Peer's fork digest in their ENR matches that of
|
||||
// our localnodes.
|
||||
// 5. Peer's fork digest in their ENR matches that of our localnodes.
|
||||
func (s *Service) filterPeer(node *enode.Node) bool {
|
||||
// Ignore nil node entries passed in.
|
||||
if node == nil {
|
||||
@@ -628,22 +695,24 @@ func (s *Service) filterPeer(node *enode.Node) bool {
|
||||
// This checks our set max peers in our config, and
|
||||
// determines whether our currently connected and
|
||||
// active peers are above our set max peer limit.
|
||||
func (s *Service) isPeerAtLimit(inbound bool) bool {
|
||||
numOfConns := len(s.host.Network().Peers())
|
||||
func (s *Service) isPeerAtLimit(direction connectivityDirection) bool {
|
||||
maxPeers := int(s.cfg.MaxPeers)
|
||||
// If we are measuring the limit for inbound peers
|
||||
// we apply the high watermark buffer.
|
||||
if inbound {
|
||||
|
||||
// If we are measuring the limit for inbound peers we apply the high watermark buffer.
|
||||
if direction == inbound {
|
||||
maxPeers += highWatermarkBuffer
|
||||
maxInbound := s.peers.InboundLimit() + highWatermarkBuffer
|
||||
currInbound := len(s.peers.InboundConnected())
|
||||
// Exit early if we are at the inbound limit.
|
||||
if currInbound >= maxInbound {
|
||||
inboundCount := len(s.peers.InboundConnected())
|
||||
|
||||
// Return early if we are at the inbound limit.
|
||||
if inboundCount >= maxInbound {
|
||||
return true
|
||||
}
|
||||
}
|
||||
activePeers := len(s.Peers().Active())
|
||||
return activePeers >= maxPeers || numOfConns >= maxPeers
|
||||
|
||||
peerCount := len(s.host.Network().Peers())
|
||||
activePeerCount := len(s.Peers().Active())
|
||||
return activePeerCount >= maxPeers || peerCount >= maxPeers
|
||||
}
|
||||
|
||||
// isBelowOutboundPeerThreshold checks if the number of outbound peers that
|
||||
@@ -901,7 +970,7 @@ func multiAddrFromString(address string) (ma.Multiaddr, error) {
|
||||
return ma.NewMultiaddr(address)
|
||||
}
|
||||
|
||||
func udpVersionFromIP(ipAddr net.IP) int {
|
||||
func udpVersionFromIP(ipAddr net.IP) udpVersion {
|
||||
if ipAddr.To4() != nil {
|
||||
return udp4
|
||||
}
|
||||
|
||||
@@ -323,16 +323,16 @@ func TestMultiAddrConversion_OK(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestStaticPeering_PeersAreAdded(t *testing.T) {
|
||||
const port = uint(6000)
|
||||
cs := startup.NewClockSynchronizer()
|
||||
cfg := &Config{
|
||||
MaxPeers: 30,
|
||||
ClockWaiter: cs,
|
||||
}
|
||||
port := 6000
|
||||
var staticPeers []string
|
||||
var hosts []host.Host
|
||||
// setup other nodes
|
||||
for i := 1; i <= 5; i++ {
|
||||
for i := uint(1); i <= 5; i++ {
|
||||
h, _, ipaddr := createHost(t, port+i)
|
||||
staticPeers = append(staticPeers, fmt.Sprintf("/ip4/%s/tcp/%d/p2p/%s", ipaddr, port+i, h.ID()))
|
||||
hosts = append(hosts, h)
|
||||
@@ -406,14 +406,14 @@ func TestInboundPeerLimit(t *testing.T) {
|
||||
_ = addPeer(t, s.peers, peerdata.ConnectionState(ethpb.ConnectionState_CONNECTED), false)
|
||||
}
|
||||
|
||||
require.Equal(t, true, s.isPeerAtLimit(false), "not at limit for outbound peers")
|
||||
require.Equal(t, false, s.isPeerAtLimit(true), "at limit for inbound peers")
|
||||
require.Equal(t, true, s.isPeerAtLimit(all), "not at limit for outbound peers")
|
||||
require.Equal(t, false, s.isPeerAtLimit(inbound), "at limit for inbound peers")
|
||||
|
||||
for i := 0; i < highWatermarkBuffer; i++ {
|
||||
_ = addPeer(t, s.peers, peerdata.ConnectionState(ethpb.ConnectionState_CONNECTED), false)
|
||||
}
|
||||
|
||||
require.Equal(t, true, s.isPeerAtLimit(true), "not at limit for inbound peers")
|
||||
require.Equal(t, true, s.isPeerAtLimit(inbound), "not at limit for inbound peers")
|
||||
}
|
||||
|
||||
func TestOutboundPeerThreshold(t *testing.T) {
|
||||
|
||||
@@ -134,6 +134,9 @@ func (s *Service) topicScoreParams(topic string) (*pubsub.TopicScoreParams, erro
|
||||
return defaultLightClientOptimisticUpdateTopicParams(), nil
|
||||
case strings.Contains(topic, GossipLightClientFinalityUpdateMessage):
|
||||
return defaultLightClientFinalityUpdateTopicParams(), nil
|
||||
case strings.Contains(topic, GossipInclusionList):
|
||||
// TODO: Update this properly for inclusion list
|
||||
return defaultBlockTopicParams(), nil
|
||||
default:
|
||||
return nil, errors.Errorf("unrecognized topic provided for parameter registration: %s", topic)
|
||||
}
|
||||
|
||||
@@ -25,6 +25,7 @@ var gossipTopicMappings = map[string]func() proto.Message{
|
||||
LightClientOptimisticUpdateTopicFormat: func() proto.Message { return ðpb.LightClientOptimisticUpdateAltair{} },
|
||||
LightClientFinalityUpdateTopicFormat: func() proto.Message { return ðpb.LightClientFinalityUpdateAltair{} },
|
||||
DataColumnSubnetTopicFormat: func() proto.Message { return ðpb.DataColumnSidecar{} },
|
||||
InclusionListTopicFormat: func() proto.Message { return ðpb.SignedInclusionList{} },
|
||||
}
|
||||
|
||||
// GossipTopicMappings is a function to return the assigned data type
|
||||
@@ -144,4 +145,7 @@ func init() {
|
||||
|
||||
// Specially handle Fulu objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockFulu{})] = BlockSubnetTopicFormat
|
||||
|
||||
// Specially handle InclusionList objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedInclusionList{})] = InclusionListTopicFormat
|
||||
}
|
||||
|
||||
@@ -10,6 +10,7 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/peerdata"
|
||||
prysmTime "github.com/OffchainLabs/prysm/v6/time"
|
||||
"github.com/libp2p/go-libp2p/core/host"
|
||||
"github.com/libp2p/go-libp2p/core/network"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
@@ -17,6 +18,8 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
agentVersionKey = "AgentVersion"
|
||||
|
||||
// The time to wait for a status request.
|
||||
timeForStatus = 10 * time.Second
|
||||
)
|
||||
@@ -28,12 +31,15 @@ func peerMultiaddrString(conn network.Conn) string {
|
||||
}
|
||||
|
||||
func (s *Service) connectToPeer(conn network.Conn) {
|
||||
s.peers.SetConnectionState(conn.RemotePeer(), peers.Connected)
|
||||
remotePeer := conn.RemotePeer()
|
||||
|
||||
s.peers.SetConnectionState(remotePeer, peers.Connected)
|
||||
// Go through the handshake process.
|
||||
log.WithFields(logrus.Fields{
|
||||
"direction": conn.Stat().Direction.String(),
|
||||
"multiAddr": peerMultiaddrString(conn),
|
||||
"activePeers": len(s.peers.Active()),
|
||||
"agent": agentString(remotePeer, s.Host()),
|
||||
}).Debug("Initiate peer connection")
|
||||
}
|
||||
|
||||
@@ -61,6 +67,7 @@ func (s *Service) disconnectFromPeerOnError(
|
||||
"multiaddr": peerMultiaddrString(conn),
|
||||
"direction": conn.Stat().Direction.String(),
|
||||
"remainingActivePeers": len(s.peers.Active()),
|
||||
"agent": agentString(remotePeerID, s.Host()),
|
||||
}).
|
||||
Debug("Initiate peer disconnection")
|
||||
|
||||
@@ -189,9 +196,10 @@ func (s *Service) AddDisconnectionHandler(handler func(ctx context.Context, id p
|
||||
DisconnectedF: func(net network.Network, conn network.Conn) {
|
||||
peerID := conn.RemotePeer()
|
||||
|
||||
log.WithFields(logrus.Fields{
|
||||
log := log.WithFields(logrus.Fields{
|
||||
"multiAddr": peerMultiaddrString(conn),
|
||||
"direction": conn.Stat().Direction.String(),
|
||||
"agent": agentString(peerID, s.Host()),
|
||||
})
|
||||
// Must be handled in a goroutine as this callback cannot be blocking.
|
||||
go func() {
|
||||
@@ -222,3 +230,14 @@ func (s *Service) AddDisconnectionHandler(handler func(ctx context.Context, id p
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
func agentString(pid peer.ID, hst host.Host) string {
|
||||
rawVersion, storeErr := hst.Peerstore().Get(pid, agentVersionKey)
|
||||
|
||||
result, ok := rawVersion.(string)
|
||||
if storeErr != nil || !ok {
|
||||
result = ""
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
@@ -49,7 +49,7 @@ type (
|
||||
BroadcastBlob(ctx context.Context, subnet uint64, blob *ethpb.BlobSidecar) error
|
||||
BroadcastLightClientOptimisticUpdate(ctx context.Context, update interfaces.LightClientOptimisticUpdate) error
|
||||
BroadcastLightClientFinalityUpdate(ctx context.Context, update interfaces.LightClientFinalityUpdate) error
|
||||
BroadcastDataColumn(root [fieldparams.RootLength]byte, columnSubnet uint64, dataColumnSidecar *ethpb.DataColumnSidecar, peersChecked ...chan<- bool) error
|
||||
BroadcastDataColumn(root [fieldparams.RootLength]byte, columnSubnet uint64, dataColumnSidecar *ethpb.DataColumnSidecar) error
|
||||
}
|
||||
|
||||
// SetStreamHandler configures p2p to handle streams of a certain topic ID.
|
||||
@@ -98,7 +98,7 @@ type (
|
||||
NodeID() enode.ID
|
||||
DiscoveryAddresses() ([]multiaddr.Multiaddr, error)
|
||||
RefreshPersistentSubnets()
|
||||
FindPeersWithSubnet(ctx context.Context, topic string, subIndex uint64, threshold int) (bool, error)
|
||||
FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error
|
||||
AddPingMethod(reqFunc func(ctx context.Context, id peer.ID) error)
|
||||
}
|
||||
|
||||
|
||||
@@ -393,7 +393,7 @@ func (p *Status) SetNextValidTime(pid peer.ID, nextTime time.Time) {
|
||||
peerData.NextValidTime = nextTime
|
||||
}
|
||||
|
||||
// RandomizeBackOff adds extra backoff period during which peer will not be dialed.
|
||||
// RandomizeBackOff adds extra backoff period during which peer won't be dialed.
|
||||
func (p *Status) RandomizeBackOff(pid peer.ID) {
|
||||
p.store.Lock()
|
||||
defer p.store.Unlock()
|
||||
|
||||
@@ -21,9 +21,9 @@ import (
|
||||
|
||||
const (
|
||||
// overlay parameters
|
||||
gossipSubD = 8 // topic stable mesh target count
|
||||
gossipSubDlo = 6 // topic stable mesh low watermark
|
||||
gossipSubDhi = 12 // topic stable mesh high watermark
|
||||
gossipSubD = 3 // topic stable mesh target count
|
||||
gossipSubDlo = 2 // topic stable mesh low watermark
|
||||
gossipSubDhi = 4 // topic stable mesh high watermark
|
||||
|
||||
// gossip parameters
|
||||
gossipSubMcacheLen = 6 // number of windows to retain full messages in cache for `IWANT` responses
|
||||
|
||||
@@ -37,24 +37,28 @@ import (
|
||||
|
||||
var _ runtime.Service = (*Service)(nil)
|
||||
|
||||
// In the event that we are at our peer limit, we
|
||||
// stop looking for new peers and instead poll
|
||||
// for the current peer limit status for the time period
|
||||
// defined below.
|
||||
var pollingPeriod = 6 * time.Second
|
||||
const (
|
||||
// When looking for new nodes, if not enough nodes are found,
|
||||
// we stop after this spent time.
|
||||
batchPeriod = 2 * time.Second
|
||||
|
||||
// When looking for new nodes, if not enough nodes are found,
|
||||
// we stop after this spent time.
|
||||
var batchPeriod = 2 * time.Second
|
||||
// maxBadResponses is the maximum number of bad responses from a peer before we stop talking to it.
|
||||
maxBadResponses = 5
|
||||
)
|
||||
|
||||
// Refresh rate of ENR set at twice per slot.
|
||||
var refreshRate = slots.DivideSlotBy(2)
|
||||
var (
|
||||
// Refresh rate of ENR set at twice per slot.
|
||||
refreshRate = slots.DivideSlotBy(2)
|
||||
|
||||
// maxBadResponses is the maximum number of bad responses from a peer before we stop talking to it.
|
||||
const maxBadResponses = 5
|
||||
// maxDialTimeout is the timeout for a single peer dial.
|
||||
maxDialTimeout = params.BeaconConfig().RespTimeoutDuration()
|
||||
|
||||
// maxDialTimeout is the timeout for a single peer dial.
|
||||
var maxDialTimeout = params.BeaconConfig().RespTimeoutDuration()
|
||||
// In the event that we are at our peer limit, we
|
||||
// stop looking for new peers and instead poll
|
||||
// for the current peer limit status for the time period
|
||||
// defined below.
|
||||
pollingPeriod = 6 * time.Second
|
||||
)
|
||||
|
||||
// Service for managing peer to peer (p2p) networking.
|
||||
type Service struct {
|
||||
@@ -251,6 +255,7 @@ func (s *Service) Start() {
|
||||
"inboundTCP": inboundTCPCount,
|
||||
"outboundTCP": outboundTCPCount,
|
||||
"total": total,
|
||||
"target": s.cfg.MaxPeers,
|
||||
}
|
||||
|
||||
if features.Get().EnableQUIC {
|
||||
|
||||
@@ -13,6 +13,7 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
|
||||
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
|
||||
"github.com/OffchainLabs/prysm/v6/network/forks"
|
||||
@@ -72,7 +73,7 @@ func (mockListener) RandomNodes() enode.Iterator {
|
||||
|
||||
func (mockListener) RebootListener() error { panic("implement me") }
|
||||
|
||||
func createHost(t *testing.T, port int) (host.Host, *ecdsa.PrivateKey, net.IP) {
|
||||
func createHost(t *testing.T, port uint) (host.Host, *ecdsa.PrivateKey, net.IP) {
|
||||
_, pkey := createAddrAndPrivKey(t)
|
||||
ipAddr := net.ParseIP("127.0.0.1")
|
||||
listen, err := multiaddr.NewMultiaddr(fmt.Sprintf("/ip4/%s/tcp/%d", ipAddr, port))
|
||||
@@ -185,21 +186,33 @@ func TestService_Start_NoDiscoverFlag(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestListenForNewNodes(t *testing.T) {
|
||||
const (
|
||||
port = uint(2000)
|
||||
testPollingPeriod = 1 * time.Second
|
||||
peerCount = 5
|
||||
)
|
||||
|
||||
params.SetupTestConfigCleanup(t)
|
||||
|
||||
// Setup bootnode.
|
||||
notifier := &mock.MockStateNotifier{}
|
||||
cfg := &Config{StateNotifier: notifier, PingInterval: testPingInterval, DisableLivenessCheck: true}
|
||||
port := 2000
|
||||
cfg.UDPPort = uint(port)
|
||||
cfg := &Config{
|
||||
StateNotifier: &mock.MockStateNotifier{},
|
||||
PingInterval: testPingInterval,
|
||||
DisableLivenessCheck: true,
|
||||
UDPPort: port,
|
||||
}
|
||||
|
||||
_, pkey := createAddrAndPrivKey(t)
|
||||
ipAddr := net.ParseIP("127.0.0.1")
|
||||
genesisTime := prysmTime.Now()
|
||||
var gvr [32]byte
|
||||
var gvr [fieldparams.RootLength]byte
|
||||
|
||||
s := &Service{
|
||||
cfg: cfg,
|
||||
genesisTime: genesisTime,
|
||||
genesisValidatorsRoot: gvr[:],
|
||||
}
|
||||
|
||||
bootListener, err := s.createListener(ipAddr, pkey)
|
||||
require.NoError(t, err)
|
||||
defer bootListener.Close()
|
||||
@@ -210,35 +223,40 @@ func TestListenForNewNodes(t *testing.T) {
|
||||
|
||||
// Use shorter period for testing.
|
||||
currentPeriod := pollingPeriod
|
||||
pollingPeriod = 1 * time.Second
|
||||
pollingPeriod = testPollingPeriod
|
||||
defer func() {
|
||||
pollingPeriod = currentPeriod
|
||||
}()
|
||||
|
||||
bootNode := bootListener.Self()
|
||||
|
||||
var listeners []*listenerWrapper
|
||||
var hosts []host.Host
|
||||
// setup other nodes.
|
||||
// Setup other nodes.
|
||||
cs := startup.NewClockSynchronizer()
|
||||
cfg = &Config{
|
||||
Discv5BootStrapAddrs: []string{bootNode.String()},
|
||||
PingInterval: testPingInterval,
|
||||
DisableLivenessCheck: true,
|
||||
MaxPeers: 30,
|
||||
ClockWaiter: cs,
|
||||
}
|
||||
for i := 1; i <= 5; i++ {
|
||||
listeners := make([]*listenerWrapper, 0, peerCount)
|
||||
hosts := make([]host.Host, 0, peerCount)
|
||||
|
||||
for i := uint(1); i <= peerCount; i++ {
|
||||
cfg = &Config{
|
||||
Discv5BootStrapAddrs: []string{bootNode.String()},
|
||||
PingInterval: testPingInterval,
|
||||
DisableLivenessCheck: true,
|
||||
MaxPeers: peerCount,
|
||||
ClockWaiter: cs,
|
||||
UDPPort: port + i,
|
||||
TCPPort: port + i,
|
||||
}
|
||||
|
||||
h, pkey, ipAddr := createHost(t, port+i)
|
||||
cfg.UDPPort = uint(port + i)
|
||||
cfg.TCPPort = uint(port + i)
|
||||
|
||||
s := &Service{
|
||||
cfg: cfg,
|
||||
genesisTime: genesisTime,
|
||||
genesisValidatorsRoot: gvr[:],
|
||||
}
|
||||
|
||||
listener, err := s.startDiscoveryV5(ipAddr, pkey)
|
||||
assert.NoError(t, err, "Could not start discovery for node")
|
||||
require.NoError(t, err, "Could not start discovery for node")
|
||||
|
||||
listeners = append(listeners, listener)
|
||||
hosts = append(hosts, h)
|
||||
}
|
||||
@@ -263,19 +281,26 @@ func TestListenForNewNodes(t *testing.T) {
|
||||
|
||||
s, err = NewService(t.Context(), cfg)
|
||||
require.NoError(t, err)
|
||||
exitRoutine := make(chan bool)
|
||||
go func() {
|
||||
s.Start()
|
||||
<-exitRoutine
|
||||
}()
|
||||
time.Sleep(1 * time.Second)
|
||||
|
||||
require.NoError(t, cs.SetClock(startup.NewClock(genesisTime, gvr)))
|
||||
go s.Start()
|
||||
|
||||
time.Sleep(4 * time.Second)
|
||||
assert.Equal(t, 5, len(s.host.Network().Peers()), "Not all peers added to peerstore")
|
||||
require.NoError(t, s.Stop())
|
||||
exitRoutine <- true
|
||||
err = cs.SetClock(startup.NewClock(genesisTime, gvr))
|
||||
require.NoError(t, err, "Could not set clock in service")
|
||||
|
||||
actualPeerCount := len(s.host.Network().Peers())
|
||||
for range 40 {
|
||||
if actualPeerCount == peerCount {
|
||||
break
|
||||
}
|
||||
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
actualPeerCount = len(s.host.Network().Peers())
|
||||
}
|
||||
|
||||
assert.Equal(t, peerCount, actualPeerCount, "Not all peers added to peerstore")
|
||||
|
||||
err = s.Stop()
|
||||
require.NoError(t, err, "Failed to stop service")
|
||||
}
|
||||
|
||||
func TestPeer_Disconnect(t *testing.T) {
|
||||
|
||||
@@ -2,6 +2,7 @@ package p2p
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"strings"
|
||||
"sync"
|
||||
@@ -11,6 +12,7 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
|
||||
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
|
||||
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/wrapper"
|
||||
@@ -23,7 +25,6 @@ import (
|
||||
"github.com/holiman/uint256"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/go-bitfield"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
var (
|
||||
@@ -57,249 +58,297 @@ const blobSubnetLockerVal = 110
|
||||
const dataColumnSubnetVal = 150
|
||||
|
||||
// nodeFilter returns a function that filters nodes based on the subnet topic and subnet index.
|
||||
func (s *Service) nodeFilter(topic string, index uint64) (func(node *enode.Node) bool, error) {
|
||||
func (s *Service) nodeFilter(topic string, indices map[uint64]int) (func(node *enode.Node) (map[uint64]bool, error), error) {
|
||||
switch {
|
||||
case strings.Contains(topic, GossipAttestationMessage):
|
||||
return s.filterPeerForAttSubnet(index), nil
|
||||
return s.filterPeerForAttSubnet(indices), nil
|
||||
case strings.Contains(topic, GossipSyncCommitteeMessage):
|
||||
return s.filterPeerForSyncSubnet(index), nil
|
||||
return s.filterPeerForSyncSubnet(indices), nil
|
||||
case strings.Contains(topic, GossipBlobSidecarMessage):
|
||||
return s.filterPeerForBlobSubnet(), nil
|
||||
return s.filterPeerForBlobSubnet(indices), nil
|
||||
case strings.Contains(topic, GossipDataColumnSidecarMessage):
|
||||
return s.filterPeerForDataColumnsSubnet(index), nil
|
||||
return s.filterPeerForDataColumnsSubnet(indices), nil
|
||||
default:
|
||||
return nil, errors.Errorf("no subnet exists for provided topic: %s", topic)
|
||||
}
|
||||
}
|
||||
|
||||
// searchForPeers performs a network search for peers subscribed to a particular subnet.
|
||||
// It exits as soon as one of these conditions is met:
|
||||
// - It looped through `batchSize` nodes.
|
||||
// - It found `peersToFindCount“ peers corresponding to the `filter` criteria.
|
||||
// - Iterator is exhausted.
|
||||
func searchForPeers(
|
||||
iterator enode.Iterator,
|
||||
batchPeriod time.Duration,
|
||||
peersToFindCount uint,
|
||||
filter func(node *enode.Node) bool,
|
||||
) []*enode.Node {
|
||||
nodeFromNodeID := make(map[enode.ID]*enode.Node)
|
||||
start := time.Now()
|
||||
// FindAndDialPeersWithSubnets ensures that our node is connected to at least `minimumPeersPerSubnet`
|
||||
// peers for each subnet listed in `subnets`.
|
||||
// If, for all subnets, the threshold is met, then this function immediately returns.
|
||||
// Otherwise, it searches for new peers for defective subnets, and dials them.
|
||||
// If `ctx“ is canceled while searching for peers, search is stopped, but new found peers are still dialed.
|
||||
// In this case, the function returns an error.
|
||||
func (s *Service) FindAndDialPeersWithSubnets(
|
||||
ctx context.Context,
|
||||
topicFormat string,
|
||||
digest [fieldparams.VersionLength]byte,
|
||||
minimumPeersPerSubnet int,
|
||||
subnets map[uint64]bool,
|
||||
) error {
|
||||
ctx, span := trace.StartSpan(ctx, "p2p.FindAndDialPeersWithSubnet")
|
||||
defer span.End()
|
||||
|
||||
for time.Since(start) < batchPeriod && uint(len(nodeFromNodeID)) < peersToFindCount && iterator.Next() {
|
||||
// Return early if the discovery listener isn't set.
|
||||
if s.dv5Listener == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Restrict dials if limit is applied.
|
||||
maxConcurrentDials := math.MaxInt
|
||||
if flags.MaxDialIsActive() {
|
||||
maxConcurrentDials = flags.Get().MaxConcurrentDials
|
||||
}
|
||||
|
||||
defectiveSubnets := s.defectiveSubnets(topicFormat, digest, minimumPeersPerSubnet, subnets)
|
||||
for len(defectiveSubnets) > 0 {
|
||||
// Stop the search/dialing loop if the context is canceled.
|
||||
if err := ctx.Err(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
peersToDial, err := func() ([]*enode.Node, error) {
|
||||
ctx, cancel := context.WithTimeout(ctx, batchPeriod)
|
||||
defer cancel()
|
||||
|
||||
peersToDial, err := s.findPeersWithSubnets(ctx, topicFormat, digest, minimumPeersPerSubnet, defectiveSubnets)
|
||||
if err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
return nil, errors.Wrap(err, "find peers with subnets")
|
||||
}
|
||||
|
||||
return peersToDial, nil
|
||||
}()
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Dial new peers in batches.
|
||||
s.dialPeers(s.ctx, maxConcurrentDials, peersToDial)
|
||||
|
||||
defectiveSubnets = s.defectiveSubnets(topicFormat, digest, minimumPeersPerSubnet, subnets)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// findPeersWithSubnets finds peers subscribed to defective subnets in batches
|
||||
// until enough peers are found or the context is canceled.
|
||||
// It returns new peers found during the search.
|
||||
func (s *Service) findPeersWithSubnets(
|
||||
ctx context.Context,
|
||||
topicFormat string,
|
||||
digest [fieldparams.VersionLength]byte,
|
||||
minimumPeersPerSubnet int,
|
||||
defectiveSubnetsOrigin map[uint64]int,
|
||||
) ([]*enode.Node, error) {
|
||||
// Copy the defective subnets map to avoid modifying the original map.
|
||||
defectiveSubnets := make(map[uint64]int, len(defectiveSubnetsOrigin))
|
||||
for k, v := range defectiveSubnetsOrigin {
|
||||
defectiveSubnets[k] = v
|
||||
}
|
||||
|
||||
// Create an discovery iterator to find new peers.
|
||||
iterator := s.dv5Listener.RandomNodes()
|
||||
|
||||
// `iterator.Next` can block indefinitely. `iterator.Close` unblocks it.
|
||||
// So it is important to close the iterator when the context is done to ensure
|
||||
// that the search does not hang indefinitely.
|
||||
go func() {
|
||||
<-ctx.Done()
|
||||
iterator.Close()
|
||||
}()
|
||||
|
||||
// Retrieve the filter function that will be used to filter nodes based on the defective subnets.
|
||||
filter, err := s.nodeFilter(topicFormat, defectiveSubnets)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "node filter")
|
||||
}
|
||||
|
||||
// Crawl the network for peers subscribed to the defective subnets.
|
||||
nodeByNodeID := make(map[enode.ID]*enode.Node)
|
||||
for len(defectiveSubnets) > 0 && iterator.Next() {
|
||||
if err := ctx.Err(); err != nil {
|
||||
// Convert the map to a slice.
|
||||
peersToDial := make([]*enode.Node, 0, len(nodeByNodeID))
|
||||
for _, node := range nodeByNodeID {
|
||||
peersToDial = append(peersToDial, node)
|
||||
}
|
||||
|
||||
return peersToDial, err
|
||||
}
|
||||
|
||||
// Get all needed subnets that the node is subscribed to.
|
||||
// Skip nodes that are not subscribed to any of the defective subnets.
|
||||
node := iterator.Node()
|
||||
|
||||
// Filter out nodes that do not meet the criteria.
|
||||
if !filter(node) {
|
||||
nodeSubnets, err := filter(node)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "filter node")
|
||||
}
|
||||
if len(nodeSubnets) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
// Remove duplicates, keeping the node with higher seq.
|
||||
prevNode, ok := nodeFromNodeID[node.ID()]
|
||||
if ok && prevNode.Seq() > node.Seq() {
|
||||
existing, ok := nodeByNodeID[node.ID()]
|
||||
if ok && existing.Seq() > node.Seq() {
|
||||
continue
|
||||
}
|
||||
nodeByNodeID[node.ID()] = node
|
||||
|
||||
nodeFromNodeID[node.ID()] = node
|
||||
// We found a new peer. Modify the defective subnets map
|
||||
// and the filter accordingly.
|
||||
for subnet := range defectiveSubnets {
|
||||
if !nodeSubnets[subnet] {
|
||||
continue
|
||||
}
|
||||
|
||||
defectiveSubnets[subnet]--
|
||||
|
||||
if defectiveSubnets[subnet] == 0 {
|
||||
delete(defectiveSubnets, subnet)
|
||||
}
|
||||
|
||||
filter, err = s.nodeFilter(topicFormat, defectiveSubnets)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "node filter")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Convert the map to a slice.
|
||||
nodes := make([]*enode.Node, 0, len(nodeFromNodeID))
|
||||
for _, node := range nodeFromNodeID {
|
||||
nodes = append(nodes, node)
|
||||
peersToDial := make([]*enode.Node, 0, len(nodeByNodeID))
|
||||
for _, node := range nodeByNodeID {
|
||||
peersToDial = append(peersToDial, node)
|
||||
}
|
||||
|
||||
return nodes
|
||||
return peersToDial, nil
|
||||
}
|
||||
|
||||
// dialPeer dials a peer in a separate goroutine.
|
||||
func (s *Service) dialPeer(ctx context.Context, wg *sync.WaitGroup, node *enode.Node) {
|
||||
info, _, err := convertToAddrInfo(node)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if info == nil {
|
||||
return
|
||||
}
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
if err := s.connectWithPeer(ctx, *info); err != nil {
|
||||
log.WithError(err).Tracef("Could not connect with peer %s", info.String())
|
||||
// defectiveSubnets returns a map of subnets that have fewer than the minimum peer count.
|
||||
func (s *Service) defectiveSubnets(
|
||||
topicFormat string,
|
||||
digest [fieldparams.VersionLength]byte,
|
||||
minimumPeersPerSubnet int,
|
||||
subnets map[uint64]bool,
|
||||
) map[uint64]int {
|
||||
missingCountPerSubnet := make(map[uint64]int, len(subnets))
|
||||
for subnet := range subnets {
|
||||
topic := fmt.Sprintf(topicFormat, digest, subnet) + s.Encoding().ProtocolSuffix()
|
||||
peers := s.pubsub.ListPeers(topic)
|
||||
peerCount := len(peers)
|
||||
if peerCount < minimumPeersPerSubnet {
|
||||
missingCountPerSubnet[subnet] = minimumPeersPerSubnet - peerCount
|
||||
}
|
||||
}
|
||||
|
||||
wg.Done()
|
||||
}()
|
||||
return missingCountPerSubnet
|
||||
}
|
||||
|
||||
// FindPeersWithSubnet performs a network search for peers
|
||||
// subscribed to a particular subnet. Then it tries to connect
|
||||
// with those peers. This method will block until either:
|
||||
// - the required amount of peers are found, or
|
||||
// - the context is terminated.
|
||||
// On some edge cases, this method may hang indefinitely while peers
|
||||
// are actually found. In such a case, the user should cancel the context
|
||||
// and re-run the method again.
|
||||
func (s *Service) FindPeersWithSubnet(
|
||||
ctx context.Context,
|
||||
topic string,
|
||||
index uint64,
|
||||
threshold int,
|
||||
) (bool, error) {
|
||||
const minLogInterval = 1 * time.Minute
|
||||
// dialPeers dials multiple peers concurrently up to `maxConcurrentDials` at a time.
|
||||
// In case of a dial failure, it logs the error but continues dialing other peers.
|
||||
func (s *Service) dialPeers(ctx context.Context, maxConcurrentDials int, nodes []*enode.Node) uint {
|
||||
var mut sync.Mutex
|
||||
|
||||
ctx, span := trace.StartSpan(ctx, "p2p.FindPeersWithSubnet")
|
||||
defer span.End()
|
||||
|
||||
span.SetAttributes(trace.Int64Attribute("index", int64(index))) // lint:ignore uintcast -- It's safe to do this for tracing.
|
||||
|
||||
if s.dv5Listener == nil {
|
||||
// Return if discovery isn't set
|
||||
return false, nil
|
||||
}
|
||||
|
||||
topic += s.Encoding().ProtocolSuffix()
|
||||
iterator := s.dv5Listener.RandomNodes()
|
||||
defer iterator.Close()
|
||||
|
||||
filter, err := s.nodeFilter(topic, index)
|
||||
if err != nil {
|
||||
return false, errors.Wrap(err, "node filter")
|
||||
}
|
||||
|
||||
peersSummary := func(topic string, threshold int) (int, int) {
|
||||
// Retrieve how many peers we have for this topic.
|
||||
peerCountForTopic := len(s.pubsub.ListPeers(topic))
|
||||
|
||||
// Compute how many peers we are missing to reach the threshold.
|
||||
missingPeerCountForTopic := max(0, threshold-peerCountForTopic)
|
||||
|
||||
return peerCountForTopic, missingPeerCountForTopic
|
||||
}
|
||||
|
||||
// Compute how many peers we are missing to reach the threshold.
|
||||
peerCountForTopic, missingPeerCountForTopic := peersSummary(topic, threshold)
|
||||
|
||||
// Exit early if we have enough peers.
|
||||
if missingPeerCountForTopic == 0 {
|
||||
return true, nil
|
||||
}
|
||||
|
||||
log := log.WithFields(logrus.Fields{
|
||||
"topic": topic,
|
||||
"targetPeerCount": threshold,
|
||||
})
|
||||
|
||||
log.WithField("currentPeerCount", peerCountForTopic).Debug("Searching for new peers for a subnet - start")
|
||||
|
||||
lastLogTime := time.Now()
|
||||
|
||||
wg := new(sync.WaitGroup)
|
||||
for {
|
||||
// If the context is done, we can exit the loop. This is the unhappy path.
|
||||
if err := ctx.Err(); err != nil {
|
||||
return false, errors.Errorf(
|
||||
"unable to find requisite number of peers for topic %s - only %d out of %d peers available after searching",
|
||||
topic, peerCountForTopic, threshold,
|
||||
)
|
||||
counter := uint(0)
|
||||
for start := 0; start < len(nodes); start += maxConcurrentDials {
|
||||
if ctx.Err() != nil {
|
||||
return counter
|
||||
}
|
||||
|
||||
// Search for new peers in the network.
|
||||
nodes := searchForPeers(iterator, batchPeriod, uint(missingPeerCountForTopic), filter)
|
||||
|
||||
// Restrict dials if limit is applied.
|
||||
maxConcurrentDials := math.MaxInt
|
||||
if flags.MaxDialIsActive() {
|
||||
maxConcurrentDials = flags.Get().MaxConcurrentDials
|
||||
}
|
||||
|
||||
// Dial the peers in batches.
|
||||
for start := 0; start < len(nodes); start += maxConcurrentDials {
|
||||
stop := min(start+maxConcurrentDials, len(nodes))
|
||||
for _, node := range nodes[start:stop] {
|
||||
s.dialPeer(ctx, wg, node)
|
||||
var wg sync.WaitGroup
|
||||
stop := min(start+maxConcurrentDials, len(nodes))
|
||||
for _, node := range nodes[start:stop] {
|
||||
log := log.WithField("nodeID", node.ID())
|
||||
info, _, err := convertToAddrInfo(node)
|
||||
if err != nil {
|
||||
log.WithError(err).Debug("Could not convert node to addr info")
|
||||
continue
|
||||
}
|
||||
|
||||
// Wait for all dials to be completed.
|
||||
wg.Wait()
|
||||
if info == nil {
|
||||
log.Debug("Nil addr info")
|
||||
continue
|
||||
}
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
if err := s.connectWithPeer(ctx, *info); err != nil {
|
||||
log.WithError(err).WithField("info", info.String()).Debug("Could not connect with peer")
|
||||
return
|
||||
}
|
||||
|
||||
mut.Lock()
|
||||
defer mut.Unlock()
|
||||
counter++
|
||||
}()
|
||||
}
|
||||
|
||||
peerCountForTopic, missingPeerCountForTopic := peersSummary(topic, threshold)
|
||||
|
||||
// If we have enough peers, we can exit the loop. This is the happy path.
|
||||
if missingPeerCountForTopic == 0 {
|
||||
break
|
||||
}
|
||||
|
||||
if time.Since(lastLogTime) > minLogInterval {
|
||||
lastLogTime = time.Now()
|
||||
log.WithField("currentPeerCount", peerCountForTopic).Debug("Searching for new peers for a subnet - continue")
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
log.WithField("currentPeerCount", threshold).Debug("Searching for new peers for a subnet - success")
|
||||
return true, nil
|
||||
return counter
|
||||
}
|
||||
|
||||
// returns a method with filters peers specifically for a particular attestation subnet.
|
||||
func (s *Service) filterPeerForAttSubnet(index uint64) func(node *enode.Node) bool {
|
||||
return func(node *enode.Node) bool {
|
||||
// filterPeerForAttSubnet returns a method with filters peers specifically for a particular attestation subnet.
|
||||
func (s *Service) filterPeerForAttSubnet(indices map[uint64]int) func(node *enode.Node) (map[uint64]bool, error) {
|
||||
return func(node *enode.Node) (map[uint64]bool, error) {
|
||||
if !s.filterPeer(node) {
|
||||
return false
|
||||
return map[uint64]bool{}, nil
|
||||
}
|
||||
|
||||
subnets, err := attSubnets(node.Record())
|
||||
subnets, err := attestationSubnets(node.Record())
|
||||
if err != nil {
|
||||
return false
|
||||
return nil, errors.Wrap(err, "attestation subnets")
|
||||
}
|
||||
|
||||
return subnets[index]
|
||||
return intersect(indices, subnets), nil
|
||||
}
|
||||
}
|
||||
|
||||
// returns a method with filters peers specifically for a particular sync subnet.
|
||||
func (s *Service) filterPeerForSyncSubnet(index uint64) func(node *enode.Node) bool {
|
||||
return func(node *enode.Node) bool {
|
||||
func (s *Service) filterPeerForSyncSubnet(indices map[uint64]int) func(node *enode.Node) (map[uint64]bool, error) {
|
||||
return func(node *enode.Node) (map[uint64]bool, error) {
|
||||
if !s.filterPeer(node) {
|
||||
return false
|
||||
return map[uint64]bool{}, nil
|
||||
}
|
||||
|
||||
subnets, err := syncSubnets(node.Record())
|
||||
if err != nil {
|
||||
return false
|
||||
return nil, errors.Wrap(err, "sync subnets")
|
||||
}
|
||||
indExists := false
|
||||
for _, comIdx := range subnets {
|
||||
if comIdx == index {
|
||||
indExists = true
|
||||
break
|
||||
}
|
||||
}
|
||||
return indExists
|
||||
|
||||
return intersect(indices, subnets), nil
|
||||
}
|
||||
}
|
||||
|
||||
// returns a method with filters peers specifically for a particular blob subnet.
|
||||
// All peers are supposed to be subscribed to all blob subnets.
|
||||
func (s *Service) filterPeerForBlobSubnet() func(_ *enode.Node) bool {
|
||||
return func(_ *enode.Node) bool {
|
||||
return true
|
||||
func (s *Service) filterPeerForBlobSubnet(indices map[uint64]int) func(_ *enode.Node) (map[uint64]bool, error) {
|
||||
result := make(map[uint64]bool, len(indices))
|
||||
for i := range indices {
|
||||
result[i] = true
|
||||
}
|
||||
|
||||
return func(_ *enode.Node) (map[uint64]bool, error) {
|
||||
return result, nil
|
||||
}
|
||||
}
|
||||
|
||||
// returns a method with filters peers specifically for a particular data column subnet.
|
||||
func (s *Service) filterPeerForDataColumnsSubnet(index uint64) func(node *enode.Node) bool {
|
||||
return func(node *enode.Node) bool {
|
||||
func (s *Service) filterPeerForDataColumnsSubnet(indices map[uint64]int) func(node *enode.Node) (map[uint64]bool, error) {
|
||||
return func(node *enode.Node) (map[uint64]bool, error) {
|
||||
if !s.filterPeer(node) {
|
||||
return false
|
||||
return map[uint64]bool{}, nil
|
||||
}
|
||||
|
||||
subnets, err := dataColumnSubnets(node.ID(), node.Record())
|
||||
if err != nil {
|
||||
return false
|
||||
return nil, errors.Wrap(err, "data column subnets")
|
||||
}
|
||||
|
||||
return subnets[index]
|
||||
return intersect(indices, subnets), nil
|
||||
}
|
||||
}
|
||||
|
||||
@@ -475,43 +524,47 @@ func initializeSyncCommSubnets(node *enode.LocalNode) *enode.LocalNode {
|
||||
|
||||
// Reads the attestation subnets entry from a node's ENR and determines
|
||||
// the committee indices of the attestation subnets the node is subscribed to.
|
||||
func attSubnets(record *enr.Record) (map[uint64]bool, error) {
|
||||
func attestationSubnets(record *enr.Record) (map[uint64]bool, error) {
|
||||
bitV, err := attBitvector(record)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
committeeIdxs := make(map[uint64]bool)
|
||||
// lint:ignore uintcast -- subnet count can be safely cast to int.
|
||||
if len(bitV) != byteCount(int(attestationSubnetCount)) {
|
||||
return committeeIdxs, errors.Errorf("invalid bitvector provided, it has a size of %d", len(bitV))
|
||||
return nil, errors.Wrap(err, "att bit vector")
|
||||
}
|
||||
|
||||
for i := uint64(0); i < attestationSubnetCount; i++ {
|
||||
// lint:ignore uintcast -- subnet count can be safely cast to int.
|
||||
if len(bitV) != byteCount(int(attestationSubnetCount)) {
|
||||
return nil, errors.Errorf("invalid bitvector provided, it has a size of %d", len(bitV))
|
||||
}
|
||||
|
||||
indices := make(map[uint64]bool, attestationSubnetCount)
|
||||
for i := range attestationSubnetCount {
|
||||
if bitV.BitAt(i) {
|
||||
committeeIdxs[i] = true
|
||||
indices[i] = true
|
||||
}
|
||||
}
|
||||
return committeeIdxs, nil
|
||||
|
||||
return indices, nil
|
||||
}
|
||||
|
||||
// Reads the sync subnets entry from a node's ENR and determines
|
||||
// the committee indices of the sync subnets the node is subscribed to.
|
||||
func syncSubnets(record *enr.Record) ([]uint64, error) {
|
||||
func syncSubnets(record *enr.Record) (map[uint64]bool, error) {
|
||||
bitV, err := syncBitvector(record)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, errors.Wrap(err, "sync bit vector")
|
||||
}
|
||||
|
||||
// lint:ignore uintcast -- subnet count can be safely cast to int.
|
||||
if len(bitV) != byteCount(int(syncCommsSubnetCount)) {
|
||||
return []uint64{}, errors.Errorf("invalid bitvector provided, it has a size of %d", len(bitV))
|
||||
return nil, errors.Errorf("invalid bitvector provided, it has a size of %d", len(bitV))
|
||||
}
|
||||
var committeeIdxs []uint64
|
||||
for i := uint64(0); i < syncCommsSubnetCount; i++ {
|
||||
|
||||
indices := make(map[uint64]bool, syncCommsSubnetCount)
|
||||
for i := range syncCommsSubnetCount {
|
||||
if bitV.BitAt(i) {
|
||||
committeeIdxs = append(committeeIdxs, i)
|
||||
indices[i] = true
|
||||
}
|
||||
}
|
||||
return committeeIdxs, nil
|
||||
return indices, nil
|
||||
}
|
||||
|
||||
// Retrieve the data columns subnets from a node's ENR and node ID.
|
||||
@@ -585,3 +638,16 @@ func byteCount(bitCount int) int {
|
||||
}
|
||||
return numOfBytes
|
||||
}
|
||||
|
||||
// interesect intersects two maps and returns a new map containing only the keys
|
||||
// that are present in both maps.
|
||||
func intersect(left map[uint64]int, right map[uint64]bool) map[uint64]bool {
|
||||
result := make(map[uint64]bool, min(len(left), len(right)))
|
||||
for i := range left {
|
||||
if right[i] {
|
||||
result[i] = true
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
@@ -5,7 +5,6 @@ import (
|
||||
"crypto/rand"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@@ -22,7 +21,7 @@ import (
|
||||
"github.com/prysmaticlabs/go-bitfield"
|
||||
)
|
||||
|
||||
func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
|
||||
func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) {
|
||||
// Topology of this test:
|
||||
//
|
||||
//
|
||||
@@ -37,7 +36,12 @@ func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
|
||||
// In our case: The node i is subscribed to subnet i, with i = 1, 2, 3
|
||||
|
||||
// Define the genesis validators root, to ensure everybody is on the same network.
|
||||
const genesisValidatorRootStr = "0xdeadbeefcafecafedeadbeefcafecafedeadbeefcafecafedeadbeefcafecafe"
|
||||
const (
|
||||
genesisValidatorRootStr = "0xdeadbeefcafecafedeadbeefcafecafedeadbeefcafecafedeadbeefcafecafe"
|
||||
subnetCount = 3
|
||||
minimumPeersPerSubnet = 1
|
||||
)
|
||||
|
||||
genesisValidatorsRoot, err := hex.DecodeString(genesisValidatorRootStr[2:])
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -87,13 +91,12 @@ func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
|
||||
|
||||
// Create 3 nodes, each subscribed to a different subnet.
|
||||
// Each node is connected to the bootstrap node.
|
||||
services := make([]*Service, 0, 3)
|
||||
services := make([]*Service, 0, subnetCount)
|
||||
|
||||
for i := 1; i <= 3; i++ {
|
||||
subnet := uint64(i)
|
||||
for i := uint64(1); i <= subnetCount; i++ {
|
||||
service, err := NewService(ctx, &Config{
|
||||
Discv5BootStrapAddrs: []string{bootNodeENR},
|
||||
MaxPeers: 30,
|
||||
MaxPeers: 0, // Set to 0 to ensure that peers are discovered via subnets search, and not generic peers discovery.
|
||||
UDPPort: uint(2000 + i),
|
||||
TCPPort: uint(3000 + i),
|
||||
QUICPort: uint(3000 + i),
|
||||
@@ -115,12 +118,13 @@ func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
|
||||
|
||||
// Set the ENR `attnets`, used by Prysm to filter peers by subnet.
|
||||
bitV := bitfield.NewBitvector64()
|
||||
bitV.SetBitAt(subnet, true)
|
||||
bitV.SetBitAt(i, true)
|
||||
entry := enr.WithEntry(attSubnetEnrKey, &bitV)
|
||||
service.dv5Listener.LocalNode().Set(entry)
|
||||
|
||||
// Join and subscribe to the subnet, needed by libp2p.
|
||||
topic, err := service.pubsub.Join(fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + "/ssz_snappy")
|
||||
topicName := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, i) + "/ssz_snappy"
|
||||
topic, err := service.pubsub.Join(topicName)
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = topic.Subscribe()
|
||||
@@ -160,37 +164,18 @@ func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
}()
|
||||
|
||||
// Look up 3 different subnets.
|
||||
exists := make([]bool, 0, 3)
|
||||
for i := 1; i <= 3; i++ {
|
||||
subnet := uint64(i)
|
||||
topic := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet)
|
||||
subnets := map[uint64]bool{1: true, 2: true, 3: true}
|
||||
defectiveSubnets := service.defectiveSubnets(AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets)
|
||||
require.Equal(t, subnetCount, len(defectiveSubnets))
|
||||
|
||||
exist := false
|
||||
ctxWithTimeOut, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
// This for loop is used to ensure we don't get stuck in `FindPeersWithSubnet`.
|
||||
// Read the documentation of `FindPeersWithSubnet` for more details.
|
||||
for j := 0; j < 3; j++ {
|
||||
ctxWithTimeOut, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
err = service.FindAndDialPeersWithSubnets(ctxWithTimeOut, AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets)
|
||||
require.NoError(t, err)
|
||||
|
||||
exist, err = service.FindPeersWithSubnet(ctxWithTimeOut, topic, subnet, 1)
|
||||
require.NoError(t, err)
|
||||
|
||||
if exist {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
require.NoError(t, err)
|
||||
exists = append(exists, exist)
|
||||
|
||||
}
|
||||
|
||||
// Check if all peers are found.
|
||||
for _, exist := range exists {
|
||||
require.Equal(t, true, exist, "Peer with subnet doesn't exist")
|
||||
}
|
||||
defectiveSubnets = service.defectiveSubnets(AttestationSubnetTopicFormat, bootNodeForkDigest, minimumPeersPerSubnet, subnets)
|
||||
require.Equal(t, 0, len(defectiveSubnets))
|
||||
}
|
||||
|
||||
func Test_AttSubnets(t *testing.T) {
|
||||
@@ -305,37 +290,34 @@ func Test_AttSubnets(t *testing.T) {
|
||||
wantErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
db, err := enode.OpenDB("")
|
||||
assert.NoError(t, err)
|
||||
require.NoError(t, err)
|
||||
|
||||
priv, _, err := crypto.GenerateSecp256k1Key(rand.Reader)
|
||||
assert.NoError(t, err)
|
||||
require.NoError(t, err)
|
||||
|
||||
convertedKey, err := ecdsaprysm.ConvertFromInterfacePrivKey(priv)
|
||||
assert.NoError(t, err)
|
||||
require.NoError(t, err)
|
||||
|
||||
localNode := enode.NewLocalNode(db, convertedKey)
|
||||
record := tt.record(localNode)
|
||||
|
||||
got, err := attSubnets(record)
|
||||
got, err := attestationSubnets(record)
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("syncSubnets() error = %v, wantErr %v", err, tt.wantErr)
|
||||
t.Errorf("attestationSubnets() error = %v, wantErr %v", err, tt.wantErr)
|
||||
return
|
||||
}
|
||||
|
||||
if tt.wantErr {
|
||||
assert.ErrorContains(t, tt.errContains, err)
|
||||
require.ErrorContains(t, tt.errContains, err)
|
||||
}
|
||||
|
||||
want := make(map[uint64]bool, len(tt.want))
|
||||
require.Equal(t, len(tt.want), len(got))
|
||||
for _, subnet := range tt.want {
|
||||
want[subnet] = true
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(got, want) {
|
||||
t.Errorf("syncSubnets() got = %v, want %v", got, want)
|
||||
require.Equal(t, true, got[subnet])
|
||||
}
|
||||
})
|
||||
}
|
||||
@@ -494,11 +476,14 @@ func Test_SyncSubnets(t *testing.T) {
|
||||
t.Errorf("syncSubnets() error = %v, wantErr %v", err, tt.wantErr)
|
||||
return
|
||||
}
|
||||
|
||||
if tt.wantErr {
|
||||
assert.ErrorContains(t, tt.errContains, err)
|
||||
}
|
||||
if !reflect.DeepEqual(got, tt.want) {
|
||||
t.Errorf("syncSubnets() got = %v, want %v", got, tt.want)
|
||||
|
||||
require.Equal(t, len(tt.want), len(got))
|
||||
for _, subnet := range tt.want {
|
||||
require.Equal(t, true, got[subnet])
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@@ -68,9 +68,9 @@ func (*FakeP2P) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// FindPeersWithSubnet mocks the p2p func.
|
||||
func (*FakeP2P) FindPeersWithSubnet(_ context.Context, _ string, _ uint64, _ int) (bool, error) {
|
||||
return false, nil
|
||||
// FindAndDialPeersWithSubnets mocks the p2p func.
|
||||
func (*FakeP2P) FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// RefreshPersistentSubnets mocks the p2p func.
|
||||
@@ -167,7 +167,7 @@ func (*FakeP2P) BroadcastLightClientFinalityUpdate(_ context.Context, _ interfac
|
||||
}
|
||||
|
||||
// BroadcastDataColumn -- fake.
|
||||
func (*FakeP2P) BroadcastDataColumn(_ [fieldparams.RootLength]byte, _ uint64, _ *ethpb.DataColumnSidecar, _ ...chan<- bool) error {
|
||||
func (*FakeP2P) BroadcastDataColumn(_ [fieldparams.RootLength]byte, _ uint64, _ *ethpb.DataColumnSidecar) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
||||
@@ -63,7 +63,7 @@ func (m *MockBroadcaster) BroadcastLightClientFinalityUpdate(_ context.Context,
|
||||
}
|
||||
|
||||
// BroadcastDataColumn broadcasts a data column for mock.
|
||||
func (m *MockBroadcaster) BroadcastDataColumn([fieldparams.RootLength]byte, uint64, *ethpb.DataColumnSidecar, ...chan<- bool) error {
|
||||
func (m *MockBroadcaster) BroadcastDataColumn([fieldparams.RootLength]byte, uint64, *ethpb.DataColumnSidecar) error {
|
||||
m.BroadcastCalled.Store(true)
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -4,6 +4,7 @@ import (
|
||||
"context"
|
||||
"errors"
|
||||
|
||||
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
"github.com/ethereum/go-ethereum/p2p/enr"
|
||||
"github.com/libp2p/go-libp2p/core/host"
|
||||
@@ -56,9 +57,9 @@ func (m *MockPeerManager) DiscoveryAddresses() ([]multiaddr.Multiaddr, error) {
|
||||
// RefreshPersistentSubnets .
|
||||
func (*MockPeerManager) RefreshPersistentSubnets() {}
|
||||
|
||||
// FindPeersWithSubnet .
|
||||
func (*MockPeerManager) FindPeersWithSubnet(_ context.Context, _ string, _ uint64, _ int) (bool, error) {
|
||||
return true, nil
|
||||
// FindAndDialPeersWithSubnet .
|
||||
func (*MockPeerManager) FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// AddPingMethod .
|
||||
|
||||
@@ -224,7 +224,7 @@ func (p *TestP2P) BroadcastLightClientFinalityUpdate(_ context.Context, _ interf
|
||||
}
|
||||
|
||||
// BroadcastDataColumn broadcasts a data column for mock.
|
||||
func (p *TestP2P) BroadcastDataColumn([fieldparams.RootLength]byte, uint64, *ethpb.DataColumnSidecar, ...chan<- bool) error {
|
||||
func (p *TestP2P) BroadcastDataColumn([fieldparams.RootLength]byte, uint64, *ethpb.DataColumnSidecar) error {
|
||||
p.BroadcastCalled.Store(true)
|
||||
return nil
|
||||
}
|
||||
@@ -408,9 +408,9 @@ func (p *TestP2P) Peers() *peers.Status {
|
||||
return p.peers
|
||||
}
|
||||
|
||||
// FindPeersWithSubnet mocks the p2p func.
|
||||
func (*TestP2P) FindPeersWithSubnet(_ context.Context, _ string, _ uint64, _ int) (bool, error) {
|
||||
return false, nil
|
||||
// FindAndDialPeersWithSubnets mocks the p2p func.
|
||||
func (*TestP2P) FindAndDialPeersWithSubnets(ctx context.Context, topicFormat string, digest [fieldparams.VersionLength]byte, minimumPeersPerSubnet int, subnets map[uint64]bool) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// RefreshPersistentSubnets mocks the p2p func.
|
||||
|
||||
@@ -36,6 +36,7 @@ const (
|
||||
GossipLightClientOptimisticUpdateMessage = "light_client_optimistic_update"
|
||||
// GossipDataColumnSidecarMessage is the name for the data column sidecar message type.
|
||||
GossipDataColumnSidecarMessage = "data_column_sidecar"
|
||||
GossipInclusionList = "inclusion_list"
|
||||
|
||||
// Topic Formats
|
||||
//
|
||||
@@ -65,4 +66,5 @@ const (
|
||||
LightClientOptimisticUpdateTopicFormat = GossipProtocolAndDigest + GossipLightClientOptimisticUpdateMessage
|
||||
// DataColumnSubnetTopicFormat is the topic format for the data column subnet.
|
||||
DataColumnSubnetTopicFormat = GossipProtocolAndDigest + GossipDataColumnSidecarMessage + "_%d"
|
||||
InclusionListTopicFormat = GossipProtocolAndDigest + GossipInclusionList
|
||||
)
|
||||
|
||||
@@ -550,12 +550,10 @@ func (s *Service) GetAttestationData(
|
||||
return nil, &RpcError{Reason: Unavailable, Err: errOptimisticMode}
|
||||
}
|
||||
|
||||
headRoot, err := s.HeadFetcher.HeadRoot(ctx)
|
||||
if err != nil {
|
||||
return nil, &RpcError{Reason: Internal, Err: errors.Wrap(err, "could not get head root")}
|
||||
}
|
||||
headRoot := s.ChainInfoFetcher.GetAttesterHead() // Attesters vote based on IL constrained head root.
|
||||
|
||||
targetEpoch := slots.ToEpoch(req.Slot)
|
||||
targetRoot, err := s.HeadFetcher.TargetRootForEpoch(bytesutil.ToBytes32(headRoot), targetEpoch)
|
||||
targetRoot, err := s.HeadFetcher.TargetRootForEpoch(headRoot, targetEpoch)
|
||||
if err != nil {
|
||||
return nil, &RpcError{Reason: Internal, Err: errors.Wrap(err, "could not get target root")}
|
||||
}
|
||||
@@ -565,7 +563,7 @@ func (s *Service) GetAttestationData(
|
||||
return nil, &RpcError{Reason: Internal, Err: errors.Wrap(err, "could not get head state")}
|
||||
}
|
||||
if coreTime.CurrentEpoch(headState) < slots.ToEpoch(req.Slot) { // Ensure justified checkpoint safety by processing head state across the boundary.
|
||||
headState, err = transition.ProcessSlotsUsingNextSlotCache(ctx, headState, headRoot, req.Slot)
|
||||
headState, err = transition.ProcessSlotsUsingNextSlotCache(ctx, headState, headRoot[:], req.Slot)
|
||||
if err != nil {
|
||||
return nil, &RpcError{Reason: Internal, Err: errors.Errorf("could not process slots up to %d: %v", req.Slot, err)}
|
||||
}
|
||||
@@ -574,7 +572,7 @@ func (s *Service) GetAttestationData(
|
||||
|
||||
if err = s.AttestationCache.Put(&cache.AttestationConsensusData{
|
||||
Slot: req.Slot,
|
||||
HeadRoot: headRoot,
|
||||
HeadRoot: headRoot[:],
|
||||
Target: forkchoicetypes.Checkpoint{
|
||||
Epoch: targetEpoch,
|
||||
Root: targetRoot,
|
||||
@@ -590,7 +588,7 @@ func (s *Service) GetAttestationData(
|
||||
return ðpb.AttestationData{
|
||||
Slot: req.Slot,
|
||||
CommitteeIndex: committeeIndex,
|
||||
BeaconBlockRoot: headRoot,
|
||||
BeaconBlockRoot: headRoot[:],
|
||||
Source: ðpb.Checkpoint{
|
||||
Epoch: justifiedCheckpoint.Epoch,
|
||||
Root: justifiedCheckpoint.Root,
|
||||
|
||||
@@ -577,6 +577,12 @@ func TestGetSpec(t *testing.T) {
|
||||
assert.Equal(t, "102", v)
|
||||
case "BLOB_SIDECAR_SUBNET_COUNT_ELECTRA":
|
||||
assert.Equal(t, "103", v)
|
||||
case "DOMAIN_INCLUSION_LIST_COMMITTEE":
|
||||
assert.Equal(t, "0x00000000", v)
|
||||
case "EIP7805_FORK_VERSION":
|
||||
assert.Equal(t, "0x0a000000", v)
|
||||
case "EIP7805_FORK_EPOCH":
|
||||
assert.Equal(t, "18446744073709551615", v)
|
||||
default:
|
||||
t.Errorf("Incorrect key: %s", k)
|
||||
}
|
||||
|
||||
@@ -64,6 +64,8 @@ const (
|
||||
PayloadAttributesTopic = "payload_attributes"
|
||||
// BlobSidecarTopic represents a new blob sidecar event topic
|
||||
BlobSidecarTopic = "blob_sidecar"
|
||||
// InclusionListTopic represents a new inclusion list event topic
|
||||
InclusionListTopic = "inclusion_list"
|
||||
// ProposerSlashingTopic represents a new proposer slashing event topic
|
||||
ProposerSlashingTopic = "proposer_slashing"
|
||||
// AttesterSlashingTopic represents a new attester slashing event topic
|
||||
@@ -104,6 +106,7 @@ var opsFeedEventTopics = map[feed.EventType]string{
|
||||
operation.SyncCommitteeContributionReceived: SyncCommitteeContributionTopic,
|
||||
operation.BLSToExecutionChangeReceived: BLSToExecutionChangeTopic,
|
||||
operation.BlobSidecarReceived: BlobSidecarTopic,
|
||||
operation.InclusionListReceived: InclusionListTopic,
|
||||
operation.AttesterSlashingReceived: AttesterSlashingTopic,
|
||||
operation.ProposerSlashingReceived: ProposerSlashingTopic,
|
||||
operation.BlockGossipReceived: BlockGossipTopic,
|
||||
@@ -444,6 +447,8 @@ func topicForEvent(event *feed.Event) string {
|
||||
return BLSToExecutionChangeTopic
|
||||
case *operation.BlobSidecarReceivedData:
|
||||
return BlobSidecarTopic
|
||||
case *operation.InclusionListReceivedData:
|
||||
return InclusionListTopic
|
||||
case *operation.AttesterSlashingReceivedData:
|
||||
return AttesterSlashingTopic
|
||||
case *operation.ProposerSlashingReceivedData:
|
||||
@@ -576,6 +581,13 @@ func (s *Server) lazyReaderForEvent(ctx context.Context, event *feed.Event, topi
|
||||
KzgCommitment: hexutil.Encode(v.Blob.KzgCommitment),
|
||||
})
|
||||
}, nil
|
||||
case *operation.InclusionListReceivedData:
|
||||
return func() io.Reader {
|
||||
return jsonMarshalReader(eventName, structs.InclusionListEvent{
|
||||
Version: "eip7805",
|
||||
Data: structs.SignedInclusionListFromConsensus(v.SignedInclusionList),
|
||||
})
|
||||
}, nil
|
||||
case *operation.AttesterSlashingReceivedData:
|
||||
switch slashing := v.AttesterSlashing.(type) {
|
||||
case *eth.AttesterSlashing:
|
||||
|
||||
@@ -118,6 +118,7 @@ func operationEventsFixtures(t *testing.T) (*topicRequest, []*feed.Event) {
|
||||
SyncCommitteeContributionTopic,
|
||||
BLSToExecutionChangeTopic,
|
||||
BlobSidecarTopic,
|
||||
InclusionListTopic,
|
||||
AttesterSlashingTopic,
|
||||
ProposerSlashingTopic,
|
||||
BlockGossipTopic,
|
||||
@@ -207,6 +208,20 @@ func operationEventsFixtures(t *testing.T) (*topicRequest, []*feed.Event) {
|
||||
Blob: &vblob,
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: operation.InclusionListReceived,
|
||||
Data: &operation.InclusionListReceivedData{
|
||||
SignedInclusionList: ð.SignedInclusionList{
|
||||
Message: ð.InclusionList{
|
||||
Slot: 0,
|
||||
ValidatorIndex: 0,
|
||||
InclusionListCommitteeRoot: make([]byte, fieldparams.RootLength),
|
||||
Transactions: [][]byte{},
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: operation.AttesterSlashingReceived,
|
||||
Data: &operation.AttesterSlashingReceivedData{
|
||||
|
||||
@@ -52,9 +52,10 @@ func TestLightClientHandler_GetLightClientBootstrap(t *testing.T) {
|
||||
bootstrap, err := lightclient.NewLightClientBootstrapFromBeaconState(l.Ctx, slot, l.State, l.Block)
|
||||
require.NoError(t, err)
|
||||
|
||||
lcStore := lightclient.NewLightClientStore(dbtesting.SetupDB(t))
|
||||
db := dbtesting.SetupDB(t)
|
||||
lcStore := lightclient.NewLightClientStore(db)
|
||||
|
||||
err = lcStore.SaveLightClientBootstrap(l.Ctx, blockRoot, bootstrap)
|
||||
err = db.SaveLightClientBootstrap(l.Ctx, blockRoot[:], bootstrap)
|
||||
require.NoError(t, err)
|
||||
|
||||
s := &Server{
|
||||
@@ -95,9 +96,10 @@ func TestLightClientHandler_GetLightClientBootstrap(t *testing.T) {
|
||||
bootstrap, err := lightclient.NewLightClientBootstrapFromBeaconState(l.Ctx, slot, l.State, l.Block)
|
||||
require.NoError(t, err)
|
||||
|
||||
lcStore := lightclient.NewLightClientStore(dbtesting.SetupDB(t))
|
||||
db := dbtesting.SetupDB(t)
|
||||
lcStore := lightclient.NewLightClientStore(db)
|
||||
|
||||
err = lcStore.SaveLightClientBootstrap(l.Ctx, blockRoot, bootstrap)
|
||||
err = db.SaveLightClientBootstrap(l.Ctx, blockRoot[:], bootstrap)
|
||||
require.NoError(t, err)
|
||||
|
||||
s := &Server{
|
||||
|
||||
@@ -1365,6 +1365,7 @@ func TestGetAttestationData(t *testing.T) {
|
||||
FinalizedFetcher: chain,
|
||||
AttestationCache: cache.NewAttestationDataCache(),
|
||||
OptimisticModeFetcher: chain,
|
||||
ChainInfoFetcher: chain,
|
||||
},
|
||||
}
|
||||
|
||||
@@ -1524,6 +1525,7 @@ func TestGetAttestationData(t *testing.T) {
|
||||
HeadFetcher: chain,
|
||||
FinalizedFetcher: chain,
|
||||
OptimisticModeFetcher: chain,
|
||||
ChainInfoFetcher: chain,
|
||||
},
|
||||
}
|
||||
|
||||
@@ -1683,6 +1685,7 @@ func TestGetAttestationData(t *testing.T) {
|
||||
HeadFetcher: chain,
|
||||
GenesisTimeFetcher: chain,
|
||||
FinalizedFetcher: chain,
|
||||
ChainInfoFetcher: chain,
|
||||
},
|
||||
}
|
||||
|
||||
@@ -1853,6 +1856,7 @@ func TestGetAttestationData(t *testing.T) {
|
||||
HeadFetcher: chain,
|
||||
GenesisTimeFetcher: chain,
|
||||
FinalizedFetcher: chain,
|
||||
ChainInfoFetcher: chain,
|
||||
},
|
||||
}
|
||||
|
||||
|
||||
@@ -11,6 +11,7 @@ go_library(
|
||||
"duties_v2.go",
|
||||
"exit.go",
|
||||
"log.go",
|
||||
"inclusion_list.go",
|
||||
"proposer.go",
|
||||
"proposer_altair.go",
|
||||
"proposer_attestations.go",
|
||||
|
||||
@@ -69,6 +69,7 @@ func TestAttestationDataAtSlot_HandlesFarAwayJustifiedEpoch(t *testing.T) {
|
||||
GenesisTimeFetcher: &mock.ChainService{Genesis: time.Now().Add(time.Duration(-1*offset) * time.Second)},
|
||||
FinalizedFetcher: &mock.ChainService{CurrentJustifiedCheckPoint: justifiedCheckpoint},
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: false},
|
||||
ChainInfoFetcher: &mock.ChainService{TargetRoot: blockRoot, Root: blockRoot[:], State: beaconState},
|
||||
},
|
||||
}
|
||||
|
||||
|
||||
@@ -185,6 +185,7 @@ func TestGetAttestationData_OK(t *testing.T) {
|
||||
FinalizedFetcher: &mock.ChainService{CurrentJustifiedCheckPoint: justifiedCheckpoint},
|
||||
AttestationCache: cache.NewAttestationDataCache(),
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: false},
|
||||
ChainInfoFetcher: &mock.ChainService{TargetRoot: targetRoot, Root: blockRoot[:], State: beaconState},
|
||||
},
|
||||
}
|
||||
|
||||
@@ -314,6 +315,7 @@ func TestGetAttestationData_Optimistic(t *testing.T) {
|
||||
HeadFetcher: &mock.ChainService{Optimistic: false, State: beaconState},
|
||||
FinalizedFetcher: &mock.ChainService{CurrentJustifiedCheckPoint: ðpb.Checkpoint{}},
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: false},
|
||||
ChainInfoFetcher: &mock.ChainService{Optimistic: false, State: beaconState},
|
||||
},
|
||||
}
|
||||
_, err = as.GetAttestationData(t.Context(), ðpb.AttestationDataRequest{})
|
||||
@@ -426,6 +428,9 @@ func TestGetAttestationData_SucceedsInFirstEpoch(t *testing.T) {
|
||||
GenesisTimeFetcher: &mock.ChainService{Genesis: prysmTime.Now().Add(time.Duration(-1*offset) * time.Second)},
|
||||
FinalizedFetcher: &mock.ChainService{CurrentJustifiedCheckPoint: justifiedCheckpoint},
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: false},
|
||||
ChainInfoFetcher: &mock.ChainService{
|
||||
TargetRoot: targetRoot, Root: blockRoot[:], State: beaconState,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
@@ -495,6 +500,7 @@ func TestGetAttestationData_CommitteeIndexIsZeroPostElectra(t *testing.T) {
|
||||
FinalizedFetcher: &mock.ChainService{CurrentJustifiedCheckPoint: justifiedCheckpoint},
|
||||
AttestationCache: cache.NewAttestationDataCache(),
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: false},
|
||||
ChainInfoFetcher: &mock.ChainService{TargetRoot: targetRoot, Root: blockRoot[:], State: beaconState},
|
||||
},
|
||||
}
|
||||
|
||||
|
||||
@@ -113,6 +113,7 @@ func (vs *Server) duties(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.
|
||||
assignment.Committee = ca.Committee
|
||||
assignment.AttesterSlot = ca.AttesterSlot
|
||||
assignment.CommitteeIndex = ca.CommitteeIndex
|
||||
assignment.InclusionListCommitteeSlot = ca.InclusionListCommitteeSlot
|
||||
}
|
||||
// Save the next epoch assignments.
|
||||
ca, ok = nextEpochAssignments[idx]
|
||||
@@ -120,6 +121,7 @@ func (vs *Server) duties(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.
|
||||
nextAssignment.Committee = ca.Committee
|
||||
nextAssignment.AttesterSlot = ca.AttesterSlot
|
||||
nextAssignment.CommitteeIndex = ca.CommitteeIndex
|
||||
nextAssignment.InclusionListCommitteeSlot = ca.InclusionListCommitteeSlot
|
||||
}
|
||||
} else {
|
||||
// If the validator isn't in the beacon state, try finding their deposit to determine their status.
|
||||
|
||||
@@ -8,6 +8,7 @@ import (
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/rpc/core"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
|
||||
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
|
||||
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
|
||||
@@ -143,11 +144,12 @@ type dutiesMetadata struct {
|
||||
}
|
||||
|
||||
type metadata struct {
|
||||
committeesAtSlot uint64
|
||||
proposalSlots map[primitives.ValidatorIndex][]primitives.Slot
|
||||
startSlot primitives.Slot
|
||||
committeesBySlot [][][]primitives.ValidatorIndex
|
||||
liteAssignment *helpers.LiteAssignment
|
||||
committeesAtSlot uint64
|
||||
proposalSlots map[primitives.ValidatorIndex][]primitives.Slot
|
||||
startSlot primitives.Slot
|
||||
committeesBySlot [][][]primitives.ValidatorIndex
|
||||
liteAssignment *helpers.LiteAssignment
|
||||
inclusionListCommitteesBySlot [][]primitives.ValidatorIndex
|
||||
}
|
||||
|
||||
func loadDutiesMetadata(ctx context.Context, s state.BeaconState, reqEpoch primitives.Epoch) (*dutiesMetadata, error) {
|
||||
@@ -193,6 +195,18 @@ func loadMetadata(ctx context.Context, s state.BeaconState, reqEpoch primitives.
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Compute inclusion list committees for each slot in the epoch
|
||||
meta.inclusionListCommitteesBySlot = make([][]primitives.ValidatorIndex, params.BeaconConfig().SlotsPerEpoch)
|
||||
for i := primitives.Slot(0); i < params.BeaconConfig().SlotsPerEpoch; i++ {
|
||||
slot := meta.startSlot + i
|
||||
inclusionListCommittee, err := helpers.GetInclusionListCommittee(ctx, s, slot)
|
||||
if err != nil {
|
||||
// Skip inclusion list committee computation if not supported for this slot
|
||||
continue
|
||||
}
|
||||
meta.inclusionListCommitteesBySlot[i] = inclusionListCommittee
|
||||
}
|
||||
|
||||
return meta, nil
|
||||
}
|
||||
|
||||
@@ -215,11 +229,37 @@ func (vs *Server) buildValidatorDuty(
|
||||
assignment.ProposerSlots = meta.current.proposalSlots[idx]
|
||||
populateCommitteeFields(assignment, meta.current.liteAssignment)
|
||||
|
||||
// Check for inclusion list committee assignment in current epoch
|
||||
for slotOffset, inclusionListCommittee := range meta.current.inclusionListCommitteesBySlot {
|
||||
for _, validatorIndex := range inclusionListCommittee {
|
||||
if validatorIndex == idx {
|
||||
assignment.InclusionListCommitteeSlot = meta.current.startSlot + primitives.Slot(slotOffset)
|
||||
break
|
||||
}
|
||||
}
|
||||
if assignment.InclusionListCommitteeSlot != 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
nextAssignment.ValidatorIndex = idx
|
||||
nextAssignment.Status = statusEnum
|
||||
nextAssignment.CommitteesAtSlot = meta.next.committeesAtSlot
|
||||
populateCommitteeFields(nextAssignment, meta.next.liteAssignment)
|
||||
|
||||
// Check for inclusion list committee assignment in next epoch
|
||||
for slotOffset, inclusionListCommittee := range meta.next.inclusionListCommitteesBySlot {
|
||||
for _, validatorIndex := range inclusionListCommittee {
|
||||
if validatorIndex == idx {
|
||||
nextAssignment.InclusionListCommitteeSlot = meta.next.startSlot + primitives.Slot(slotOffset)
|
||||
break
|
||||
}
|
||||
}
|
||||
if nextAssignment.InclusionListCommitteeSlot != 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Sync committee flags
|
||||
if coreTime.HigherEqualThanAltairVersionAndEpoch(s, reqEpoch) {
|
||||
inSync, err := helpers.IsCurrentPeriodSyncCommittee(s, idx)
|
||||
|
||||
79
beacon-chain/rpc/prysm/v1alpha1/validator/inclusion_list.go
Normal file
79
beacon-chain/rpc/prysm/v1alpha1/validator/inclusion_list.go
Normal file
@@ -0,0 +1,79 @@
|
||||
package validator
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/encoding/ssz"
|
||||
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
|
||||
"github.com/OffchainLabs/prysm/v6/time/slots"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/protobuf/types/known/emptypb"
|
||||
)
|
||||
|
||||
// GetInclusionList retrieves the inclusion list for the specified slot.
|
||||
// The slot must be the current or next slot. The inclusion list is built using
|
||||
// committee indices, the execution payload header from beacon state, and the transactions from the execution engine.
|
||||
func (vs *Server) GetInclusionList(ctx context.Context, request *ethpb.GetInclusionListRequest) (*ethpb.InclusionList, error) {
|
||||
currentSlot := vs.TimeFetcher.CurrentSlot()
|
||||
if request.Slot != currentSlot && request.Slot+1 != currentSlot {
|
||||
return nil, status.Errorf(codes.InvalidArgument, "requested slot %d is not current or next slot", request.Slot)
|
||||
}
|
||||
|
||||
st, err := vs.HeadFetcher.HeadState(ctx)
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "failed to get head state: %v", err)
|
||||
}
|
||||
st, err = transition.ProcessSlotsIfPossible(ctx, st, request.Slot)
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "failed to process slots: %v", err)
|
||||
}
|
||||
indices, err := helpers.GetInclusionListCommittee(ctx, st, request.Slot)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
root, err := ssz.InclusionListRoot(indices)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
header, err := st.LatestExecutionPayloadHeader()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Fetch the transactions associated with the inclusion list.
|
||||
txs, err := vs.ExecutionEngineCaller.GetInclusionList(ctx, [32]byte(header.BlockHash()))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return ðpb.InclusionList{
|
||||
Slot: request.Slot,
|
||||
InclusionListCommitteeRoot: root[:],
|
||||
Transactions: txs,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// SubmitInclusionList broadcasts a signed inclusion list to the P2P network and caches it locally.
|
||||
func (vs *Server) SubmitInclusionList(ctx context.Context, il *ethpb.SignedInclusionList) (*emptypb.Empty, error) {
|
||||
slotStartTime := slots.UnsafeStartTime(vs.TimeFetcher.GenesisTime(), il.Message.Slot)
|
||||
currentTime := time.Now()
|
||||
isBeforeFreezeDeadline := vs.TimeFetcher.CurrentSlot() == il.Message.Slot &&
|
||||
currentTime.Sub(slotStartTime) < time.Duration(params.BeaconConfig().InclusionListFreezeDeadLine)*time.Second
|
||||
if !isBeforeFreezeDeadline {
|
||||
return nil, status.Errorf(codes.InvalidArgument, "inclusion list submission is after freeze deadline")
|
||||
}
|
||||
|
||||
if err := vs.P2P.Broadcast(ctx, il); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
vs.InclusionLists.Add(il.Message.Slot, il.Message.ValidatorIndex, il.Message.Transactions, isBeforeFreezeDeadline)
|
||||
|
||||
return &emptypb.Empty{}, nil
|
||||
}
|
||||
@@ -80,6 +80,7 @@ type Server struct {
|
||||
ClockWaiter startup.ClockWaiter
|
||||
CoreService *core.Service
|
||||
AttestationStateFetcher blockchain.AttestationStateFetcher
|
||||
InclusionLists *cache.InclusionLists
|
||||
}
|
||||
|
||||
// Deprecated: The gRPC API will remain the default and fully supported through v8 (expected in 2026) but will be eventually removed in favor of REST API.
|
||||
|
||||
@@ -24,13 +24,10 @@ func (vs *Server) GetSyncMessageBlockRoot(
|
||||
return nil, err
|
||||
}
|
||||
|
||||
r, err := vs.HeadFetcher.HeadRoot(ctx)
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "Could not retrieve head root: %v", err)
|
||||
}
|
||||
r := vs.ForkchoiceFetcher.GetAttesterHead() // Sync committee vote based on IL constrained head root.
|
||||
|
||||
return ðpb.SyncMessageBlockRootResponse{
|
||||
Root: r,
|
||||
Root: r[:],
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -80,24 +77,22 @@ func (vs *Server) GetSyncCommitteeContribution(
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "Could not get sync subcommittee messages: %v", err)
|
||||
}
|
||||
headRoot, err := vs.HeadFetcher.HeadRoot(ctx)
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "Could not get head root: %v", err)
|
||||
}
|
||||
headRoot := vs.ForkchoiceFetcher.GetAttesterHead()
|
||||
|
||||
sig, aggregatedBits, err := vs.CoreService.AggregatedSigAndAggregationBits(
|
||||
ctx,
|
||||
ðpb.AggregatedSigAndAggregationBitsRequest{
|
||||
Msgs: msgs,
|
||||
Slot: req.Slot,
|
||||
SubnetId: req.SubnetId,
|
||||
BlockRoot: headRoot,
|
||||
BlockRoot: headRoot[:],
|
||||
})
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Internal, "Could not get contribution data: %v", err)
|
||||
}
|
||||
contribution := ðpb.SyncCommitteeContribution{
|
||||
Slot: req.Slot,
|
||||
BlockRoot: headRoot,
|
||||
BlockRoot: headRoot[:],
|
||||
SubcommitteeIndex: req.SubnetId,
|
||||
AggregationBits: aggregatedBits,
|
||||
Signature: sig,
|
||||
|
||||
@@ -25,14 +25,14 @@ import (
|
||||
)
|
||||
|
||||
func TestGetSyncMessageBlockRoot_OK(t *testing.T) {
|
||||
r := []byte{'a'}
|
||||
r := [32]byte{'a'}
|
||||
server := &Server{
|
||||
HeadFetcher: &mock.ChainService{Root: r},
|
||||
TimeFetcher: &mock.ChainService{Genesis: time.Now()},
|
||||
TimeFetcher: &mock.ChainService{Genesis: time.Now()},
|
||||
ForkchoiceFetcher: &mock.ChainService{Root: r[:]},
|
||||
}
|
||||
res, err := server.GetSyncMessageBlockRoot(t.Context(), &emptypb.Empty{})
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, r, res.Root)
|
||||
require.DeepEqual(t, r[:], res.Root)
|
||||
}
|
||||
|
||||
func TestGetSyncMessageBlockRoot_Optimistic(t *testing.T) {
|
||||
@@ -42,7 +42,7 @@ func TestGetSyncMessageBlockRoot_Optimistic(t *testing.T) {
|
||||
params.OverrideBeaconConfig(cfg)
|
||||
|
||||
server := &Server{
|
||||
HeadFetcher: &mock.ChainService{},
|
||||
ForkchoiceFetcher: &mock.ChainService{},
|
||||
TimeFetcher: &mock.ChainService{Genesis: time.Now()},
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: true},
|
||||
}
|
||||
@@ -53,7 +53,7 @@ func TestGetSyncMessageBlockRoot_Optimistic(t *testing.T) {
|
||||
require.ErrorContains(t, errOptimisticMode.Error(), err)
|
||||
|
||||
server = &Server{
|
||||
HeadFetcher: &mock.ChainService{},
|
||||
ForkchoiceFetcher: &mock.ChainService{},
|
||||
TimeFetcher: &mock.ChainService{Genesis: time.Now()},
|
||||
OptimisticModeFetcher: &mock.ChainService{Optimistic: false},
|
||||
}
|
||||
@@ -118,6 +118,7 @@ func TestGetSyncCommitteeContribution_FiltersDuplicates(t *testing.T) {
|
||||
HeadFetcher: headFetcher,
|
||||
P2P: &mockp2p.MockBroadcaster{},
|
||||
TimeFetcher: &mock.ChainService{Genesis: time.Now()},
|
||||
ForkchoiceFetcher: headFetcher,
|
||||
}
|
||||
secKey, err := bls.RandKey()
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -123,6 +123,7 @@ type Config struct {
|
||||
TrackedValidatorsCache *cache.TrackedValidatorsCache
|
||||
PayloadIDCache *cache.PayloadIDCache
|
||||
LCStore *lightClient.Store
|
||||
InclusionListsCache *cache.InclusionLists
|
||||
}
|
||||
|
||||
// NewService instantiates a new RPC service instance that will
|
||||
@@ -212,6 +213,7 @@ func NewService(ctx context.Context, cfg *Config) *Service {
|
||||
FinalizedFetcher: s.cfg.FinalizationFetcher,
|
||||
ReplayerBuilder: ch,
|
||||
OptimisticModeFetcher: s.cfg.OptimisticModeFetcher,
|
||||
ChainInfoFetcher: s.cfg.ChainInfoFetcher,
|
||||
}
|
||||
validatorServer := &validatorv1alpha1.Server{
|
||||
Ctx: s.ctx,
|
||||
@@ -252,6 +254,7 @@ func NewService(ctx context.Context, cfg *Config) *Service {
|
||||
TrackedValidatorsCache: s.cfg.TrackedValidatorsCache,
|
||||
PayloadIDCache: s.cfg.PayloadIDCache,
|
||||
AttestationStateFetcher: s.cfg.AttestationReceiver,
|
||||
InclusionLists: s.cfg.InclusionListsCache,
|
||||
}
|
||||
s.validatorServer = validatorServer
|
||||
nodeServer := &nodev1alpha1.Server{
|
||||
|
||||
@@ -14,6 +14,7 @@ go_library(
|
||||
"error.go",
|
||||
"fork_watcher.go",
|
||||
"fuzz_exports.go", # keep
|
||||
"inclusion_list.go",
|
||||
"log.go",
|
||||
"metrics.go",
|
||||
"options.go",
|
||||
@@ -120,6 +121,7 @@ go_library(
|
||||
"//crypto/bls:go_default_library",
|
||||
"//crypto/rand:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//encoding/ssz:go_default_library",
|
||||
"//encoding/ssz/equality:go_default_library",
|
||||
"//io/file:go_default_library",
|
||||
"//math:go_default_library",
|
||||
|
||||
@@ -109,7 +109,12 @@ func isValidStreamError(err error) bool {
|
||||
|
||||
func closeStream(stream network.Stream, log *logrus.Entry) {
|
||||
if err := stream.Close(); isValidStreamError(err) {
|
||||
log.WithError(err).Debugf("Could not reset stream with protocol %s", stream.Protocol())
|
||||
log.WithError(err).
|
||||
WithFields(logrus.Fields{
|
||||
"protocol": stream.Protocol(),
|
||||
"peer": stream.Conn().RemotePeer(),
|
||||
}).
|
||||
Debug("Could not close stream")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -118,7 +123,12 @@ func closeStreamAndWait(stream network.Stream, log *logrus.Entry) {
|
||||
_err := stream.Reset()
|
||||
_ = _err
|
||||
if isValidStreamError(err) {
|
||||
log.WithError(err).Debugf("Could not reset stream with protocol %s", stream.Protocol())
|
||||
log.WithError(err).
|
||||
WithFields(logrus.Fields{
|
||||
"protocol": stream.Protocol(),
|
||||
"peer": stream.Conn().RemotePeer(),
|
||||
}).
|
||||
Debug("Could not reset stream")
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
180
beacon-chain/sync/inclusion_list.go
Normal file
180
beacon-chain/sync/inclusion_list.go
Normal file
@@ -0,0 +1,180 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed"
|
||||
opfeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/operation"
|
||||
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/helpers"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/encoding/ssz"
|
||||
eth "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
|
||||
prysmTime "github.com/OffchainLabs/prysm/v6/time"
|
||||
"github.com/OffchainLabs/prysm/v6/time/slots"
|
||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
// validateInclusionList validates an incoming inclusion list message.
|
||||
// Returns appropriate validation results based on the following rules:
|
||||
// [REJECT] The slot `message.slot` is equal to the previous or current slot.
|
||||
// [IGNORE] The slot `message.slot` is equal to the current slot, or it is equal to the previous slot and the current time
|
||||
//
|
||||
// is less than attestation_deadline seconds into the slot.
|
||||
//
|
||||
// [IGNORE] The inclusion_list_committee for slot `message.slot` on the current branch corresponds to `message.inclusion_list_committee_root`,
|
||||
//
|
||||
// as determined by `hash_tree_root(inclusion_list_committee) == message.inclusion_list_committee_root`.
|
||||
//
|
||||
// [REJECT] The validator index `message.validator_index` is within the inclusion_list_committee corresponding to `message.inclusion_list_committee_root`.
|
||||
// [REJECT] The transactions `message.transactions` length is within the upper bound MAX_TRANSACTIONS_PER_INCLUSION_LIST.
|
||||
// [IGNORE] The message is either the first or second valid message received from the validator with index `message.validator_index`.
|
||||
// [REJECT] The signature of `inclusion_list.signature` is valid with respect to the validator index.
|
||||
func (s *Service) validateInclusionList(ctx context.Context, id peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
|
||||
// Skip self-published messages.
|
||||
if id == s.cfg.p2p.PeerID() {
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
// Ignore if the node is currently syncing.
|
||||
if s.cfg.initialSync.Syncing() {
|
||||
return pubsub.ValidationIgnore, nil
|
||||
}
|
||||
|
||||
// Validate topic presence.
|
||||
if msg.Topic == nil {
|
||||
return pubsub.ValidationReject, errInvalidTopic
|
||||
}
|
||||
|
||||
// Decode the pubsub message into the appropriate type.
|
||||
m, err := s.decodePubsubMessage(msg)
|
||||
if err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
il, ok := m.(*eth.SignedInclusionList)
|
||||
if !ok {
|
||||
return pubsub.ValidationReject, errWrongMessage
|
||||
}
|
||||
|
||||
// Check for nil inclusion list.
|
||||
if err := helpers.ValidateNilSignedInclusionList(il); err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
|
||||
// Validate slot constraints.
|
||||
currentSlot := s.cfg.clock.CurrentSlot()
|
||||
if il.Message.Slot != currentSlot && il.Message.Slot+1 != currentSlot {
|
||||
return pubsub.ValidationReject, errors.New("slot %d is not equal to the previous %d or current %d slot")
|
||||
}
|
||||
secondsSinceSlotStart, err := slots.SinceSlotStart(currentSlot, s.cfg.chain.GenesisTime(), prysmTime.Now())
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
deadline := params.BeaconConfig().SecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
|
||||
if il.Message.Slot+1 == currentSlot && uint64(secondsSinceSlotStart.Seconds()) > deadline {
|
||||
return pubsub.ValidationIgnore, errors.New("slot is equal to the previous slot and the current time is more than attestation_deadline seconds into the slot")
|
||||
}
|
||||
|
||||
// Fetch the current head state.
|
||||
st, err := s.cfg.chain.HeadState(ctx)
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
|
||||
// Validate inclusion list committee root.
|
||||
committee, err := helpers.GetInclusionListCommittee(ctx, st, il.Message.Slot)
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
root, err := ssz.InclusionListRoot(committee)
|
||||
if err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
if root != [32]byte(il.Message.InclusionListCommitteeRoot) {
|
||||
return pubsub.ValidationReject, errors.New("inclusion_list_committee_root does not match the inclusion_list_committee")
|
||||
}
|
||||
|
||||
// Validate validator index is within the committee.
|
||||
var included bool
|
||||
for _, i := range committee {
|
||||
if i == il.Message.ValidatorIndex {
|
||||
included = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !included {
|
||||
return pubsub.ValidationReject, errors.New("validator_index is not within the inclusion_list_committee")
|
||||
}
|
||||
|
||||
// Validate transaction size.
|
||||
totalSize := 0
|
||||
for _, transaction := range il.Message.Transactions {
|
||||
totalSize += len(transaction)
|
||||
}
|
||||
if totalSize > 8*1024 {
|
||||
return pubsub.ValidationReject, errors.New("total size of transactions exceeds 8KB")
|
||||
}
|
||||
|
||||
// Check for duplicate inclusion list from the validator.
|
||||
if s.inclusionLists.SeenTwice(il.Message.Slot, il.Message.ValidatorIndex) {
|
||||
return pubsub.ValidationReject, errors.New("inclusion list seen twice")
|
||||
}
|
||||
|
||||
// Validate the inclusion list signature.
|
||||
if err := helpers.ValidateInclusionListSignature(ctx, st, il); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
|
||||
msg.ValidatorData = il
|
||||
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
// subscriberInclusionList handles incoming inclusion list messages by adding them to the local inclusion list cache.
|
||||
func (s *Service) subscriberInclusionList(ctx context.Context, msg proto.Message) error {
|
||||
il, ok := msg.(*eth.SignedInclusionList)
|
||||
if !ok {
|
||||
return fmt.Errorf("message was not type *ethpb.SignedInclusionList, type=%T", msg)
|
||||
}
|
||||
if il == nil {
|
||||
return errors.New("nil inclusion list")
|
||||
}
|
||||
|
||||
startTime, err := slots.StartTime(s.cfg.clock.GenesisTime(), il.Message.Slot)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "could not compute start time for slot %d", il.Message.Slot)
|
||||
}
|
||||
timeNow := time.Now()
|
||||
isBeforeFreezeDeadline := s.cfg.clock.CurrentSlot() == il.Message.Slot &&
|
||||
timeNow.Sub(startTime) < time.Duration(params.BeaconConfig().InclusionListFreezeDeadLine)*time.Second
|
||||
|
||||
slotStartTime := slots.UnsafeStartTime(s.cfg.chain.GenesisTime(), il.Message.Slot)
|
||||
ilTxByteSize := 0
|
||||
for _, transaction := range il.Message.Transactions {
|
||||
ilTxByteSize += len(transaction)
|
||||
}
|
||||
|
||||
log.WithFields(logrus.Fields{
|
||||
"slot": il.Message.Slot,
|
||||
"committeeRoot": fmt.Sprintf("%#x", il.Message.InclusionListCommitteeRoot),
|
||||
"txCount": len(il.Message.Transactions),
|
||||
"sinceSlotStart": time.Since(slotStartTime),
|
||||
"ilTxByteSize": ilTxByteSize,
|
||||
}).Info("Inclusion list verified and received")
|
||||
|
||||
s.inclusionLists.Add(il.Message.Slot, il.Message.ValidatorIndex, il.Message.Transactions, isBeforeFreezeDeadline)
|
||||
|
||||
s.cfg.operationNotifier.OperationFeed().Send(&feed.Event{
|
||||
Type: opfeed.InclusionListReceived,
|
||||
Data: &opfeed.InclusionListReceivedData{
|
||||
SignedInclusionList: il,
|
||||
},
|
||||
})
|
||||
|
||||
return nil
|
||||
}
|
||||
@@ -236,7 +236,7 @@ func (s *Service) updateMetrics() {
|
||||
if err != nil {
|
||||
log.WithError(err).Debugf("Could not compute fork digest")
|
||||
}
|
||||
indices := s.aggregatorSubnetIndices(s.cfg.clock.CurrentSlot())
|
||||
indices := aggregatorSubnetIndices(s.cfg.clock.CurrentSlot())
|
||||
syncIndices := cache.SyncSubnetIDs.GetAllSubnets(slots.ToEpoch(s.cfg.clock.CurrentSlot()))
|
||||
attTopic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.Attestation{})]
|
||||
syncTopic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.SyncCommitteeMessage{})]
|
||||
|
||||
@@ -215,6 +215,14 @@ func WithSlasherEnabled(enabled bool) Option {
|
||||
}
|
||||
}
|
||||
|
||||
// WithInclusionListsCache allows sync pkg to access inclusion lists cache.
|
||||
func WithInclusionListsCache(c *cache.InclusionLists) Option {
|
||||
return func(s *Service) error {
|
||||
s.inclusionLists = c
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// WithLightClientStore allows the sync package to access light client data.
|
||||
func WithLightClientStore(lcs *lightClient.Store) Option {
|
||||
return func(s *Service) error {
|
||||
|
||||
@@ -125,6 +125,10 @@ func (s *Service) rpcHandlerByTopicFromEpoch(epoch primitives.Epoch) (map[string
|
||||
// Get the beacon config.
|
||||
beaconConfig := params.BeaconConfig()
|
||||
|
||||
if epoch >= beaconConfig.Eip7805ForkEpoch {
|
||||
return s.rpcHandlerByTopicFromFork(version.Focil)
|
||||
}
|
||||
|
||||
if epoch >= beaconConfig.FuluForkEpoch {
|
||||
return s.rpcHandlerByTopicFromFork(version.Fulu)
|
||||
}
|
||||
|
||||
@@ -76,6 +76,12 @@ func WriteBlockChunk(stream libp2pcore.Stream, tor blockchain.TemporalOracle, en
|
||||
return err
|
||||
}
|
||||
obtainedCtx = digest[:]
|
||||
case version.Focil:
|
||||
digest, err := forks.ForkDigestFromEpoch(params.BeaconConfig().Eip7805ForkEpoch, valRoot[:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
obtainedCtx = digest[:]
|
||||
default:
|
||||
return errors.Wrapf(ErrUnrecognizedVersion, "block version %d is not recognized", blk.Version())
|
||||
}
|
||||
|
||||
@@ -42,7 +42,7 @@ func (s *Service) goodbyeRPCHandler(_ context.Context, msg interface{}, stream l
|
||||
return fmt.Errorf("wrong message type for goodbye, got %T, wanted *uint64", msg)
|
||||
}
|
||||
if err := s.rateLimiter.validateRequest(stream, 1); err != nil {
|
||||
log.WithError(err).Debug("Goodbye message from rate-limited peer.")
|
||||
log.WithError(err).Debug("Goodbye message from rate-limited peer")
|
||||
} else {
|
||||
s.rateLimiter.add(stream, 1)
|
||||
}
|
||||
@@ -65,7 +65,12 @@ func (s *Service) disconnectBadPeer(ctx context.Context, id peer.ID, badPeerErr
|
||||
log.WithError(err).Debug("Error when disconnecting with bad peer")
|
||||
}
|
||||
|
||||
log.WithError(badPeerErr).WithField("peerID", id).Debug("Initiate peer disconnection")
|
||||
log.WithError(badPeerErr).
|
||||
WithFields(logrus.Fields{
|
||||
"peerID": id,
|
||||
"agent": agentString(id, s.cfg.p2p.Host()),
|
||||
}).
|
||||
Debug("Sent peer disconnection")
|
||||
}
|
||||
|
||||
// A custom goodbye method that is used by our connection handler, in the
|
||||
|
||||
@@ -81,7 +81,7 @@ func TestRPC_LightClientBootstrap(t *testing.T) {
|
||||
blockRoot, err := l.Block.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
require.NoError(t, r.lcStore.SaveLightClientBootstrap(ctx, blockRoot, bootstrap))
|
||||
require.NoError(t, r.cfg.beaconDB.SaveLightClientBootstrap(ctx, blockRoot[:], bootstrap))
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(1)
|
||||
|
||||
@@ -217,6 +217,7 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
|
||||
log.WithFields(logrus.Fields{
|
||||
"peer": remotePeer,
|
||||
"error": err,
|
||||
"agent": agentString(remotePeer, s.cfg.p2p.Host()),
|
||||
}).Debug("Invalid status message from peer")
|
||||
|
||||
var respCode byte
|
||||
|
||||
@@ -178,6 +178,7 @@ type Service struct {
|
||||
slasherEnabled bool
|
||||
lcStore *lightClient.Store
|
||||
dataColumnLogCh chan dataColumnLogEntry
|
||||
inclusionLists *cache.InclusionLists
|
||||
}
|
||||
|
||||
// NewService initializes new regular sync service.
|
||||
|
||||
@@ -2,11 +2,9 @@ package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"runtime/debug"
|
||||
"slices"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
@@ -21,7 +19,6 @@ import (
|
||||
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
|
||||
"github.com/OffchainLabs/prysm/v6/config/params"
|
||||
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
|
||||
"github.com/OffchainLabs/prysm/v6/container/slice"
|
||||
"github.com/OffchainLabs/prysm/v6/monitoring/tracing"
|
||||
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
|
||||
"github.com/OffchainLabs/prysm/v6/network/forks"
|
||||
@@ -32,17 +29,50 @@ import (
|
||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
||||
"github.com/libp2p/go-libp2p/core/host"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
const pubsubMessageTimeout = 30 * time.Second
|
||||
|
||||
// wrappedVal represents a gossip validator which also returns an error along with the result.
|
||||
type wrappedVal func(context.Context, peer.ID, *pubsub.Message) (pubsub.ValidationResult, error)
|
||||
type (
|
||||
// wrappedVal represents a gossip validator which also returns an error along with the result.
|
||||
wrappedVal func(context.Context, peer.ID, *pubsub.Message) (pubsub.ValidationResult, error)
|
||||
|
||||
// subHandler represents handler for a given subscription.
|
||||
type subHandler func(context.Context, proto.Message) error
|
||||
// subHandler represents handler for a given subscription.
|
||||
subHandler func(context.Context, proto.Message) error
|
||||
|
||||
// parameters used for the `subscribeWithParameters` function.
|
||||
subscribeParameters struct {
|
||||
topicFormat string
|
||||
validate wrappedVal
|
||||
handle subHandler
|
||||
digest [4]byte
|
||||
|
||||
// getSubnetsToJoin is a function that returns all subnets the node should join.
|
||||
getSubnetsToJoin func(currentSlot primitives.Slot) map[uint64]bool
|
||||
|
||||
// getSubnetsRequiringPeers is a function that returns all subnets that require peers to be found
|
||||
// but for which no subscriptions are needed.
|
||||
getSubnetsRequiringPeers func(currentSlot primitives.Slot) map[uint64]bool
|
||||
}
|
||||
|
||||
// parameters used for the `subscribeToSubnets` function.
|
||||
subscribeToSubnetsParameters struct {
|
||||
subscriptionBySubnet map[uint64]*pubsub.Subscription
|
||||
topicFormat string
|
||||
digest [4]byte
|
||||
genesisValidatorsRoot [fieldparams.RootLength]byte
|
||||
genesisTime time.Time
|
||||
currentSlot primitives.Slot
|
||||
validate wrappedVal
|
||||
handle subHandler
|
||||
getSubnetsToJoin func(currentSlot primitives.Slot) map[uint64]bool
|
||||
}
|
||||
)
|
||||
|
||||
var errInvalidDigest = errors.New("invalid digest")
|
||||
|
||||
// noopValidator is a no-op that only decodes the message, but does not check its contents.
|
||||
func (s *Service) noopValidator(_ context.Context, _ peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
|
||||
@@ -55,28 +85,36 @@ func (s *Service) noopValidator(_ context.Context, _ peer.ID, msg *pubsub.Messag
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
func sliceFromCount(count uint64) []uint64 {
|
||||
result := make([]uint64, 0, count)
|
||||
|
||||
func mapFromCount(count uint64) map[uint64]bool {
|
||||
result := make(map[uint64]bool, count)
|
||||
for item := range count {
|
||||
result = append(result, item)
|
||||
result[item] = true
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
func (s *Service) activeSyncSubnetIndices(currentSlot primitives.Slot) []uint64 {
|
||||
if flags.Get().SubscribeToAllSubnets {
|
||||
return sliceFromCount(params.BeaconConfig().SyncCommitteeSubnetCount)
|
||||
func mapFromSlice(slices ...[]uint64) map[uint64]bool {
|
||||
result := make(map[uint64]bool)
|
||||
|
||||
for _, slice := range slices {
|
||||
for _, item := range slice {
|
||||
result[item] = true
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
func (s *Service) activeSyncSubnetIndices(currentSlot primitives.Slot) map[uint64]bool {
|
||||
if flags.Get().SubscribeToAllSubnets {
|
||||
return mapFromCount(params.BeaconConfig().SyncCommitteeSubnetCount)
|
||||
}
|
||||
|
||||
// Get the current epoch.
|
||||
currentEpoch := slots.ToEpoch(currentSlot)
|
||||
subscriptions := cache.SyncSubnetIDs.GetAllSubnets(currentEpoch)
|
||||
|
||||
// Retrieve the subnets we want to subscribe to.
|
||||
subs := cache.SyncSubnetIDs.GetAllSubnets(currentEpoch)
|
||||
|
||||
return slice.SetUint64(subs)
|
||||
return mapFromSlice(subscriptions)
|
||||
}
|
||||
|
||||
// Register PubSub subscribers
|
||||
@@ -111,14 +149,14 @@ func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
|
||||
s.attesterSlashingSubscriber,
|
||||
digest,
|
||||
)
|
||||
s.subscribeWithParameters(
|
||||
p2p.AttestationSubnetTopicFormat,
|
||||
s.validateCommitteeIndexBeaconAttestation,
|
||||
s.committeeIndexBeaconAttestationSubscriber,
|
||||
digest,
|
||||
s.persistentAndAggregatorSubnetIndices,
|
||||
s.attesterSubnetIndices,
|
||||
)
|
||||
s.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.AttestationSubnetTopicFormat,
|
||||
validate: s.validateCommitteeIndexBeaconAttestation,
|
||||
handle: s.committeeIndexBeaconAttestationSubscriber,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: s.persistentAndAggregatorSubnetIndices,
|
||||
getSubnetsRequiringPeers: attesterSubnetIndices,
|
||||
})
|
||||
|
||||
// New gossip topic in Altair
|
||||
if params.BeaconConfig().AltairForkEpoch <= epoch {
|
||||
@@ -128,14 +166,15 @@ func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
|
||||
s.syncContributionAndProofSubscriber,
|
||||
digest,
|
||||
)
|
||||
s.subscribeWithParameters(
|
||||
p2p.SyncCommitteeSubnetTopicFormat,
|
||||
s.validateSyncCommitteeMessage,
|
||||
s.syncCommitteeMessageSubscriber,
|
||||
digest,
|
||||
s.activeSyncSubnetIndices,
|
||||
func(currentSlot primitives.Slot) []uint64 { return []uint64{} },
|
||||
)
|
||||
|
||||
s.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.SyncCommitteeSubnetTopicFormat,
|
||||
validate: s.validateSyncCommitteeMessage,
|
||||
handle: s.syncCommitteeMessageSubscriber,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: s.activeSyncSubnetIndices,
|
||||
})
|
||||
|
||||
if features.Get().EnableLightClient {
|
||||
s.subscribe(
|
||||
p2p.LightClientOptimisticUpdateTopicFormat,
|
||||
@@ -164,42 +203,54 @@ func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
|
||||
|
||||
// New gossip topic in Deneb, removed in Electra
|
||||
if params.BeaconConfig().DenebForkEpoch <= epoch && epoch < params.BeaconConfig().ElectraForkEpoch {
|
||||
s.subscribeWithParameters(
|
||||
p2p.BlobSubnetTopicFormat,
|
||||
s.validateBlob,
|
||||
s.blobSubscriber,
|
||||
digest,
|
||||
func(currentSlot primitives.Slot) []uint64 {
|
||||
return sliceFromCount(params.BeaconConfig().BlobsidecarSubnetCount)
|
||||
s.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.BlobSubnetTopicFormat,
|
||||
validate: s.validateBlob,
|
||||
handle: s.blobSubscriber,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: func(primitives.Slot) map[uint64]bool {
|
||||
return mapFromCount(params.BeaconConfig().BlobsidecarSubnetCount)
|
||||
},
|
||||
func(currentSlot primitives.Slot) []uint64 { return []uint64{} },
|
||||
)
|
||||
})
|
||||
}
|
||||
|
||||
// New gossip topic in Electra, removed in Fulu
|
||||
if params.BeaconConfig().ElectraForkEpoch <= epoch && epoch < params.BeaconConfig().FuluForkEpoch {
|
||||
s.subscribeWithParameters(
|
||||
p2p.BlobSubnetTopicFormat,
|
||||
s.validateBlob,
|
||||
s.blobSubscriber,
|
||||
digest,
|
||||
func(currentSlot primitives.Slot) []uint64 {
|
||||
return sliceFromCount(params.BeaconConfig().BlobsidecarSubnetCountElectra)
|
||||
s.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.BlobSubnetTopicFormat,
|
||||
validate: s.validateBlob,
|
||||
handle: s.blobSubscriber,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: func(currentSlot primitives.Slot) map[uint64]bool {
|
||||
return mapFromCount(params.BeaconConfig().BlobsidecarSubnetCountElectra)
|
||||
},
|
||||
func(currentSlot primitives.Slot) []uint64 { return []uint64{} },
|
||||
})
|
||||
s.subscribe( // Hack for kurtosis starting from genesis
|
||||
p2p.InclusionListTopicFormat,
|
||||
s.validateInclusionList,
|
||||
s.subscriberInclusionList,
|
||||
digest,
|
||||
)
|
||||
}
|
||||
|
||||
if params.BeaconConfig().Eip7805ForkEpoch <= epoch {
|
||||
s.subscribe(
|
||||
p2p.InclusionListTopicFormat,
|
||||
s.validateInclusionList,
|
||||
s.subscriberInclusionList,
|
||||
digest,
|
||||
)
|
||||
}
|
||||
|
||||
// New gossip topic in Fulu.
|
||||
if params.BeaconConfig().FuluForkEpoch <= epoch {
|
||||
s.subscribeWithParameters(
|
||||
p2p.DataColumnSubnetTopicFormat,
|
||||
s.validateDataColumn,
|
||||
s.dataColumnSubscriber,
|
||||
digest,
|
||||
s.dataColumnSubnetIndices,
|
||||
func(currentSlot primitives.Slot) []uint64 { return []uint64{} },
|
||||
)
|
||||
s.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.DataColumnSubnetTopicFormat,
|
||||
validate: s.validateDataColumn,
|
||||
handle: s.dataColumnSubscriber,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: s.dataColumnSubnetIndices,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -379,197 +430,185 @@ func (s *Service) wrapAndReportValidation(topic string, v wrappedVal) (string, p
|
||||
}
|
||||
}
|
||||
|
||||
// pruneSubscriptions unsubscribe from topics we are currently subscribed to but that are
|
||||
// pruneSubscriptions unsubscribes from topics we are currently subscribed to but that are
|
||||
// not in the list of wanted subnets.
|
||||
// This function mutates the `subscriptionBySubnet` map, which is used to keep track of the current subscriptions.
|
||||
func (s *Service) pruneSubscriptions(
|
||||
subscriptions map[uint64]*pubsub.Subscription,
|
||||
wantedSubs []uint64,
|
||||
subscriptionBySubnet map[uint64]*pubsub.Subscription,
|
||||
wantedSubnets map[uint64]bool,
|
||||
topicFormat string,
|
||||
digest [4]byte,
|
||||
) {
|
||||
for k, v := range subscriptions {
|
||||
var wanted bool
|
||||
for _, idx := range wantedSubs {
|
||||
if k == idx {
|
||||
wanted = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if !wanted && v != nil {
|
||||
v.Cancel()
|
||||
fullTopic := fmt.Sprintf(topicFormat, digest, k) + s.cfg.p2p.Encoding().ProtocolSuffix()
|
||||
s.unSubscribeFromTopic(fullTopic)
|
||||
delete(subscriptions, k)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// searchForPeers searches for peers in the given subnets.
|
||||
func (s *Service) searchForPeers(
|
||||
ctx context.Context,
|
||||
topicFormat string,
|
||||
digest [4]byte,
|
||||
currentSlot primitives.Slot,
|
||||
getSubnetsToSubscribe func(currentSlot primitives.Slot) []uint64,
|
||||
getSubnetsToFindPeersOnly func(currentSlot primitives.Slot) []uint64,
|
||||
) {
|
||||
// Retrieve the subnets we want to subscribe to.
|
||||
subnetsToSubscribeIndex := getSubnetsToSubscribe(currentSlot)
|
||||
|
||||
// Retrieve the subnets we want to find peers for.
|
||||
subnetsToFindPeersOnlyIndex := getSubnetsToFindPeersOnly(currentSlot)
|
||||
|
||||
// Combine the subnets to subscribe and the subnets to find peers for.
|
||||
subnetsToFindPeersIndex := slice.SetUint64(append(subnetsToSubscribeIndex, subnetsToFindPeersOnlyIndex...))
|
||||
|
||||
// Find new peers for wanted subnets if needed.
|
||||
for _, subnetIndex := range subnetsToFindPeersIndex {
|
||||
topic := fmt.Sprintf(topicFormat, digest, subnetIndex)
|
||||
|
||||
// Check if we have enough peers in the subnet. Skip if we do.
|
||||
if s.enoughPeersAreConnected(topic) {
|
||||
for subnet, subscription := range subscriptionBySubnet {
|
||||
if subscription == nil {
|
||||
// Should not happen, but just in case.
|
||||
delete(subscriptionBySubnet, subnet)
|
||||
continue
|
||||
}
|
||||
|
||||
// Not enough peers in the subnet, we need to search for more.
|
||||
_, err := s.cfg.p2p.FindPeersWithSubnet(ctx, topic, subnetIndex, flags.Get().MinimumPeersPerSubnet)
|
||||
if err != nil {
|
||||
log.WithError(err).Debug("Could not search for peers")
|
||||
if wantedSubnets[subnet] {
|
||||
// Nothing to prune.
|
||||
continue
|
||||
}
|
||||
|
||||
// We are subscribed to a subnet that is no longer wanted.
|
||||
subscription.Cancel()
|
||||
fullTopic := fmt.Sprintf(topicFormat, digest, subnet) + s.cfg.p2p.Encoding().ProtocolSuffix()
|
||||
s.unSubscribeFromTopic(fullTopic)
|
||||
delete(subscriptionBySubnet, subnet)
|
||||
}
|
||||
}
|
||||
|
||||
// subscribeToSubnets subscribes to needed subnets, unsubscribe from unneeded ones and search for more peers if needed.
|
||||
// Returns `true` if the digest is valid (wrt. the current epoch), `false` otherwise.
|
||||
func (s *Service) subscribeToSubnets(
|
||||
topicFormat string,
|
||||
digest [4]byte,
|
||||
genesisValidatorsRoot [fieldparams.RootLength]byte,
|
||||
genesisTime time.Time,
|
||||
subscriptions map[uint64]*pubsub.Subscription,
|
||||
currentSlot primitives.Slot,
|
||||
validate wrappedVal,
|
||||
handle subHandler,
|
||||
getSubnetsToSubscribe func(currentSlot primitives.Slot) []uint64,
|
||||
) bool {
|
||||
// subscribeToSubnets subscribes to needed subnets and unsubscribe from unneeded ones.
|
||||
// This functions mutates the `subscriptionBySubnet` map, which is used to keep track of the current subscriptions.
|
||||
func (s *Service) subscribeToSubnets(p subscribeToSubnetsParameters) error {
|
||||
// Do not subscribe if not synced.
|
||||
if s.chainStarted.IsSet() && s.cfg.initialSync.Syncing() {
|
||||
return true
|
||||
return nil
|
||||
}
|
||||
|
||||
// Check the validity of the digest.
|
||||
valid, err := isDigestValid(digest, genesisTime, genesisValidatorsRoot)
|
||||
valid, err := isDigestValid(p.digest, p.genesisTime, p.genesisValidatorsRoot)
|
||||
if err != nil {
|
||||
log.Error(err)
|
||||
return true
|
||||
return errors.Wrap(err, "is digest valid")
|
||||
}
|
||||
|
||||
// Unsubscribe from all subnets if the digest is not valid. It's likely to be the case after a hard fork.
|
||||
// Unsubscribe from all subnets if digest is not valid. It's likely to be the case after a hard fork.
|
||||
if !valid {
|
||||
description := topicFormat
|
||||
if pos := strings.LastIndex(topicFormat, "/"); pos != -1 {
|
||||
description = topicFormat[pos+1:]
|
||||
}
|
||||
|
||||
if pos := strings.LastIndex(description, "_"); pos != -1 {
|
||||
description = description[:pos]
|
||||
}
|
||||
|
||||
log.WithFields(logrus.Fields{
|
||||
"digest": fmt.Sprintf("%#x", digest),
|
||||
"subnets": description,
|
||||
}).Debug("Subnets with this digest are no longer valid, unsubscribing from all of them")
|
||||
s.pruneSubscriptions(subscriptions, []uint64{}, topicFormat, digest)
|
||||
return false
|
||||
wantedSubnets := map[uint64]bool{}
|
||||
s.pruneSubscriptions(p.subscriptionBySubnet, wantedSubnets, p.topicFormat, p.digest)
|
||||
return errInvalidDigest
|
||||
}
|
||||
|
||||
// Retrieve the subnets we want to subscribe to.
|
||||
subnetsToSubscribeIndex := getSubnetsToSubscribe(currentSlot)
|
||||
// Retrieve the subnets we want to join.
|
||||
subnetsToJoin := p.getSubnetsToJoin(p.currentSlot)
|
||||
|
||||
// Remove subscriptions that are no longer wanted.
|
||||
s.pruneSubscriptions(subscriptions, subnetsToSubscribeIndex, topicFormat, digest)
|
||||
s.pruneSubscriptions(p.subscriptionBySubnet, subnetsToJoin, p.topicFormat, p.digest)
|
||||
|
||||
// Subscribe to wanted subnets.
|
||||
for _, subnetIndex := range subnetsToSubscribeIndex {
|
||||
subnetTopic := fmt.Sprintf(topicFormat, digest, subnetIndex)
|
||||
// Subscribe to wanted and not already registered subnets.
|
||||
for subnet := range subnetsToJoin {
|
||||
subnetTopic := fmt.Sprintf(p.topicFormat, p.digest, subnet)
|
||||
|
||||
// Check if subscription exists.
|
||||
if _, exists := subscriptions[subnetIndex]; exists {
|
||||
continue
|
||||
if _, exists := p.subscriptionBySubnet[subnet]; !exists {
|
||||
subscription := s.subscribeWithBase(subnetTopic, p.validate, p.handle)
|
||||
p.subscriptionBySubnet[subnet] = subscription
|
||||
}
|
||||
|
||||
// We need to subscribe to the subnet.
|
||||
subscription := s.subscribeWithBase(subnetTopic, validate, handle)
|
||||
subscriptions[subnetIndex] = subscription
|
||||
}
|
||||
return true
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// subscribeWithParameters subscribes to a list of subnets.
|
||||
func (s *Service) subscribeWithParameters(
|
||||
topicFormat string,
|
||||
validate wrappedVal,
|
||||
handle subHandler,
|
||||
digest [4]byte,
|
||||
getSubnetsToSubscribe func(currentSlot primitives.Slot) []uint64,
|
||||
getSubnetsToFindPeersOnly func(currentSlot primitives.Slot) []uint64,
|
||||
) {
|
||||
// Initialize the subscriptions map.
|
||||
subscriptions := make(map[uint64]*pubsub.Subscription)
|
||||
|
||||
// Retrieve the genesis validators root.
|
||||
func (s *Service) subscribeWithParameters(p subscribeParameters) {
|
||||
minimumPeersPerSubnet := flags.Get().MinimumPeersPerSubnet
|
||||
subscriptionBySubnet := make(map[uint64]*pubsub.Subscription)
|
||||
genesisValidatorsRoot := s.cfg.clock.GenesisValidatorsRoot()
|
||||
|
||||
// Retrieve the epoch of the fork corresponding to the digest.
|
||||
_, epoch, err := forks.RetrieveForkDataFromDigest(digest, genesisValidatorsRoot[:])
|
||||
if err != nil {
|
||||
panic(err) // lint:nopanic -- Impossible condition.
|
||||
}
|
||||
|
||||
// Retrieve the base protobuf message.
|
||||
base := p2p.GossipTopicMappings(topicFormat, epoch)
|
||||
if base == nil {
|
||||
panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topicFormat)) // lint:nopanic -- Impossible condition.
|
||||
}
|
||||
|
||||
// Retrieve the genesis time.
|
||||
genesisTime := s.cfg.clock.GenesisTime()
|
||||
|
||||
// Define a ticker ticking every slot.
|
||||
secondsPerSlot := params.BeaconConfig().SecondsPerSlot
|
||||
ticker := slots.NewSlotTicker(genesisTime, secondsPerSlot)
|
||||
|
||||
// Retrieve the current slot.
|
||||
secondsPerSlotDuration := time.Duration(secondsPerSlot) * time.Second
|
||||
currentSlot := s.cfg.clock.CurrentSlot()
|
||||
neededSubnets := computeAllNeededSubnets(currentSlot, p.getSubnetsToJoin, p.getSubnetsRequiringPeers)
|
||||
|
||||
// Subscribe to subnets.
|
||||
s.subscribeToSubnets(topicFormat, digest, genesisValidatorsRoot, genesisTime, subscriptions, currentSlot, validate, handle, getSubnetsToSubscribe)
|
||||
shortTopicFormat := p.topicFormat
|
||||
shortTopicFormatLen := len(shortTopicFormat)
|
||||
if shortTopicFormatLen >= 3 && shortTopicFormat[shortTopicFormatLen-3:] == "_%d" {
|
||||
shortTopicFormat = shortTopicFormat[:shortTopicFormatLen-3]
|
||||
}
|
||||
|
||||
// Derive a new context and cancel function.
|
||||
ctx, cancel := context.WithCancel(s.ctx)
|
||||
shortTopic := fmt.Sprintf(shortTopicFormat, p.digest)
|
||||
|
||||
parameters := subscribeToSubnetsParameters{
|
||||
subscriptionBySubnet: subscriptionBySubnet,
|
||||
topicFormat: p.topicFormat,
|
||||
digest: p.digest,
|
||||
genesisValidatorsRoot: genesisValidatorsRoot,
|
||||
genesisTime: genesisTime,
|
||||
currentSlot: currentSlot,
|
||||
validate: p.validate,
|
||||
handle: p.handle,
|
||||
getSubnetsToJoin: p.getSubnetsToJoin,
|
||||
}
|
||||
|
||||
err := s.subscribeToSubnets(parameters)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not subscribe to subnets")
|
||||
}
|
||||
|
||||
// Subscribe to expected subnets and search for peers if needed at every slot.
|
||||
go func() {
|
||||
// Search for peers.
|
||||
s.searchForPeers(ctx, topicFormat, digest, currentSlot, getSubnetsToSubscribe, getSubnetsToFindPeersOnly)
|
||||
func() {
|
||||
ctx, cancel := context.WithTimeout(s.ctx, secondsPerSlotDuration)
|
||||
defer cancel()
|
||||
|
||||
if err := s.cfg.p2p.FindAndDialPeersWithSubnets(ctx, p.topicFormat, p.digest, minimumPeersPerSubnet, neededSubnets); err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
log.WithError(err).Debug("Could not find peers with subnets")
|
||||
}
|
||||
}()
|
||||
|
||||
slotTicker := slots.NewSlotTicker(genesisTime, secondsPerSlot)
|
||||
defer slotTicker.Done()
|
||||
|
||||
for {
|
||||
select {
|
||||
case currentSlot := <-ticker.C():
|
||||
isDigestValid := s.subscribeToSubnets(topicFormat, digest, genesisValidatorsRoot, genesisTime, subscriptions, currentSlot, validate, handle, getSubnetsToSubscribe)
|
||||
case currentSlot := <-slotTicker.C():
|
||||
parameters.currentSlot = currentSlot
|
||||
if err := s.subscribeToSubnets(parameters); err != nil {
|
||||
if errors.Is(err, errInvalidDigest) {
|
||||
log.WithField("topics", shortTopic).Debug("Digest is invalid, stopping subscription")
|
||||
return
|
||||
}
|
||||
|
||||
// Stop the ticker if the digest is not valid. Likely to happen after a hard fork.
|
||||
if !isDigestValid {
|
||||
ticker.Done()
|
||||
return
|
||||
log.WithError(err).Error("Could not subscribe to subnets")
|
||||
continue
|
||||
}
|
||||
|
||||
// Search for peers.
|
||||
s.searchForPeers(ctx, topicFormat, digest, currentSlot, getSubnetsToSubscribe, getSubnetsToFindPeersOnly)
|
||||
func() {
|
||||
ctx, cancel := context.WithTimeout(s.ctx, secondsPerSlotDuration)
|
||||
defer cancel()
|
||||
|
||||
if err := s.cfg.p2p.FindAndDialPeersWithSubnets(ctx, p.topicFormat, p.digest, minimumPeersPerSubnet, neededSubnets); err != nil && !errors.Is(err, context.DeadlineExceeded) {
|
||||
log.WithError(err).Debug("Could not find peers with subnets")
|
||||
}
|
||||
}()
|
||||
|
||||
case <-s.ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Warn the user if we are not subscribed to enough peers in the subnets.
|
||||
go func() {
|
||||
log := log.WithField("minimum", minimumPeersPerSubnet)
|
||||
logTicker := time.NewTicker(5 * time.Minute)
|
||||
defer logTicker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-logTicker.C:
|
||||
subnetsToFindPeersIndex := computeAllNeededSubnets(currentSlot, p.getSubnetsToJoin, p.getSubnetsRequiringPeers)
|
||||
|
||||
isSubnetWithMissingPeers := false
|
||||
// Find new peers for wanted subnets if needed.
|
||||
for index := range subnetsToFindPeersIndex {
|
||||
topic := fmt.Sprintf(p.topicFormat, p.digest, index)
|
||||
|
||||
// Check if we have enough peers in the subnet. Skip if we do.
|
||||
if count := s.connectedPeersCount(topic); count < minimumPeersPerSubnet {
|
||||
isSubnetWithMissingPeers = true
|
||||
log.WithFields(logrus.Fields{
|
||||
"topic": topic,
|
||||
"actual": count,
|
||||
}).Warning("Not enough connected peers")
|
||||
}
|
||||
}
|
||||
|
||||
if !isSubnetWithMissingPeers {
|
||||
log.WithField("topic", shortTopic).Info("All subnets have enough connected peers")
|
||||
}
|
||||
|
||||
case <-s.ctx.Done():
|
||||
cancel()
|
||||
ticker.Done()
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -591,76 +630,82 @@ func (s *Service) unSubscribeFromTopic(topic string) {
|
||||
}
|
||||
}
|
||||
|
||||
// enoughPeersAreConnected checks if we have enough peers which are subscribed to the same subnet.
|
||||
func (s *Service) enoughPeersAreConnected(subnetTopic string) bool {
|
||||
// connectedPeersCount counts how many peer for a given topic are connected to the node.
|
||||
func (s *Service) connectedPeersCount(subnetTopic string) int {
|
||||
topic := subnetTopic + s.cfg.p2p.Encoding().ProtocolSuffix()
|
||||
threshold := flags.Get().MinimumPeersPerSubnet
|
||||
|
||||
peersWithSubnet := s.cfg.p2p.PubSub().ListPeers(topic)
|
||||
peersWithSubnetCount := len(peersWithSubnet)
|
||||
|
||||
return peersWithSubnetCount >= threshold
|
||||
return len(peersWithSubnet)
|
||||
}
|
||||
|
||||
func (s *Service) dataColumnSubnetIndices(_ primitives.Slot) []uint64 {
|
||||
func (s *Service) dataColumnSubnetIndices(primitives.Slot) map[uint64]bool {
|
||||
nodeID := s.cfg.p2p.NodeID()
|
||||
custodyGroupCount := s.cfg.custodyInfo.CustodyGroupSamplingSize(peerdas.Target)
|
||||
|
||||
nodeInfo, _, err := peerdas.Info(nodeID, custodyGroupCount)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not retrieve peer info")
|
||||
return []uint64{}
|
||||
return nil
|
||||
}
|
||||
|
||||
return sliceFromMap(nodeInfo.DataColumnsSubnets, true /*sorted*/)
|
||||
return nodeInfo.DataColumnsSubnets
|
||||
}
|
||||
|
||||
func (s *Service) persistentAndAggregatorSubnetIndices(currentSlot primitives.Slot) []uint64 {
|
||||
func (s *Service) persistentAndAggregatorSubnetIndices(currentSlot primitives.Slot) map[uint64]bool {
|
||||
if flags.Get().SubscribeToAllSubnets {
|
||||
return sliceFromCount(params.BeaconConfig().AttestationSubnetCount)
|
||||
return mapFromCount(params.BeaconConfig().AttestationSubnetCount)
|
||||
}
|
||||
|
||||
persistentSubnetIndices := s.persistentSubnetIndices()
|
||||
aggregatorSubnetIndices := s.aggregatorSubnetIndices(currentSlot)
|
||||
persistentSubnetIndices := persistentSubnetIndices()
|
||||
aggregatorSubnetIndices := aggregatorSubnetIndices(currentSlot)
|
||||
|
||||
// Combine subscriptions to get all requested subscriptions.
|
||||
return slice.SetUint64(append(persistentSubnetIndices, aggregatorSubnetIndices...))
|
||||
return mapFromSlice(persistentSubnetIndices, aggregatorSubnetIndices)
|
||||
}
|
||||
|
||||
// filters out required peers for the node to function, not
|
||||
// pruning peers who are in our attestation subnets.
|
||||
func (s *Service) filterNeededPeers(pids []peer.ID) []peer.ID {
|
||||
minimumPeersPerSubnet := flags.Get().MinimumPeersPerSubnet
|
||||
currentSlot := s.cfg.clock.CurrentSlot()
|
||||
|
||||
// Exit early if nothing to filter.
|
||||
if len(pids) == 0 {
|
||||
return pids
|
||||
}
|
||||
|
||||
digest, err := s.currentForkDigest()
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not compute fork digest")
|
||||
return pids
|
||||
}
|
||||
currSlot := s.cfg.clock.CurrentSlot()
|
||||
wantedSubs := s.persistentAndAggregatorSubnetIndices(currSlot)
|
||||
wantedSubs = slice.SetUint64(append(wantedSubs, s.attesterSubnetIndices(currSlot)...))
|
||||
|
||||
wantedSubnets := make(map[uint64]bool)
|
||||
for subnet := range s.persistentAndAggregatorSubnetIndices(currentSlot) {
|
||||
wantedSubnets[subnet] = true
|
||||
}
|
||||
|
||||
for subnet := range attesterSubnetIndices(currentSlot) {
|
||||
wantedSubnets[subnet] = true
|
||||
}
|
||||
|
||||
topic := p2p.GossipTypeMapping[reflect.TypeOf(ðpb.Attestation{})]
|
||||
|
||||
// Map of peers in subnets
|
||||
peerMap := make(map[peer.ID]bool)
|
||||
|
||||
for _, sub := range wantedSubs {
|
||||
subnetTopic := fmt.Sprintf(topic, digest, sub) + s.cfg.p2p.Encoding().ProtocolSuffix()
|
||||
ps := s.cfg.p2p.PubSub().ListPeers(subnetTopic)
|
||||
if len(ps) > flags.Get().MinimumPeersPerSubnet {
|
||||
for subnet := range wantedSubnets {
|
||||
subnetTopic := fmt.Sprintf(topic, digest, subnet) + s.cfg.p2p.Encoding().ProtocolSuffix()
|
||||
peers := s.cfg.p2p.PubSub().ListPeers(subnetTopic)
|
||||
if len(peers) > minimumPeersPerSubnet {
|
||||
// In the event we have more than the minimum, we can
|
||||
// mark the remaining as viable for pruning.
|
||||
ps = ps[:flags.Get().MinimumPeersPerSubnet]
|
||||
peers = peers[:minimumPeersPerSubnet]
|
||||
}
|
||||
|
||||
// Add peer to peer map.
|
||||
for _, p := range ps {
|
||||
// Even if the peer id has
|
||||
// already been seen we still set
|
||||
// it, as the outcome is the same.
|
||||
peerMap[p] = true
|
||||
for _, peer := range peers {
|
||||
// Even if the peer ID has already been seen we still set it,
|
||||
// as the outcome is the same.
|
||||
peerMap[peer] = true
|
||||
}
|
||||
}
|
||||
|
||||
@@ -716,6 +761,34 @@ func isDigestValid(digest [4]byte, genesis time.Time, genValRoot [32]byte) (bool
|
||||
return retDigest == digest, nil
|
||||
}
|
||||
|
||||
// computeAllNeededSubnets computes the subnets we want to join
|
||||
// and the subnets for which we want to find peers.
|
||||
func computeAllNeededSubnets(
|
||||
currentSlot primitives.Slot,
|
||||
getSubnetsToJoin func(currentSlot primitives.Slot) map[uint64]bool,
|
||||
getSubnetsRequiringPeers func(currentSlot primitives.Slot) map[uint64]bool,
|
||||
) map[uint64]bool {
|
||||
// Retrieve the subnets we want to join.
|
||||
subnetsToJoin := getSubnetsToJoin(currentSlot)
|
||||
|
||||
// Retrieve the subnets we want to find peers into.
|
||||
subnetsRequiringPeers := make(map[uint64]bool)
|
||||
if getSubnetsRequiringPeers != nil {
|
||||
subnetsRequiringPeers = getSubnetsRequiringPeers(currentSlot)
|
||||
}
|
||||
|
||||
// Combine the two maps to get all needed subnets.
|
||||
neededSubnets := make(map[uint64]bool, len(subnetsToJoin)+len(subnetsRequiringPeers))
|
||||
for subnet := range subnetsToJoin {
|
||||
neededSubnets[subnet] = true
|
||||
}
|
||||
for subnet := range subnetsRequiringPeers {
|
||||
neededSubnets[subnet] = true
|
||||
}
|
||||
|
||||
return neededSubnets
|
||||
}
|
||||
|
||||
func agentString(pid peer.ID, hst host.Host) string {
|
||||
rawVersion, storeErr := hst.Peerstore().Get(pid, "AgentVersion")
|
||||
agString, ok := rawVersion.(string)
|
||||
@@ -742,17 +815,3 @@ func errorIsIgnored(err error) bool {
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// sliceFromMap returns a sorted list of keys from a map.
|
||||
func sliceFromMap(m map[uint64]bool, sorted ...bool) []uint64 {
|
||||
result := make([]uint64, 0, len(m))
|
||||
for k := range m {
|
||||
result = append(result, k)
|
||||
}
|
||||
|
||||
if len(sorted) > 0 && sorted[0] {
|
||||
slices.Sort(result)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
@@ -35,11 +35,11 @@ func (s *Service) committeeIndexBeaconAttestationSubscriber(_ context.Context, m
|
||||
}
|
||||
}
|
||||
|
||||
func (*Service) persistentSubnetIndices() []uint64 {
|
||||
func persistentSubnetIndices() []uint64 {
|
||||
return cache.SubnetIDs.GetAllSubnets()
|
||||
}
|
||||
|
||||
func (*Service) aggregatorSubnetIndices(currentSlot primitives.Slot) []uint64 {
|
||||
func aggregatorSubnetIndices(currentSlot primitives.Slot) []uint64 {
|
||||
endEpoch := slots.ToEpoch(currentSlot) + 1
|
||||
endSlot := params.BeaconConfig().SlotsPerEpoch.Mul(uint64(endEpoch))
|
||||
var commIds []uint64
|
||||
@@ -49,12 +49,16 @@ func (*Service) aggregatorSubnetIndices(currentSlot primitives.Slot) []uint64 {
|
||||
return slice.SetUint64(commIds)
|
||||
}
|
||||
|
||||
func (*Service) attesterSubnetIndices(currentSlot primitives.Slot) []uint64 {
|
||||
func attesterSubnetIndices(currentSlot primitives.Slot) map[uint64]bool {
|
||||
endEpoch := slots.ToEpoch(currentSlot) + 1
|
||||
endSlot := params.BeaconConfig().SlotsPerEpoch.Mul(uint64(endEpoch))
|
||||
var commIds []uint64
|
||||
|
||||
subnets := make(map[uint64]bool, int(endSlot-currentSlot+1))
|
||||
for i := currentSlot; i <= endSlot; i++ {
|
||||
commIds = append(commIds, cache.SubnetIDs.GetAttesterSubnetIDs(i)...)
|
||||
for _, subnetId := range cache.SubnetIDs.GetAttesterSubnetIDs(i) {
|
||||
subnets[subnetId] = true
|
||||
}
|
||||
}
|
||||
return slice.SetUint64(commIds)
|
||||
|
||||
return subnets
|
||||
}
|
||||
|
||||
@@ -310,7 +310,7 @@ func TestRevalidateSubscription_CorrectlyFormatsTopic(t *testing.T) {
|
||||
subscriptions[2], err = r.cfg.p2p.SubscribeToTopic(fullTopic)
|
||||
require.NoError(t, err)
|
||||
|
||||
r.pruneSubscriptions(subscriptions, []uint64{2}, defaultTopic, digest)
|
||||
r.pruneSubscriptions(subscriptions, map[uint64]bool{2: true}, defaultTopic, digest)
|
||||
require.LogsDoNotContain(t, hook, "Could not unregister topic validator")
|
||||
}
|
||||
|
||||
@@ -482,8 +482,7 @@ func TestFilterSubnetPeers(t *testing.T) {
|
||||
p2 := createPeer(t, subnet10, subnet20)
|
||||
p3 := createPeer(t)
|
||||
|
||||
// Connect to all
|
||||
// peers.
|
||||
// Connect to all peers.
|
||||
p.Connect(p1)
|
||||
p.Connect(p2)
|
||||
p.Connect(p3)
|
||||
@@ -540,7 +539,11 @@ func TestSubscribeWithSyncSubnets_DynamicOK(t *testing.T) {
|
||||
cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte("pubkey"), currEpoch, []uint64{0, 1}, 10*time.Second)
|
||||
digest, err := r.currentForkDigest()
|
||||
assert.NoError(t, err)
|
||||
r.subscribeWithParameters(p2p.SyncCommitteeSubnetTopicFormat, nil, nil, digest, r.activeSyncSubnetIndices, func(currentSlot primitives.Slot) []uint64 { return []uint64{} })
|
||||
r.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.SyncCommitteeSubnetTopicFormat,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: r.activeSyncSubnetIndices,
|
||||
})
|
||||
time.Sleep(2 * time.Second)
|
||||
assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics()))
|
||||
topicMap := map[string]bool{}
|
||||
@@ -589,7 +592,11 @@ func TestSubscribeWithSyncSubnets_DynamicSwitchFork(t *testing.T) {
|
||||
digest, err := signing.ComputeForkDigest(params.BeaconConfig().GenesisForkVersion, genRoot[:])
|
||||
assert.NoError(t, err)
|
||||
|
||||
r.subscribeWithParameters(p2p.SyncCommitteeSubnetTopicFormat, nil, nil, digest, r.activeSyncSubnetIndices, func(currentSlot primitives.Slot) []uint64 { return []uint64{} })
|
||||
r.subscribeWithParameters(subscribeParameters{
|
||||
topicFormat: p2p.SyncCommitteeSubnetTopicFormat,
|
||||
digest: digest,
|
||||
getSubnetsToJoin: r.activeSyncSubnetIndices,
|
||||
})
|
||||
time.Sleep(2 * time.Second)
|
||||
assert.Equal(t, 2, len(r.cfg.p2p.PubSub().GetTopics()))
|
||||
topicMap := map[string]bool{}
|
||||
|
||||
3
changelog/bastin_checkpoints-only-bootstraps.md
Normal file
3
changelog/bastin_checkpoints-only-bootstraps.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Changed
|
||||
|
||||
- Change LC Bootstrap logic to only save bootstraps on finalized checkpoints instead of every block.
|
||||
2
changelog/manu-subscriptions.md
Normal file
2
changelog/manu-subscriptions.md
Normal file
@@ -0,0 +1,2 @@
|
||||
### Fixed
|
||||
- Subnets subscription: Avoid dynamic subscribing blocking in case not enough peers per subnets are found.
|
||||
3
changelog/tt_focil.md
Normal file
3
changelog/tt_focil.md
Normal file
@@ -0,0 +1,3 @@
|
||||
### Fixed
|
||||
|
||||
- FOCIL
|
||||
@@ -127,6 +127,7 @@ type BeaconChainConfig struct {
|
||||
DomainApplicationMask [4]byte `yaml:"DOMAIN_APPLICATION_MASK" spec:"true"` // DomainApplicationMask defines the BLS signature domain for application mask.
|
||||
DomainApplicationBuilder [4]byte `yaml:"DOMAIN_APPLICATION_BUILDER" spec:"true"` // DomainApplicationBuilder defines the BLS signature domain for application builder.
|
||||
DomainBLSToExecutionChange [4]byte `yaml:"DOMAIN_BLS_TO_EXECUTION_CHANGE" spec:"true"` // DomainBLSToExecutionChange defines the BLS signature domain to change withdrawal addresses to ETH1 prefix
|
||||
DomainInclusionListCommittee [4]byte `yaml:"DOMAIN_INCLUSION_LIST_COMMITTEE" spec:"true"` // DomainInclusionListCommittee defines the BLS signature domain for inclusion list committee root.
|
||||
|
||||
// Prysm constants.
|
||||
GenesisValidatorsRoot [32]byte // GenesisValidatorsRoot is the root hash of the genesis validators.
|
||||
@@ -171,6 +172,8 @@ type BeaconChainConfig struct {
|
||||
ElectraForkEpoch primitives.Epoch `yaml:"ELECTRA_FORK_EPOCH" spec:"true"` // ElectraForkEpoch is used to represent the assigned fork epoch for electra.
|
||||
FuluForkVersion []byte `yaml:"FULU_FORK_VERSION" spec:"true"` // FuluForkVersion is used to represent the fork version for fulu.
|
||||
FuluForkEpoch primitives.Epoch `yaml:"FULU_FORK_EPOCH" spec:"true"` // FuluForkEpoch is used to represent the assigned fork epoch for fulu.
|
||||
Eip7805ForkVersion []byte `yaml:"EIP7805_FORK_VERSION" spec:"true"` // Eip7805ForkVersion is used to represent the fork version for fulu.
|
||||
Eip7805ForkEpoch primitives.Epoch `yaml:"EIP7805_FORK_EPOCH" spec:"true"` // Eip7805ForkEpoch is used to represent the assigned fork epoch for fulu.
|
||||
|
||||
ForkVersionSchedule map[[fieldparams.VersionLength]byte]primitives.Epoch // Schedule of fork epochs by version.
|
||||
ForkVersionNames map[[fieldparams.VersionLength]byte]string // Human-readable names of fork versions.
|
||||
@@ -314,6 +317,9 @@ type BeaconChainConfig struct {
|
||||
// DeprecatedMaxBlobsPerBlockFulu defines the max blobs that could exist in a block post Fulu hard fork.
|
||||
// Deprecated: This field is no longer supported. Avoid using it.
|
||||
DeprecatedMaxBlobsPerBlockFulu int `yaml:"MAX_BLOBS_PER_BLOCK_FULU" spec:"true"`
|
||||
|
||||
InclusionListCommitteeSize uint64
|
||||
InclusionListFreezeDeadLine uint64
|
||||
}
|
||||
|
||||
func (b *BeaconChainConfig) VersionToForkEpochMap() map[int]primitives.Epoch {
|
||||
@@ -379,6 +385,7 @@ func ConfigForkVersions(b *BeaconChainConfig) map[[fieldparams.VersionLength]byt
|
||||
bytesutil.ToBytes4(b.DenebForkVersion): version.Deneb,
|
||||
bytesutil.ToBytes4(b.ElectraForkVersion): version.Electra,
|
||||
bytesutil.ToBytes4(b.FuluForkVersion): version.Fulu,
|
||||
bytesutil.ToBytes4(b.Eip7805ForkVersion): version.Focil,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -427,6 +434,7 @@ func (b *BeaconChainConfig) TargetBlobsPerBlock(slot primitives.Slot) int {
|
||||
return b.DeprecatedMaxBlobsPerBlock / 2
|
||||
}
|
||||
|
||||
// MaxBlobsPerBlock returns the maximum number of blobs per block for the given slot.
|
||||
func (b *BeaconChainConfig) MaxBlobsPerBlock(slot primitives.Slot) int {
|
||||
epoch := primitives.Epoch(slot.DivSlot(b.SlotsPerEpoch))
|
||||
|
||||
@@ -449,6 +457,7 @@ func (b *BeaconChainConfig) MaxBlobsPerBlock(slot primitives.Slot) int {
|
||||
if epoch >= b.ElectraForkEpoch {
|
||||
return b.DeprecatedMaxBlobsPerBlockElectra
|
||||
}
|
||||
|
||||
return b.DeprecatedMaxBlobsPerBlock
|
||||
}
|
||||
|
||||
|
||||
@@ -34,8 +34,6 @@ var placeholderFields = []string{
|
||||
"EIP7441_FORK_VERSION",
|
||||
"EIP7732_FORK_EPOCH",
|
||||
"EIP7732_FORK_VERSION",
|
||||
"EIP7805_FORK_EPOCH",
|
||||
"EIP7805_FORK_VERSION",
|
||||
"EPOCHS_PER_SHUFFLING_PHASE",
|
||||
"MAX_BYTES_PER_INCLUSION_LIST",
|
||||
"MAX_REQUEST_BLOB_SIDECARS_FULU",
|
||||
|
||||
@@ -222,6 +222,8 @@ var mainnetBeaconConfig = &BeaconChainConfig{
|
||||
ElectraForkEpoch: mainnetElectraForkEpoch,
|
||||
FuluForkVersion: []byte{6, 0, 0, 0},
|
||||
FuluForkEpoch: mainnetFuluForkEpoch,
|
||||
Eip7805ForkEpoch: math.MaxUint64,
|
||||
Eip7805ForkVersion: []byte{10, 0, 0, 0},
|
||||
|
||||
// New values introduced in Altair hard fork 1.
|
||||
// Participation flag indices.
|
||||
@@ -339,8 +341,8 @@ var mainnetBeaconConfig = &BeaconChainConfig{
|
||||
AttestationSubnetPrefixBits: 6,
|
||||
SubnetsPerNode: 2,
|
||||
NodeIdBits: 256,
|
||||
|
||||
BlobSchedule: []BlobScheduleEntry{},
|
||||
InclusionListCommitteeSize: 16,
|
||||
InclusionListFreezeDeadLine: 8,
|
||||
}
|
||||
|
||||
// MainnetTestConfig provides a version of the mainnet config that has a different name
|
||||
|
||||
@@ -97,6 +97,8 @@ func MinimalSpecConfig() *BeaconChainConfig {
|
||||
minimalConfig.ElectraForkEpoch = math.MaxUint64
|
||||
minimalConfig.FuluForkVersion = []byte{6, 0, 0, 1}
|
||||
minimalConfig.FuluForkEpoch = math.MaxUint64
|
||||
minimalConfig.Eip7805ForkVersion = []byte{10, 0, 0, 1}
|
||||
minimalConfig.Eip7805ForkEpoch = math.MaxUint64
|
||||
|
||||
minimalConfig.SyncCommitteeSize = 32
|
||||
minimalConfig.InactivityScoreBias = 4
|
||||
|
||||
@@ -713,6 +713,14 @@ func (b *BeaconBlock) Version() int {
|
||||
return b.version
|
||||
}
|
||||
|
||||
func (b *BeaconBlock) NotSatisfyingInclusionList() bool {
|
||||
return b.notSatisfyingInclusionList
|
||||
}
|
||||
|
||||
func (b *BeaconBlock) MarkInclusionListNotSatisfied() {
|
||||
b.notSatisfyingInclusionList = true
|
||||
}
|
||||
|
||||
// HashTreeRoot returns the ssz root of the block.
|
||||
func (b *BeaconBlock) HashTreeRoot() ([field_params.RootLength]byte, error) {
|
||||
pb, err := b.Proto()
|
||||
|
||||
@@ -63,12 +63,13 @@ var _ interfaces.ReadOnlyBeaconBlockBody = &BeaconBlockBody{}
|
||||
|
||||
// BeaconBlock is the main beacon block structure. It can represent any block type.
|
||||
type BeaconBlock struct {
|
||||
version int
|
||||
slot primitives.Slot
|
||||
proposerIndex primitives.ValidatorIndex
|
||||
parentRoot [field_params.RootLength]byte
|
||||
stateRoot [field_params.RootLength]byte
|
||||
body *BeaconBlockBody
|
||||
version int
|
||||
slot primitives.Slot
|
||||
proposerIndex primitives.ValidatorIndex
|
||||
parentRoot [field_params.RootLength]byte
|
||||
stateRoot [field_params.RootLength]byte
|
||||
body *BeaconBlockBody
|
||||
notSatisfyingInclusionList bool
|
||||
}
|
||||
|
||||
// SignedBeaconBlock is the main signed beacon block structure. It can represent any block type.
|
||||
|
||||
@@ -47,6 +47,8 @@ type ReadOnlyBeaconBlock interface {
|
||||
ssz.HashRoot
|
||||
Version() int
|
||||
AsSignRequestObject() (validatorpb.SignRequestObject, error)
|
||||
NotSatisfyingInclusionList() bool
|
||||
MarkInclusionListNotSatisfied()
|
||||
}
|
||||
|
||||
// ReadOnlyBeaconBlockBody describes the method set employed by an object
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user