Validator custody: Update to the latest specification. (#15532)

* Validator custody: Update to the latest specfication.

* Update beacon-chain/blockchain/process_block.go

Co-authored-by: terence <terence@prysmaticlabs.com>

* Fix James' comment.

* Fix James' comment.

* Fix James' comment.

---------

Co-authored-by: terence <terence@prysmaticlabs.com>
Co-authored-by: james-prysm <90280386+james-prysm@users.noreply.github.com>
This commit is contained in:
Manu NALEPA
2025-08-02 08:21:08 +02:00
committed by GitHub
parent ae4b982a6c
commit 149e220b61
84 changed files with 2005 additions and 667 deletions

View File

@@ -73,6 +73,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
@@ -194,6 +195,7 @@ go_test(
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@com_github_holiman_uint256//:go_default_library",
"@com_github_libp2p_go_libp2p//core/peer:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",

View File

@@ -7,7 +7,6 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightclient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/execution"
@@ -235,14 +234,6 @@ func WithSyncChecker(checker Checker) Option {
}
}
// WithCustodyInfo sets the custody info for the blockchain service.
func WithCustodyInfo(custodyInfo *peerdas.CustodyInfo) Option {
return func(s *Service) error {
s.cfg.CustodyInfo = custodyInfo
return nil
}
}
// WithSlasherEnabled sets whether the slasher is enabled or not.
func WithSlasherEnabled(enabled bool) Option {
return func(s *Service) error {

View File

@@ -666,7 +666,9 @@ func (s *Service) areDataColumnsAvailable(
root [fieldparams.RootLength]byte,
block interfaces.ReadOnlyBeaconBlock,
) error {
// We are only required to check within MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS.
samplesPerSlot := params.BeaconConfig().SamplesPerSlot
// We are only required to check within MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS
blockSlot, currentSlot := block.Slot(), s.CurrentSlot()
blockEpoch, currentEpoch := slots.ToEpoch(blockSlot), slots.ToEpoch(currentSlot)
if !params.WithinDAPeriod(blockEpoch, currentEpoch) {
@@ -689,16 +691,20 @@ func (s *Service) areDataColumnsAvailable(
}
// All columns to sample need to be available for the block to be considered available.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.10/specs/fulu/das-core.md#custody-sampling
nodeID := s.cfg.P2P.NodeID()
// Prevent custody group count to change during the rest of the function.
s.cfg.CustodyInfo.Mut.RLock()
defer s.cfg.CustodyInfo.Mut.RUnlock()
// Get the custody group sampling size for the node.
custodyGroupSamplingSize := s.cfg.CustodyInfo.CustodyGroupSamplingSize(peerdas.Actual)
peerInfo, _, err := peerdas.Info(nodeID, custodyGroupSamplingSize)
custodyGroupCount, err := s.cfg.P2P.CustodyGroupCount()
if err != nil {
return errors.Wrap(err, "custody group count")
}
// Compute the sampling size.
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/das-core.md#custody-sampling
samplingSize := max(samplesPerSlot, custodyGroupCount)
// Get the peer info for the node.
peerInfo, _, err := peerdas.Info(nodeID, samplingSize)
if err != nil {
return errors.Wrap(err, "peer info")
}

View File

@@ -2894,7 +2894,6 @@ func TestIsDataAvailable(t *testing.T) {
}
params := testIsAvailableParams{
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{})},
columnsToSave: indices,
blobKzgCommitmentsCount: 3,
}
@@ -2907,7 +2906,6 @@ func TestIsDataAvailable(t *testing.T) {
t.Run("Fulu - no missing data columns", func(t *testing.T) {
params := testIsAvailableParams{
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{})},
columnsToSave: []uint64{1, 17, 19, 42, 75, 87, 102, 117, 119}, // 119 is not needed
blobKzgCommitmentsCount: 3,
}
@@ -2922,7 +2920,7 @@ func TestIsDataAvailable(t *testing.T) {
startWaiting := make(chan bool)
testParams := testIsAvailableParams{
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{}), WithStartWaitingDataColumnSidecars(startWaiting)},
options: []Option{WithStartWaitingDataColumnSidecars(startWaiting)},
columnsToSave: []uint64{1, 17, 19, 75, 102, 117, 119}, // 119 is not needed, 42 and 87 are missing
blobKzgCommitmentsCount: 3,
@@ -2959,6 +2957,9 @@ func TestIsDataAvailable(t *testing.T) {
require.NoError(t, err)
}()
ctx, cancel := context.WithTimeout(ctx, time.Second*2)
defer cancel()
err = service.isDataAvailable(ctx, root, signed)
require.NoError(t, err)
})
@@ -2971,10 +2972,6 @@ func TestIsDataAvailable(t *testing.T) {
startWaiting := make(chan bool)
var custodyInfo peerdas.CustodyInfo
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(cgc)
custodyInfo.ToAdvertiseGroupCount.Set(cgc)
minimumColumnsCountToReconstruct := peerdas.MinimumColumnsCountToReconstruct()
indices := make([]uint64, 0, minimumColumnsCountToReconstruct-missingColumns)
@@ -2983,12 +2980,14 @@ func TestIsDataAvailable(t *testing.T) {
}
testParams := testIsAvailableParams{
options: []Option{WithCustodyInfo(&custodyInfo), WithStartWaitingDataColumnSidecars(startWaiting)},
options: []Option{WithStartWaitingDataColumnSidecars(startWaiting)},
columnsToSave: indices,
blobKzgCommitmentsCount: 3,
}
ctx, _, service, root, signed := testIsAvailableSetup(t, testParams)
_, _, err := service.cfg.P2P.UpdateCustodyInfo(0, cgc)
require.NoError(t, err)
block := signed.Block()
slot := block.Slot()
proposerIndex := block.ProposerIndex()
@@ -3020,6 +3019,9 @@ func TestIsDataAvailable(t *testing.T) {
require.NoError(t, err)
}()
ctx, cancel := context.WithTimeout(ctx, time.Second*2)
defer cancel()
err = service.isDataAvailable(ctx, root, signed)
require.NoError(t, err)
})
@@ -3028,7 +3030,7 @@ func TestIsDataAvailable(t *testing.T) {
startWaiting := make(chan bool)
params := testIsAvailableParams{
options: []Option{WithCustodyInfo(&peerdas.CustodyInfo{}), WithStartWaitingDataColumnSidecars(startWaiting)},
options: []Option{WithStartWaitingDataColumnSidecars(startWaiting)},
blobKzgCommitmentsCount: 3,
}

View File

@@ -16,7 +16,6 @@ import (
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/core/peerdas"
coreTime "github.com/OffchainLabs/prysm/v6/beacon-chain/core/time"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
@@ -31,6 +30,7 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state/stategen"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
@@ -97,7 +97,6 @@ type config struct {
FinalizedStateAtStartUp state.BeaconState
ExecutionEngineCaller execution.EngineCaller
SyncChecker Checker
CustodyInfo *peerdas.CustodyInfo
}
// Checker is an interface used to determine if a node is in initial sync
@@ -296,6 +295,20 @@ func (s *Service) StartFromSavedState(saved state.BeaconState) error {
if err := s.clockSetter.SetClock(startup.NewClock(s.genesisTime, vr)); err != nil {
return errors.Wrap(err, "failed to initialize blockchain service")
}
if !params.FuluEnabled() {
return nil
}
earliestAvailableSlot, custodySubnetCount, err := s.updateCustodyInfoInDB(saved.Slot())
if err != nil {
return errors.Wrap(err, "could not get and save custody group count")
}
if _, _, err := s.cfg.P2P.UpdateCustodyInfo(earliestAvailableSlot, custodySubnetCount); err != nil {
return errors.Wrap(err, "update custody info")
}
return nil
}
@@ -516,6 +529,57 @@ func (s *Service) removeStartupState() {
s.cfg.FinalizedStateAtStartUp = nil
}
// UpdateCustodyInfoInDB updates the custody information in the database.
// It returns the (potentially updated) custody group count and the earliest available slot.
func (s *Service) updateCustodyInfoInDB(slot primitives.Slot) (primitives.Slot, uint64, error) {
isSubscribedToAllDataSubnets := flags.Get().SubscribeAllDataSubnets
beaconConfig := params.BeaconConfig()
custodyRequirement := beaconConfig.CustodyRequirement
// Check if the node was previously subscribed to all data subnets, and if so,
// store the new status accordingly.
wasSubscribedToAllDataSubnets, err := s.cfg.BeaconDB.UpdateSubscribedToAllDataSubnets(s.ctx, isSubscribedToAllDataSubnets)
if err != nil {
log.WithError(err).Error("Could not update subscription status to all data subnets")
}
// Warn the user if the node was previously subscribed to all data subnets and is not any more.
if wasSubscribedToAllDataSubnets && !isSubscribedToAllDataSubnets {
log.Warnf(
"Because the flag `--%s` was previously used, the node will still subscribe to all data subnets.",
flags.SubscribeAllDataSubnets.Name,
)
}
// Compute the custody group count.
custodyGroupCount := custodyRequirement
if isSubscribedToAllDataSubnets {
custodyGroupCount = beaconConfig.NumberOfColumns
}
// Safely compute the fulu fork slot.
fuluForkSlot, err := fuluForkSlot()
if err != nil {
return 0, 0, errors.Wrap(err, "fulu fork slot")
}
// If slot is before the fulu fork slot, then use the earliest stored slot as the reference slot.
if slot < fuluForkSlot {
slot, err = s.cfg.BeaconDB.EarliestSlot(s.ctx)
if err != nil {
return 0, 0, errors.Wrap(err, "earliest slot")
}
}
earliestAvailableSlot, custodyGroupCount, err := s.cfg.BeaconDB.UpdateCustodyInfo(s.ctx, slot, custodyGroupCount)
if err != nil {
return 0, 0, errors.Wrap(err, "update custody info")
}
return earliestAvailableSlot, custodyGroupCount, nil
}
func spawnCountdownIfPreGenesis(ctx context.Context, genesisTime time.Time, db db.HeadAccessDatabase) {
currentTime := prysmTime.Now()
if currentTime.After(genesisTime) {
@@ -532,3 +596,19 @@ func spawnCountdownIfPreGenesis(ctx context.Context, genesisTime time.Time, db d
}
go slots.CountdownToGenesis(ctx, genesisTime, uint64(gState.NumValidators()), gRoot)
}
func fuluForkSlot() (primitives.Slot, error) {
beaconConfig := params.BeaconConfig()
fuluForkEpoch := beaconConfig.FuluForkEpoch
if fuluForkEpoch == beaconConfig.FarFutureEpoch {
return beaconConfig.FarFutureSlot, nil
}
forkFuluSlot, err := slots.EpochStart(fuluForkEpoch)
if err != nil {
return 0, errors.Wrap(err, "epoch start")
}
return forkFuluSlot, nil
}

View File

@@ -30,6 +30,7 @@ import (
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/libp2p/go-libp2p/core/peer"
"google.golang.org/protobuf/proto"
)
@@ -54,6 +55,7 @@ type mockBroadcaster struct {
type mockAccessor struct {
mockBroadcaster
mockCustodyManager
p2pTesting.MockPeerManager
}
@@ -97,6 +99,43 @@ func (mb *mockBroadcaster) BroadcastBLSChanges(_ context.Context, _ []*ethpb.Sig
var _ p2p.Broadcaster = (*mockBroadcaster)(nil)
// mockCustodyManager is a mock implementation of p2p.CustodyManager
type mockCustodyManager struct {
mut sync.RWMutex
earliestAvailableSlot primitives.Slot
custodyGroupCount uint64
}
func (dch *mockCustodyManager) EarliestAvailableSlot() (primitives.Slot, error) {
dch.mut.RLock()
defer dch.mut.RUnlock()
return dch.earliestAvailableSlot, nil
}
func (dch *mockCustodyManager) CustodyGroupCount() (uint64, error) {
dch.mut.RLock()
defer dch.mut.RUnlock()
return dch.custodyGroupCount, nil
}
func (dch *mockCustodyManager) UpdateCustodyInfo(earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error) {
dch.mut.Lock()
defer dch.mut.Unlock()
dch.earliestAvailableSlot = earliestAvailableSlot
dch.custodyGroupCount = custodyGroupCount
return earliestAvailableSlot, custodyGroupCount, nil
}
func (dch *mockCustodyManager) CustodyGroupCountFromPeer(peer.ID) uint64 {
return 0
}
var _ p2p.CustodyManager = (*mockCustodyManager)(nil)
type testServiceRequirements struct {
ctx context.Context
db db.Database

View File

@@ -102,13 +102,13 @@ func ProcessWithdrawalRequests(ctx context.Context, st state.BeaconState, wrs []
return nil, err
} else if n == params.BeaconConfig().PendingPartialWithdrawalsLimit && !isFullExitRequest {
// if the PendingPartialWithdrawalsLimit is met, the user would have paid for a partial withdrawal that's not included
log.Debugln("Skipping execution layer withdrawal request, PendingPartialWithdrawalsLimit reached")
log.Debug("Skipping execution layer withdrawal request, PendingPartialWithdrawalsLimit reached")
continue
}
vIdx, exists := st.ValidatorIndexByPubkey(bytesutil.ToBytes48(wr.ValidatorPubkey))
if !exists {
log.Debugf("Skipping execution layer withdrawal request, validator index for %s not found\n", hexutil.Encode(wr.ValidatorPubkey))
log.WithField("validator", hexutil.Encode(wr.ValidatorPubkey)).Debug("Skipping execution layer withdrawal request, validator index not found")
continue
}
validator, err := st.ValidatorAtIndexReadOnly(vIdx)
@@ -120,23 +120,23 @@ func ProcessWithdrawalRequests(ctx context.Context, st state.BeaconState, wrs []
wc := validator.GetWithdrawalCredentials()
isCorrectSourceAddress := bytes.Equal(wc[12:], wr.SourceAddress)
if !hasCorrectCredential || !isCorrectSourceAddress {
log.Debugln("Skipping execution layer withdrawal request, wrong withdrawal credentials")
log.Debug("Skipping execution layer withdrawal request, wrong withdrawal credentials")
continue
}
// Verify the validator is active.
if !helpers.IsActiveValidatorUsingTrie(validator, currentEpoch) {
log.Debugln("Skipping execution layer withdrawal request, validator not active")
log.Debug("Skipping execution layer withdrawal request, validator not active")
continue
}
// Verify the validator has not yet submitted an exit.
if validator.ExitEpoch() != params.BeaconConfig().FarFutureEpoch {
log.Debugln("Skipping execution layer withdrawal request, validator has submitted an exit already")
log.Debug("Skipping execution layer withdrawal request, validator has submitted an exit already")
continue
}
// Verify the validator has been active long enough.
if currentEpoch < validator.ActivationEpoch().AddEpoch(params.BeaconConfig().ShardCommitteePeriod) {
log.Debugln("Skipping execution layer withdrawal request, validator has not been active long enough")
log.Debug("Skipping execution layer withdrawal request, validator has not been active long enough")
continue
}

View File

@@ -15,7 +15,7 @@ import (
)
// UpgradeToFulu updates inputs a generic state to return the version Fulu state.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/fork.md#upgrading-the-state
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/fork.md#upgrading-the-state
func UpgradeToFulu(ctx context.Context, beaconState state.BeaconState) (state.BeaconState, error) {
currentSyncCommittee, err := beaconState.CurrentSyncCommittee()
if err != nil {

View File

@@ -16,7 +16,6 @@ go_library(
deps = [
"//beacon-chain/blockchain/kzg:go_default_library",
"//beacon-chain/state:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
@@ -53,7 +52,6 @@ go_test(
":go_default_library",
"//beacon-chain/blockchain/kzg:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",

View File

@@ -31,15 +31,8 @@ var (
maxUint256 = &uint256.Int{math.MaxUint64, math.MaxUint64, math.MaxUint64, math.MaxUint64}
)
type CustodyType int
const (
Target CustodyType = iota
Actual
)
// CustodyGroups computes the custody groups the node should participate in for custody.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/das-core.md#get_custody_groups
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/das-core.md#get_custody_groups
func CustodyGroups(nodeId enode.ID, custodyGroupCount uint64) ([]uint64, error) {
numberOfCustodyGroups := params.BeaconConfig().NumberOfCustodyGroups
@@ -102,7 +95,7 @@ func CustodyGroups(nodeId enode.ID, custodyGroupCount uint64) ([]uint64, error)
}
// ComputeColumnsForCustodyGroup computes the columns for a given custody group.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/das-core.md#compute_columns_for_custody_group
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/das-core.md#compute_columns_for_custody_group
func ComputeColumnsForCustodyGroup(custodyGroup uint64) ([]uint64, error) {
beaconConfig := params.BeaconConfig()
numberOfCustodyGroups := beaconConfig.NumberOfCustodyGroups
@@ -127,7 +120,7 @@ func ComputeColumnsForCustodyGroup(custodyGroup uint64) ([]uint64, error) {
// DataColumnSidecars computes the data column sidecars from the signed block, cells and cell proofs.
// The returned value contains pointers to function parameters.
// (If the caller alterates `cellsAndProofs` afterwards, the returned value will be modified as well.)
// https://github.com/ethereum/consensus-specs/blob/v1.6.0-alpha.3/specs/fulu/validator.md#get_data_column_sidecars_from_block
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/validator.md#get_data_column_sidecars_from_block
func DataColumnSidecars(signedBlock interfaces.ReadOnlySignedBeaconBlock, cellsAndProofs []kzg.CellsAndProofs) ([]*ethpb.DataColumnSidecar, error) {
if signedBlock == nil || signedBlock.IsNil() || len(cellsAndProofs) == 0 {
return nil, nil
@@ -176,19 +169,6 @@ func ComputeCustodyGroupForColumn(columnIndex uint64) (uint64, error) {
return columnIndex % numberOfCustodyGroups, nil
}
// CustodyGroupSamplingSize returns the number of custody groups the node should sample from.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/das-core.md#custody-sampling
func (custodyInfo *CustodyInfo) CustodyGroupSamplingSize(ct CustodyType) uint64 {
custodyGroupCount := custodyInfo.TargetGroupCount.Get()
if ct == Actual {
custodyGroupCount = custodyInfo.ActualGroupCount()
}
samplesPerSlot := params.BeaconConfig().SamplesPerSlot
return max(samplesPerSlot, custodyGroupCount)
}
// CustodyColumns computes the custody columns from the custody groups.
func CustodyColumns(custodyGroups []uint64) (map[uint64]bool, error) {
numberOfCustodyGroups := params.BeaconConfig().NumberOfCustodyGroups
@@ -219,7 +199,7 @@ func CustodyColumns(custodyGroups []uint64) (map[uint64]bool, error) {
// the KZG commitment includion proofs and cells and cell proofs.
// The returned value contains pointers to function parameters.
// (If the caller alterates input parameters afterwards, the returned value will be modified as well.)
// https://github.com/ethereum/consensus-specs/blob/v1.6.0-alpha.3/specs/fulu/validator.md#get_data_column_sidecars
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/validator.md#get_data_column_sidecars
func dataColumnsSidecars(
signedBlockHeader *ethpb.SignedBeaconBlockHeader,
blobKzgCommitments [][]byte,

View File

@@ -104,62 +104,6 @@ func TestComputeCustodyGroupForColumn(t *testing.T) {
})
}
func TestCustodyGroupSamplingSize(t *testing.T) {
testCases := []struct {
name string
custodyType peerdas.CustodyType
validatorsCustodyRequirement uint64
toAdvertiseCustodyGroupCount uint64
expected uint64
}{
{
name: "target, lower than samples per slot",
custodyType: peerdas.Target,
validatorsCustodyRequirement: 2,
expected: 8,
},
{
name: "target, higher than samples per slot",
custodyType: peerdas.Target,
validatorsCustodyRequirement: 100,
expected: 100,
},
{
name: "actual, lower than samples per slot",
custodyType: peerdas.Actual,
validatorsCustodyRequirement: 3,
toAdvertiseCustodyGroupCount: 4,
expected: 8,
},
{
name: "actual, higher than samples per slot",
custodyType: peerdas.Actual,
validatorsCustodyRequirement: 100,
toAdvertiseCustodyGroupCount: 101,
expected: 100,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Create a custody info.
custodyInfo := peerdas.CustodyInfo{}
// Set the validators custody requirement for target custody group count.
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(tc.validatorsCustodyRequirement)
// Set the to advertise custody group count.
custodyInfo.ToAdvertiseGroupCount.Set(tc.toAdvertiseCustodyGroupCount)
// Compute the custody group sampling size.
actual := custodyInfo.CustodyGroupSamplingSize(tc.custodyType)
// Check the result.
require.Equal(t, tc.expected, actual)
})
}
}
func TestCustodyColumns(t *testing.T) {
t.Run("group too large", func(t *testing.T) {
_, err := peerdas.CustodyColumns([]uint64{1_000_000})

View File

@@ -4,45 +4,17 @@ import (
"encoding/binary"
"sync"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/ethereum/go-ethereum/p2p/enode"
lru "github.com/hashicorp/golang-lru"
"github.com/pkg/errors"
)
// info contains all useful peerDAS related information regarding a peer.
type (
info struct {
CustodyGroups map[uint64]bool
CustodyColumns map[uint64]bool
DataColumnsSubnets map[uint64]bool
}
targetCustodyGroupCount struct {
mut sync.RWMutex
validatorsCustodyRequirement uint64
}
toAdverstiseCustodyGroupCount struct {
mut sync.RWMutex
value uint64
}
CustodyInfo struct {
// Mut is a mutex to be used by caller to ensure neither
// TargetCustodyGroupCount nor ToAdvertiseCustodyGroupCount are being modified.
// (This is not necessary to use this mutex for any data protection.)
Mut sync.RWMutex
// TargetGroupCount represents the target number of custody groups we should custody
// regarding the validators we are tracking.
TargetGroupCount targetCustodyGroupCount
// ToAdvertiseGroupCount represents the number of custody groups to advertise to the network.
ToAdvertiseGroupCount toAdverstiseCustodyGroupCount
}
)
type info struct {
CustodyGroups map[uint64]bool
CustodyColumns map[uint64]bool
DataColumnsSubnets map[uint64]bool
}
const (
nodeInfoCacheSize = 200
@@ -109,61 +81,6 @@ func Info(nodeID enode.ID, custodyGroupCount uint64) (*info, bool, error) {
return result, false, nil
}
// ActualGroupCount returns the actual custody group count.
func (custodyInfo *CustodyInfo) ActualGroupCount() uint64 {
return min(custodyInfo.TargetGroupCount.Get(), custodyInfo.ToAdvertiseGroupCount.Get())
}
// CustodyGroupCount returns the number of groups we should participate in for custody.
func (tcgc *targetCustodyGroupCount) Get() uint64 {
// If subscribed to all subnets, return the number of custody groups.
if flags.Get().SubscribeAllDataSubnets {
return params.BeaconConfig().NumberOfCustodyGroups
}
tcgc.mut.RLock()
defer tcgc.mut.RUnlock()
// If no validators are tracked, return the default custody requirement.
if tcgc.validatorsCustodyRequirement == 0 {
return params.BeaconConfig().CustodyRequirement
}
// Return the validators custody requirement.
return tcgc.validatorsCustodyRequirement
}
// setValidatorsCustodyRequirement sets the validators custody requirement.
func (tcgc *targetCustodyGroupCount) SetValidatorsCustodyRequirement(value uint64) {
tcgc.mut.Lock()
defer tcgc.mut.Unlock()
tcgc.validatorsCustodyRequirement = value
}
// Get returns the to advertise custody group count.
func (tacgc *toAdverstiseCustodyGroupCount) Get() uint64 {
// If subscribed to all subnets, return the number of custody groups.
if flags.Get().SubscribeAllDataSubnets {
return params.BeaconConfig().NumberOfCustodyGroups
}
custodyRequirement := params.BeaconConfig().CustodyRequirement
tacgc.mut.RLock()
defer tacgc.mut.RUnlock()
return max(tacgc.value, custodyRequirement)
}
// Set sets the to advertise custody group count.
func (tacgc *toAdverstiseCustodyGroupCount) Set(value uint64) {
tacgc.mut.Lock()
defer tacgc.mut.Unlock()
tacgc.value = value
}
// createInfoCacheIfNeeded creates a new cache if it doesn't exist.
func createInfoCacheIfNeeded() error {
nodeInfoCacheMut.Lock()

View File

@@ -4,7 +4,6 @@ import (
"testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v6/testing/require"
"github.com/ethereum/go-ethereum/p2p/enode"
)
@@ -26,108 +25,3 @@ func TestInfo(t *testing.T) {
require.DeepEqual(t, expectedDataColumnsSubnets, actual.DataColumnsSubnets)
}
}
func TestTargetCustodyGroupCount(t *testing.T) {
testCases := []struct {
name string
subscribeToAllColumns bool
validatorsCustodyRequirement uint64
expected uint64
}{
{
name: "subscribed to all data subnets",
subscribeToAllColumns: true,
validatorsCustodyRequirement: 100,
expected: 128,
},
{
name: "no validators attached",
subscribeToAllColumns: false,
validatorsCustodyRequirement: 0,
expected: 4,
},
{
name: "some validators attached",
subscribeToAllColumns: false,
validatorsCustodyRequirement: 100,
expected: 100,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Subscribe to all subnets if needed.
if tc.subscribeToAllColumns {
resetFlags := flags.Get()
gFlags := new(flags.GlobalFlags)
gFlags.SubscribeAllDataSubnets = true
flags.Init(gFlags)
defer flags.Init(resetFlags)
}
var custodyInfo peerdas.CustodyInfo
// Set the validators custody requirement.
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(tc.validatorsCustodyRequirement)
// Get the target custody group count.
actual := custodyInfo.TargetGroupCount.Get()
// Compare the expected and actual values.
require.Equal(t, tc.expected, actual)
})
}
}
func TestToAdvertiseCustodyGroupCount(t *testing.T) {
testCases := []struct {
name string
subscribeToAllColumns bool
toAdvertiseCustodyGroupCount uint64
expected uint64
}{
{
name: "subscribed to all subnets",
subscribeToAllColumns: true,
toAdvertiseCustodyGroupCount: 100,
expected: 128,
},
{
name: "higher than custody requirement",
subscribeToAllColumns: false,
toAdvertiseCustodyGroupCount: 100,
expected: 100,
},
{
name: "lower than custody requirement",
subscribeToAllColumns: false,
toAdvertiseCustodyGroupCount: 1,
expected: 4,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
// Subscribe to all subnets if needed.
if tc.subscribeToAllColumns {
resetFlags := flags.Get()
gFlags := new(flags.GlobalFlags)
gFlags.SubscribeAllDataSubnets = true
flags.Init(gFlags)
defer flags.Init(resetFlags)
}
// Create a custody info.
var custodyInfo peerdas.CustodyInfo
// Set the to advertise custody group count.
custodyInfo.ToAdvertiseGroupCount.Set(tc.toAdvertiseCustodyGroupCount)
// Get the to advertise custody group count.
actual := custodyInfo.ToAdvertiseGroupCount.Get()
// Compare the expected and actual values.
require.Equal(t, tc.expected, actual)
})
}
}

View File

@@ -10,10 +10,7 @@ import (
"github.com/pkg/errors"
)
const (
CustodyGroupCountEnrKey = "cgc"
kzgPosition = 11 // The index of the KZG commitment list in the Body
)
const kzgPosition = 11 // The index of the KZG commitment list in the Body
var (
ErrIndexTooLarge = errors.New("column index is larger than the specified columns count")
@@ -27,13 +24,13 @@ var (
ErrCannotLoadCustodyGroupCount = errors.New("cannot load the custody group count from peer")
)
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/p2p-interface.md#custody-group-count
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#custody-group-count
type Cgc uint64
func (Cgc) ENRKey() string { return CustodyGroupCountEnrKey }
func (Cgc) ENRKey() string { return params.BeaconNetworkConfig().CustodyGroupCountKey }
// VerifyDataColumnSidecar verifies if the data column sidecar is valid.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/p2p-interface.md#verify_data_column_sidecar
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#verify_data_column_sidecar
func VerifyDataColumnSidecar(sidecar blocks.RODataColumn) error {
// The sidecar index must be within the valid range.
numberOfColumns := params.BeaconConfig().NumberOfColumns
@@ -60,7 +57,7 @@ func VerifyDataColumnSidecar(sidecar blocks.RODataColumn) error {
// while we are verifying all the KZG proofs from multiple sidecars in a batch.
// This is done to improve performance since the internal KZG library is way more
// efficient when verifying in batch.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/p2p-interface.md#verify_data_column_sidecar_kzg_proofs
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#verify_data_column_sidecar_kzg_proofs
func VerifyDataColumnsSidecarKZGProofs(sidecars []blocks.RODataColumn) error {
// Compute the total count.
count := 0
@@ -96,7 +93,7 @@ func VerifyDataColumnsSidecarKZGProofs(sidecars []blocks.RODataColumn) error {
}
// VerifyDataColumnSidecarInclusionProof verifies if the given KZG commitments included in the given beacon block.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/p2p-interface.md#verify_data_column_sidecar_inclusion_proof
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#verify_data_column_sidecar_inclusion_proof
func VerifyDataColumnSidecarInclusionProof(sidecar blocks.RODataColumn) error {
if sidecar.SignedBlockHeader == nil || sidecar.SignedBlockHeader.Header == nil {
return ErrNilBlockHeader
@@ -128,7 +125,7 @@ func VerifyDataColumnSidecarInclusionProof(sidecar blocks.RODataColumn) error {
}
// ComputeSubnetForDataColumnSidecar computes the subnet for a data column sidecar.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/p2p-interface.md#compute_subnet_for_data_column_sidecar
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#compute_subnet_for_data_column_sidecar
func ComputeSubnetForDataColumnSidecar(columnIndex uint64) uint64 {
dataColumnSidecarSubnetCount := params.BeaconConfig().DataColumnSidecarSubnetCount
return columnIndex % dataColumnSidecarSubnetCount

View File

@@ -8,7 +8,7 @@ import (
)
// ValidatorsCustodyRequirement returns the number of custody groups regarding the validator indices attached to the beacon node.
// https://github.com/ethereum/consensus-specs/blob/v1.5.0-beta.5/specs/fulu/validator.md#validator-custody
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/validator.md#validator-custody
func ValidatorsCustodyRequirement(state beaconState.ReadOnlyBeaconState, validatorsIndex map[primitives.ValidatorIndex]bool) (uint64, error) {
totalNodeBalance := uint64(0)
for index := range validatorsIndex {

View File

@@ -39,10 +39,8 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/verification:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",

View File

@@ -22,8 +22,8 @@ type LazilyPersistentStoreColumn struct {
store *filesystem.DataColumnStorage
nodeID enode.ID
cache *dataColumnCache
custodyInfo *peerdas.CustodyInfo
newDataColumnsVerifier verification.NewDataColumnsVerifier
custodyGroupCount uint64
}
var _ AvailabilityStore = &LazilyPersistentStoreColumn{}
@@ -38,13 +38,18 @@ type DataColumnsVerifier interface {
// NewLazilyPersistentStoreColumn creates a new LazilyPersistentStoreColumn.
// WARNING: The resulting LazilyPersistentStoreColumn is NOT thread-safe.
func NewLazilyPersistentStoreColumn(store *filesystem.DataColumnStorage, nodeID enode.ID, newDataColumnsVerifier verification.NewDataColumnsVerifier, custodyInfo *peerdas.CustodyInfo) *LazilyPersistentStoreColumn {
func NewLazilyPersistentStoreColumn(
store *filesystem.DataColumnStorage,
nodeID enode.ID,
newDataColumnsVerifier verification.NewDataColumnsVerifier,
custodyGroupCount uint64,
) *LazilyPersistentStoreColumn {
return &LazilyPersistentStoreColumn{
store: store,
nodeID: nodeID,
cache: newDataColumnCache(),
custodyInfo: custodyInfo,
newDataColumnsVerifier: newDataColumnsVerifier,
custodyGroupCount: custodyGroupCount,
}
}
@@ -155,6 +160,8 @@ func (s *LazilyPersistentStoreColumn) IsDataAvailable(ctx context.Context, curre
// fullCommitmentsToCheck returns the commitments to check for a given block.
func (s *LazilyPersistentStoreColumn) fullCommitmentsToCheck(nodeID enode.ID, block blocks.ROBlock, currentSlot primitives.Slot) (*safeCommitmentsArray, error) {
samplesPerSlot := params.BeaconConfig().SamplesPerSlot
// Return early for blocks that are pre-Fulu.
if block.Version() < version.Fulu {
return &safeCommitmentsArray{}, nil
@@ -183,11 +190,9 @@ func (s *LazilyPersistentStoreColumn) fullCommitmentsToCheck(nodeID enode.ID, bl
return &safeCommitmentsArray{}, nil
}
// Retrieve the groups count.
custodyGroupCount := s.custodyInfo.ActualGroupCount()
// Retrieve peer info.
peerInfo, _, err := peerdas.Info(nodeID, custodyGroupCount)
samplingSize := max(s.custodyGroupCount, samplesPerSlot)
peerInfo, _, err := peerdas.Info(nodeID, samplingSize)
if err != nil {
return nil, errors.Wrap(err, "peer info")
}

View File

@@ -4,10 +4,8 @@ import (
"context"
"testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/verification"
"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/blocks"
@@ -29,7 +27,7 @@ var commitments = [][]byte{
func TestPersist(t *testing.T) {
t.Run("no sidecars", func(t *testing.T) {
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, &peerdas.CustodyInfo{})
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, 0)
err := lazilyPersistentStoreColumns.Persist(0)
require.NoError(t, err)
require.Equal(t, 0, len(lazilyPersistentStoreColumns.cache.entries))
@@ -44,7 +42,7 @@ func TestPersist(t *testing.T) {
}
roSidecars, _ := roSidecarsFromDataColumnParamsByBlockRoot(t, dataColumnParamsByBlockRoot)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, &peerdas.CustodyInfo{})
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, 0)
err := lazilyPersistentStoreColumns.Persist(0, roSidecars...)
require.ErrorIs(t, err, errMixedRoots)
@@ -59,7 +57,7 @@ func TestPersist(t *testing.T) {
}
roSidecars, _ := roSidecarsFromDataColumnParamsByBlockRoot(t, dataColumnParamsByBlockRoot)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, &peerdas.CustodyInfo{})
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, 0)
err := lazilyPersistentStoreColumns.Persist(1_000_000, roSidecars...)
require.NoError(t, err)
@@ -76,7 +74,7 @@ func TestPersist(t *testing.T) {
}
roSidecars, roDataColumns := roSidecarsFromDataColumnParamsByBlockRoot(t, dataColumnParamsByBlockRoot)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, &peerdas.CustodyInfo{})
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, nil, 0)
err := lazilyPersistentStoreColumns.Persist(slot, roSidecars...)
require.NoError(t, err)
@@ -114,7 +112,7 @@ func TestIsDataAvailable(t *testing.T) {
signedRoBlock := newSignedRoBlock(t, signedBeaconBlockFulu)
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, newDataColumnsVerifier, &peerdas.CustodyInfo{})
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, newDataColumnsVerifier, 0)
err := lazilyPersistentStoreColumns.IsDataAvailable(ctx, 0 /*current slot*/, signedRoBlock)
require.NoError(t, err)
@@ -135,9 +133,9 @@ func TestIsDataAvailable(t *testing.T) {
root := signedRoBlock.Root()
dataColumnStorage := filesystem.NewEphemeralDataColumnStorage(t)
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, newDataColumnsVerifier, &peerdas.CustodyInfo{})
lazilyPersistentStoreColumns := NewLazilyPersistentStoreColumn(dataColumnStorage, enode.ID{}, newDataColumnsVerifier, 0)
indices := [...]uint64{1, 17, 87, 102}
indices := [...]uint64{1, 17, 19, 42, 75, 87, 102, 117}
dataColumnsParams := make([]util.DataColumnParam, 0, len(indices))
for _, index := range indices {
dataColumnParams := util.DataColumnParam{
@@ -221,14 +219,10 @@ func TestFullCommitmentsToCheck(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
resetFlags := flags.Get()
gFlags := new(flags.GlobalFlags)
gFlags.SubscribeAllDataSubnets = true
flags.Init(gFlags)
defer flags.Init(resetFlags)
numberOfColumns := params.BeaconConfig().NumberOfColumns
b := tc.block(t)
s := NewLazilyPersistentStoreColumn(nil, enode.ID{}, nil, &peerdas.CustodyInfo{})
s := NewLazilyPersistentStoreColumn(nil, enode.ID{}, nil, numberOfColumns)
commitmentsArray, err := s.fullCommitmentsToCheck(enode.ID{}, b, tc.slot)
require.NoError(t, err)

View File

@@ -33,6 +33,7 @@ type ReadOnlyDatabase interface {
IsFinalizedBlock(ctx context.Context, blockRoot [32]byte) bool
FinalizedChildBlock(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error)
HighestRootsBelowSlot(ctx context.Context, slot primitives.Slot) (primitives.Slot, [][32]byte, error)
EarliestSlot(ctx context.Context) (primitives.Slot, error)
// State related methods.
State(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error)
StateOrError(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error)
@@ -56,12 +57,11 @@ type ReadOnlyDatabase interface {
// Fee recipients operations.
FeeRecipientByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (common.Address, error)
RegistrationByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (*ethpb.ValidatorRegistrationV1, error)
// light client operations
// Light client operations
LightClientUpdates(ctx context.Context, startPeriod, endPeriod uint64) (map[uint64]interfaces.LightClientUpdate, error)
LightClientUpdate(ctx context.Context, period uint64) (interfaces.LightClientUpdate, error)
LightClientBootstrap(ctx context.Context, blockRoot []byte) (interfaces.LightClientBootstrap, error)
// origin checkpoint sync support
// Origin checkpoint sync support
OriginCheckpointBlockRoot(ctx context.Context) ([32]byte, error)
BackfillStatus(context.Context) (*dbval.BackfillStatus, error)
}
@@ -102,6 +102,10 @@ type NoHeadAccessDatabase interface {
CleanUpDirtyStates(ctx context.Context, slotsPerArchivedPoint primitives.Slot) error
DeleteHistoricalDataBeforeSlot(ctx context.Context, slot primitives.Slot, batchSize int) (int, error)
// Custody operations.
UpdateSubscribedToAllDataSubnets(ctx context.Context, subscribed bool) (bool, error)
UpdateCustodyInfo(ctx context.Context, earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error)
}
// HeadAccessDatabase defines a struct with access to reading chain head data.

View File

@@ -8,6 +8,7 @@ go_library(
"backup.go",
"blocks.go",
"checkpoint.go",
"custody.go",
"deposit_contract.go",
"encoding.go",
"error.go",
@@ -83,6 +84,7 @@ go_test(
"backup_test.go",
"blocks_test.go",
"checkpoint_test.go",
"custody_test.go",
"deposit_contract_test.go",
"encoding_test.go",
"execution_chain_test.go",

View File

@@ -860,6 +860,47 @@ func (s *Store) SaveRegistrationsByValidatorIDs(ctx context.Context, ids []primi
})
}
// EarliestStoredSlot returns the earliest slot in the database.
func (s *Store) EarliestSlot(ctx context.Context) (primitives.Slot, error) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
_, span := trace.StartSpan(ctx, "BeaconDB.EarliestSlot")
defer span.End()
earliestAvailableSlot := primitives.Slot(0)
err := s.db.View(func(tx *bolt.Tx) error {
// Retrieve the root corresponding to the earliest available block.
c := tx.Bucket(blockSlotIndicesBucket).Cursor()
k, v := c.First()
if k == nil || v == nil {
return ErrNotFound
}
slot := bytesutil.BytesToSlotBigEndian(k)
// The genesis block may be indexed in this bucket, even if we started from a checkpoint.
// Because of this, we check the next block. If the next block is still in the genesis epoch,
// then we consider we have the whole chain.
if slot != 0 {
earliestAvailableSlot = slot
}
k, v = c.Next()
if k == nil || v == nil {
// Only the genesis block is available.
return nil
}
slot = bytesutil.BytesToSlotBigEndian(k)
if slot < slotsPerEpoch {
// We are still in the genesis epoch, so we consider we have the whole chain.
return nil
}
earliestAvailableSlot = slot
return nil
})
return earliestAvailableSlot, err
}
type slotRoot struct {
slot primitives.Slot
root [32]byte
@@ -883,7 +924,7 @@ func (s *Store) slotRootsInRange(ctx context.Context, start, end primitives.Slot
c := bkt.Cursor()
for k, v := c.Seek(key); ; /* rely on internal checks to exit */ k, v = c.Prev() {
if len(k) == 0 && len(v) == 0 {
// The `edge`` variable and this `if` deal with 2 edge cases:
// The `edge` variable and this `if` deal with 2 edge cases:
// - Seeking past the end of the bucket (the `end` param is higher than the highest slot).
// - Seeking before the beginning of the bucket (the `start` param is lower than the lowest slot).
// In both of these cases k,v will be nil and we can handle the same way using `edge` to

View File

@@ -1,6 +1,7 @@
package kv
import (
"context"
"fmt"
"testing"
"time"
@@ -1327,3 +1328,86 @@ func TestStore_RegistrationsByValidatorID(t *testing.T) {
want := errors.Wrap(ErrNotFoundFeeRecipient, "validator id 3")
require.Equal(t, want.Error(), err.Error())
}
// Block creates a phase0 beacon block at the specified slot and saves it to the database.
func createAndSaveBlock(t *testing.T, ctx context.Context, db *Store, slot primitives.Slot) {
block := util.NewBeaconBlock()
block.Block.Slot = slot
wrappedBlock, err := blocks.NewSignedBeaconBlock(block)
require.NoError(t, err)
require.NoError(t, db.SaveBlock(ctx, wrappedBlock))
}
func TestStore_EarliestSlot(t *testing.T) {
ctx := t.Context()
t.Run("empty database returns ErrNotFound", func(t *testing.T) {
db := setupDB(t)
slot, err := db.EarliestSlot(ctx)
require.ErrorIs(t, err, ErrNotFound)
assert.Equal(t, primitives.Slot(0), slot)
})
t.Run("database with only genesis block", func(t *testing.T) {
db := setupDB(t)
// Create and save genesis block (slot 0)
createAndSaveBlock(t, ctx, db, 0)
slot, err := db.EarliestSlot(ctx)
require.NoError(t, err)
assert.Equal(t, primitives.Slot(0), slot)
})
t.Run("database with genesis and blocks in genesis epoch", func(t *testing.T) {
db := setupDB(t)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
// Create and save genesis block (slot 0)
createAndSaveBlock(t, ctx, db, 0)
// Create and save a block in the genesis epoch
createAndSaveBlock(t, ctx, db, primitives.Slot(slotsPerEpoch-1))
slot, err := db.EarliestSlot(ctx)
require.NoError(t, err)
assert.Equal(t, primitives.Slot(0), slot)
})
t.Run("database with genesis and blocks beyond genesis epoch", func(t *testing.T) {
db := setupDB(t)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
// Create and save genesis block (slot 0)
createAndSaveBlock(t, ctx, db, 0)
// Create and save a block beyond the genesis epoch
nextEpochSlot := primitives.Slot(slotsPerEpoch)
createAndSaveBlock(t, ctx, db, nextEpochSlot)
slot, err := db.EarliestSlot(ctx)
require.NoError(t, err)
assert.Equal(t, nextEpochSlot, slot)
})
t.Run("database starting from checkpoint (non-zero earliest slot)", func(t *testing.T) {
db := setupDB(t)
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
// Simulate starting from a checkpoint by creating blocks starting from a later slot
checkpointSlot := primitives.Slot(slotsPerEpoch * 10) // 10 epochs later
nextEpochSlot := checkpointSlot + slotsPerEpoch
// Create and save first block at checkpoint slot
createAndSaveBlock(t, ctx, db, checkpointSlot)
// Create and save another block in the next epoch
createAndSaveBlock(t, ctx, db, nextEpochSlot)
slot, err := db.EarliestSlot(ctx)
require.NoError(t, err)
assert.Equal(t, nextEpochSlot, slot)
})
}

View File

@@ -0,0 +1,129 @@
package kv
import (
"context"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
bolt "go.etcd.io/bbolt"
)
// UpdateCustodyInfo atomically updates the custody group count only it is greater than the stored one.
// In this case, it also updates the earliest available slot with the provided value.
// It returns the (potentially updated) custody group count and earliest available slot.
func (s *Store) UpdateCustodyInfo(ctx context.Context, earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error) {
_, span := trace.StartSpan(ctx, "BeaconDB.UpdateCustodyInfo")
defer span.End()
storedGroupCount, storedEarliestAvailableSlot := uint64(0), primitives.Slot(0)
if err := s.db.Update(func(tx *bolt.Tx) error {
// Retrieve the custody bucket.
bucket, err := tx.CreateBucketIfNotExists(custodyBucket)
if err != nil {
return errors.Wrap(err, "create custody bucket")
}
// Retrieve the stored custody group count.
storedGroupCountBytes := bucket.Get(groupCountKey)
if len(storedGroupCountBytes) != 0 {
storedGroupCount = bytesutil.BytesToUint64BigEndian(storedGroupCountBytes)
}
// Retrieve the stored earliest available slot.
storedEarliestAvailableSlotBytes := bucket.Get(earliestAvailableSlotKey)
if len(storedEarliestAvailableSlotBytes) != 0 {
storedEarliestAvailableSlot = primitives.Slot(bytesutil.BytesToUint64BigEndian(storedEarliestAvailableSlotBytes))
}
// Exit early if the new custody group count is lower than or equal to the stored one.
if custodyGroupCount <= storedGroupCount {
return nil
}
storedGroupCount, storedEarliestAvailableSlot = custodyGroupCount, earliestAvailableSlot
// Store the earliest available slot.
bytes := bytesutil.Uint64ToBytesBigEndian(uint64(earliestAvailableSlot))
if err := bucket.Put(earliestAvailableSlotKey, bytes); err != nil {
return errors.Wrap(err, "put earliest available slot")
}
// Store the custody group count.
bytes = bytesutil.Uint64ToBytesBigEndian(custodyGroupCount)
if err := bucket.Put(groupCountKey, bytes); err != nil {
return errors.Wrap(err, "put custody group count")
}
return nil
}); err != nil {
return 0, 0, err
}
log.WithFields(logrus.Fields{
"earliestAvailableSlot": storedEarliestAvailableSlot,
"groupCount": storedGroupCount,
}).Debug("Custody info")
return storedEarliestAvailableSlot, storedGroupCount, nil
}
// UpdateSubscribedToAllDataSubnets updates the "subscribed to all data subnets" status in the database
// only if `subscribed` is `true`.
// It returns the previous subscription status.
func (s *Store) UpdateSubscribedToAllDataSubnets(ctx context.Context, subscribed bool) (bool, error) {
_, span := trace.StartSpan(ctx, "BeaconDB.UpdateSubscribedToAllDataSubnets")
defer span.End()
result := false
if !subscribed {
if err := s.db.View(func(tx *bolt.Tx) error {
// Retrieve the custody bucket.
bucket := tx.Bucket(custodyBucket)
if bucket == nil {
return nil
}
// Retrieve the subscribe all data subnets flag.
bytes := bucket.Get(subscribeAllDataSubnetsKey)
if len(bytes) == 0 {
return nil
}
if bytes[0] == 1 {
result = true
}
return nil
}); err != nil {
return false, err
}
return result, nil
}
if err := s.db.Update(func(tx *bolt.Tx) error {
// Retrieve the custody bucket.
bucket, err := tx.CreateBucketIfNotExists(custodyBucket)
if err != nil {
return errors.Wrap(err, "create custody bucket")
}
bytes := bucket.Get(subscribeAllDataSubnetsKey)
if len(bytes) != 0 && bytes[0] == 1 {
result = true
}
if err := bucket.Put(subscribeAllDataSubnetsKey, []byte{1}); err != nil {
return errors.Wrap(err, "put subscribe all data subnets")
}
return nil
}); err != nil {
return false, err
}
return result, nil
}

View File

@@ -0,0 +1,176 @@
package kv
import (
"context"
"testing"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v6/testing/require"
bolt "go.etcd.io/bbolt"
)
// getCustodyInfoFromDB reads the custody info directly from the database for testing purposes.
func getCustodyInfoFromDB(t *testing.T, db *Store) (primitives.Slot, uint64) {
t.Helper()
var earliestSlot primitives.Slot
var groupCount uint64
err := db.db.View(func(tx *bolt.Tx) error {
bucket := tx.Bucket(custodyBucket)
if bucket == nil {
return nil
}
// Read group count
groupCountBytes := bucket.Get(groupCountKey)
if len(groupCountBytes) != 0 {
groupCount = bytesutil.BytesToUint64BigEndian(groupCountBytes)
}
// Read earliest available slot
earliestSlotBytes := bucket.Get(earliestAvailableSlotKey)
if len(earliestSlotBytes) != 0 {
earliestSlot = primitives.Slot(bytesutil.BytesToUint64BigEndian(earliestSlotBytes))
}
return nil
})
require.NoError(t, err)
return earliestSlot, groupCount
}
// getSubscriptionStatusFromDB reads the subscription status directly from the database for testing purposes.
func getSubscriptionStatusFromDB(t *testing.T, db *Store) bool {
t.Helper()
var subscribed bool
err := db.db.View(func(tx *bolt.Tx) error {
bucket := tx.Bucket(custodyBucket)
if bucket == nil {
return nil
}
bytes := bucket.Get(subscribeAllDataSubnetsKey)
if len(bytes) != 0 && bytes[0] == 1 {
subscribed = true
}
return nil
})
require.NoError(t, err)
return subscribed
}
func TestUpdateCustodyInfo(t *testing.T) {
ctx := t.Context()
t.Run("initial update with empty database", func(t *testing.T) {
const (
earliestSlot = primitives.Slot(100)
groupCount = uint64(5)
)
db := setupDB(t)
slot, count, err := db.UpdateCustodyInfo(ctx, earliestSlot, groupCount)
require.NoError(t, err)
require.Equal(t, earliestSlot, slot)
require.Equal(t, groupCount, count)
storedSlot, storedCount := getCustodyInfoFromDB(t, db)
require.Equal(t, earliestSlot, storedSlot)
require.Equal(t, groupCount, storedCount)
})
t.Run("update with higher group count", func(t *testing.T) {
const (
initialSlot = primitives.Slot(100)
initialCount = uint64(5)
earliestSlot = primitives.Slot(200)
groupCount = uint64(10)
)
db := setupDB(t)
_, _, err := db.UpdateCustodyInfo(ctx, initialSlot, initialCount)
require.NoError(t, err)
slot, count, err := db.UpdateCustodyInfo(ctx, earliestSlot, groupCount)
require.NoError(t, err)
require.Equal(t, earliestSlot, slot)
require.Equal(t, groupCount, count)
storedSlot, storedCount := getCustodyInfoFromDB(t, db)
require.Equal(t, earliestSlot, storedSlot)
require.Equal(t, groupCount, storedCount)
})
t.Run("update with lower group count should not update", func(t *testing.T) {
const (
initialSlot = primitives.Slot(200)
initialCount = uint64(10)
earliestSlot = primitives.Slot(300)
groupCount = uint64(8)
)
db := setupDB(t)
_, _, err := db.UpdateCustodyInfo(ctx, initialSlot, initialCount)
require.NoError(t, err)
slot, count, err := db.UpdateCustodyInfo(ctx, earliestSlot, groupCount)
require.NoError(t, err)
require.Equal(t, initialSlot, slot)
require.Equal(t, initialCount, count)
storedSlot, storedCount := getCustodyInfoFromDB(t, db)
require.Equal(t, initialSlot, storedSlot)
require.Equal(t, initialCount, storedCount)
})
}
func TestUpdateSubscribedToAllDataSubnets(t *testing.T) {
ctx := context.Background()
t.Run("initial update with empty database - set to false", func(t *testing.T) {
db := setupDB(t)
prev, err := db.UpdateSubscribedToAllDataSubnets(ctx, false)
require.NoError(t, err)
require.Equal(t, false, prev)
stored := getSubscriptionStatusFromDB(t, db)
require.Equal(t, false, stored)
})
t.Run("attempt to update from true to false (should not change)", func(t *testing.T) {
db := setupDB(t)
_, err := db.UpdateSubscribedToAllDataSubnets(ctx, true)
require.NoError(t, err)
prev, err := db.UpdateSubscribedToAllDataSubnets(ctx, false)
require.NoError(t, err)
require.Equal(t, true, prev)
stored := getSubscriptionStatusFromDB(t, db)
require.Equal(t, true, stored)
})
t.Run("attempt to update from true to false (should not change)", func(t *testing.T) {
db := setupDB(t)
_, err := db.UpdateSubscribedToAllDataSubnets(ctx, true)
require.NoError(t, err)
prev, err := db.UpdateSubscribedToAllDataSubnets(ctx, true)
require.NoError(t, err)
require.Equal(t, true, prev)
stored := getSubscriptionStatusFromDB(t, db)
require.Equal(t, true, stored)
})
}

View File

@@ -123,6 +123,7 @@ var Buckets = [][]byte{
feeRecipientBucket,
registrationBucket,
custodyBucket,
}
// KVStoreOption is a functional option that modifies a kv.Store.

View File

@@ -70,4 +70,10 @@ var (
// Migrations
migrationsBucket = []byte("migrations")
// Custody
custodyBucket = []byte("custody")
groupCountKey = []byte("group-count")
earliestAvailableSlotKey = []byte("earliest-available-slot")
subscribeAllDataSubnetsKey = []byte("subscribe-all-data-subnets")
)

View File

@@ -43,6 +43,7 @@ func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error
return errors.Wrap(err, "failed to initialize origin block w/ bytes + config+fork")
}
blk := wblk.Block()
slot := blk.Slot()
blockRoot, err := blk.HashTreeRoot()
if err != nil {
@@ -51,43 +52,43 @@ func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error
pr := blk.ParentRoot()
bf := &dbval.BackfillStatus{
LowSlot: uint64(wblk.Block().Slot()),
LowSlot: uint64(slot),
LowRoot: blockRoot[:],
LowParentRoot: pr[:],
OriginRoot: blockRoot[:],
OriginSlot: uint64(wblk.Block().Slot()),
OriginSlot: uint64(slot),
}
if err = s.SaveBackfillStatus(ctx, bf); err != nil {
return errors.Wrap(err, "unable to save backfill status data to db for checkpoint sync")
}
log.WithField("root", fmt.Sprintf("%#x", blockRoot)).Info("Saving checkpoint block to db")
log.WithField("root", fmt.Sprintf("%#x", blockRoot)).Info("Saving checkpoint data into database")
if err := s.SaveBlock(ctx, wblk); err != nil {
return errors.Wrap(err, "could not save checkpoint block")
return errors.Wrap(err, "save block")
}
// save state
log.WithField("blockRoot", fmt.Sprintf("%#x", blockRoot)).Info("Calling SaveState")
if err = s.SaveState(ctx, state, blockRoot); err != nil {
return errors.Wrap(err, "could not save state")
return errors.Wrap(err, "save state")
}
if err = s.SaveStateSummary(ctx, &ethpb.StateSummary{
Slot: state.Slot(),
Root: blockRoot[:],
}); err != nil {
return errors.Wrap(err, "could not save state summary")
return errors.Wrap(err, "save state summary")
}
// mark block as head of chain, so that processing will pick up from this point
if err = s.SaveHeadBlockRoot(ctx, blockRoot); err != nil {
return errors.Wrap(err, "could not save head block root")
return errors.Wrap(err, "save head block root")
}
// save origin block root in a special key, to be used when the canonical
// origin (start of chain, ie alternative to genesis) block or state is needed
if err = s.SaveOriginCheckpointBlockRoot(ctx, blockRoot); err != nil {
return errors.Wrap(err, "could not save origin block root")
return errors.Wrap(err, "save origin checkpoint block root")
}
// rebuild the checkpoint from the block
@@ -96,15 +97,18 @@ func (s *Store) SaveOrigin(ctx context.Context, serState, serBlock []byte) error
if err != nil {
return err
}
chkpt := &ethpb.Checkpoint{
Epoch: primitives.Epoch(slotEpoch),
Root: blockRoot[:],
}
if err = s.SaveJustifiedCheckpoint(ctx, chkpt); err != nil {
return errors.Wrap(err, "could not mark checkpoint sync block as justified")
return errors.Wrap(err, "save justified checkpoint")
}
if err = s.SaveFinalizedCheckpoint(ctx, chkpt); err != nil {
return errors.Wrap(err, "could not mark checkpoint sync block as finalized")
return errors.Wrap(err, "save finalized checkpoint")
}
return nil

View File

@@ -23,7 +23,6 @@ go_library(
"//beacon-chain/cache:go_default_library",
"//beacon-chain/cache/depositsnapshot:go_default_library",
"//beacon-chain/core/light-client:go_default_library",
"//beacon-chain/core/peerdas:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/db/filesystem:go_default_library",
"//beacon-chain/db/kv:go_default_library",

View File

@@ -26,7 +26,6 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache/depositsnapshot"
lightclient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/kv"
@@ -124,7 +123,6 @@ type BeaconNode struct {
BlobStorageOptions []filesystem.BlobStorageOption
DataColumnStorage *filesystem.DataColumnStorage
DataColumnStorageOptions []filesystem.DataColumnStorageOption
custodyInfo *peerdas.CustodyInfo
verifyInitWaiter *verification.InitializerWaiter
syncChecker *initialsync.SyncChecker
slasherEnabled bool
@@ -166,7 +164,6 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
serviceFlagOpts: &serviceFlagOpts{},
initialSyncComplete: make(chan struct{}),
syncChecker: &initialsync.SyncChecker{},
custodyInfo: &peerdas.CustodyInfo{},
slasherEnabled: cliCtx.Bool(flags.SlasherFlag.Name),
}
@@ -717,7 +714,6 @@ func (b *BeaconNode) registerP2P(cliCtx *cli.Context) error {
StateNotifier: b,
DB: b.db,
ClockWaiter: b.clockWaiter,
CustodyInfo: b.custodyInfo,
})
if err != nil {
return err
@@ -800,7 +796,6 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
blockchain.WithTrackedValidatorsCache(b.trackedValidatorsCache),
blockchain.WithPayloadIDCache(b.payloadIDCache),
blockchain.WithSyncChecker(b.syncChecker),
blockchain.WithCustodyInfo(b.custodyInfo),
blockchain.WithSlasherEnabled(b.slasherEnabled),
blockchain.WithLightClientStore(b.lcStore),
)
@@ -888,7 +883,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithDataColumnStorage(b.DataColumnStorage),
regularsync.WithVerifierWaiter(b.verifyInitWaiter),
regularsync.WithAvailableBlocker(bFillStore),
regularsync.WithCustodyInfo(b.custodyInfo),
regularsync.WithTrackedValidatorsCache(b.trackedValidatorsCache),
regularsync.WithSlasherEnabled(b.slasherEnabled),
regularsync.WithLightClientStore(b.lcStore),
regularsync.WithBatchVerifierLimit(b.cliCtx.Int(flags.BatchVerifierLimit.Name)),

View File

@@ -227,6 +227,7 @@ func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
bootListener, err := s.createListener(ipAddr, pkey)
require.NoError(t, err)
@@ -255,6 +256,7 @@ func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
listener, err := s.startDiscoveryV5(ipAddr, pkey)
// Set for 2nd peer
@@ -695,6 +697,7 @@ func TestService_BroadcastDataColumn(t *testing.T) {
subnetsLock: make(map[uint64]*sync.RWMutex),
subnetsLockLock: sync.Mutex{},
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{ScorerParams: &scorers.Config{}}),
custodyInfo: &custodyInfo{},
}
// Create a listener.

View File

@@ -4,7 +4,6 @@ import (
"time"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
)
@@ -40,7 +39,6 @@ type Config struct {
StateNotifier statefeed.Notifier
DB db.ReadOnlyDatabase
ClockWaiter startup.ClockWaiter
CustodyInfo *peerdas.CustodyInfo
}
// validateConfig validates whether the values provided are accurate and will set

View File

@@ -3,11 +3,114 @@ package p2p
import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/time/slots"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
var _ DataColumnsHandler = (*Service)(nil)
var _ CustodyManager = (*Service)(nil)
// EarliestAvailableSlot returns the earliest available slot.
func (s *Service) EarliestAvailableSlot() (primitives.Slot, error) {
s.custodyInfoLock.RLock()
defer s.custodyInfoLock.RUnlock()
if s.custodyInfo == nil {
return 0, errors.New("no custody info available")
}
return s.custodyInfo.earliestAvailableSlot, nil
}
// CustodyGroupCount returns the custody group count.
func (s *Service) CustodyGroupCount() (uint64, error) {
s.custodyInfoLock.Lock()
defer s.custodyInfoLock.Unlock()
if s.custodyInfo == nil {
return 0, errors.New("no custody info available")
}
return s.custodyInfo.groupCount, nil
}
// UpdateCustodyInfo updates the stored custody group count to the incoming one
// if the incoming one is greater than the stored one. In this case, the
// incoming earliest available slot should be greater than or equal to the
// stored one or an error is returned.
//
// - If there is no stored custody info, or
// - If the incoming earliest available slot is greater than or equal to the
// fulu fork slot and the incoming custody group count is greater than the
// number of samples per slot
//
// then the stored earliest available slot is updated to the incoming one.
//
// This function returns a boolean indicating whether the custody info was
// updated and the (possibly updated) custody info itself.
//
// Rationale:
// - The custody group count can only be increased (specification)
// - If the custody group count is increased before Fulu, we can still serve
// all the data, since there is no sharding before Fulu. As a consequence
// we do not need to update the earliest available slot in this case.
// - If the custody group count is increased after Fulu, but to a value less
// than or equal to the number of samples per slot, we can still serve all
// the data, since we store all sampled data column sidecars in all cases.
// As a consequence, we do not need to update the earliest available slot
// - If the custody group count is increased after Fulu to a value higher than
// the number of samples per slot, then, until the backfill is complete, we
// are unable to serve the data column sidecars corresponding to the new
// custody groups. As a consequence, we need to update the earliest
// available slot to inform the peers that we are not able to serve data
// column sidecars before this point.
func (s *Service) UpdateCustodyInfo(earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error) {
samplesPerSlot := params.BeaconConfig().SamplesPerSlot
s.custodyInfoLock.Lock()
defer s.custodyInfoLock.Unlock()
if s.custodyInfo == nil {
s.custodyInfo = &custodyInfo{
earliestAvailableSlot: earliestAvailableSlot,
groupCount: custodyGroupCount,
}
return earliestAvailableSlot, custodyGroupCount, nil
}
inMemory := s.custodyInfo
if custodyGroupCount <= inMemory.groupCount {
return inMemory.earliestAvailableSlot, inMemory.groupCount, nil
}
if earliestAvailableSlot < inMemory.earliestAvailableSlot {
return 0, 0, errors.Errorf(
"earliest available slot %d is less than the current one %d. (custody group count: %d, current one: %d)",
earliestAvailableSlot, inMemory.earliestAvailableSlot, custodyGroupCount, inMemory.groupCount,
)
}
if custodyGroupCount <= samplesPerSlot {
inMemory.groupCount = custodyGroupCount
return inMemory.earliestAvailableSlot, custodyGroupCount, nil
}
fuluForkSlot, err := fuluForkSlot()
if err != nil {
return 0, 0, errors.Wrap(err, "fulu fork slot")
}
if earliestAvailableSlot < fuluForkSlot {
inMemory.groupCount = custodyGroupCount
return inMemory.earliestAvailableSlot, custodyGroupCount, nil
}
inMemory.earliestAvailableSlot = earliestAvailableSlot
inMemory.groupCount = custodyGroupCount
return earliestAvailableSlot, custodyGroupCount, nil
}
// CustodyGroupCountFromPeer retrieves custody group count from a peer.
// It first tries to get the custody group count from the peer's metadata,
@@ -72,3 +175,19 @@ func (s *Service) custodyGroupCountFromPeerENR(pid peer.ID) uint64 {
return custodyGroupCount
}
func fuluForkSlot() (primitives.Slot, error) {
beaconConfig := params.BeaconConfig()
fuluForkEpoch := beaconConfig.FuluForkEpoch
if fuluForkEpoch == beaconConfig.FarFutureEpoch {
return beaconConfig.FarFutureSlot, nil
}
forkFuluSlot, err := slots.EpochStart(fuluForkEpoch)
if err != nil {
return 0, errors.Wrap(err, "epoch start")
}
return forkFuluSlot, nil
}

View File

@@ -1,12 +1,15 @@
package p2p
import (
"context"
"strings"
"testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v6/consensus-types/wrapper"
pb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1/metadata"
@@ -15,6 +18,174 @@ import (
"github.com/libp2p/go-libp2p/core/network"
)
func TestEarliestAvailableSlot(t *testing.T) {
t.Run("No custody info available", func(t *testing.T) {
service := &Service{
custodyInfo: nil,
}
_, err := service.EarliestAvailableSlot()
require.NotNil(t, err)
})
t.Run("Valid custody info", func(t *testing.T) {
const expected primitives.Slot = 100
service := &Service{
custodyInfo: &custodyInfo{
earliestAvailableSlot: expected,
},
}
slot, err := service.EarliestAvailableSlot()
require.NoError(t, err)
require.Equal(t, expected, slot)
})
}
func TestCustodyGroupCount(t *testing.T) {
t.Run("No custody info available", func(t *testing.T) {
service := &Service{
custodyInfo: nil,
}
_, err := service.CustodyGroupCount()
require.NotNil(t, err)
require.Equal(t, true, strings.Contains(err.Error(), "no custody info available"))
})
t.Run("Valid custody info", func(t *testing.T) {
const expected uint64 = 5
service := &Service{
custodyInfo: &custodyInfo{
groupCount: expected,
},
}
count, err := service.CustodyGroupCount()
require.NoError(t, err)
require.Equal(t, expected, count)
})
}
func TestUpdateCustodyInfo(t *testing.T) {
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig()
config.SamplesPerSlot = 8
config.FuluForkEpoch = 10
params.OverrideBeaconConfig(config)
testCases := []struct {
name string
initialCustodyInfo *custodyInfo
inputSlot primitives.Slot
inputGroupCount uint64
expectedUpdated bool
expectedSlot primitives.Slot
expectedGroupCount uint64
expectedErr string
}{
{
name: "First time setting custody info",
initialCustodyInfo: nil,
inputSlot: 100,
inputGroupCount: 5,
expectedUpdated: true,
expectedSlot: 100,
expectedGroupCount: 5,
},
{
name: "Group count decrease - no update",
initialCustodyInfo: &custodyInfo{
earliestAvailableSlot: 50,
groupCount: 10,
},
inputSlot: 60,
inputGroupCount: 8,
expectedUpdated: false,
expectedSlot: 50,
expectedGroupCount: 10,
},
{
name: "Earliest slot decrease - error",
initialCustodyInfo: &custodyInfo{
earliestAvailableSlot: 100,
groupCount: 5,
},
inputSlot: 50,
inputGroupCount: 10,
expectedErr: "earliest available slot 50 is less than the current one 100",
},
{
name: "Group count increase but <= samples per slot",
initialCustodyInfo: &custodyInfo{
earliestAvailableSlot: 50,
groupCount: 5,
},
inputSlot: 60,
inputGroupCount: 8,
expectedUpdated: true,
expectedSlot: 50,
expectedGroupCount: 8,
},
{
name: "Group count increase > samples per slot, before Fulu fork",
initialCustodyInfo: &custodyInfo{
earliestAvailableSlot: 50,
groupCount: 5,
},
inputSlot: 60,
inputGroupCount: 15,
expectedUpdated: true,
expectedSlot: 50,
expectedGroupCount: 15,
},
{
name: "Group count increase > samples per slot, after Fulu fork",
initialCustodyInfo: &custodyInfo{
earliestAvailableSlot: 50,
groupCount: 5,
},
inputSlot: 500,
inputGroupCount: 15,
expectedUpdated: true,
expectedSlot: 500,
expectedGroupCount: 15,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
service := &Service{
custodyInfo: tc.initialCustodyInfo,
}
slot, groupCount, err := service.UpdateCustodyInfo(tc.inputSlot, tc.inputGroupCount)
if tc.expectedErr != "" {
require.NotNil(t, err)
require.Equal(t, true, strings.Contains(err.Error(), tc.expectedErr))
return
}
require.NoError(t, err)
require.Equal(t, tc.expectedSlot, slot)
require.Equal(t, tc.expectedGroupCount, groupCount)
if tc.expectedUpdated {
require.NotNil(t, service.custodyInfo)
require.Equal(t, tc.expectedSlot, service.custodyInfo.earliestAvailableSlot)
require.Equal(t, tc.expectedGroupCount, service.custodyInfo.groupCount)
}
})
}
}
func TestCustodyGroupCountFromPeer(t *testing.T) {
const (
expectedENR uint64 = 7
@@ -109,3 +280,59 @@ func TestCustodyGroupCountFromPeer(t *testing.T) {
}
}
func TestCustodyGroupCountFromPeerENR(t *testing.T) {
const (
expectedENR uint64 = 7
pid = "test-id"
)
cgc := peerdas.Cgc(expectedENR)
custodyRequirement := params.BeaconConfig().CustodyRequirement
testCases := []struct {
name string
record *enr.Record
expected uint64
wantErr bool
}{
{
name: "No ENR record",
record: nil,
expected: custodyRequirement,
},
{
name: "Empty ENR record",
record: &enr.Record{},
expected: custodyRequirement,
},
{
name: "Valid ENR with custody group count",
record: func() *enr.Record {
record := &enr.Record{}
record.Set(cgc)
return record
}(),
expected: expectedENR,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
peers := peers.NewStatus(context.Background(), &peers.StatusConfig{
ScorerParams: &scorers.Config{},
})
if tc.record != nil {
peers.Add(tc.record, pid, nil, network.DirOutbound)
}
service := &Service{
peers: peers,
}
actual := service.custodyGroupCountFromPeerENR(pid)
require.Equal(t, tc.expected, actual)
})
}
}

View File

@@ -236,22 +236,40 @@ func (s *Service) RefreshPersistentSubnets() {
// Get the sync subnet bitfield in our metadata.
currentBitSInMetadata := s.Metadata().SyncnetsBitfield()
// Is our sync bitvector record up to date?
isBitSUpToDate := bytes.Equal(bitS, inRecordBitS) && bytes.Equal(bitS, currentBitSInMetadata)
// Compare current epoch with the Fulu fork epoch.
fuluForkEpoch := params.BeaconConfig().FuluForkEpoch
custodyGroupCount, inRecordCustodyGroupCount := uint64(0), uint64(0)
if params.FuluEnabled() {
// Get the custody group count we store in our record.
inRecordCustodyGroupCount, err = peerdas.CustodyGroupCountFromRecord(record)
if err != nil {
log.WithError(err).Error("Could not retrieve custody group count")
return
}
custodyGroupCount, err = s.CustodyGroupCount()
if err != nil {
log.WithError(err).Error("Could not retrieve custody group count")
return
}
}
// We add `1` to the current epoch because we want to prepare one epoch before the Fulu fork.
if currentEpoch+1 < fuluForkEpoch {
// Is our custody group count record up to date?
isCustodyGroupCountUpToDate := custodyGroupCount == inRecordCustodyGroupCount
// Altair behaviour.
if metadataVersion == version.Altair && isBitVUpToDate && isBitSUpToDate {
if metadataVersion == version.Altair && isBitVUpToDate && isBitSUpToDate && (!params.FuluEnabled() || isCustodyGroupCountUpToDate) {
// Nothing to do, return early.
return
}
// Some data have changed, update our record and metadata.
s.updateSubnetRecordWithMetadataV2(bitV, bitS)
s.updateSubnetRecordWithMetadataV2(bitV, bitS, custodyGroupCount)
// Ping all peers to inform them of new metadata
s.pingPeersAndLogEnr()
@@ -259,16 +277,6 @@ func (s *Service) RefreshPersistentSubnets() {
return
}
// Get the current custody group count.
custodyGroupCount := s.cfg.CustodyInfo.ActualGroupCount()
// Get the custody group count we store in our record.
inRecordCustodyGroupCount, err := peerdas.CustodyGroupCountFromRecord(record)
if err != nil {
log.WithError(err).Error("Could not retrieve custody subnet count")
return
}
// Get the custody group count in our metadata.
inMetadataCustodyGroupCount := s.Metadata().CustodyGroupCount()
@@ -565,11 +573,6 @@ func (s *Service) createLocalNode(
localNode.Set(quicEntry)
}
if params.FuluEnabled() {
custodyGroupCount := s.cfg.CustodyInfo.ActualGroupCount()
localNode.Set(peerdas.Cgc(custodyGroupCount))
}
localNode.SetFallbackIP(ipAddr)
localNode.SetFallbackUDP(udpPort)
@@ -581,6 +584,16 @@ func (s *Service) createLocalNode(
localNode = initializeAttSubnets(localNode)
localNode = initializeSyncCommSubnets(localNode)
if params.FuluEnabled() {
custodyGroupCount, err := s.CustodyGroupCount()
if err != nil {
return nil, errors.Wrap(err, "could not retrieve custody group count")
}
custodyGroupCountEntry := peerdas.Cgc(custodyGroupCount)
localNode.Set(custodyGroupCountEntry)
}
if s.cfg != nil && s.cfg.HostAddress != "" {
hostIP := net.ParseIP(s.cfg.HostAddress)
if hostIP.To4() == nil && hostIP.To16() == nil {

View File

@@ -16,7 +16,6 @@ import (
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/peerdata"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers/scorers"
@@ -65,6 +64,7 @@ func TestCreateListener(t *testing.T) {
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
cfg: &Config{UDPPort: uint(port)},
custodyInfo: &custodyInfo{},
}
listener, err := s.createListener(ipAddr, pkey)
require.NoError(t, err)
@@ -91,6 +91,7 @@ func TestStartDiscV5_DiscoverAllPeers(t *testing.T) {
cfg: &Config{UDPPort: uint(port), PingInterval: testPingInterval, DisableLivenessCheck: true},
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
bootListener, err := s.createListener(ipAddr, pkey)
require.NoError(t, err)
@@ -116,6 +117,7 @@ func TestStartDiscV5_DiscoverAllPeers(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
listener, err := s.startDiscoveryV5(ipAddr, pkey)
assert.NoError(t, err, "Could not start discovery for node")
@@ -157,27 +159,27 @@ func TestCreateLocalNode(t *testing.T) {
}{
{
name: "valid config",
cfg: &Config{CustodyInfo: &peerdas.CustodyInfo{}},
cfg: &Config{},
expectedError: false,
},
{
name: "invalid host address",
cfg: &Config{HostAddress: "invalid", CustodyInfo: &peerdas.CustodyInfo{}},
cfg: &Config{HostAddress: "invalid"},
expectedError: true,
},
{
name: "valid host address",
cfg: &Config{HostAddress: "192.168.0.1", CustodyInfo: &peerdas.CustodyInfo{}},
cfg: &Config{HostAddress: "192.168.0.1"},
expectedError: false,
},
{
name: "invalid host DNS",
cfg: &Config{HostDNS: "invalid", CustodyInfo: &peerdas.CustodyInfo{}},
cfg: &Config{HostDNS: "invalid"},
expectedError: true,
},
{
name: "valid host DNS",
cfg: &Config{HostDNS: "www.google.com", CustodyInfo: &peerdas.CustodyInfo{}},
cfg: &Config{HostDNS: "www.google.com"},
expectedError: false,
},
}
@@ -191,6 +193,8 @@ func TestCreateLocalNode(t *testing.T) {
quicPort = 3000
)
custodyRequirement := params.BeaconConfig().CustodyRequirement
// Create a private key.
address, privKey := createAddrAndPrivKey(t)
@@ -199,6 +203,7 @@ func TestCreateLocalNode(t *testing.T) {
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
cfg: tt.cfg,
custodyInfo: &custodyInfo{groupCount: custodyRequirement},
}
localNode, err := service.createLocalNode(privKey, address, udpPort, tcpPort, quicPort)
@@ -210,7 +215,7 @@ func TestCreateLocalNode(t *testing.T) {
require.NoError(t, err)
expectedAddress := address
if tt.cfg.HostAddress != "" {
if tt.cfg != nil && tt.cfg.HostAddress != "" {
expectedAddress = net.ParseIP(tt.cfg.HostAddress)
}
@@ -250,8 +255,8 @@ func TestCreateLocalNode(t *testing.T) {
// Check cgc config.
custodyGroupCount := new(uint64)
require.NoError(t, localNode.Node().Record().Load(enr.WithEntry(peerdas.CustodyGroupCountEnrKey, custodyGroupCount)))
require.Equal(t, params.BeaconConfig().CustodyRequirement, *custodyGroupCount)
require.NoError(t, localNode.Node().Record().Load(enr.WithEntry(params.BeaconNetworkConfig().CustodyGroupCountKey, custodyGroupCount)))
require.Equal(t, custodyRequirement, *custodyGroupCount)
})
}
}
@@ -263,6 +268,7 @@ func TestRebootDiscoveryListener(t *testing.T) {
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
cfg: &Config{UDPPort: uint(port)},
custodyInfo: &custodyInfo{},
}
createListener := func() (*discover.UDPv5, error) {
@@ -295,6 +301,7 @@ func TestMultiAddrsConversion_InvalidIPAddr(t *testing.T) {
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
cfg: &Config{},
custodyInfo: &custodyInfo{},
}
node, err := s.createLocalNode(pkey, addr, 0, 0, 0)
require.NoError(t, err)
@@ -313,6 +320,7 @@ func TestMultiAddrConversion_OK(t *testing.T) {
},
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
custodyInfo: &custodyInfo{},
}
listener, err := s.createListener(ipAddr, pkey)
require.NoError(t, err)
@@ -386,6 +394,7 @@ func TestHostIsResolved(t *testing.T) {
},
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
custodyInfo: &custodyInfo{},
}
ip, key := createAddrAndPrivKey(t)
list, err := s.createListener(ip, key)
@@ -455,6 +464,7 @@ func TestUDPMultiAddress(t *testing.T) {
cfg: &Config{UDPPort: uint(port)},
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
createListener := func() (*discover.UDPv5, error) {
@@ -655,7 +665,7 @@ func checkPingCountCacheMetadataRecord(
if expected.custodyGroupCount != nil {
// Check custody subnet count in ENR.
var actualCustodyGroupCount uint64
err := service.dv5Listener.LocalNode().Node().Record().Load(enr.WithEntry(peerdas.CustodyGroupCountEnrKey, &actualCustodyGroupCount))
err := service.dv5Listener.LocalNode().Node().Record().Load(enr.WithEntry(params.BeaconNetworkConfig().CustodyGroupCountKey, &actualCustodyGroupCount))
require.NoError(t, err)
require.Equal(t, *expected.custodyGroupCount, actualCustodyGroupCount)
@@ -818,10 +828,11 @@ func TestRefreshPersistentSubnets(t *testing.T) {
actualPingCount++
return nil
},
cfg: &Config{UDPPort: 2000, CustodyInfo: &peerdas.CustodyInfo{}},
cfg: &Config{UDPPort: 2000},
peers: p2p.Peers(),
genesisTime: time.Now().Add(-time.Duration(tc.epochSinceGenesis*secondsPerEpoch) * time.Second),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
custodyInfo: &custodyInfo{groupCount: custodyGroupCount},
}
// Set the listener and the metadata.

View File

@@ -40,6 +40,7 @@ func TestStartDiscv5_DifferentForkDigests(t *testing.T) {
},
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
bootListener, err := s.createListener(ipAddr, pkey)
require.NoError(t, err)
@@ -73,6 +74,7 @@ func TestStartDiscv5_DifferentForkDigests(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: root,
custodyInfo: &custodyInfo{},
}
listener, err := s.startDiscoveryV5(ipAddr, pkey)
assert.NoError(t, err, "Could not start discovery for node")
@@ -134,6 +136,7 @@ func TestStartDiscv5_SameForkDigests_DifferentNextForkData(t *testing.T) {
cfg: &Config{UDPPort: uint(port), PingInterval: testPingInterval, DisableLivenessCheck: true},
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
bootListener, err := s.createListener(ipAddr, pkey)
require.NoError(t, err)
@@ -168,6 +171,7 @@ func TestStartDiscv5_SameForkDigests_DifferentNextForkData(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
listener, err := s.startDiscoveryV5(ipAddr, pkey)
assert.NoError(t, err, "Could not start discovery for node")

View File

@@ -7,6 +7,7 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1/metadata"
"github.com/ethereum/go-ethereum/p2p/enode"
@@ -32,13 +33,14 @@ type (
ConnectionHandler
PeersProvider
MetadataProvider
DataColumnsHandler
CustodyManager
}
// Accessor provides access to the Broadcaster and PeerManager interfaces.
// Accessor provides access to the Broadcaster, PeerManager and CustodyManager interfaces.
Accessor interface {
Broadcaster
PeerManager
CustodyManager
}
// Broadcaster broadcasts messages to peers over the p2p pubsub protocol.
@@ -118,8 +120,11 @@ type (
MetadataSeq() uint64
}
// DataColumnsHandler abstracts some data columns related methods.
DataColumnsHandler interface {
// CustodyManager abstracts some data columns related methods.
CustodyManager interface {
EarliestAvailableSlot() (primitives.Slot, error)
CustodyGroupCount() (uint64, error)
UpdateCustodyInfo(earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error)
CustodyGroupCountFromPeer(peer.ID) uint64
}
)

View File

@@ -54,7 +54,7 @@ type PeerData struct {
NextValidTime time.Time
// Chain related data.
MetaData metadata.Metadata
ChainState *ethpb.Status
ChainState *ethpb.StatusV2
ChainStateLastUpdated time.Time
ChainStateValidationError error
// Scorers internal data.

View File

@@ -112,7 +112,7 @@ func (s *PeerStatusScorer) BadPeers() []peer.ID {
}
// SetPeerStatus sets chain state data for a given peer.
func (s *PeerStatusScorer) SetPeerStatus(pid peer.ID, chainState *pb.Status, validationError error) {
func (s *PeerStatusScorer) SetPeerStatus(pid peer.ID, chainState *pb.StatusV2, validationError error) {
s.store.Lock()
defer s.store.Unlock()
@@ -130,14 +130,14 @@ func (s *PeerStatusScorer) SetPeerStatus(pid peer.ID, chainState *pb.Status, val
// PeerStatus gets the chain state of the given remote peer.
// This can return nil if there is no known chain state for the peer.
// This will error if the peer does not exist.
func (s *PeerStatusScorer) PeerStatus(pid peer.ID) (*pb.Status, error) {
func (s *PeerStatusScorer) PeerStatus(pid peer.ID) (*pb.StatusV2, error) {
s.store.RLock()
defer s.store.RUnlock()
return s.peerStatusNoLock(pid)
}
// peerStatusNoLock lock-free version of PeerStatus.
func (s *PeerStatusScorer) peerStatusNoLock(pid peer.ID) (*pb.Status, error) {
func (s *PeerStatusScorer) peerStatusNoLock(pid peer.ID) (*pb.StatusV2, error) {
if peerData, ok := s.store.PeerData(pid); ok {
if peerData.ChainState == nil {
return nil, peerdata.ErrNoPeerStatus

View File

@@ -35,7 +35,7 @@ func TestScorers_PeerStatus_Score(t *testing.T) {
name: "existent bad peer",
update: func(scorer *scorers.PeerStatusScorer) {
scorer.SetHeadSlot(0)
scorer.SetPeerStatus("peer1", &pb.Status{
scorer.SetPeerStatus("peer1", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: 64,
}, p2ptypes.ErrWrongForkDigestVersion)
@@ -48,7 +48,7 @@ func TestScorers_PeerStatus_Score(t *testing.T) {
name: "existent peer no head slot for the host node is known",
update: func(scorer *scorers.PeerStatusScorer) {
scorer.SetHeadSlot(0)
scorer.SetPeerStatus("peer1", &pb.Status{
scorer.SetPeerStatus("peer1", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: 64,
}, nil)
@@ -61,7 +61,7 @@ func TestScorers_PeerStatus_Score(t *testing.T) {
name: "existent peer head is before ours",
update: func(scorer *scorers.PeerStatusScorer) {
scorer.SetHeadSlot(128)
scorer.SetPeerStatus("peer1", &pb.Status{
scorer.SetPeerStatus("peer1", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: 64,
}, nil)
@@ -75,12 +75,12 @@ func TestScorers_PeerStatus_Score(t *testing.T) {
update: func(scorer *scorers.PeerStatusScorer) {
headSlot := primitives.Slot(128)
scorer.SetHeadSlot(headSlot)
scorer.SetPeerStatus("peer1", &pb.Status{
scorer.SetPeerStatus("peer1", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: headSlot + 64,
}, nil)
// Set another peer to a higher score.
scorer.SetPeerStatus("peer2", &pb.Status{
scorer.SetPeerStatus("peer2", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: headSlot + 128,
}, nil)
@@ -95,7 +95,7 @@ func TestScorers_PeerStatus_Score(t *testing.T) {
update: func(scorer *scorers.PeerStatusScorer) {
headSlot := primitives.Slot(128)
scorer.SetHeadSlot(headSlot)
scorer.SetPeerStatus("peer1", &pb.Status{
scorer.SetPeerStatus("peer1", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: headSlot + 64,
}, nil)
@@ -108,7 +108,7 @@ func TestScorers_PeerStatus_Score(t *testing.T) {
name: "existent peer no max known slot",
update: func(scorer *scorers.PeerStatusScorer) {
scorer.SetHeadSlot(0)
scorer.SetPeerStatus("peer1", &pb.Status{
scorer.SetPeerStatus("peer1", &pb.StatusV2{
HeadRoot: make([]byte, 32),
HeadSlot: 0,
}, nil)
@@ -141,7 +141,7 @@ func TestScorers_PeerStatus_IsBadPeer(t *testing.T) {
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid))
assert.NoError(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid))
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid, &pb.Status{}, p2ptypes.ErrWrongForkDigestVersion)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid, &pb.StatusV2{}, p2ptypes.ErrWrongForkDigestVersion)
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer(pid))
assert.NotNil(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid))
}
@@ -160,9 +160,9 @@ func TestScorers_PeerStatus_BadPeers(t *testing.T) {
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid3))
assert.NoError(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid3))
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid1, &pb.Status{}, p2ptypes.ErrWrongForkDigestVersion)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid2, &pb.Status{}, nil)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid3, &pb.Status{}, p2ptypes.ErrWrongForkDigestVersion)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid1, &pb.StatusV2{}, p2ptypes.ErrWrongForkDigestVersion)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid2, &pb.StatusV2{}, nil)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid3, &pb.StatusV2{}, p2ptypes.ErrWrongForkDigestVersion)
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer(pid1))
assert.NotNil(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid1))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid2))
@@ -179,12 +179,12 @@ func TestScorers_PeerStatus_PeerStatus(t *testing.T) {
})
status, err := peerStatuses.Scorers().PeerStatusScorer().PeerStatus("peer1")
require.ErrorContains(t, peerdata.ErrPeerUnknown.Error(), err)
assert.Equal(t, (*pb.Status)(nil), status)
assert.Equal(t, (*pb.StatusV2)(nil), status)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus("peer1", &pb.Status{
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus("peer1", &pb.StatusV2{
HeadSlot: 128,
}, nil)
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus("peer2", &pb.Status{
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus("peer2", &pb.StatusV2{
HeadSlot: 128,
}, p2ptypes.ErrInvalidEpoch)
status, err = peerStatuses.Scorers().PeerStatusScorer().PeerStatus("peer1")

View File

@@ -205,14 +205,14 @@ func (p *Status) ENR(pid peer.ID) (*enr.Record, error) {
}
// SetChainState sets the chain state of the given remote peer.
func (p *Status) SetChainState(pid peer.ID, chainState *pb.Status) {
func (p *Status) SetChainState(pid peer.ID, chainState *pb.StatusV2) {
p.scorers.PeerStatusScorer().SetPeerStatus(pid, chainState, nil)
}
// ChainState gets the chain state of the given remote peer.
// This will error if the peer does not exist.
// This will error if there is no known chain state for the peer.
func (p *Status) ChainState(pid peer.ID) (*pb.Status, error) {
func (p *Status) ChainState(pid peer.ID) (*pb.StatusV2, error) {
return p.scorers.PeerStatusScorer().PeerStatus(pid)
}

View File

@@ -289,7 +289,7 @@ func TestPeerChainState(t *testing.T) {
require.NoError(t, err)
finalizedEpoch := primitives.Epoch(123)
p.SetChainState(id, &pb.Status{FinalizedEpoch: finalizedEpoch})
p.SetChainState(id, &pb.StatusV2{FinalizedEpoch: finalizedEpoch})
resChainState, err := p.ChainState(id)
require.NoError(t, err)
@@ -324,7 +324,7 @@ func TestPeerWithNilChainState(t *testing.T) {
resChainState, err := p.ChainState(id)
require.Equal(t, peerdata.ErrNoPeerStatus, err)
var nothing *pb.Status
var nothing *pb.StatusV2
require.Equal(t, resChainState, nothing)
}
@@ -616,7 +616,7 @@ func TestTrimmedOrderedPeers(t *testing.T) {
// Peer 1
pid1 := addPeer(t, p, peers.Connected)
p.SetChainState(pid1, &pb.Status{
p.SetChainState(pid1, &pb.StatusV2{
HeadSlot: 3 * params.BeaconConfig().SlotsPerEpoch,
FinalizedEpoch: 3,
FinalizedRoot: mockroot3[:],
@@ -624,7 +624,7 @@ func TestTrimmedOrderedPeers(t *testing.T) {
// Peer 2
pid2 := addPeer(t, p, peers.Connected)
p.SetChainState(pid2, &pb.Status{
p.SetChainState(pid2, &pb.StatusV2{
HeadSlot: 4 * params.BeaconConfig().SlotsPerEpoch,
FinalizedEpoch: 4,
FinalizedRoot: mockroot4[:],
@@ -632,7 +632,7 @@ func TestTrimmedOrderedPeers(t *testing.T) {
// Peer 3
pid3 := addPeer(t, p, peers.Connected)
p.SetChainState(pid3, &pb.Status{
p.SetChainState(pid3, &pb.StatusV2{
HeadSlot: 5 * params.BeaconConfig().SlotsPerEpoch,
FinalizedEpoch: 5,
FinalizedRoot: mockroot5[:],
@@ -640,7 +640,7 @@ func TestTrimmedOrderedPeers(t *testing.T) {
// Peer 4
pid4 := addPeer(t, p, peers.Connected)
p.SetChainState(pid4, &pb.Status{
p.SetChainState(pid4, &pb.StatusV2{
HeadSlot: 2 * params.BeaconConfig().SlotsPerEpoch,
FinalizedEpoch: 2,
FinalizedRoot: mockroot2[:],
@@ -648,7 +648,7 @@ func TestTrimmedOrderedPeers(t *testing.T) {
// Peer 5
pid5 := addPeer(t, p, peers.Connected)
p.SetChainState(pid5, &pb.Status{
p.SetChainState(pid5, &pb.StatusV2{
HeadSlot: 2 * params.BeaconConfig().SlotsPerEpoch,
FinalizedEpoch: 2,
FinalizedRoot: mockroot2[:],
@@ -1012,7 +1012,7 @@ func TestStatus_BestPeer(t *testing.T) {
},
})
for _, peerConfig := range tt.peers {
p.SetChainState(addPeer(t, p, peers.Connected), &pb.Status{
p.SetChainState(addPeer(t, p, peers.Connected), &pb.StatusV2{
FinalizedEpoch: peerConfig.finalizedEpoch,
HeadSlot: peerConfig.headSlot,
})
@@ -1039,7 +1039,7 @@ func TestBestFinalized_returnsMaxValue(t *testing.T) {
for i := 0; i <= maxPeers+100; i++ {
p.Add(new(enr.Record), peer.ID(rune(i)), nil, network.DirOutbound)
p.SetConnectionState(peer.ID(rune(i)), peers.Connected)
p.SetChainState(peer.ID(rune(i)), &pb.Status{
p.SetChainState(peer.ID(rune(i)), &pb.StatusV2{
FinalizedEpoch: 10,
})
}
@@ -1062,7 +1062,7 @@ func TestStatus_BestNonFinalized(t *testing.T) {
for i, headSlot := range peerSlots {
p.Add(new(enr.Record), peer.ID(rune(i)), nil, network.DirOutbound)
p.SetConnectionState(peer.ID(rune(i)), peers.Connected)
p.SetChainState(peer.ID(rune(i)), &pb.Status{
p.SetChainState(peer.ID(rune(i)), &pb.StatusV2{
HeadSlot: headSlot,
})
}
@@ -1085,17 +1085,17 @@ func TestStatus_CurrentEpoch(t *testing.T) {
})
// Peer 1
pid1 := addPeer(t, p, peers.Connected)
p.SetChainState(pid1, &pb.Status{
p.SetChainState(pid1, &pb.StatusV2{
HeadSlot: params.BeaconConfig().SlotsPerEpoch * 4,
})
// Peer 2
pid2 := addPeer(t, p, peers.Connected)
p.SetChainState(pid2, &pb.Status{
p.SetChainState(pid2, &pb.StatusV2{
HeadSlot: params.BeaconConfig().SlotsPerEpoch * 5,
})
// Peer 3
pid3 := addPeer(t, p, peers.Connected)
p.SetChainState(pid3, &pb.Status{
p.SetChainState(pid3, &pb.StatusV2{
HeadSlot: params.BeaconConfig().SlotsPerEpoch * 4,
})

View File

@@ -108,6 +108,8 @@ const (
RPCDataColumnSidecarsByRangeTopicV1 = protocolPrefix + DataColumnSidecarsByRangeName + SchemaVersionV1
// V2 RPC Topics
// RPCStatusTopicV2 defines the v1 topic for the status rpc method.
RPCStatusTopicV2 = protocolPrefix + StatusMessageName + SchemaVersionV2
// RPCBlocksByRangeTopicV2 defines v2 the topic for the blocks by range rpc method.
RPCBlocksByRangeTopicV2 = protocolPrefix + BeaconBlocksByRangeMessageName + SchemaVersionV2
// RPCBlocksByRootTopicV2 defines the v2 topic for the blocks by root rpc method.
@@ -130,6 +132,7 @@ var (
RPCTopicMappings = map[string]interface{}{
// RPC Status Message
RPCStatusTopicV1: new(pb.Status),
RPCStatusTopicV2: new(pb.StatusV2),
// RPC Goodbye Message
RPCGoodByeTopicV1: new(primitives.SSZUint64),
@@ -201,6 +204,7 @@ var (
// Maps all the RPC messages which are to updated in fulu.
fuluMapping = map[string]string{
StatusMessageName: SchemaVersionV2,
MetadataMessageName: SchemaVersionV3,
}

View File

@@ -141,6 +141,11 @@ func TestTopicFromMessage_CorrectType(t *testing.T) {
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/beacon_blocks_by_range/2", topic)
// Modified in fulu fork.
topic, err = TopicFromMessage(StatusMessageName, fuluForkEpoch)
require.NoError(t, err)
require.Equal(t, "/eth2/beacon_chain/req/status/2", topic)
// Modified both in altair and fulu fork.
topic, err = TopicFromMessage(MetadataMessageName, fuluForkEpoch)
require.NoError(t, err)

View File

@@ -16,6 +16,7 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/config/features"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
leakybucket "github.com/OffchainLabs/prysm/v6/container/leaky-bucket"
"github.com/OffchainLabs/prysm/v6/monitoring/tracing/trace"
prysmnetwork "github.com/OffchainLabs/prysm/v6/network"
@@ -62,33 +63,42 @@ var (
)
// Service for managing peer to peer (p2p) networking.
type Service struct {
started bool
isPreGenesis bool
pingMethod func(ctx context.Context, id peer.ID) error
pingMethodLock sync.RWMutex
cancel context.CancelFunc
cfg *Config
peers *peers.Status
addrFilter *multiaddr.Filters
ipLimiter *leakybucket.Collector
privKey *ecdsa.PrivateKey
metaData metadata.Metadata
pubsub *pubsub.PubSub
joinedTopics map[string]*pubsub.Topic
joinedTopicsLock sync.RWMutex
subnetsLock map[uint64]*sync.RWMutex
subnetsLockLock sync.Mutex // Lock access to subnetsLock
initializationLock sync.Mutex
dv5Listener ListenerRebooter
startupErr error
ctx context.Context
host host.Host
genesisTime time.Time
genesisValidatorsRoot []byte
activeValidatorCount uint64
peerDisconnectionTime *cache.Cache
}
type (
Service struct {
started bool
isPreGenesis bool
pingMethod func(ctx context.Context, id peer.ID) error
pingMethodLock sync.RWMutex
cancel context.CancelFunc
cfg *Config
peers *peers.Status
addrFilter *multiaddr.Filters
ipLimiter *leakybucket.Collector
privKey *ecdsa.PrivateKey
metaData metadata.Metadata
pubsub *pubsub.PubSub
joinedTopics map[string]*pubsub.Topic
joinedTopicsLock sync.RWMutex
subnetsLock map[uint64]*sync.RWMutex
subnetsLockLock sync.Mutex // Lock access to subnetsLock
initializationLock sync.Mutex
dv5Listener ListenerRebooter
startupErr error
ctx context.Context
host host.Host
genesisTime time.Time
genesisValidatorsRoot []byte
activeValidatorCount uint64
peerDisconnectionTime *cache.Cache
custodyInfo *custodyInfo
custodyInfoLock sync.RWMutex // Lock access to custodyInfo
}
custodyInfo struct {
earliestAvailableSlot primitives.Slot
groupCount uint64
}
)
// NewService initializes a new p2p service compatible with shared.Service interface. No
// connections are made until the Start function is called during the service registry startup.

View File

@@ -114,6 +114,7 @@ func TestService_Start_OnlyStartsOnce(t *testing.T) {
s, err := NewService(t.Context(), cfg)
require.NoError(t, err)
s.dv5Listener = &mockListener{}
s.custodyInfo = &custodyInfo{}
exitRoutine := make(chan bool)
go func() {
s.Start()
@@ -211,6 +212,7 @@ func TestListenForNewNodes(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: gvr[:],
custodyInfo: &custodyInfo{},
}
bootListener, err := s.createListener(ipAddr, pkey)
@@ -252,6 +254,7 @@ func TestListenForNewNodes(t *testing.T) {
cfg: cfg,
genesisTime: genesisTime,
genesisValidatorsRoot: gvr[:],
custodyInfo: &custodyInfo{},
}
listener, err := s.startDiscoveryV5(ipAddr, pkey)
@@ -281,6 +284,7 @@ func TestListenForNewNodes(t *testing.T) {
s, err = NewService(t.Context(), cfg)
require.NoError(t, err)
s.custodyInfo = &custodyInfo{}
go s.Start()

View File

@@ -390,11 +390,23 @@ func (s *Service) updateSubnetRecordWithMetadata(bitV bitfield.Bitvector64) {
// with a new value for a bitfield of subnets tracked. It also record's
// the sync committee subnet in the enr. It also updates the node's
// metadata by increasing the sequence number and the subnets tracked by the node.
func (s *Service) updateSubnetRecordWithMetadataV2(bitVAtt bitfield.Bitvector64, bitVSync bitfield.Bitvector4) {
func (s *Service) updateSubnetRecordWithMetadataV2(
bitVAtt bitfield.Bitvector64,
bitVSync bitfield.Bitvector4,
custodyGroupCount uint64,
) {
entry := enr.WithEntry(attSubnetEnrKey, &bitVAtt)
subEntry := enr.WithEntry(syncCommsSubnetEnrKey, &bitVSync)
s.dv5Listener.LocalNode().Set(entry)
s.dv5Listener.LocalNode().Set(subEntry)
localNode := s.dv5Listener.LocalNode()
localNode.Set(entry)
localNode.Set(subEntry)
if params.FuluEnabled() {
custodyGroupCountEntry := enr.WithEntry(custodyGroupCountEnrKey, custodyGroupCount)
localNode.Set(custodyGroupCountEntry)
}
s.metaData = wrapper.WrappedMetadataV1(&pb.MetaDataV1{
SeqNumber: s.metaData.SequenceNumber() + 1,
Attnets: bitVAtt,
@@ -421,10 +433,8 @@ func (s *Service) updateSubnetRecordWithMetadataV3(
localNode.Set(syncSubnetsEntry)
localNode.Set(custodyGroupCountEntry)
newSeqNumber := s.metaData.SequenceNumber() + 1
s.metaData = wrapper.WrappedMetadataV2(&pb.MetaDataV2{
SeqNumber: newSeqNumber,
SeqNumber: s.metaData.SequenceNumber() + 1,
Attnets: bitVAtt,
Syncnets: bitVSync,
CustodyGroupCount: custodyGroupCount,

View File

@@ -74,6 +74,7 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) {
cfg: &Config{UDPPort: 2000, TCPPort: 3000, QUICPort: 3000, DisableLivenessCheck: true, PingInterval: testPingInterval},
genesisTime: genesisTime,
genesisValidatorsRoot: genesisValidatorsRoot,
custodyInfo: &custodyInfo{},
}
bootNodeForkDigest, err := bootNodeService.currentForkDigest()
@@ -108,6 +109,7 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) {
service.genesisTime = genesisTime
service.genesisValidatorsRoot = genesisValidatorsRoot
service.custodyInfo = &custodyInfo{}
nodeForkDigest, err := service.currentForkDigest()
require.NoError(t, err)
@@ -157,6 +159,7 @@ func TestStartDiscV5_FindAndDialPeersWithSubnet(t *testing.T) {
service.genesisTime = genesisTime
service.genesisValidatorsRoot = genesisValidatorsRoot
service.custodyInfo = &custodyInfo{}
service.Start()
defer func() {

View File

@@ -15,6 +15,7 @@ go_library(
importpath = "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing",
visibility = [
"//beacon-chain:__subpackages__",
"//testing:__subpackages__",
],
deps = [
"//beacon-chain/core/peerdas:go_default_library",
@@ -24,6 +25,7 @@ go_library(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/metadata:go_default_library",
"//testing/require:go_default_library",

View File

@@ -7,6 +7,7 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1/metadata"
"github.com/ethereum/go-ethereum/p2p/enode"
@@ -196,6 +197,22 @@ func (*FakeP2P) InterceptUpgraded(network.Conn) (allow bool, reason control.Disc
return true, 0
}
// EarliestAvailableSlot -- fake.
func (*FakeP2P) EarliestAvailableSlot() (primitives.Slot, error) {
return 0, nil
}
// CustodyGroupCount -- fake.
func (*FakeP2P) CustodyGroupCount() (uint64, error) {
return 0, nil
}
// UpdateCustodyInfo -- fake.
func (s *FakeP2P) UpdateCustodyInfo(earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error) {
return earliestAvailableSlot, custodyGroupCount, nil
}
// CustodyGroupCountFromPeer -- fake.
func (*FakeP2P) CustodyGroupCountFromPeer(peer.ID) uint64 {
return 0
}

View File

@@ -65,7 +65,7 @@ func (m *MockPeersProvider) Peers() *peers.Status {
}
m.peers.Add(createENR(), id0, ma0, network.DirInbound)
m.peers.SetConnectionState(id0, peers.Connected)
m.peers.SetChainState(id0, &pb.Status{FinalizedEpoch: 10})
m.peers.SetChainState(id0, &pb.StatusV2{FinalizedEpoch: 10})
id1, err := peer.Decode(MockRawPeerId1)
if err != nil {
log.WithError(err).Debug("Cannot decode")
@@ -76,7 +76,7 @@ func (m *MockPeersProvider) Peers() *peers.Status {
}
m.peers.Add(createENR(), id1, ma1, network.DirOutbound)
m.peers.SetConnectionState(id1, peers.Connected)
m.peers.SetChainState(id1, &pb.Status{FinalizedEpoch: 11})
m.peers.SetChainState(id1, &pb.StatusV2{FinalizedEpoch: 11})
}
return m.peers
}

View File

@@ -6,6 +6,7 @@ import (
"bytes"
"context"
"fmt"
"sync"
"sync/atomic"
"testing"
"time"
@@ -17,6 +18,7 @@ import (
fieldparams "github.com/OffchainLabs/prysm/v6/config/fieldparams"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1/metadata"
"github.com/OffchainLabs/prysm/v6/testing/require"
@@ -48,16 +50,19 @@ const (
// TestP2P represents a p2p implementation that can be used for testing.
type TestP2P struct {
t *testing.T
BHost host.Host
EnodeID enode.ID
pubsub *pubsub.PubSub
joinedTopics map[string]*pubsub.Topic
BroadcastCalled atomic.Bool
DelaySend bool
Digest [4]byte
peers *peers.Status
LocalMetadata metadata.Metadata
t *testing.T
BHost host.Host
EnodeID enode.ID
pubsub *pubsub.PubSub
joinedTopics map[string]*pubsub.Topic
BroadcastCalled atomic.Bool
DelaySend bool
Digest [4]byte
peers *peers.Status
LocalMetadata metadata.Metadata
custodyInfoMut sync.RWMutex // protects custodyGroupCount and earliestAvailableSlot
earliestAvailableSlot primitives.Slot
custodyGroupCount uint64
}
// NewTestP2P initializes a new p2p test service.
@@ -461,6 +466,34 @@ func (*TestP2P) InterceptUpgraded(network.Conn) (allow bool, reason control.Disc
return true, 0
}
// EarliestAvailableSlot .
func (s *TestP2P) EarliestAvailableSlot() (primitives.Slot, error) {
s.custodyInfoMut.RLock()
defer s.custodyInfoMut.RUnlock()
return s.earliestAvailableSlot, nil
}
// CustodyGroupCount .
func (s *TestP2P) CustodyGroupCount() (uint64, error) {
s.custodyInfoMut.RLock()
defer s.custodyInfoMut.RUnlock()
return s.custodyGroupCount, nil
}
// UpdateCustodyInfo .
func (s *TestP2P) UpdateCustodyInfo(earliestAvailableSlot primitives.Slot, custodyGroupCount uint64) (primitives.Slot, uint64, error) {
s.custodyInfoMut.Lock()
defer s.custodyInfoMut.Unlock()
s.earliestAvailableSlot = earliestAvailableSlot
s.custodyGroupCount = custodyGroupCount
return s.earliestAvailableSlot, s.custodyGroupCount, nil
}
// CustodyGroupCountFromPeer .
func (s *TestP2P) CustodyGroupCountFromPeer(pid peer.ID) uint64 {
// By default, we assume the peer custodies the minimum number of groups.
custodyRequirement := params.BeaconConfig().CustodyRequirement

View File

@@ -109,6 +109,8 @@ func (ds *Server) getPeer(pid peer.ID) (*ethpb.DebugPeerResponse, error) {
peerInfo.MetadataV0 = metadata.MetadataObjV0()
case metadata.MetadataObjV1() != nil:
peerInfo.MetadataV1 = metadata.MetadataObjV1()
case metadata.MetadataObjV2() != nil:
peerInfo.MetadataV2 = metadata.MetadataObjV2()
}
}
addresses := peerStore.Addrs(pid)
@@ -127,7 +129,7 @@ func (ds *Server) getPeer(pid peer.ID) (*ethpb.DebugPeerResponse, error) {
if err != nil {
// In the event chain state is non existent, we
// initialize with the zero value.
pStatus = new(ethpb.Status)
pStatus = new(ethpb.StatusV2)
}
lastUpdated, err := peers.ChainStateLastUpdated(pid)
if err != nil {
@@ -150,6 +152,16 @@ func (ds *Server) getPeer(pid peer.ID) (*ethpb.DebugPeerResponse, error) {
BehaviourPenalty: float32(bPenalty),
ValidationError: errorToString(peers.Scorers().ValidationError(pid)),
}
// Convert statusV2 into status
peerStatus := &ethpb.Status{
ForkDigest: pStatus.ForkDigest,
FinalizedRoot: pStatus.FinalizedRoot,
FinalizedEpoch: pStatus.FinalizedEpoch,
HeadRoot: pStatus.HeadRoot,
HeadSlot: pStatus.HeadSlot,
}
return &ethpb.DebugPeerResponse{
ListeningAddresses: stringAddrs,
Direction: pbDirection,
@@ -157,7 +169,7 @@ func (ds *Server) getPeer(pid peer.ID) (*ethpb.DebugPeerResponse, error) {
PeerId: pid.String(),
Enr: enr,
PeerInfo: peerInfo,
PeerStatus: pStatus,
PeerStatus: peerStatus,
LastUpdated: unixTime,
ScoreInfo: scoreInfo,
}, nil

View File

@@ -259,7 +259,7 @@ func (s *State) latestAncestor(ctx context.Context, blockRoot [32]byte) (state.B
defer span.End()
if s.isFinalizedRoot(blockRoot) {
finalizedState := s.finalizedState()
finalizedState := s.FinalizedState()
if finalizedState != nil {
return finalizedState, nil
}
@@ -297,7 +297,7 @@ func (s *State) latestAncestor(ctx context.Context, blockRoot [32]byte) (state.B
// Does the state exist in finalized info cache.
if s.isFinalizedRoot(parentRoot) {
return s.finalizedState(), nil
return s.FinalizedState(), nil
}
// Does the state exist in epoch boundary cache.

View File

@@ -196,7 +196,7 @@ func (s *State) isFinalizedRoot(r [32]byte) bool {
}
// Returns the cached and copied finalized state.
func (s *State) finalizedState() state.BeaconState {
func (s *State) FinalizedState() state.BeaconState {
s.finalizedInfo.lock.RLock()
defer s.finalizedInfo.lock.RUnlock()
return s.finalizedInfo.state.Copy()

View File

@@ -32,5 +32,5 @@ func TestResume(t *testing.T) {
require.DeepSSZEqual(t, beaconState.ToProtoUnsafe(), resumeState.ToProtoUnsafe())
assert.Equal(t, params.BeaconConfig().SlotsPerEpoch, service.finalizedInfo.slot, "Did not get watned slot")
assert.Equal(t, service.finalizedInfo.root, root, "Did not get wanted root")
assert.NotNil(t, service.finalizedState(), "Wanted a non nil finalized state")
assert.NotNil(t, service.FinalizedState(), "Wanted a non nil finalized state")
}

View File

@@ -7,6 +7,7 @@ go_library(
"block_batcher.go",
"broadcast_bls_changes.go",
"context.go",
"custody.go",
"data_columns_reconstruct.go",
"deadlines.go",
"decode_pubsub.go",
@@ -166,6 +167,7 @@ go_test(
"block_batcher_test.go",
"broadcast_bls_changes_test.go",
"context_test.go",
"custody_test.go",
"data_columns_reconstruct_test.go",
"decode_pubsub_test.go",
"error_test.go",

View File

@@ -0,0 +1,130 @@
package sync
import (
"strings"
"time"
"github.com/OffchainLabs/prysm/v6/async"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
var nilFinalizedStateError = errors.New("finalized state is nil")
func (s *Service) maintainCustodyInfo() {
const interval = 1 * time.Minute
async.RunEvery(s.ctx, interval, func() {
if err := s.updateCustodyInfoIfNeeded(); err != nil {
log.WithError(err).Error("Failed to update custody info")
}
})
}
func (s *Service) updateCustodyInfoIfNeeded() error {
const minimumPeerCount = 1
// Get our actual custody group count.
actualCustodyGrounpCount, err := s.cfg.p2p.CustodyGroupCount()
if err != nil {
return errors.Wrap(err, "p2p custody group count")
}
// Get our target custody group count.
targetCustodyGroupCount, err := s.custodyGroupCount()
if err != nil {
return errors.Wrap(err, "custody group count")
}
// If the actual custody group count is already equal to the target, skip the update.
if actualCustodyGrounpCount >= targetCustodyGroupCount {
return nil
}
// Check that all subscribed data column sidecars topics have at least `minimumPeerCount` peers.
topics := s.cfg.p2p.PubSub().GetTopics()
enoughPeers := true
for _, topic := range topics {
if !strings.Contains(topic, p2p.GossipDataColumnSidecarMessage) {
continue
}
if peers := s.cfg.p2p.PubSub().ListPeers(topic); len(peers) < minimumPeerCount {
// If a topic has fewer than the minimum required peers, log a warning.
log.WithFields(logrus.Fields{
"topic": topic,
"peerCount": len(peers),
"minimumPeerCount": minimumPeerCount,
}).Debug("Insufficient peers for data column sidecar topic to maintain custody count")
enoughPeers = false
}
}
if !enoughPeers {
return nil
}
headROBlock, err := s.cfg.chain.HeadBlock(s.ctx)
if err != nil {
return errors.Wrap(err, "head block")
}
headSlot := headROBlock.Block().Slot()
storedEarliestSlot, storedGroupCount, err := s.cfg.p2p.UpdateCustodyInfo(headSlot, targetCustodyGroupCount)
if err != nil {
return errors.Wrap(err, "p2p update custody info")
}
if _, _, err := s.cfg.beaconDB.UpdateCustodyInfo(s.ctx, storedEarliestSlot, storedGroupCount); err != nil {
return errors.Wrap(err, "beacon db update custody info")
}
return nil
}
// custodyGroupCount computes the custody group count based on the custody requirement,
// the validators custody requirement, and whether the node is subscribed to all data subnets.
func (s *Service) custodyGroupCount() (uint64, error) {
beaconConfig := params.BeaconConfig()
if flags.Get().SubscribeAllDataSubnets {
return beaconConfig.NumberOfCustodyGroups, nil
}
validatorsCustodyRequirement, err := s.validatorsCustodyRequirement()
if err != nil {
return 0, errors.Wrap(err, "validators custody requirement")
}
return max(beaconConfig.CustodyRequirement, validatorsCustodyRequirement), nil
}
// validatorsCustodyRequirements computes the custody requirements based on the
// finalized state and the tracked validators.
func (s *Service) validatorsCustodyRequirement() (uint64, error) {
// Get the indices of the tracked validators.
indices := s.trackedValidatorsCache.Indices()
// Return early if no validators are tracked.
if len(indices) == 0 {
return 0, nil
}
// Retrieve the finalized state.
finalizedState := s.cfg.stateGen.FinalizedState()
if finalizedState == nil || finalizedState.IsNil() {
return 0, nilFinalizedStateError
}
// Compute the validators custody requirements.
result, err := peerdas.ValidatorsCustodyRequirement(finalizedState, indices)
if err != nil {
return 0, errors.Wrap(err, "validators custody requirements")
}
return result, nil
}

View File

@@ -0,0 +1,195 @@
package sync
import (
"context"
"strings"
"testing"
"time"
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
dbtesting "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v6/config/params"
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/primitives"
eth "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/testing/require"
)
type testSetup struct {
service *Service
p2pService *p2ptest.TestP2P
beaconDB db.Database
ctx context.Context
initialSlot primitives.Slot
initialCount uint64
}
func setupCustodyTest(t *testing.T, withChain bool) *testSetup {
ctx := t.Context()
p2pService := p2ptest.NewTestP2P(t)
beaconDB := dbtesting.SetupDB(t)
const (
initialEarliestSlot = primitives.Slot(50)
initialCustodyCount = uint64(5)
)
_, _, err := p2pService.UpdateCustodyInfo(initialEarliestSlot, initialCustodyCount)
require.NoError(t, err)
dbEarliestAvailableSlot, dbCustodyCount, err := beaconDB.UpdateCustodyInfo(ctx, initialEarliestSlot, initialCustodyCount)
require.NoError(t, err)
require.Equal(t, initialEarliestSlot, dbEarliestAvailableSlot)
require.Equal(t, initialCustodyCount, dbCustodyCount)
cfg := &config{
p2p: p2pService,
beaconDB: beaconDB,
}
if withChain {
const headSlot = primitives.Slot(100)
block, err := blocks.NewSignedBeaconBlock(&eth.SignedBeaconBlock{
Block: &eth.BeaconBlock{
Body: &eth.BeaconBlockBody{},
Slot: headSlot,
},
})
require.NoError(t, err)
cfg.chain = &mock.ChainService{
Genesis: time.Now(),
ValidAttestation: true,
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
},
Block: block,
}
}
service := &Service{
ctx: ctx,
cfg: cfg,
trackedValidatorsCache: cache.NewTrackedValidatorsCache(),
}
return &testSetup{
service: service,
p2pService: p2pService,
beaconDB: beaconDB,
ctx: ctx,
initialSlot: initialEarliestSlot,
initialCount: initialCustodyCount,
}
}
func (ts *testSetup) assertCustodyInfo(t *testing.T, expectedSlot primitives.Slot, expectedCount uint64) {
p2pEarliestSlot, err := ts.p2pService.EarliestAvailableSlot()
require.NoError(t, err)
require.Equal(t, expectedSlot, p2pEarliestSlot)
p2pCustodyCount, err := ts.p2pService.CustodyGroupCount()
require.NoError(t, err)
require.Equal(t, expectedCount, p2pCustodyCount)
dbEarliestSlot, dbCustodyCount, err := ts.beaconDB.UpdateCustodyInfo(ts.ctx, 0, 0)
require.NoError(t, err)
require.Equal(t, expectedSlot, dbEarliestSlot)
require.Equal(t, expectedCount, dbCustodyCount)
}
func withSubscribeAllDataSubnets(t *testing.T, fn func()) {
originalFlag := flags.Get().SubscribeAllDataSubnets
defer func() {
flags.Get().SubscribeAllDataSubnets = originalFlag
}()
flags.Get().SubscribeAllDataSubnets = true
fn()
}
func TestUpdateCustodyInfoIfNeeded(t *testing.T) {
params.SetupTestConfigCleanup(t)
beaconConfig := params.BeaconConfig()
beaconConfig.NumberOfCustodyGroups = 128
beaconConfig.CustodyRequirement = 4
beaconConfig.SamplesPerSlot = 8
params.OverrideBeaconConfig(beaconConfig)
t.Run("Skip update when actual custody count >= target", func(t *testing.T) {
setup := setupCustodyTest(t, false)
err := setup.service.updateCustodyInfoIfNeeded()
require.NoError(t, err)
setup.assertCustodyInfo(t, setup.initialSlot, setup.initialCount)
})
t.Run("not enough peers in some subnets", func(t *testing.T) {
const randomTopic = "aTotalRandomTopicName"
require.Equal(t, false, strings.Contains(randomTopic, p2p.GossipDataColumnSidecarMessage))
withSubscribeAllDataSubnets(t, func() {
setup := setupCustodyTest(t, false)
_, err := setup.service.cfg.p2p.SubscribeToTopic(p2p.GossipDataColumnSidecarMessage)
require.NoError(t, err)
_, err = setup.service.cfg.p2p.SubscribeToTopic(randomTopic)
require.NoError(t, err)
err = setup.service.updateCustodyInfoIfNeeded()
require.NoError(t, err)
setup.assertCustodyInfo(t, setup.initialSlot, setup.initialCount)
})
})
t.Run("should update", func(t *testing.T) {
withSubscribeAllDataSubnets(t, func() {
setup := setupCustodyTest(t, true)
err := setup.service.updateCustodyInfoIfNeeded()
require.NoError(t, err)
const expectedSlot = primitives.Slot(100)
setup.assertCustodyInfo(t, expectedSlot, beaconConfig.NumberOfCustodyGroups)
})
})
}
func TestCustodyGroupCount(t *testing.T) {
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig()
config.NumberOfCustodyGroups = 10
config.CustodyRequirement = 3
params.OverrideBeaconConfig(config)
t.Run("SubscribeAllDataSubnets enabled returns NumberOfCustodyGroups", func(t *testing.T) {
withSubscribeAllDataSubnets(t, func() {
service := &Service{
ctx: context.Background(),
}
result, err := service.custodyGroupCount()
require.NoError(t, err)
require.Equal(t, config.NumberOfCustodyGroups, result)
})
})
t.Run("No tracked validators returns CustodyRequirement", func(t *testing.T) {
service := &Service{
ctx: context.Background(),
trackedValidatorsCache: cache.NewTrackedValidatorsCache(),
}
result, err := service.custodyGroupCount()
require.NoError(t, err)
require.Equal(t, config.CustodyRequirement, result)
})
}

View File

@@ -32,6 +32,7 @@ func (s *Service) reconstructSaveBroadcastDataColumnSidecars(
root [fieldparams.RootLength]byte,
) error {
startTime := time.Now()
samplesPerSlot := params.BeaconConfig().SamplesPerSlot
// Lock to prevent concurrent reconstructions.
s.reconstructionLock.Lock()
@@ -49,8 +50,13 @@ func (s *Service) reconstructSaveBroadcastDataColumnSidecars(
// Retrieve our local node info.
nodeID := s.cfg.p2p.NodeID()
custodyGroupCount := s.cfg.custodyInfo.ActualGroupCount()
localNodeInfo, _, err := peerdas.Info(nodeID, custodyGroupCount)
custodyGroupCount, err := s.cfg.p2p.CustodyGroupCount()
if err != nil {
return errors.Wrap(err, "custody group count")
}
samplingSize := max(custodyGroupCount, samplesPerSlot)
localNodeInfo, _, err := peerdas.Info(nodeID, samplingSize)
if err != nil {
return errors.Wrap(err, "peer info")
}
@@ -155,10 +161,12 @@ func (s *Service) broadcastMissingDataColumnSidecars(
// Get the node ID.
nodeID := s.cfg.p2p.NodeID()
// Get the custody group count.
custodyGroupCount := s.cfg.custodyInfo.ActualGroupCount()
// Retrieve the local node info.
custodyGroupCount, err := s.cfg.p2p.CustodyGroupCount()
if err != nil {
return errors.Wrap(err, "custody group count")
}
localNodeInfo, _, err := peerdas.Info(nodeID, custodyGroupCount)
if err != nil {
return errors.Wrap(err, "peerdas info")

View File

@@ -65,15 +65,10 @@ func TestReconstructDataColumns(t *testing.T) {
err := storage.Save(verifiedRoDataColumns[:minimumCount])
require.NoError(t, err)
custodyInfo := &peerdas.CustodyInfo{}
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(cgc)
custodyInfo.ToAdvertiseGroupCount.Set(cgc)
service := NewService(
ctx,
WithP2P(p2ptest.NewTestP2P(t)),
WithDataColumnStorage(storage),
WithCustodyInfo(custodyInfo),
WithChainService(&mockChain.ChainService{}),
)
@@ -121,14 +116,9 @@ func TestBroadcastMissingDataColumnSidecars(t *testing.T) {
slot, proposerIndex := block.Slot(), block.ProposerIndex()
t.Run("no missing sidecars", func(t *testing.T) {
custodyInfo := &peerdas.CustodyInfo{}
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(cgc)
custodyInfo.ToAdvertiseGroupCount.Set(cgc)
service := NewService(
ctx,
WithP2P(p2ptest.NewTestP2P(t)),
WithCustodyInfo(custodyInfo),
)
for _, index := range [...]uint64{1, 17, 19, 42, 75, 87, 102, 117} {
@@ -141,10 +131,6 @@ func TestBroadcastMissingDataColumnSidecars(t *testing.T) {
})
t.Run("some missing sidecars", func(t *testing.T) {
custodyInfo := &peerdas.CustodyInfo{}
custodyInfo.TargetGroupCount.SetValidatorsCustodyRequirement(cgc)
custodyInfo.ToAdvertiseGroupCount.Set(cgc)
toSave := make([]blocks.VerifiedRODataColumn, 0, 2)
for _, index := range [...]uint64{42, 87} {
toSave = append(toSave, verifiedRoDataColumns[index])
@@ -158,9 +144,10 @@ func TestBroadcastMissingDataColumnSidecars(t *testing.T) {
service := NewService(
ctx,
WithP2P(p2p),
WithCustodyInfo(custodyInfo),
WithDataColumnStorage(storage),
)
_, _, err = service.cfg.p2p.UpdateCustodyInfo(0, cgc)
require.NoError(t, err)
for _, index := range [...]uint64{1, 17, 19, 102, 117} { // 42, 75 and 87 are missing
key := computeCacheKey(slot, proposerIndex, index)
@@ -186,6 +173,5 @@ func TestBroadcastMissingDataColumnSidecars(t *testing.T) {
}
require.Equal(t, true, p2p.BroadcastCalled.Load())
})
}

View File

@@ -7,7 +7,7 @@ import (
"github.com/OffchainLabs/prysm/v6/async/abool"
mockChain "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/cache"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptest "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
@@ -47,7 +47,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
chain: chainService,
clock: startup.NewClock(gt, vr),
initialSync: &mockSync.Sync{IsSyncing: false},
custodyInfo: &peerdas.CustodyInfo{},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -83,7 +82,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
chain: chainService,
clock: startup.NewClock(gt, vr),
initialSync: &mockSync.Sync{IsSyncing: false},
custodyInfo: &peerdas.CustodyInfo{},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -128,7 +126,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
chain: chainService,
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
initialSync: &mockSync.Sync{IsSyncing: false},
custodyInfo: &peerdas.CustodyInfo{},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -171,7 +168,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
chain: chainService,
clock: startup.NewClock(gt, vr),
initialSync: &mockSync.Sync{IsSyncing: false},
custodyInfo: &peerdas.CustodyInfo{},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -216,7 +212,6 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
chain: chainService,
clock: startup.NewClock(gt, vr),
initialSync: &mockSync.Sync{IsSyncing: false},
custodyInfo: &peerdas.CustodyInfo{},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
@@ -261,10 +256,10 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
chain: chainService,
clock: startup.NewClock(gt, vr),
initialSync: &mockSync.Sync{IsSyncing: false},
custodyInfo: &peerdas.CustodyInfo{},
},
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
chainStarted: abool.New(),
subHandler: newSubTopicHandler(),
trackedValidatorsCache: cache.NewTrackedValidatorsCache(),
}
return r
},

View File

@@ -387,7 +387,7 @@ func TestBlocksFetcher_findForkWithPeer(t *testing.T) {
defer func() {
assert.NoError(t, p1.Disconnect(p2.PeerID()))
}()
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{
p1.Peers().SetChainState(p2.PeerID(), &ethpb.StatusV2{
HeadRoot: nil,
HeadSlot: 0,
})

View File

@@ -232,7 +232,7 @@ func connectPeer(t *testing.T, host *p2pt.TestP2P, datum *peerData, peerStatus *
peerStatus.Add(new(enr.Record), p.PeerID(), nil, network.DirOutbound)
peerStatus.SetConnectionState(p.PeerID(), peers.Connected)
peerStatus.SetChainState(p.PeerID(), &ethpb.Status{
peerStatus.SetChainState(p.PeerID(), &ethpb.StatusV2{
ForkDigest: params.BeaconConfig().GenesisForkVersion,
FinalizedRoot: []byte(fmt.Sprintf("finalized_root %d", datum.finalizedEpoch)),
FinalizedEpoch: datum.finalizedEpoch,
@@ -331,7 +331,7 @@ func connectPeerHavingBlocks(
peerStatus.Add(new(enr.Record), p.PeerID(), nil, network.DirOutbound)
peerStatus.SetConnectionState(p.PeerID(), peers.Connected)
peerStatus.SetChainState(p.PeerID(), &ethpb.Status{
peerStatus.SetChainState(p.PeerID(), &ethpb.StatusV2{
ForkDigest: params.BeaconConfig().GenesisForkVersion,
FinalizedRoot: []byte(fmt.Sprintf("finalized_root %d", finalizedEpoch)),
FinalizedEpoch: finalizedEpoch,

View File

@@ -7,7 +7,6 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/operation"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/execution"
@@ -199,10 +198,10 @@ func WithAvailableBlocker(avb coverage.AvailableBlocker) Option {
}
}
// WithCustodyInfo for custody info.
func WithCustodyInfo(custodyInfo *peerdas.CustodyInfo) Option {
// WithTrackedValidatorsCache for tracked validators cache.
func WithTrackedValidatorsCache(c *cache.TrackedValidatorsCache) Option {
return func(s *Service) error {
s.cfg.custodyInfo = custodyInfo
s.trackedValidatorsCache = c
return nil
}
}

View File

@@ -51,7 +51,7 @@ func TestProcessPendingAtts_NoBlockRequestBlock(t *testing.T) {
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected")
p1.Peers().Add(new(enr.Record), p2.PeerID(), nil, network.DirOutbound)
p1.Peers().SetConnectionState(p2.PeerID(), peers.Connected)
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{})
p1.Peers().SetChainState(p2.PeerID(), &ethpb.StatusV2{})
chain := &mock.ChainService{Genesis: prysmTime.Now(), FinalizedCheckPoint: &ethpb.Checkpoint{}}
r := &Service{

View File

@@ -406,7 +406,7 @@ func TestRegularSyncBeaconBlockSubscriber_ProcessPendingBlocks_2Chains(t *testin
p1.Peers().Add(new(enr.Record), p2.PeerID(), nil, network.DirOutbound)
p1.Peers().SetConnectionState(p2.PeerID(), peers.Connected)
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{})
p1.Peers().SetChainState(p2.PeerID(), &ethpb.StatusV2{})
b0 := util.NewBeaconBlock()
util.SaveBlock(t, t.Context(), r.cfg.beaconDB, b0)
@@ -505,7 +505,7 @@ func TestRegularSyncBeaconBlockSubscriber_PruneOldPendingBlocks(t *testing.T) {
p1.Peers().Add(new(enr.Record), p1.PeerID(), nil, network.DirOutbound)
p1.Peers().SetConnectionState(p1.PeerID(), peers.Connected)
p1.Peers().SetChainState(p1.PeerID(), &ethpb.Status{})
p1.Peers().SetChainState(p1.PeerID(), &ethpb.StatusV2{})
b0 := util.NewBeaconBlock()
util.SaveBlock(t, t.Context(), r.cfg.beaconDB, b0)
@@ -611,7 +611,7 @@ func TestService_BatchRootRequest(t *testing.T) {
p1.Peers().Add(new(enr.Record), p2.PeerID(), nil, network.DirOutbound)
p1.Peers().SetConnectionState(p2.PeerID(), peers.Connected)
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{FinalizedEpoch: 2})
p1.Peers().SetChainState(p2.PeerID(), &ethpb.StatusV2{FinalizedEpoch: 2})
b0 := util.NewBeaconBlock()
util.SaveBlock(t, t.Context(), r.cfg.beaconDB, b0)

View File

@@ -64,6 +64,7 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
topicMap[addEncoding(p2p.RPCPingTopicV1)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */)
// Status Message
topicMap[addEncoding(p2p.RPCStatusTopicV1)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */)
topicMap[addEncoding(p2p.RPCStatusTopicV2)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */)
// Use a single collector for block requests
blockCollector := leakybucket.NewCollector(allowedBlocksPerSecond, allowedBlocksBurst, blockBucketPeriod, false /* deleteEmptyBuckets */)

View File

@@ -17,7 +17,7 @@ import (
func TestNewRateLimiter(t *testing.T) {
rlimiter := newRateLimiter(mockp2p.NewTestP2P(t))
assert.Equal(t, len(rlimiter.limiterMap), 19, "correct number of topics not registered")
assert.Equal(t, len(rlimiter.limiterMap), 20, "correct number of topics not registered")
}
func TestNewRateLimiter_FreeCorrectly(t *testing.T) {

View File

@@ -43,13 +43,14 @@ func (s *Service) rpcHandlerByTopicFromFork(forkIndex int) (map[string]rpcHandle
// Fulu: https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/p2p-interface.md#messages
if forkIndex >= version.Fulu {
return map[string]rpcHandler{
p2p.RPCStatusTopicV2: s.statusRPCHandler, // Updated in Fulu
p2p.RPCGoodByeTopicV1: s.goodbyeRPCHandler,
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler,
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler,
p2p.RPCPingTopicV1: s.pingHandler,
p2p.RPCMetaDataTopicV3: s.metaDataHandler, // Modified in Fulu
p2p.RPCBlobSidecarsByRootTopicV1: s.blobSidecarByRootRPCHandler,
p2p.RPCBlobSidecarsByRangeTopicV1: s.blobSidecarsByRangeRPCHandler,
p2p.RPCMetaDataTopicV3: s.metaDataHandler, // Updated in Fulu
p2p.RPCBlobSidecarsByRootTopicV1: s.blobSidecarByRootRPCHandler, // Modified in Fulu
p2p.RPCBlobSidecarsByRangeTopicV1: s.blobSidecarsByRangeRPCHandler, // Modified in Fulu
p2p.RPCDataColumnSidecarsByRootTopicV1: s.dataColumnSidecarByRootRPCHandler, // Added in Fulu
p2p.RPCDataColumnSidecarsByRangeTopicV1: s.dataColumnSidecarsByRangeRPCHandler, // Added in Fulu
}, nil
@@ -60,8 +61,8 @@ func (s *Service) rpcHandlerByTopicFromFork(forkIndex int) (map[string]rpcHandle
return map[string]rpcHandler{
p2p.RPCStatusTopicV1: s.statusRPCHandler,
p2p.RPCGoodByeTopicV1: s.goodbyeRPCHandler,
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler,
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler,
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler, // Modified in Electra
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler, // Modified in Electra
p2p.RPCPingTopicV1: s.pingHandler,
p2p.RPCMetaDataTopicV2: s.metaDataHandler,
p2p.RPCBlobSidecarsByRootTopicV1: s.blobSidecarByRootRPCHandler, // Modified in Electra
@@ -90,8 +91,8 @@ func (s *Service) rpcHandlerByTopicFromFork(forkIndex int) (map[string]rpcHandle
handler := map[string]rpcHandler{
p2p.RPCStatusTopicV1: s.statusRPCHandler,
p2p.RPCGoodByeTopicV1: s.goodbyeRPCHandler,
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler, // Updated in Altair and modified in Capella
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler, // Updated in Altair and modified in Capella
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler, // Updated in Altair and modified in Bellatrix and Capella
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler, // Updated in Altair and modified in Bellatrix and Capella
p2p.RPCPingTopicV1: s.pingHandler,
p2p.RPCMetaDataTopicV2: s.metaDataHandler, // Updated in Altair
}

View File

@@ -394,7 +394,7 @@ func TestRequestPendingBlobs(t *testing.T) {
}
p1.Peers().Add(new(enr.Record), p2.PeerID(), nil, network.DirOutbound)
p1.Peers().SetConnectionState(p2.PeerID(), peers.Connected)
p1.Peers().SetChainState(p2.PeerID(), &ethpb.Status{FinalizedEpoch: 1})
p1.Peers().SetChainState(p2.PeerID(), &ethpb.StatusV2{FinalizedEpoch: 1})
s := &Service{
cfg: &config{
p2p: p1,

View File

@@ -10,6 +10,7 @@ import (
"github.com/OffchainLabs/prysm/v6/async"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/peers"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
p2ptypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/cmd/beacon-chain/flags"
"github.com/OffchainLabs/prysm/v6/config/params"
@@ -22,6 +23,7 @@ import (
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/sirupsen/logrus"
)
@@ -136,7 +138,7 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, peer peer.ID) error
headRoot, err := s.cfg.chain.HeadRoot(ctx)
if err != nil {
return errors.Wrap(err, "head root")
return errors.Wrap(err, "chain head root")
}
forkDigest, err := s.currentForkDigest()
@@ -144,26 +146,24 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, peer peer.ID) error
return errors.Wrap(err, "current fork digest")
}
cp := s.cfg.chain.FinalizedCheckpt()
resp := &pb.Status{
ForkDigest: forkDigest[:],
FinalizedRoot: cp.Root,
FinalizedEpoch: cp.Epoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.chain.HeadSlot(),
}
// Compute the current epoch.
currentSlot := s.cfg.clock.CurrentSlot()
currentEpoch := slots.ToEpoch(currentSlot)
log := log.WithField("peer", peer)
topic, err := p2p.TopicFromMessage(p2p.StatusMessageName, slots.ToEpoch(s.cfg.clock.CurrentSlot()))
// Compute the topic for the status request regarding the current epoch.
topic, err := p2p.TopicFromMessage(p2p.StatusMessageName, currentEpoch)
if err != nil {
return errors.Wrap(err, "topic from message")
}
stream, err := s.cfg.p2p.Send(ctx, resp, topic, peer)
cp := s.cfg.chain.FinalizedCheckpt()
status := s.buildStatusFromEpoch(currentEpoch, forkDigest, cp.Root, cp.Epoch, headRoot)
stream, err := s.cfg.p2p.Send(ctx, status, topic, peer)
if err != nil {
return errors.Wrap(err, "send p2p message")
return errors.Wrap(err, "p2p send")
}
defer closeStream(stream, log)
code, errMsg, err := ReadStatusCode(stream, s.cfg.p2p.Encoding())
@@ -171,16 +171,14 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, peer peer.ID) error
s.downscorePeer(peer, "statusRequestReadStatusCodeError")
return errors.Wrap(err, "read status code")
}
if code != 0 {
s.downscorePeer(peer, "statusRequestNonNullStatusCode")
return errors.New(errMsg)
}
msg := &pb.Status{}
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
s.downscorePeer(peer, "statusRequestDecodeError")
return errors.Wrap(err, "decode status message")
msg, err := s.decodeStatus(stream, currentEpoch)
if err != nil {
return errors.Wrap(err, "decode status")
}
// If validation fails, validation error is logged, and peer status scorer will mark peer as bad.
@@ -189,9 +187,35 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, peer peer.ID) error
if err := s.cfg.p2p.Peers().IsBad(peer); err != nil {
s.disconnectBadPeer(s.ctx, peer, err)
}
return err
}
func (s *Service) decodeStatus(stream network.Stream, epoch primitives.Epoch) (*pb.StatusV2, error) {
if epoch >= params.BeaconConfig().FuluForkEpoch {
msg := new(pb.StatusV2)
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
return nil, errors.Wrap(err, "decode with max length")
}
return msg, nil
}
msg := new(pb.Status)
if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil {
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
return nil, errors.Wrap(err, "decode with max length")
}
status, err := statusV2(msg)
if err != nil {
return nil, errors.Wrap(err, "status data")
}
return status, nil
}
func (s *Service) reValidatePeer(ctx context.Context, id peer.ID) error {
s.cfg.p2p.Peers().Scorers().PeerStatusScorer().SetHeadSlot(s.cfg.chain.HeadSlot())
if err := s.sendRPCStatusRequest(ctx, id); err != nil {
@@ -211,10 +235,11 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
defer cancel()
SetRPCStreamDeadlines(stream)
log := log.WithField("handler", "status")
m, ok := msg.(*pb.Status)
if !ok {
return errors.New("message is not type *pb.Status")
m, err := statusV2(msg)
if err != nil {
return errors.Wrap(err, "status data")
}
if err := s.rateLimiter.validateRequest(stream, 1); err != nil {
return err
}
@@ -275,30 +300,123 @@ func (s *Service) statusRPCHandler(ctx context.Context, msg interface{}, stream
func (s *Service) respondWithStatus(ctx context.Context, stream network.Stream) error {
headRoot, err := s.cfg.chain.HeadRoot(ctx)
if err != nil {
return err
return errors.Wrap(err, "chain head root")
}
forkDigest, err := s.currentForkDigest()
if err != nil {
return err
return errors.Wrap(err, "current fork digest")
}
cp := s.cfg.chain.FinalizedCheckpt()
resp := &pb.Status{
ForkDigest: forkDigest[:],
FinalizedRoot: cp.Root,
FinalizedEpoch: cp.Epoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.chain.HeadSlot(),
status, err := s.buildStatusFromStream(stream, forkDigest, cp.Root, cp.Epoch, headRoot)
if err != nil {
return errors.Wrap(err, "build status")
}
if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil && !isUnwantedError(err) {
log.WithError(err).Debug("Could not write to stream")
}
_, err = s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, resp)
return err
if _, err := s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, status); err != nil {
return errors.Wrap(err, "encode with max length")
}
return nil
}
func (s *Service) validateStatusMessage(ctx context.Context, msg *pb.Status) error {
func (s *Service) buildStatusFromStream(
stream libp2pcore.Stream,
forkDigest [4]byte,
finalizedRoot []byte,
FinalizedEpoch primitives.Epoch,
headRoot []byte,
) (ssz.Marshaler, error) {
// Get the stream version from the protocol.
_, _, streamVersion, err := p2p.TopicDeconstructor(string(stream.Protocol()))
if err != nil {
err := errors.Wrap(err, "topic deconstructor")
resp, err2 := s.generateErrorResponse(responseCodeServerError, types.ErrGeneric.Error())
if err2 != nil {
log.WithError(err2).Debug("Could not write to stream")
return nil, err
}
if _, err2 := stream.Write(resp); err2 != nil {
log.WithError(err2).Debug("Could not write to stream")
}
return nil, err
}
if streamVersion == p2p.SchemaVersionV2 {
earliestAvailableSlot, err := s.cfg.p2p.EarliestAvailableSlot()
if err != nil {
return nil, errors.Wrap(err, "earliest available slot")
}
status := &pb.StatusV2{
ForkDigest: forkDigest[:],
FinalizedRoot: finalizedRoot,
FinalizedEpoch: FinalizedEpoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.chain.HeadSlot(),
EarliestAvailableSlot: earliestAvailableSlot,
}
return status, nil
}
status := &pb.Status{
ForkDigest: forkDigest[:],
FinalizedRoot: finalizedRoot,
FinalizedEpoch: FinalizedEpoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.chain.HeadSlot(),
}
return status, nil
}
func (s *Service) buildStatusFromEpoch(
epoch primitives.Epoch,
forkDigest [4]byte,
finalizedRoot []byte,
FinalizedEpoch primitives.Epoch,
headRoot []byte,
) ssz.Marshaler {
// Get the stream version from the protocol.
if epoch >= params.BeaconConfig().FuluForkEpoch {
status := &pb.StatusV2{
ForkDigest: forkDigest[:],
FinalizedRoot: finalizedRoot,
FinalizedEpoch: FinalizedEpoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.chain.HeadSlot(),
EarliestAvailableSlot: 0,
}
return status
}
status := &pb.Status{
ForkDigest: forkDigest[:],
FinalizedRoot: finalizedRoot,
FinalizedEpoch: FinalizedEpoch,
HeadRoot: headRoot,
HeadSlot: s.cfg.chain.HeadSlot(),
}
return status
}
func (s *Service) validateStatusMessage(ctx context.Context, genericMsg interface{}) error {
msg, err := statusV2(genericMsg)
if err != nil {
return errors.Wrap(err, "status data")
}
forkDigest, err := s.currentForkDigest()
if err != nil {
return err
@@ -368,3 +486,24 @@ func (s *Service) validateStatusMessage(ctx context.Context, msg *pb.Status) err
}
return p2ptypes.ErrInvalidEpoch
}
func statusV2(msg interface{}) (*pb.StatusV2, error) {
if status, ok := msg.(*pb.StatusV2); ok {
return status, nil
}
if status, ok := msg.(*pb.Status); ok {
status := &pb.StatusV2{
ForkDigest: status.ForkDigest,
FinalizedRoot: status.FinalizedRoot,
FinalizedEpoch: status.FinalizedEpoch,
HeadRoot: status.HeadRoot,
HeadSlot: status.HeadSlot,
EarliestAvailableSlot: 0, // Default value for StatusV2
}
return status, nil
}
return nil, errors.New("message is not type *pb.Status or *pb.StatusV2")
}

View File

@@ -6,6 +6,8 @@ import (
"testing"
"time"
beaconState "github.com/OffchainLabs/prysm/v6/beacon-chain/state"
"github.com/OffchainLabs/prysm/v6/async/abool"
mock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/transition"
@@ -399,6 +401,7 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
if util.WaitTimeout(&wg, 1*time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
if util.WaitTimeout(&wg2, 1*time.Second) {
t.Fatal("Did not receive stream within 1 sec")
}
@@ -425,80 +428,148 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
}
func TestStatusRPCRequest_RequestSent(t *testing.T) {
p1 := p2ptest.NewTestP2P(t)
p2 := p2ptest.NewTestP2P(t)
beaconConfig := params.BeaconConfig()
// Set up a head state with data we expect.
head := util.NewBeaconBlock()
head.Block.Slot = 111
headRoot, err := head.Block.HashTreeRoot()
require.NoError(t, err)
finalized := util.NewBeaconBlock()
finalized.Block.Slot = 40
finalizedRoot, err := finalized.Block.HashTreeRoot()
require.NoError(t, err)
genesisState, err := transition.GenesisBeaconState(t.Context(), nil, 0, &ethpb.Eth1Data{})
require.NoError(t, err)
require.NoError(t, genesisState.SetSlot(111))
require.NoError(t, genesisState.UpdateBlockRootAtIndex(111%uint64(params.BeaconConfig().SlotsPerHistoricalRoot), headRoot))
finalizedCheckpt := &ethpb.Checkpoint{
Epoch: 5,
Root: finalizedRoot[:],
}
testCases := []struct {
name string
fuluForkEpoch primitives.Epoch
topic string
streamHandler func(service *Service, stream network.Stream, genesisState beaconState.BeaconState, beaconRoot, headRoot, finalizedRoot []byte)
}{
{
name: "before fulu",
fuluForkEpoch: beaconConfig.FarFutureEpoch,
topic: "/eth2/beacon_chain/req/status/1/ssz_snappy",
streamHandler: func(service *Service, stream network.Stream, genesisState beaconState.BeaconState, beaconRoot, headRoot, finalizedRoot []byte) {
out := &ethpb.Status{}
require.NoError(t, service.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
chain := &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
Fork: &ethpb.Fork{
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
CurrentVersion: params.BeaconConfig().GenesisForkVersion,
digest, err := service.currentForkDigest()
require.NoError(t, err)
expected := &ethpb.Status{
ForkDigest: digest[:],
HeadSlot: genesisState.Slot(),
HeadRoot: headRoot,
FinalizedEpoch: 5,
FinalizedRoot: finalizedRoot,
}
if !proto.Equal(out, expected) {
t.Errorf("Did not receive expected message. Got %+v wanted %+v", out, expected)
}
err = service.respondWithStatus(context.Background(), stream)
require.NoError(t, err)
},
},
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
}
r := &Service{
cfg: &config{
p2p: p1,
chain: chain,
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
{
name: "after fulu",
fuluForkEpoch: 0,
topic: "/eth2/beacon_chain/req/status/2/ssz_snappy",
streamHandler: func(service *Service, stream network.Stream, genesisState beaconState.BeaconState, beaconRoot, headRoot, finalizedRoot []byte) {
out := &ethpb.StatusV2{}
assert.NoError(t, service.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
digest, err := service.currentForkDigest()
assert.NoError(t, err)
expected := &ethpb.StatusV2{
ForkDigest: digest[:],
HeadSlot: genesisState.Slot(),
HeadRoot: headRoot,
FinalizedEpoch: 5,
FinalizedRoot: finalizedRoot,
EarliestAvailableSlot: 0,
}
if !proto.Equal(out, expected) {
t.Errorf("Did not receive expected message. Got %+v wanted %+v", out, expected)
}
err = service.respondWithStatus(t.Context(), stream)
require.NoError(t, err)
},
},
ctx: t.Context(),
rateLimiter: newRateLimiter(p1),
}
// Setup streams
pcl := protocol.ID("/eth2/beacon_chain/req/status/1/ssz_snappy")
topic := string(pcl)
r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(1, 1, time.Second, false)
var wg sync.WaitGroup
wg.Add(1)
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
out := &ethpb.Status{}
assert.NoError(t, r.cfg.p2p.Encoding().DecodeWithMaxLength(stream, out))
digest, err := r.currentForkDigest()
assert.NoError(t, err)
expected := &ethpb.Status{
ForkDigest: digest[:],
HeadSlot: genesisState.Slot(),
HeadRoot: headRoot[:],
FinalizedEpoch: 5,
FinalizedRoot: finalizedRoot[:],
}
if !proto.Equal(out, expected) {
t.Errorf("Did not receive expected message. Got %+v wanted %+v", out, expected)
}
})
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
params.SetupTestConfigCleanup(t)
cfg := params.BeaconConfig().Copy()
cfg.FuluForkEpoch = tc.fuluForkEpoch
cfg.ForkVersionSchedule[bytesutil.ToBytes4(cfg.FuluForkVersion)] = cfg.FuluForkEpoch
params.OverrideBeaconConfig(cfg)
p1.AddConnectionHandler(r.sendRPCStatusRequest, nil)
p1.Connect(p2)
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
p1.Connect(p2)
if util.WaitTimeout(&wg, 1*time.Second) {
t.Fatal("Did not receive stream within 1 sec")
// Set up a head state with data we expect.
head := util.NewBeaconBlock()
head.Block.Slot = 111
headRoot, err := head.Block.HashTreeRoot()
require.NoError(t, err)
finalized := util.NewBeaconBlock()
finalized.Block.Slot = 40
finalizedRoot, err := finalized.Block.HashTreeRoot()
require.NoError(t, err)
genesisState, err := transition.GenesisBeaconState(context.Background(), nil, 0, &ethpb.Eth1Data{})
require.NoError(t, err)
require.NoError(t, genesisState.SetSlot(111))
require.NoError(t, genesisState.UpdateBlockRootAtIndex(111%uint64(params.BeaconConfig().SlotsPerHistoricalRoot), headRoot))
finalizedCheckpt := &ethpb.Checkpoint{
Epoch: 5,
Root: finalizedRoot[:],
}
chain := &mock.ChainService{
State: genesisState,
FinalizedCheckPoint: finalizedCheckpt,
Root: headRoot[:],
Fork: &ethpb.Fork{
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
CurrentVersion: params.BeaconConfig().GenesisForkVersion,
},
Genesis: time.Now(),
ValidatorsRoot: [32]byte{'A'},
}
r := &Service{
cfg: &config{
p2p: p1,
chain: chain,
clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot),
},
ctx: context.Background(),
rateLimiter: newRateLimiter(p1),
}
// Setup streams
pcl := protocol.ID(tc.topic)
r.rateLimiter.limiterMap[tc.topic] = leakybucket.NewCollector(1, 1, time.Second, false)
var wg sync.WaitGroup
wg.Add(1)
p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) {
defer wg.Done()
tc.streamHandler(r, stream, genesisState, chain.Root, headRoot[:], finalizedRoot[:])
})
err = r.sendRPCStatusRequest(context.Background(), p2.BHost.ID())
require.ErrorIs(t, err, p2ptypes.ErrInvalidEpoch)
if util.WaitTimeout(&wg, 1*time.Hour) {
t.Fatal("Did not receive stream within 1 sec")
}
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to continue being connected")
})
}
assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to continue being connected")
}
func TestStatusRPCRequest_FinalizedBlockExists(t *testing.T) {

View File

@@ -9,19 +9,6 @@ import (
"sync"
"time"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
p2ptypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/crypto/rand"
lru "github.com/hashicorp/golang-lru"
pubsub "github.com/libp2p/go-libp2p-pubsub"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/peer"
gcache "github.com/patrickmn/go-cache"
"github.com/pkg/errors"
"github.com/trailofbits/go-mutexasserts"
"github.com/OffchainLabs/prysm/v6/async"
"github.com/OffchainLabs/prysm/v6/async/abool"
"github.com/OffchainLabs/prysm/v6/async/event"
@@ -30,6 +17,7 @@ import (
blockfeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/block"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/operation"
statefeed "github.com/OffchainLabs/prysm/v6/beacon-chain/core/feed/state"
lightClient "github.com/OffchainLabs/prysm/v6/beacon-chain/core/light-client"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
"github.com/OffchainLabs/prysm/v6/beacon-chain/execution"
@@ -39,6 +27,7 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/operations/synccommittee"
"github.com/OffchainLabs/prysm/v6/beacon-chain/operations/voluntaryexits"
"github.com/OffchainLabs/prysm/v6/beacon-chain/p2p"
p2ptypes "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/types"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state/stategen"
"github.com/OffchainLabs/prysm/v6/beacon-chain/sync/backfill/coverage"
@@ -48,11 +37,20 @@ import (
"github.com/OffchainLabs/prysm/v6/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v6/consensus-types/interfaces"
leakybucket "github.com/OffchainLabs/prysm/v6/container/leaky-bucket"
"github.com/OffchainLabs/prysm/v6/crypto/rand"
ethpb "github.com/OffchainLabs/prysm/v6/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v6/runtime"
prysmTime "github.com/OffchainLabs/prysm/v6/time"
"github.com/OffchainLabs/prysm/v6/time/slots"
lru "github.com/hashicorp/golang-lru"
pubsub "github.com/libp2p/go-libp2p-pubsub"
libp2pcore "github.com/libp2p/go-libp2p/core"
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/peer"
gcache "github.com/patrickmn/go-cache"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
"github.com/trailofbits/go-mutexasserts"
)
var _ runtime.Service = (*Service)(nil)
@@ -108,7 +106,6 @@ type config struct {
stateNotifier statefeed.Notifier
blobStorage *filesystem.BlobStorage
dataColumnStorage *filesystem.DataColumnStorage
custodyInfo *peerdas.CustodyInfo
batchVerifierLimit int
}
@@ -176,6 +173,7 @@ type Service struct {
availableBlocker coverage.AvailableBlocker
reconstructionLock sync.Mutex
reconstructionRandGen *rand.Rand
trackedValidatorsCache *cache.TrackedValidatorsCache
ctxMap ContextByteVersions
slasherEnabled bool
lcStore *lightClient.Store
@@ -265,9 +263,15 @@ func (s *Service) Start() {
return nil
})
s.cfg.p2p.AddPingMethod(s.sendPingRequest)
s.processPendingBlocksQueue()
s.processPendingAttsQueue()
s.maintainPeerStatuses()
if params.FuluEnabled() {
s.maintainCustodyInfo()
}
s.resyncIfBehind()
// Update sync metrics.

View File

@@ -624,9 +624,15 @@ func (s *Service) connectedPeersCount(subnetTopic string) int {
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)
samplingSize, err := s.samplingSize()
if err != nil {
log.WithError(err).Error("Could not retrieve sampling size")
return nil
}
// Compute the subnets to subscribe to.
nodeInfo, _, err := peerdas.Info(nodeID, samplingSize)
if err != nil {
log.WithError(err).Error("Could not retrieve peer info")
return nil
@@ -635,6 +641,30 @@ func (s *Service) dataColumnSubnetIndices(primitives.Slot) map[uint64]bool {
return nodeInfo.DataColumnsSubnets
}
// samplingSize computes the sampling size based on the samples per slot value,
// the validators custody requirement, and whether the node is subscribed to all data subnets.
// https://github.com/ethereum/consensus-specs/blob/master/specs/fulu/das-core.md#custody-sampling
func (s *Service) samplingSize() (uint64, error) {
beaconConfig := params.BeaconConfig()
if flags.Get().SubscribeAllDataSubnets {
return beaconConfig.DataColumnSidecarSubnetCount, nil
}
// Compute the validators custody requirement.
validatorsCustodyRequirement, err := s.validatorsCustodyRequirement()
if err != nil {
return 0, errors.Wrap(err, "validators custody requirement")
}
custodyGroupCount, err := s.cfg.p2p.CustodyGroupCount()
if err != nil {
return 0, errors.Wrap(err, "custody group count")
}
return max(beaconConfig.SamplesPerSlot, validatorsCustodyRequirement, custodyGroupCount), nil
}
func (s *Service) persistentAndAggregatorSubnetIndices(currentSlot primitives.Slot) map[uint64]bool {
if flags.Get().SubscribeToAllSubnets {
return mapFromCount(params.BeaconConfig().AttestationSubnetCount)

View File

@@ -121,12 +121,13 @@ func (s *Service) processDataColumnSidecarsFromExecution(ctx context.Context, ro
}
nodeID := s.cfg.p2p.NodeID()
custodyGroupCount, err := s.cfg.p2p.CustodyGroupCount()
if err != nil {
log.WithError(err).Error("Failed to get custody group count")
return
}
s.cfg.custodyInfo.Mut.RLock()
defer s.cfg.custodyInfo.Mut.RUnlock()
groupCount := s.cfg.custodyInfo.ActualGroupCount()
info, _, err := peerdas.Info(nodeID, groupCount)
info, _, err := peerdas.Info(nodeID, custodyGroupCount)
if err != nil {
log.WithError(err).Error("Failed to get peer info")
return

View File

@@ -7,7 +7,6 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain"
"github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/kzg"
chainMock "github.com/OffchainLabs/prysm/v6/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v6/beacon-chain/db/filesystem"
dbtest "github.com/OffchainLabs/prysm/v6/beacon-chain/db/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/execution"
@@ -200,6 +199,8 @@ func TestReconstructAndBroadcastBlobs(t *testing.T) {
})
t.Run("data columns", func(t *testing.T) {
custodyRequirement := params.BeaconConfig().CustodyRequirement
// load trusted setup
err := kzg.Start()
require.NoError(t, err)
@@ -268,11 +269,13 @@ func TestReconstructAndBroadcastBlobs(t *testing.T) {
DataColumnSidecars: tt.dataColumnSidecars,
},
operationNotifier: &chainMock.MockOperationNotifier{},
custodyInfo: &peerdas.CustodyInfo{},
},
seenDataColumnCache: newSlotAwareCache(1),
}
_, _, err := s.cfg.p2p.UpdateCustodyInfo(0, custodyRequirement)
require.NoError(t, err)
kzgCommitments := make([][]byte, 0, tt.blobCount)
for range tt.blobCount {
kzgCommitment := make([]byte, 48)

3
changelog/manu-cgc.md Normal file
View File

@@ -0,0 +1,3 @@
### Changed
- Update validator custody to the latest specification, including the new status message.

View File

@@ -336,7 +336,7 @@ var (
// SubscribeAllDataSubnets enables subscription to all data subnets.
SubscribeAllDataSubnets = &cli.BoolFlag{
Name: "subscribe-all-data-subnets",
Usage: "Enable subscription to all data subnets.",
Usage: "Enable subscription to all data subnets. Once set, unsetting this flag won't have any effect.",
}
// BatchVerifierLimit sets the maximum number of signatures to batch verify at once.
BatchVerifierLimit = &cli.IntFlag{

View File

@@ -126,7 +126,7 @@ func (c *client) Send(
return nil, errors.Wrap(err, "could not open new stream")
}
// do not encode anything if we are sending a metadata request
if baseTopic != p2p.RPCMetaDataTopicV1 && baseTopic != p2p.RPCMetaDataTopicV2 {
if baseTopic != p2p.RPCMetaDataTopicV1 && baseTopic != p2p.RPCMetaDataTopicV2 && baseTopic != p2p.RPCMetaDataTopicV3 {
castedMsg, ok := message.(ssz.Marshaler)
if !ok {
return nil, errors.Errorf("%T does not support the ssz marshaller interface", message)

View File

@@ -50,7 +50,7 @@ func (c *client) registerRPCHandler(baseTopic string, handle rpcHandler) {
// since metadata requests do not have any data in the payload, we
// do not decode anything.
if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 {
if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 || baseTopic == p2p.RPCMetaDataTopicV3 {
if err := handle(context.Background(), base, stream); err != nil {
if !errors.Is(err, p2ptypes.ErrWrongForkDigestVersion) {
log.WithError(err).Debug("Could not handle p2p RPC")

View File

@@ -269,9 +269,9 @@ type BeaconChainConfig struct {
// Values introduced in Fulu upgrade
NumberOfColumns uint64 `yaml:"NUMBER_OF_COLUMNS" spec:"true"` // NumberOfColumns in the extended data matrix.
SamplesPerSlot uint64 `yaml:"SAMPLES_PER_SLOT" spec:"true"` // SamplesPerSlot refers to the number of random samples a node queries per slot.
SamplesPerSlot uint64 `yaml:"SAMPLES_PER_SLOT" spec:"true"` // SamplesPerSlot is the minimum number of samples for an honest node.
NumberOfCustodyGroups uint64 `yaml:"NUMBER_OF_CUSTODY_GROUPS" spec:"true"` // NumberOfCustodyGroups available for nodes to custody.
CustodyRequirement uint64 `yaml:"CUSTODY_REQUIREMENT" spec:"true"` // CustodyRequirement refers to the minimum amount of subnets a peer must custody and serve samples from.
CustodyRequirement uint64 `yaml:"CUSTODY_REQUIREMENT" spec:"true"` // CustodyRequirement is minimum number of custody groups an honest node custodies and serves samples from.
MinEpochsForDataColumnSidecarsRequest primitives.Epoch `yaml:"MIN_EPOCHS_FOR_DATA_COLUMN_SIDECARS_REQUESTS" spec:"true"` // MinEpochsForDataColumnSidecarsRequest is the minimum number of epochs the node will keep the data columns for.
MaxCellsInExtendedMatrix uint64 `yaml:"MAX_CELLS_IN_EXTENDED_MATRIX"` // MaxCellsInExtendedMatrix is the full data of one-dimensional erasure coding extended blobs (in row major format).
DataColumnSidecarSubnetCount uint64 `yaml:"DATA_COLUMN_SIDECAR_SUBNET_COUNT" spec:"true"` // DataColumnSidecarSubnetCount is the number of data column sidecar subnets used in the gossipsub protocol
@@ -478,9 +478,8 @@ func (b *BeaconChainConfig) MaxBlobsPerBlockAtEpoch(epoch primitives.Epoch) int
return b.DeprecatedMaxBlobsPerBlock
}
// DenebEnabled centralizes the check to determine if code paths
// that are specific to deneb should be allowed to execute. This will make it easier to find call sites that do this
// kind of check and remove them post-deneb.
// DenebEnabled centralizes the check to determine if code paths that are specific to deneb should be allowed to execute.
// This will make it easier to find call sites that do this kind of check and remove them post-deneb.
func DenebEnabled() bool {
return BeaconConfig().DenebForkEpoch < math.MaxUint64
}

View File

@@ -25,6 +25,7 @@ go_library(
"//beacon-chain/forkchoice:go_default_library",
"//beacon-chain/forkchoice/doubly-linked-tree:go_default_library",
"//beacon-chain/operations/attestations:go_default_library",
"//beacon-chain/p2p/testing:go_default_library",
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",

View File

@@ -16,6 +16,7 @@ import (
"github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice"
doublylinkedtree "github.com/OffchainLabs/prysm/v6/beacon-chain/forkchoice/doubly-linked-tree"
"github.com/OffchainLabs/prysm/v6/beacon-chain/operations/attestations"
p2pTesting "github.com/OffchainLabs/prysm/v6/beacon-chain/p2p/testing"
"github.com/OffchainLabs/prysm/v6/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state"
"github.com/OffchainLabs/prysm/v6/beacon-chain/state/stategen"
@@ -82,6 +83,7 @@ func startChainService(t testing.TB,
blockchain.WithDataColumnStorage(filesystem.NewEphemeralDataColumnStorage(t)),
blockchain.WithSyncChecker(mock.MockChecker{}),
blockchain.WithGenesisTime(genesis),
blockchain.WithP2PBroadcaster(&p2pTesting.TestP2P{}),
)
service, err := blockchain.NewService(t.Context(), opts...)
require.NoError(t, err)