Compare commits

..

15 Commits

Author SHA1 Message Date
Manu NALEPA
f27404e829 Stop calling twice writeErrorResponseToStream in case of rate limit. 2024-09-17 15:34:56 +02:00
Manu NALEPA
d97faab4e7 beaconBlocksByRangeRPCHandler: Add remote Peer ID in logs. 2024-09-17 12:13:18 +02:00
Manu NALEPA
d0b232b86c validPeersExist --> enoughPeersAreConnected 2024-09-17 12:13:18 +02:00
Manu NALEPA
536fec55c4 Add more logs about peer disconnection. 2024-09-17 12:13:10 +02:00
Manu NALEPA
eebee6e0fe Add hidden debug flag: --data-columns-reject-slot-multiple. 2024-09-16 17:52:58 +02:00
Manu NALEPA
b1f66b6882 isDataAvailable: Improve logging. 2024-09-16 10:43:30 +02:00
Manu NALEPA
ee349da18f dataColumnSidecarByRootRPCHandler: Improve logging. 2024-09-16 10:42:58 +02:00
Manu NALEPA
f40b142d31 validateDataColumn: Refactor logging. 2024-09-16 10:42:38 +02:00
Manu NALEPA
b2de1a4026 GetBeaconBlock: Improve logs.
Rationale: A `begin` log should always be followed by a `success` log or a `failure` log.
2024-09-15 22:22:03 +02:00
Manu NALEPA
b3c95f1ea3 areDataColumnsAvailable: Improve logs. 2024-09-15 22:22:03 +02:00
Manu NALEPA
560b1951ed Data columns: Use subnet sampling only.
(Instead of peer sampling.)

aaa
2024-09-15 22:22:03 +02:00
Manu NALEPA
411339f546 sendBatchRootRequest: Do send requests to peers that custodies a superset of our columns.
Before this commit, we sent "data columns by root requests" for data columns peers do not custody.
2024-09-13 17:11:27 +02:00
Manu NALEPA
e8273791c0 sendBatchRootRequest: Refactor and add comments. 2024-09-13 17:11:25 +02:00
Manu NALEPA
56df2d000b Fix CPU usage in small devnets (#14446)
* `CustodyCountFromRemotePeer`: Set happy path in the outer scope.

* `FindPeersWithSubnet`: Improve logging.

* `listenForNewNodes`: Avoid infinite loop in a small subnet.

* Address Nishant's comment.

* FIx Nishant's comment.
2024-09-13 12:27:08 +02:00
Nishant Das
ec35840722 Revert "Change Custody Count to Uint8 (#14386)" (#14415)
This reverts commit bd7ec3fa97.
2024-09-04 10:27:34 +02:00
86 changed files with 992 additions and 773 deletions

View File

@@ -546,8 +546,9 @@ func missingDataColumns(bs *filesystem.BlobStorage, root [32]byte, expected map[
// closed, the context hits cancellation/timeout, or notifications have been received for all the missing sidecars.
func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed interfaces.ReadOnlySignedBeaconBlock) error {
if coreTime.PeerDASIsActive(signed.Block().Slot()) {
return s.isDataColumnsAvailable(ctx, root, signed)
return s.areDataColumnsAvailable(ctx, root, signed)
}
if signed.Version() < version.Deneb {
return nil
}
@@ -624,7 +625,17 @@ func (s *Service) isDataAvailable(ctx context.Context, root [32]byte, signed int
}
}
func (s *Service) isDataColumnsAvailable(ctx context.Context, root [32]byte, signed interfaces.ReadOnlySignedBeaconBlock) error {
// uint64MapToSortedSlice produces a sorted uint64 slice from a map.
func uint64MapToSortedSlice(input map[uint64]bool) []uint64 {
output := make([]uint64, 0, len(input))
for idx := range input {
output = append(output, idx)
}
slices.Sort[[]uint64](output)
return output
}
func (s *Service) areDataColumnsAvailable(ctx context.Context, root [32]byte, signed interfaces.ReadOnlySignedBeaconBlock) error {
if signed.Version() < version.Deneb {
return nil
}
@@ -653,7 +664,12 @@ func (s *Service) isDataColumnsAvailable(ctx context.Context, root [32]byte, sig
return nil
}
colMap, err := peerdas.CustodyColumns(s.cfg.P2P.NodeID(), peerdas.CustodySubnetCount())
// All columns to sample need to be available for the block to be considered available.
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#subnet-sampling
nodeID := s.cfg.P2P.NodeID()
subnetSamplingSize := peerdas.SubnetSamplingSize()
colMap, err := peerdas.CustodyColumns(nodeID, subnetSamplingSize)
if err != nil {
return errors.Wrap(err, "custody columns")
}
@@ -698,34 +714,37 @@ func (s *Service) isDataColumnsAvailable(ctx context.Context, root [32]byte, sig
nextSlot := slots.BeginsAt(signed.Block().Slot()+1, s.genesisTime)
// Avoid logging if DA check is called after next slot start.
if nextSlot.After(time.Now()) {
// Compute sorted slice of expected columns.
expected := make([]uint64, 0, len(colMap))
for col := range colMap {
expected = append(expected, col)
}
slices.Sort[[]uint64](expected)
// Compute sorted slice of missing columns.
missing := make([]uint64, 0, len(missingMap))
for col := range missingMap {
missing = append(missing, col)
}
slices.Sort[[]uint64](missing)
nst := time.AfterFunc(time.Until(nextSlot), func() {
if len(missingMap) == 0 {
missingMapCount := uint64(len(missingMap))
if missingMapCount == 0 {
return
}
var (
expected interface{} = "all"
missing interface{} = "all"
)
numberOfColumns := params.BeaconConfig().NumberOfColumns
colMapCount := uint64(len(colMap))
if colMapCount < numberOfColumns {
expected = uint64MapToSortedSlice(colMap)
}
if missingMapCount < numberOfColumns {
missing = uint64MapToSortedSlice(missingMap)
}
log.WithFields(logrus.Fields{
"slot": signed.Block().Slot(),
"root": fmt.Sprintf("%#x", root),
"columnsExpected": expected,
"columnsWaiting": missing,
}).Error("Still waiting for data columns DA check at slot end.")
}).Error("Some data columns are still unavailable at slot end")
})
defer nst.Stop()
}
@@ -756,12 +775,15 @@ func (s *Service) isDataColumnsAvailable(ctx context.Context, root [32]byte, sig
return nil
}
case <-ctx.Done():
missingIndexes := make([]uint64, 0, len(missingMap))
for val := range missingMap {
copiedVal := val
missingIndexes = append(missingIndexes, copiedVal)
var missingIndices interface{} = "all"
numberOfColumns := params.BeaconConfig().NumberOfColumns
missingIndicesCount := uint64(len(missingMap))
if missingIndicesCount < numberOfColumns {
missingIndices = uint64MapToSortedSlice(missingMap)
}
return errors.Wrapf(ctx.Err(), "context deadline waiting for data column sidecars slot: %d, BlockRoot: %#x, missing %v", block.Slot(), root, missingIndexes)
return errors.Wrapf(ctx.Err(), "context deadline waiting for data column sidecars slot: %d, BlockRoot: %#x, missing %v", block.Slot(), root, missingIndices)
}
}
}

View File

@@ -34,7 +34,7 @@ const (
)
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/p2p-interface.md#the-discovery-domain-discv5
type Csc uint8
type Csc uint64
func (Csc) ENRKey() string { return CustodySubnetCountEnrKey }
@@ -431,11 +431,20 @@ func VerifyDataColumnSidecarKZGProofs(sc blocks.RODataColumn) (bool, error) {
// CustodySubnetCount returns the number of subnets the node should participate in for custody.
func CustodySubnetCount() uint64 {
count := params.BeaconConfig().CustodyRequirement
if flags.Get().SubscribeToAllSubnets {
count = params.BeaconConfig().DataColumnSidecarSubnetCount
return params.BeaconConfig().DataColumnSidecarSubnetCount
}
return count
return params.BeaconConfig().CustodyRequirement
}
// SubnetSamplingSize returns the number of subnets the node should sample from.
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/das-core.md#subnet-sampling
func SubnetSamplingSize() uint64 {
samplesPerSlot := params.BeaconConfig().SamplesPerSlot
custodySubnetCount := CustodySubnetCount()
return max(samplesPerSlot, custodySubnetCount)
}
// CustodyColumnCount returns the number of columns the node should custody.

View File

@@ -18,7 +18,6 @@ go_library(
"handshake.go",
"info.go",
"interfaces.go",
"iterator.go",
"log.go",
"message_id.go",
"monitoring.go",

View File

@@ -402,7 +402,7 @@ func (s *Service) broadcastObject(ctx context.Context, obj ssz.Marshaler, topic
}
func attestationToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string {
return fmt.Sprintf(BeaconAttestationSubnetTopicFormat, forkDigest, subnet)
return fmt.Sprintf(AttestationSubnetTopicFormat, forkDigest, subnet)
}
func syncCommitteeToTopic(subnet uint64, forkDigest [fieldparams.VersionLength]byte) string {

View File

@@ -103,8 +103,8 @@ func TestService_Broadcast_ReturnsErr_TopicNotMapped(t *testing.T) {
}
func TestService_Attestation_Subnet(t *testing.T) {
if gtm := GossipTypeMapping[reflect.TypeOf(&ethpb.Attestation{})]; gtm != BeaconAttestationSubnetTopicFormat {
t.Errorf("Constant is out of date. Wanted %s, got %s", BeaconAttestationSubnetTopicFormat, gtm)
if gtm := GossipTypeMapping[reflect.TypeOf(&ethpb.Attestation{})]; gtm != AttestationSubnetTopicFormat {
t.Errorf("Constant is out of date. Wanted %s, got %s", AttestationSubnetTopicFormat, gtm)
}
tests := []struct {
@@ -170,7 +170,7 @@ func TestService_BroadcastAttestation(t *testing.T) {
msg := util.HydrateAttestation(&ethpb.Attestation{AggregationBits: bitfield.NewBitlist(7)})
subnet := uint64(5)
topic := BeaconAttestationSubnetTopicFormat
topic := AttestationSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(msg)] = topic
digest, err := p.currentForkDigest()
require.NoError(t, err)
@@ -230,11 +230,11 @@ func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
require.NoError(t, err)
defer bootListener.Close()
// Use shorter period for testing.
currentPeriod := pollingPeriod
pollingPeriod = 1 * time.Second
// Use smaller batch size for testing.
currentBatchSize := batchSize
batchSize = 2
defer func() {
pollingPeriod = currentPeriod
batchSize = currentBatchSize
}()
bootNode := bootListener.Self()
@@ -332,7 +332,7 @@ func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
go p2.listenForNewNodes()
msg := util.HydrateAttestation(&ethpb.Attestation{AggregationBits: bitfield.NewBitlist(7)})
topic := BeaconAttestationSubnetTopicFormat
topic := AttestationSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(msg)] = topic
digest, err := p.currentForkDigest()
require.NoError(t, err)

View File

@@ -33,7 +33,7 @@ func (*Service) InterceptPeerDial(_ peer.ID) (allow bool) {
// multiaddr for the given peer.
func (s *Service) InterceptAddrDial(pid peer.ID, m multiaddr.Multiaddr) (allow bool) {
// Disallow bad peers from dialing in.
if s.peers.IsBad(pid) {
if s.peers.IsBad(pid) != nil {
return false
}
return filterConnections(s.addrFilter, m)

View File

@@ -72,26 +72,10 @@ loop:
return validPeers, nil
}
// CustodyCountFromRemotePeer retrieves the custody count from a remote peer.
func (s *Service) CustodyCountFromRemotePeer(pid peer.ID) uint64 {
func (s *Service) custodyCountFromRemotePeerEnr(pid peer.ID) uint64 {
// By default, we assume the peer custodies the minimum number of subnets.
custodyRequirement := params.BeaconConfig().CustodyRequirement
// First, try to get the custody count from the peer's metadata.
metadata, err := s.peers.Metadata(pid)
if err != nil {
log.WithError(err).WithField("peerID", pid).Debug("Failed to retrieve metadata for peer, defaulting to the ENR value")
}
if metadata != nil {
custodyCount := metadata.CustodySubnetCount()
if custodyCount > 0 {
return uint64(custodyCount)
}
}
log.WithField("peerID", pid).Debug("Failed to retrieve custody count from metadata for peer, defaulting to the ENR value")
// Retrieve the ENR of the peer.
record, err := s.peers.ENR(pid)
if err != nil {
@@ -116,3 +100,30 @@ func (s *Service) CustodyCountFromRemotePeer(pid peer.ID) uint64 {
return custodyCount
}
// CustodyCountFromRemotePeer retrieves the custody count from a remote peer.
func (s *Service) CustodyCountFromRemotePeer(pid peer.ID) uint64 {
// Try to get the custody count from the peer's metadata.
metadata, err := s.peers.Metadata(pid)
if err != nil {
log.WithError(err).WithField("peerID", pid).Debug("Failed to retrieve metadata for peer, defaulting to the ENR value")
return s.custodyCountFromRemotePeerEnr(pid)
}
// If the metadata is nil, default to the ENR value.
if metadata == nil {
log.WithField("peerID", pid).Debug("Metadata is nil, defaulting to the ENR value")
return s.custodyCountFromRemotePeerEnr(pid)
}
// Get the custody subnets count from the metadata.
custodyCount := metadata.CustodySubnetCount()
// If the custody count is null, default to the ENR value.
if custodyCount == 0 {
log.WithField("peerID", pid).Debug("The custody count extracted from the metadata equals to 0, defaulting to the ENR value")
return s.custodyCountFromRemotePeerEnr(pid)
}
return custodyCount
}

View File

@@ -121,12 +121,12 @@ func TestCustodyCountFromRemotePeer(t *testing.T) {
// Define a metadata with zero custody.
zeroMetadata := wrapper.WrappedMetadataV2(&pb.MetaDataV2{
CustodySubnetCount: []byte{0},
CustodySubnetCount: 0,
})
// Define a nominal metadata.
nominalMetadata := wrapper.WrappedMetadataV2(&pb.MetaDataV2{
CustodySubnetCount: []byte{uint8(expectedMetadata)},
CustodySubnetCount: expectedMetadata,
})
testCases := []struct {

View File

@@ -15,6 +15,7 @@ import (
ma "github.com/multiformats/go-multiaddr"
"github.com/pkg/errors"
"github.com/prysmaticlabs/go-bitfield"
"github.com/sirupsen/logrus"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas"
@@ -175,7 +176,7 @@ func (s *Service) RefreshPersistentSubnets() {
// Get the custody subnet count in our metadata.
inMetadataCustodySubnetCount := s.Metadata().CustodySubnetCount()
isCustodySubnetCountUpToDate := custodySubnetCount == inRecordCustodySubnetCount && custodySubnetCount == uint64(inMetadataCustodySubnetCount)
isCustodySubnetCountUpToDate := (custodySubnetCount == inRecordCustodySubnetCount && custodySubnetCount == inMetadataCustodySubnetCount)
if isBitVUpToDate && isBitSUpToDate && isCustodySubnetCountUpToDate {
// Nothing to do, return early.
@@ -191,7 +192,26 @@ func (s *Service) RefreshPersistentSubnets() {
// listen for new nodes watches for new nodes in the network and adds them to the peerstore.
func (s *Service) listenForNewNodes() {
iterator := filterNodes(s.ctx, s.dv5Listener.RandomNodes(), s.filterPeer)
const minLogInterval = 1 * time.Minute
peersSummary := func(threshold uint) (uint, uint) {
// Retrieve how many active peers we have.
activePeers := s.Peers().Active()
activePeerCount := uint(len(activePeers))
// Compute how many peers we are missing to reach the threshold.
if activePeerCount >= threshold {
return activePeerCount, 0
}
missingPeerCount := threshold - activePeerCount
return activePeerCount, missingPeerCount
}
var lastLogTime time.Time
iterator := s.dv5Listener.RandomNodes()
defer iterator.Close()
for {
@@ -207,17 +227,35 @@ func (s *Service) listenForNewNodes() {
time.Sleep(pollingPeriod)
continue
}
wantedCount := s.wantedPeerDials()
if wantedCount == 0 {
// Compute the number of new peers we want to dial.
activePeerCount, missingPeerCount := peersSummary(s.cfg.MaxPeers)
fields := logrus.Fields{
"currentPeerCount": activePeerCount,
"targetPeerCount": s.cfg.MaxPeers,
}
if missingPeerCount == 0 {
log.Trace("Not looking for peers, at peer limit")
time.Sleep(pollingPeriod)
continue
}
if time.Since(lastLogTime) > minLogInterval {
lastLogTime = time.Now()
log.WithFields(fields).Debug("Searching for new active peers")
}
// Restrict dials if limit is applied.
if flags.MaxDialIsActive() {
wantedCount = min(wantedCount, flags.Get().MaxConcurrentDials)
maxConcurrentDials := uint(flags.Get().MaxConcurrentDials)
missingPeerCount = min(missingPeerCount, maxConcurrentDials)
}
wantedNodes := enode.ReadNodes(iterator, wantedCount)
// Search for new peers.
wantedNodes := searchForPeers(iterator, batchSize, missingPeerCount, s.filterPeer)
wg := new(sync.WaitGroup)
for i := 0; i < len(wantedNodes); i++ {
node := wantedNodes[i]
@@ -342,7 +380,8 @@ func (s *Service) createLocalNode(
}
if params.PeerDASEnabled() {
localNode.Set(peerdas.Csc(peerdas.CustodySubnetCount()))
custodySubnetCount := peerdas.CustodySubnetCount()
localNode.Set(peerdas.Csc(custodySubnetCount))
}
localNode.SetFallbackIP(ipAddr)
@@ -427,7 +466,7 @@ func (s *Service) filterPeer(node *enode.Node) bool {
}
// Ignore bad nodes.
if s.peers.IsBad(peerData.ID) {
if s.peers.IsBad(peerData.ID) != nil {
return false
}
@@ -487,17 +526,6 @@ func (s *Service) isPeerAtLimit(inbound bool) bool {
return activePeers >= maxPeers || numOfConns >= maxPeers
}
func (s *Service) wantedPeerDials() int {
maxPeers := int(s.cfg.MaxPeers)
activePeers := len(s.Peers().Active())
wantedCount := 0
if maxPeers > activePeers {
wantedCount = maxPeers - activePeers
}
return wantedCount
}
// PeersFromStringAddrs converts peer raw ENRs into multiaddrs for p2p.
func PeersFromStringAddrs(addrs []string) ([]ma.Multiaddr, error) {
var allAddrs []ma.Multiaddr

View File

@@ -473,7 +473,7 @@ type check struct {
metadataSequenceNumber uint64
attestationSubnets []uint64
syncSubnets []uint64
custodySubnetCount *uint8
custodySubnetCount *uint64
}
func checkPingCountCacheMetadataRecord(
@@ -542,7 +542,7 @@ func checkPingCountCacheMetadataRecord(
if expected.custodySubnetCount != nil {
// Check custody subnet count in ENR.
var actualCustodySubnetCount uint8
var actualCustodySubnetCount uint64
err := service.dv5Listener.LocalNode().Node().Record().Load(enr.WithEntry(peerdas.CustodySubnetCountEnrKey, &actualCustodySubnetCount))
require.NoError(t, err)
require.Equal(t, *expected.custodySubnetCount, actualCustodySubnetCount)
@@ -565,7 +565,7 @@ func TestRefreshPersistentSubnets(t *testing.T) {
eip7594ForkEpoch = 10
)
custodySubnetCount := uint8(params.BeaconConfig().CustodyRequirement)
custodySubnetCount := params.BeaconConfig().CustodyRequirement
// Set up epochs.
defaultCfg := params.BeaconConfig()

View File

@@ -12,24 +12,24 @@ import (
// gossipTopicMappings represent the protocol ID to protobuf message type map for easy
// lookup.
var gossipTopicMappings = map[string]func() proto.Message{
BeaconBlockSubnetTopicFormat: func() proto.Message { return &ethpb.SignedBeaconBlock{} },
BeaconAttestationSubnetTopicFormat: func() proto.Message { return &ethpb.Attestation{} },
VoluntaryExitSubnetTopicFormat: func() proto.Message { return &ethpb.SignedVoluntaryExit{} },
ProposerSlashingSubnetTopicFormat: func() proto.Message { return &ethpb.ProposerSlashing{} },
AttesterSlashingSubnetTopicFormat: func() proto.Message { return &ethpb.AttesterSlashing{} },
BeaconAggregateAndProofSubnetTopicFormat: func() proto.Message { return &ethpb.SignedAggregateAttestationAndProof{} },
SyncCommitteeContributionAndProofSubnetTopicFormat: func() proto.Message { return &ethpb.SignedContributionAndProof{} },
SyncCommitteeSubnetTopicFormat: func() proto.Message { return &ethpb.SyncCommitteeMessage{} },
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return &ethpb.SignedBLSToExecutionChange{} },
BlobSubnetTopicFormat: func() proto.Message { return &ethpb.BlobSidecar{} },
DataColumnSubnetTopicFormat: func() proto.Message { return &ethpb.DataColumnSidecar{} },
BlockSubnetTopicFormat: func() proto.Message { return &ethpb.SignedBeaconBlock{} },
AttestationSubnetTopicFormat: func() proto.Message { return &ethpb.Attestation{} },
ExitSubnetTopicFormat: func() proto.Message { return &ethpb.SignedVoluntaryExit{} },
ProposerSlashingSubnetTopicFormat: func() proto.Message { return &ethpb.ProposerSlashing{} },
AttesterSlashingSubnetTopicFormat: func() proto.Message { return &ethpb.AttesterSlashing{} },
AggregateAndProofSubnetTopicFormat: func() proto.Message { return &ethpb.SignedAggregateAttestationAndProof{} },
SyncContributionAndProofSubnetTopicFormat: func() proto.Message { return &ethpb.SignedContributionAndProof{} },
SyncCommitteeSubnetTopicFormat: func() proto.Message { return &ethpb.SyncCommitteeMessage{} },
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return &ethpb.SignedBLSToExecutionChange{} },
BlobSubnetTopicFormat: func() proto.Message { return &ethpb.BlobSidecar{} },
DataColumnSubnetTopicFormat: func() proto.Message { return &ethpb.DataColumnSidecar{} },
}
// GossipTopicMappings is a function to return the assigned data type
// versioned by epoch.
func GossipTopicMappings(topic string, epoch primitives.Epoch) proto.Message {
switch topic {
case BeaconBlockSubnetTopicFormat:
case BlockSubnetTopicFormat:
if epoch >= params.BeaconConfig().ElectraForkEpoch {
return &ethpb.SignedBeaconBlockElectra{}
}
@@ -46,7 +46,7 @@ func GossipTopicMappings(topic string, epoch primitives.Epoch) proto.Message {
return &ethpb.SignedBeaconBlockAltair{}
}
return gossipMessage(topic)
case BeaconAttestationSubnetTopicFormat:
case AttestationSubnetTopicFormat:
if epoch >= params.BeaconConfig().ElectraForkEpoch {
return &ethpb.AttestationElectra{}
}
@@ -56,7 +56,7 @@ func GossipTopicMappings(topic string, epoch primitives.Epoch) proto.Message {
return &ethpb.AttesterSlashingElectra{}
}
return gossipMessage(topic)
case BeaconAggregateAndProofSubnetTopicFormat:
case AggregateAndProofSubnetTopicFormat:
if epoch >= params.BeaconConfig().ElectraForkEpoch {
return &ethpb.SignedAggregateAttestationAndProofElectra{}
}
@@ -93,16 +93,16 @@ func init() {
GossipTypeMapping[reflect.TypeOf(v())] = k
}
// Specially handle Altair objects.
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockAltair{})] = BeaconBlockSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockAltair{})] = BlockSubnetTopicFormat
// Specially handle Bellatrix objects.
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockBellatrix{})] = BeaconBlockSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockBellatrix{})] = BlockSubnetTopicFormat
// Specially handle Capella objects.
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockCapella{})] = BeaconBlockSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockCapella{})] = BlockSubnetTopicFormat
// Specially handle Deneb objects.
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockDeneb{})] = BeaconBlockSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockDeneb{})] = BlockSubnetTopicFormat
// Specially handle Electra objects.
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockElectra{})] = BeaconBlockSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.AttestationElectra{})] = BeaconAttestationSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedBeaconBlockElectra{})] = BlockSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.AttestationElectra{})] = AttestationSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.AttesterSlashingElectra{})] = AttesterSlashingSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedAggregateAttestationAndProofElectra{})] = BeaconAggregateAndProofSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedAggregateAttestationAndProofElectra{})] = AggregateAndProofSubnetTopicFormat
}

View File

@@ -44,86 +44,86 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
params.OverrideBeaconConfig(bCfg)
// Phase 0
pMessage := GossipTopicMappings(BeaconBlockSubnetTopicFormat, 0)
pMessage := GossipTopicMappings(BlockSubnetTopicFormat, 0)
_, ok := pMessage.(*ethpb.SignedBeaconBlock)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAttestationSubnetTopicFormat, 0)
pMessage = GossipTopicMappings(AttestationSubnetTopicFormat, 0)
_, ok = pMessage.(*ethpb.Attestation)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(AttesterSlashingSubnetTopicFormat, 0)
_, ok = pMessage.(*ethpb.AttesterSlashing)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAggregateAndProofSubnetTopicFormat, 0)
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, 0)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
// Altair Fork
pMessage = GossipTopicMappings(BeaconBlockSubnetTopicFormat, altairForkEpoch)
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.SignedBeaconBlockAltair)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAttestationSubnetTopicFormat, altairForkEpoch)
pMessage = GossipTopicMappings(AttestationSubnetTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.Attestation)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(AttesterSlashingSubnetTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.AttesterSlashing)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAggregateAndProofSubnetTopicFormat, altairForkEpoch)
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, altairForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
// Bellatrix Fork
pMessage = GossipTopicMappings(BeaconBlockSubnetTopicFormat, bellatrixForkEpoch)
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.SignedBeaconBlockBellatrix)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAttestationSubnetTopicFormat, bellatrixForkEpoch)
pMessage = GossipTopicMappings(AttestationSubnetTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.Attestation)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(AttesterSlashingSubnetTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.AttesterSlashing)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAggregateAndProofSubnetTopicFormat, bellatrixForkEpoch)
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, bellatrixForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
// Capella Fork
pMessage = GossipTopicMappings(BeaconBlockSubnetTopicFormat, capellaForkEpoch)
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.SignedBeaconBlockCapella)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAttestationSubnetTopicFormat, capellaForkEpoch)
pMessage = GossipTopicMappings(AttestationSubnetTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.Attestation)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(AttesterSlashingSubnetTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.AttesterSlashing)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAggregateAndProofSubnetTopicFormat, capellaForkEpoch)
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, capellaForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
// Deneb Fork
pMessage = GossipTopicMappings(BeaconBlockSubnetTopicFormat, denebForkEpoch)
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.SignedBeaconBlockDeneb)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAttestationSubnetTopicFormat, denebForkEpoch)
pMessage = GossipTopicMappings(AttestationSubnetTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.Attestation)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(AttesterSlashingSubnetTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.AttesterSlashing)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAggregateAndProofSubnetTopicFormat, denebForkEpoch)
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, denebForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProof)
assert.Equal(t, true, ok)
// Electra Fork
pMessage = GossipTopicMappings(BeaconBlockSubnetTopicFormat, electraForkEpoch)
pMessage = GossipTopicMappings(BlockSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.SignedBeaconBlockElectra)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAttestationSubnetTopicFormat, electraForkEpoch)
pMessage = GossipTopicMappings(AttestationSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.AttestationElectra)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(AttesterSlashingSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.AttesterSlashingElectra)
assert.Equal(t, true, ok)
pMessage = GossipTopicMappings(BeaconAggregateAndProofSubnetTopicFormat, electraForkEpoch)
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProofElectra)
assert.Equal(t, true, ok)
}

View File

@@ -2,7 +2,6 @@ package p2p
import (
"context"
"errors"
"fmt"
"io"
"sync"
@@ -10,6 +9,7 @@ import (
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/peerdata"
prysmTime "github.com/prysmaticlabs/prysm/v5/time"
@@ -22,7 +22,57 @@ const (
)
func peerMultiaddrString(conn network.Conn) string {
return fmt.Sprintf("%s/p2p/%s", conn.RemoteMultiaddr().String(), conn.RemotePeer().String())
remoteMultiaddr := conn.RemoteMultiaddr().String()
remotePeerID := conn.RemotePeer().String()
return fmt.Sprintf("%s/p2p/%s", remoteMultiaddr, remotePeerID)
}
func (s *Service) connectToPeer(conn network.Conn) {
s.peers.SetConnectionState(conn.RemotePeer(), peers.PeerConnected)
// Go through the handshake process.
log.WithFields(logrus.Fields{
"direction": conn.Stat().Direction.String(),
"multiAddr": peerMultiaddrString(conn),
"activePeers": len(s.peers.Active()),
}).Debug("Initiate peer connection")
}
func (s *Service) disconnectFromPeer(
conn network.Conn,
goodByeFunc func(ctx context.Context, id peer.ID) error,
badPeerErr error,
) {
// Get the remote peer ID.
remotePeerID := conn.RemotePeer()
// Get the direction of the connection.
direction := conn.Stat().Direction.String()
// Get the remote peer multiaddr.
remotePeerMultiAddr := peerMultiaddrString(conn)
// Set the peer to disconnecting state.
s.peers.SetConnectionState(remotePeerID, peers.PeerDisconnecting)
// Only attempt a goodbye if we are still connected to the peer.
if s.host.Network().Connectedness(remotePeerID) == network.Connected {
if err := goodByeFunc(context.TODO(), remotePeerID); err != nil {
log.WithError(err).Error("Unable to disconnect from peer")
}
}
// Get the remaining active peers.
activePeerCount := len(s.peers.Active())
log.
WithError(badPeerErr).
WithFields(logrus.Fields{
"multiaddr": remotePeerMultiAddr,
"direction": direction,
"remainingActivePeers": activePeerCount,
}).
Debug("Initiate peer disconnection")
s.peers.SetConnectionState(remotePeerID, peers.PeerDisconnected)
}
// AddConnectionHandler adds a callback function which handles the connection with a
@@ -59,16 +109,7 @@ func (s *Service) AddConnectionHandler(reqFunc, goodByeFunc func(ctx context.Con
s.host.Network().Notify(&network.NotifyBundle{
ConnectedF: func(net network.Network, conn network.Conn) {
remotePeer := conn.RemotePeer()
disconnectFromPeer := func() {
s.peers.SetConnectionState(remotePeer, peers.PeerDisconnecting)
// Only attempt a goodbye if we are still connected to the peer.
if s.host.Network().Connectedness(remotePeer) == network.Connected {
if err := goodByeFunc(context.TODO(), remotePeer); err != nil {
log.WithError(err).Error("Unable to disconnect from peer")
}
}
s.peers.SetConnectionState(remotePeer, peers.PeerDisconnected)
}
// Connection handler must be non-blocking as part of libp2p design.
go func() {
if peerHandshaking(remotePeer) {
@@ -77,28 +118,20 @@ func (s *Service) AddConnectionHandler(reqFunc, goodByeFunc func(ctx context.Con
return
}
defer peerFinished(remotePeer)
// Handle the various pre-existing conditions that will result in us not handshaking.
peerConnectionState, err := s.peers.ConnectionState(remotePeer)
if err == nil && (peerConnectionState == peers.PeerConnected || peerConnectionState == peers.PeerConnecting) {
log.WithField("currentState", peerConnectionState).WithField("reason", "already active").Trace("Ignoring connection request")
return
}
s.peers.Add(nil /* ENR */, remotePeer, conn.RemoteMultiaddr(), conn.Stat().Direction)
// Defensive check in the event we still get a bad peer.
if s.peers.IsBad(remotePeer) {
log.WithField("reason", "bad peer").Trace("Ignoring connection request")
disconnectFromPeer()
return
}
validPeerConnection := func() {
s.peers.SetConnectionState(conn.RemotePeer(), peers.PeerConnected)
// Go through the handshake process.
log.WithFields(logrus.Fields{
"direction": conn.Stat().Direction.String(),
"multiAddr": peerMultiaddrString(conn),
"activePeers": len(s.peers.Active()),
}).Debug("New peer connection")
s.peers.Add(nil /* ENR */, remotePeer, conn.RemoteMultiaddr(), conn.Stat().Direction)
// Defensive check in the event we still get a bad peer.
if err := s.peers.IsBad(remotePeer); err != nil {
s.disconnectFromPeer(conn, goodByeFunc, err)
return
}
// Do not perform handshake on inbound dials.
@@ -118,67 +151,83 @@ func (s *Service) AddConnectionHandler(reqFunc, goodByeFunc func(ctx context.Con
// If peer hasn't sent a status request, we disconnect with them
if _, err := s.peers.ChainState(remotePeer); errors.Is(err, peerdata.ErrPeerUnknown) || errors.Is(err, peerdata.ErrNoPeerStatus) {
statusMessageMissing.Inc()
disconnectFromPeer()
s.disconnectFromPeer(conn, goodByeFunc, errors.Wrap(err, "chain state"))
return
}
if peerExists {
updated, err := s.peers.ChainStateLastUpdated(remotePeer)
if err != nil {
disconnectFromPeer()
s.disconnectFromPeer(conn, goodByeFunc, errors.Wrap(err, "chain state last updated"))
return
}
// exit if we don't receive any current status messages from
// peer.
if updated.IsZero() || !updated.After(currentTime) {
disconnectFromPeer()
// Exit if we don't receive any current status messages from peer.
if updated.IsZero() {
s.disconnectFromPeer(conn, goodByeFunc, errors.New("is zero"))
return
}
if !updated.After(currentTime) {
s.disconnectFromPeer(conn, goodByeFunc, errors.New("did not update"))
return
}
}
validPeerConnection()
s.connectToPeer(conn)
return
}
s.peers.SetConnectionState(conn.RemotePeer(), peers.PeerConnecting)
if err := reqFunc(context.TODO(), conn.RemotePeer()); err != nil && !errors.Is(err, io.EOF) {
log.WithError(err).Trace("Handshake failed")
disconnectFromPeer()
s.disconnectFromPeer(conn, goodByeFunc, err)
return
}
validPeerConnection()
s.connectToPeer(conn)
}()
},
})
}
// AddDisconnectionHandler disconnects from peers. It handles updating the peer status.
// AddDisconnectionHandler disconnects from peers. It handles updating the peer status.
// This also calls the handler responsible for maintaining other parts of the sync or p2p system.
func (s *Service) AddDisconnectionHandler(handler func(ctx context.Context, id peer.ID) error) {
s.host.Network().Notify(&network.NotifyBundle{
DisconnectedF: func(net network.Network, conn network.Conn) {
log := log.WithField("multiAddr", peerMultiaddrString(conn))
remotePeerMultiAddr := peerMultiaddrString(conn)
peerID := conn.RemotePeer()
direction := conn.Stat().Direction.String()
log := log.WithFields(logrus.Fields{
"multiAddr": remotePeerMultiAddr,
"direction": direction,
})
// Must be handled in a goroutine as this callback cannot be blocking.
go func() {
// Exit early if we are still connected to the peer.
if net.Connectedness(conn.RemotePeer()) == network.Connected {
if net.Connectedness(peerID) == network.Connected {
return
}
priorState, err := s.peers.ConnectionState(conn.RemotePeer())
priorState, err := s.peers.ConnectionState(peerID)
if err != nil {
// Can happen if the peer has already disconnected, so...
priorState = peers.PeerDisconnected
}
s.peers.SetConnectionState(conn.RemotePeer(), peers.PeerDisconnecting)
s.peers.SetConnectionState(peerID, peers.PeerDisconnecting)
if err := handler(context.TODO(), conn.RemotePeer()); err != nil {
log.WithError(err).Error("Disconnect handler failed")
}
s.peers.SetConnectionState(conn.RemotePeer(), peers.PeerDisconnected)
s.peers.SetConnectionState(peerID, peers.PeerDisconnected)
// Only log disconnections if we were fully connected.
if priorState == peers.PeerConnected {
log.WithFields(logrus.Fields{
"direction": conn.Stat().Direction.String(),
"multiAddr": peerMultiaddrString(conn),
"activePeers": len(s.peers.Active()),
}).Debug("Peer disconnected")
activePeersCount := len(s.peers.Active())
log.WithField("remainingActivePeers", activePeersCount).Debug("Peer disconnected")
}
}()
},

View File

@@ -1,36 +0,0 @@
package p2p
import (
"context"
"github.com/ethereum/go-ethereum/p2p/enode"
)
// filterNodes wraps an iterator such that Next only returns nodes for which
// the 'check' function returns true. This custom implementation also
// checks for context deadlines so that in the event the parent context has
// expired, we do exit from the search rather than perform more network
// lookups for additional peers.
func filterNodes(ctx context.Context, it enode.Iterator, check func(*enode.Node) bool) enode.Iterator {
return &filterIter{ctx, it, check}
}
type filterIter struct {
context.Context
enode.Iterator
check func(*enode.Node) bool
}
// Next looks up for the next valid node according to our
// filter criteria.
func (f *filterIter) Next() bool {
for f.Iterator.Next() {
if f.Context.Err() != nil {
return false
}
if f.check(f.Node()) {
return true
}
}
return false
}

View File

@@ -21,7 +21,7 @@ func TestMsgID_HashesCorrectly(t *testing.T) {
genesisValidatorsRoot := bytesutil.PadTo([]byte{'A'}, 32)
d, err := forks.CreateForkDigest(time.Now(), genesisValidatorsRoot)
assert.NoError(t, err)
tpc := fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, d)
tpc := fmt.Sprintf(p2p.BlockSubnetTopicFormat, d)
invalidSnappy := [32]byte{'J', 'U', 'N', 'K'}
pMsg := &pubsubpb.Message{Data: invalidSnappy[:], Topic: &tpc}
hashedData := hash.Hash(append(params.BeaconConfig().MessageDomainInvalidSnappy[:], pMsg.Data...))
@@ -41,7 +41,7 @@ func TestMessageIDFunction_HashesCorrectlyAltair(t *testing.T) {
genesisValidatorsRoot := bytesutil.PadTo([]byte{'A'}, 32)
d, err := signing.ComputeForkDigest(params.BeaconConfig().AltairForkVersion, genesisValidatorsRoot)
assert.NoError(t, err)
tpc := fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, d)
tpc := fmt.Sprintf(p2p.BlockSubnetTopicFormat, d)
topicLen := uint64(len(tpc))
topicLenBytes := bytesutil.Uint64ToBytesLittleEndian(topicLen)
invalidSnappy := [32]byte{'J', 'U', 'N', 'K'}
@@ -71,7 +71,7 @@ func TestMessageIDFunction_HashesCorrectlyBellatrix(t *testing.T) {
genesisValidatorsRoot := bytesutil.PadTo([]byte{'A'}, 32)
d, err := signing.ComputeForkDigest(params.BeaconConfig().BellatrixForkVersion, genesisValidatorsRoot)
assert.NoError(t, err)
tpc := fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, d)
tpc := fmt.Sprintf(p2p.BlockSubnetTopicFormat, d)
topicLen := uint64(len(tpc))
topicLenBytes := bytesutil.Uint64ToBytesLittleEndian(topicLen)
invalidSnappy := [32]byte{'J', 'U', 'N', 'K'}

View File

@@ -20,6 +20,7 @@ go_library(
"//crypto/rand:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"@com_github_libp2p_go_libp2p//core/peer:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
)

View File

@@ -4,6 +4,7 @@ import (
"time"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/peerdata"
)
@@ -61,7 +62,7 @@ func (s *BadResponsesScorer) Score(pid peer.ID) float64 {
// scoreNoLock is a lock-free version of Score.
func (s *BadResponsesScorer) scoreNoLock(pid peer.ID) float64 {
if s.isBadPeerNoLock(pid) {
if s.isBadPeerNoLock(pid) != nil {
return BadPeerScore
}
score := float64(0)
@@ -116,18 +117,24 @@ func (s *BadResponsesScorer) Increment(pid peer.ID) {
// IsBadPeer states if the peer is to be considered bad.
// If the peer is unknown this will return `false`, which makes using this function easier than returning an error.
func (s *BadResponsesScorer) IsBadPeer(pid peer.ID) bool {
func (s *BadResponsesScorer) IsBadPeer(pid peer.ID) error {
s.store.RLock()
defer s.store.RUnlock()
return s.isBadPeerNoLock(pid)
}
// isBadPeerNoLock is lock-free version of IsBadPeer.
func (s *BadResponsesScorer) isBadPeerNoLock(pid peer.ID) bool {
func (s *BadResponsesScorer) isBadPeerNoLock(pid peer.ID) error {
if peerData, ok := s.store.PeerData(pid); ok {
return peerData.BadResponses >= s.config.Threshold
if peerData.BadResponses >= s.config.Threshold {
return errors.Errorf("peer exceeded bad responses threshold: got %d, threshold %d", peerData.BadResponses, s.config.Threshold)
}
return nil
}
return false
return nil
}
// BadPeers returns the peers that are considered bad.
@@ -137,7 +144,7 @@ func (s *BadResponsesScorer) BadPeers() []peer.ID {
badPeers := make([]peer.ID, 0)
for pid := range s.store.Peers() {
if s.isBadPeerNoLock(pid) {
if s.isBadPeerNoLock(pid) != nil {
badPeers = append(badPeers, pid)
}
}

View File

@@ -33,19 +33,19 @@ func TestScorers_BadResponses_Score(t *testing.T) {
assert.Equal(t, 0., scorer.Score(pid), "Unexpected score for unregistered peer")
scorer.Increment(pid)
assert.Equal(t, false, scorer.IsBadPeer(pid))
assert.NoError(t, scorer.IsBadPeer(pid))
assert.Equal(t, -2.5, scorer.Score(pid))
scorer.Increment(pid)
assert.Equal(t, false, scorer.IsBadPeer(pid))
assert.NoError(t, scorer.IsBadPeer(pid))
assert.Equal(t, float64(-5), scorer.Score(pid))
scorer.Increment(pid)
assert.Equal(t, false, scorer.IsBadPeer(pid))
assert.NoError(t, scorer.IsBadPeer(pid))
assert.Equal(t, float64(-7.5), scorer.Score(pid))
scorer.Increment(pid)
assert.Equal(t, true, scorer.IsBadPeer(pid))
assert.NotNil(t, scorer.IsBadPeer(pid))
assert.Equal(t, -100.0, scorer.Score(pid))
}
@@ -152,17 +152,17 @@ func TestScorers_BadResponses_IsBadPeer(t *testing.T) {
})
scorer := peerStatuses.Scorers().BadResponsesScorer()
pid := peer.ID("peer1")
assert.Equal(t, false, scorer.IsBadPeer(pid))
assert.NoError(t, scorer.IsBadPeer(pid))
peerStatuses.Add(nil, pid, nil, network.DirUnknown)
assert.Equal(t, false, scorer.IsBadPeer(pid))
assert.NoError(t, scorer.IsBadPeer(pid))
for i := 0; i < scorers.DefaultBadResponsesThreshold; i++ {
scorer.Increment(pid)
if i == scorers.DefaultBadResponsesThreshold-1 {
assert.Equal(t, true, scorer.IsBadPeer(pid), "Unexpected peer status")
assert.NotNil(t, scorer.IsBadPeer(pid), "Unexpected peer status")
} else {
assert.Equal(t, false, scorer.IsBadPeer(pid), "Unexpected peer status")
assert.NoError(t, scorer.IsBadPeer(pid), "Unexpected peer status")
}
}
}
@@ -185,11 +185,11 @@ func TestScorers_BadResponses_BadPeers(t *testing.T) {
scorer.Increment(pids[2])
scorer.Increment(pids[4])
}
assert.Equal(t, false, scorer.IsBadPeer(pids[0]), "Invalid peer status")
assert.Equal(t, true, scorer.IsBadPeer(pids[1]), "Invalid peer status")
assert.Equal(t, true, scorer.IsBadPeer(pids[2]), "Invalid peer status")
assert.Equal(t, false, scorer.IsBadPeer(pids[3]), "Invalid peer status")
assert.Equal(t, true, scorer.IsBadPeer(pids[4]), "Invalid peer status")
assert.NoError(t, scorer.IsBadPeer(pids[0]), "Invalid peer status")
assert.NotNil(t, scorer.IsBadPeer(pids[1]), "Invalid peer status")
assert.NotNil(t, scorer.IsBadPeer(pids[2]), "Invalid peer status")
assert.NoError(t, scorer.IsBadPeer(pids[3]), "Invalid peer status")
assert.NotNil(t, scorer.IsBadPeer(pids[4]), "Invalid peer status")
want := []peer.ID{pids[1], pids[2], pids[4]}
badPeers := scorer.BadPeers()
sort.Slice(badPeers, func(i, j int) bool {

View File

@@ -177,8 +177,8 @@ func (s *BlockProviderScorer) processedBlocksNoLock(pid peer.ID) uint64 {
// Block provider scorer cannot guarantee that lower score of a peer is indeed a sign of a bad peer.
// Therefore this scorer never marks peers as bad, and relies on scores to probabilistically sort
// out low-scorers (see WeightSorted method).
func (*BlockProviderScorer) IsBadPeer(_ peer.ID) bool {
return false
func (*BlockProviderScorer) IsBadPeer(_ peer.ID) error {
return nil
}
// BadPeers returns the peers that are considered bad.

View File

@@ -481,8 +481,8 @@ func TestScorers_BlockProvider_BadPeerMarking(t *testing.T) {
})
scorer := peerStatuses.Scorers().BlockProviderScorer()
assert.Equal(t, false, scorer.IsBadPeer("peer1"), "Unexpected status for unregistered peer")
assert.NoError(t, scorer.IsBadPeer("peer1"), "Unexpected status for unregistered peer")
scorer.IncrementProcessedBlocks("peer1", 64)
assert.Equal(t, false, scorer.IsBadPeer("peer1"))
assert.NoError(t, scorer.IsBadPeer("peer1"))
assert.Equal(t, 0, len(scorer.BadPeers()))
}

View File

@@ -2,6 +2,7 @@ package scorers
import (
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/peerdata"
pbrpc "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
@@ -51,19 +52,24 @@ func (s *GossipScorer) scoreNoLock(pid peer.ID) float64 {
}
// IsBadPeer states if the peer is to be considered bad.
func (s *GossipScorer) IsBadPeer(pid peer.ID) bool {
func (s *GossipScorer) IsBadPeer(pid peer.ID) error {
s.store.RLock()
defer s.store.RUnlock()
return s.isBadPeerNoLock(pid)
}
// isBadPeerNoLock is lock-free version of IsBadPeer.
func (s *GossipScorer) isBadPeerNoLock(pid peer.ID) bool {
func (s *GossipScorer) isBadPeerNoLock(pid peer.ID) error {
peerData, ok := s.store.PeerData(pid)
if !ok {
return false
return nil
}
return peerData.GossipScore < gossipThreshold
if peerData.GossipScore < gossipThreshold {
return errors.Errorf("gossip score below threshold: got %f - threshold %f", peerData.GossipScore, gossipThreshold)
}
return nil
}
// BadPeers returns the peers that are considered bad.
@@ -73,7 +79,7 @@ func (s *GossipScorer) BadPeers() []peer.ID {
badPeers := make([]peer.ID, 0)
for pid := range s.store.Peers() {
if s.isBadPeerNoLock(pid) {
if s.isBadPeerNoLock(pid) != nil {
badPeers = append(badPeers, pid)
}
}

View File

@@ -34,7 +34,7 @@ func TestScorers_Gossip_Score(t *testing.T) {
},
check: func(scorer *scorers.GossipScorer) {
assert.Equal(t, -101.0, scorer.Score("peer1"), "Unexpected score")
assert.Equal(t, true, scorer.IsBadPeer("peer1"), "Unexpected good peer")
assert.NotNil(t, scorer.IsBadPeer("peer1"), "Unexpected good peer")
},
},
{
@@ -44,7 +44,7 @@ func TestScorers_Gossip_Score(t *testing.T) {
},
check: func(scorer *scorers.GossipScorer) {
assert.Equal(t, 10.0, scorer.Score("peer1"), "Unexpected score")
assert.Equal(t, false, scorer.IsBadPeer("peer1"), "Unexpected bad peer")
assert.NoError(t, scorer.IsBadPeer("peer1"), "Unexpected bad peer")
_, _, topicMap, err := scorer.GossipData("peer1")
assert.NoError(t, err)
assert.Equal(t, uint64(100), topicMap["a"].TimeInMesh, "incorrect time in mesh")

View File

@@ -46,7 +46,7 @@ func (s *PeerStatusScorer) Score(pid peer.ID) float64 {
// scoreNoLock is a lock-free version of Score.
func (s *PeerStatusScorer) scoreNoLock(pid peer.ID) float64 {
if s.isBadPeerNoLock(pid) {
if s.isBadPeerNoLock(pid) != nil {
return BadPeerScore
}
score := float64(0)
@@ -67,30 +67,34 @@ func (s *PeerStatusScorer) scoreNoLock(pid peer.ID) float64 {
}
// IsBadPeer states if the peer is to be considered bad.
func (s *PeerStatusScorer) IsBadPeer(pid peer.ID) bool {
func (s *PeerStatusScorer) IsBadPeer(pid peer.ID) error {
s.store.RLock()
defer s.store.RUnlock()
return s.isBadPeerNoLock(pid)
}
// isBadPeerNoLock is lock-free version of IsBadPeer.
func (s *PeerStatusScorer) isBadPeerNoLock(pid peer.ID) bool {
func (s *PeerStatusScorer) isBadPeerNoLock(pid peer.ID) error {
peerData, ok := s.store.PeerData(pid)
if !ok {
return false
return nil
}
// Mark peer as bad, if the latest error is one of the terminal ones.
terminalErrs := []error{
p2ptypes.ErrWrongForkDigestVersion,
p2ptypes.ErrInvalidFinalizedRoot,
p2ptypes.ErrInvalidRequest,
}
for _, err := range terminalErrs {
if errors.Is(peerData.ChainStateValidationError, err) {
return true
return err
}
}
return false
return nil
}
// BadPeers returns the peers that are considered bad.
@@ -100,7 +104,7 @@ func (s *PeerStatusScorer) BadPeers() []peer.ID {
badPeers := make([]peer.ID, 0)
for pid := range s.store.Peers() {
if s.isBadPeerNoLock(pid) {
if s.isBadPeerNoLock(pid) != nil {
badPeers = append(badPeers, pid)
}
}

View File

@@ -140,12 +140,12 @@ func TestScorers_PeerStatus_IsBadPeer(t *testing.T) {
ScorerParams: &scorers.Config{},
})
pid := peer.ID("peer1")
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer(pid))
assert.Equal(t, false, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid))
assert.NoError(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid))
peerStatuses.Scorers().PeerStatusScorer().SetPeerStatus(pid, &pb.Status{}, p2ptypes.ErrWrongForkDigestVersion)
assert.Equal(t, true, peerStatuses.Scorers().IsBadPeer(pid))
assert.Equal(t, true, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid))
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer(pid))
assert.NotNil(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid))
}
func TestScorers_PeerStatus_BadPeers(t *testing.T) {
@@ -155,22 +155,22 @@ func TestScorers_PeerStatus_BadPeers(t *testing.T) {
pid1 := peer.ID("peer1")
pid2 := peer.ID("peer2")
pid3 := peer.ID("peer3")
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer(pid1))
assert.Equal(t, false, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid1))
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer(pid2))
assert.Equal(t, false, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid2))
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer(pid3))
assert.Equal(t, false, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid3))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid1))
assert.NoError(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid1))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid2))
assert.NoError(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid2))
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)
assert.Equal(t, true, peerStatuses.Scorers().IsBadPeer(pid1))
assert.Equal(t, true, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid1))
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer(pid2))
assert.Equal(t, false, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid2))
assert.Equal(t, true, peerStatuses.Scorers().IsBadPeer(pid3))
assert.Equal(t, true, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid3))
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer(pid1))
assert.NotNil(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid1))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer(pid2))
assert.NoError(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid2))
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer(pid3))
assert.NotNil(t, peerStatuses.Scorers().PeerStatusScorer().IsBadPeer(pid3))
assert.Equal(t, 2, len(peerStatuses.Scorers().PeerStatusScorer().BadPeers()))
assert.Equal(t, 2, len(peerStatuses.Scorers().BadPeers()))
}

View File

@@ -6,6 +6,7 @@ import (
"time"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/peerdata"
"github.com/prysmaticlabs/prysm/v5/config/features"
)
@@ -24,7 +25,7 @@ const BadPeerScore = gossipThreshold
// Scorer defines minimum set of methods every peer scorer must expose.
type Scorer interface {
Score(pid peer.ID) float64
IsBadPeer(pid peer.ID) bool
IsBadPeer(pid peer.ID) error
BadPeers() []peer.ID
}
@@ -124,26 +125,29 @@ func (s *Service) ScoreNoLock(pid peer.ID) float64 {
}
// IsBadPeer traverses all the scorers to see if any of them classifies peer as bad.
func (s *Service) IsBadPeer(pid peer.ID) bool {
func (s *Service) IsBadPeer(pid peer.ID) error {
s.store.RLock()
defer s.store.RUnlock()
return s.IsBadPeerNoLock(pid)
}
// IsBadPeerNoLock is a lock-free version of IsBadPeer.
func (s *Service) IsBadPeerNoLock(pid peer.ID) bool {
if s.scorers.badResponsesScorer.isBadPeerNoLock(pid) {
return true
func (s *Service) IsBadPeerNoLock(pid peer.ID) error {
if err := s.scorers.badResponsesScorer.isBadPeerNoLock(pid); err != nil {
return errors.Wrap(err, "bad responses scorer")
}
if s.scorers.peerStatusScorer.isBadPeerNoLock(pid) {
return true
if err := s.scorers.peerStatusScorer.isBadPeerNoLock(pid); err != nil {
return errors.Wrap(err, "peer status scorer")
}
if features.Get().EnablePeerScorer {
if s.scorers.gossipScorer.isBadPeerNoLock(pid) {
return true
if err := s.scorers.gossipScorer.isBadPeerNoLock(pid); err != nil {
return errors.Wrap(err, "gossip scorer")
}
}
return false
return nil
}
// BadPeers returns the peers that are considered bad by any of registered scorers.
@@ -153,7 +157,7 @@ func (s *Service) BadPeers() []peer.ID {
badPeers := make([]peer.ID, 0)
for pid := range s.store.Peers() {
if s.IsBadPeerNoLock(pid) {
if s.IsBadPeerNoLock(pid) != nil {
badPeers = append(badPeers, pid)
}
}

View File

@@ -237,7 +237,7 @@ func TestScorers_Service_loop(t *testing.T) {
for i := 0; i < s1.Params().Threshold+5; i++ {
s1.Increment(pid1)
}
assert.Equal(t, true, s1.IsBadPeer(pid1), "Peer should be marked as bad")
assert.NotNil(t, s1.IsBadPeer(pid1), "Peer should be marked as bad")
s2.IncrementProcessedBlocks("peer1", 221)
assert.Equal(t, uint64(221), s2.ProcessedBlocks("peer1"))
@@ -252,7 +252,7 @@ func TestScorers_Service_loop(t *testing.T) {
for {
select {
case <-ticker.C:
if s1.IsBadPeer(pid1) == false && s2.ProcessedBlocks("peer1") == 0 {
if s1.IsBadPeer(pid1) == nil && s2.ProcessedBlocks("peer1") == 0 {
return
}
case <-ctx.Done():
@@ -263,7 +263,7 @@ func TestScorers_Service_loop(t *testing.T) {
}()
<-done
assert.Equal(t, false, s1.IsBadPeer(pid1), "Peer should not be marked as bad")
assert.NoError(t, s1.IsBadPeer(pid1), "Peer should not be marked as bad")
assert.Equal(t, uint64(0), s2.ProcessedBlocks("peer1"), "No blocks are expected")
}
@@ -278,10 +278,10 @@ func TestScorers_Service_IsBadPeer(t *testing.T) {
},
})
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer("peer1"))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer("peer1"))
peerStatuses.Scorers().BadResponsesScorer().Increment("peer1")
peerStatuses.Scorers().BadResponsesScorer().Increment("peer1")
assert.Equal(t, true, peerStatuses.Scorers().IsBadPeer("peer1"))
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer("peer1"))
}
func TestScorers_Service_BadPeers(t *testing.T) {
@@ -295,16 +295,16 @@ func TestScorers_Service_BadPeers(t *testing.T) {
},
})
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer("peer1"))
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer("peer2"))
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer("peer3"))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer("peer1"))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer("peer2"))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer("peer3"))
assert.Equal(t, 0, len(peerStatuses.Scorers().BadPeers()))
for _, pid := range []peer.ID{"peer1", "peer3"} {
peerStatuses.Scorers().BadResponsesScorer().Increment(pid)
peerStatuses.Scorers().BadResponsesScorer().Increment(pid)
}
assert.Equal(t, true, peerStatuses.Scorers().IsBadPeer("peer1"))
assert.Equal(t, false, peerStatuses.Scorers().IsBadPeer("peer2"))
assert.Equal(t, true, peerStatuses.Scorers().IsBadPeer("peer3"))
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer("peer1"))
assert.NoError(t, peerStatuses.Scorers().IsBadPeer("peer2"))
assert.NotNil(t, peerStatuses.Scorers().IsBadPeer("peer3"))
assert.Equal(t, 2, len(peerStatuses.Scorers().BadPeers()))
}

View File

@@ -34,6 +34,7 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
ma "github.com/multiformats/go-multiaddr"
manet "github.com/multiformats/go-multiaddr/net"
"github.com/pkg/errors"
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/peerdata"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/scorers"
@@ -343,19 +344,29 @@ func (p *Status) ChainStateLastUpdated(pid peer.ID) (time.Time, error) {
// IsBad states if the peer is to be considered bad (by *any* of the registered scorers).
// If the peer is unknown this will return `false`, which makes using this function easier than returning an error.
func (p *Status) IsBad(pid peer.ID) bool {
func (p *Status) IsBad(pid peer.ID) error {
p.store.RLock()
defer p.store.RUnlock()
return p.isBad(pid)
}
// isBad is the lock-free version of IsBad.
func (p *Status) isBad(pid peer.ID) bool {
func (p *Status) isBad(pid peer.ID) error {
// Do not disconnect from trusted peers.
if p.store.IsTrustedPeer(pid) {
return false
return nil
}
return p.isfromBadIP(pid) || p.scorers.IsBadPeerNoLock(pid)
if err := p.isfromBadIP(pid); err != nil {
return errors.Wrap(err, "peer is from a bad IP")
}
if err := p.scorers.IsBadPeerNoLock(pid); err != nil {
return errors.Wrap(err, "is bad peer no lock")
}
return nil
}
// NextValidTime gets the earliest possible time it is to contact/dial
@@ -600,7 +611,7 @@ func (p *Status) Prune() {
return
}
notBadPeer := func(pid peer.ID) bool {
return !p.isBad(pid)
return p.isBad(pid) == nil
}
notTrustedPeer := func(pid peer.ID) bool {
return !p.isTrustedPeers(pid)
@@ -990,24 +1001,28 @@ func (p *Status) isTrustedPeers(pid peer.ID) bool {
// this method assumes the store lock is acquired before
// executing the method.
func (p *Status) isfromBadIP(pid peer.ID) bool {
func (p *Status) isfromBadIP(pid peer.ID) error {
peerData, ok := p.store.PeerData(pid)
if !ok {
return false
return nil
}
if peerData.Address == nil {
return false
return nil
}
ip, err := manet.ToIP(peerData.Address)
if err != nil {
return true
return errors.Wrap(err, "to ip")
}
if val, ok := p.ipTracker[ip.String()]; ok {
if val > CollocationLimit {
return true
return errors.Errorf("colocation limit exceeded: got %d - limit %d", val, CollocationLimit)
}
}
return false
return nil
}
func (p *Status) addIpToTracker(pid peer.ID) {

View File

@@ -347,7 +347,7 @@ func TestPeerBadResponses(t *testing.T) {
require.NoError(t, err)
}
assert.Equal(t, false, p.IsBad(id), "Peer marked as bad when should be good")
assert.NoError(t, p.IsBad(id), "Peer marked as bad when should be good")
address, err := ma.NewMultiaddr("/ip4/213.202.254.180/tcp/13000")
require.NoError(t, err, "Failed to create address")
@@ -358,25 +358,25 @@ func TestPeerBadResponses(t *testing.T) {
resBadResponses, err := scorer.Count(id)
require.NoError(t, err)
assert.Equal(t, 0, resBadResponses, "Unexpected bad responses")
assert.Equal(t, false, p.IsBad(id), "Peer marked as bad when should be good")
assert.NoError(t, p.IsBad(id), "Peer marked as bad when should be good")
scorer.Increment(id)
resBadResponses, err = scorer.Count(id)
require.NoError(t, err)
assert.Equal(t, 1, resBadResponses, "Unexpected bad responses")
assert.Equal(t, false, p.IsBad(id), "Peer marked as bad when should be good")
assert.NoError(t, p.IsBad(id), "Peer marked as bad when should be good")
scorer.Increment(id)
resBadResponses, err = scorer.Count(id)
require.NoError(t, err)
assert.Equal(t, 2, resBadResponses, "Unexpected bad responses")
assert.Equal(t, true, p.IsBad(id), "Peer not marked as bad when it should be")
assert.NotNil(t, p.IsBad(id), "Peer not marked as bad when it should be")
scorer.Increment(id)
resBadResponses, err = scorer.Count(id)
require.NoError(t, err)
assert.Equal(t, 3, resBadResponses, "Unexpected bad responses")
assert.Equal(t, true, p.IsBad(id), "Peer not marked as bad when it should be")
assert.NotNil(t, p.IsBad(id), "Peer not marked as bad when it should be")
}
func TestAddMetaData(t *testing.T) {
@@ -574,7 +574,7 @@ func TestPeerIPTracker(t *testing.T) {
badPeers = append(badPeers, createPeer(t, p, addr, network.DirUnknown, peerdata.PeerConnectionState(ethpb.ConnectionState_DISCONNECTED)))
}
for _, pr := range badPeers {
assert.Equal(t, true, p.IsBad(pr), "peer with bad ip is not bad")
assert.NotNil(t, p.IsBad(pr), "peer with bad ip is not bad")
}
// Add in bad peers, so that our records are trimmed out
@@ -587,7 +587,7 @@ func TestPeerIPTracker(t *testing.T) {
p.Prune()
for _, pr := range badPeers {
assert.Equal(t, false, p.IsBad(pr), "peer with good ip is regarded as bad")
assert.NoError(t, p.IsBad(pr), "peer with good ip is regarded as bad")
}
}

View File

@@ -35,22 +35,22 @@ func TestService_CanSubscribe(t *testing.T) {
tests := []test{
{
name: "block topic on current fork",
topic: fmt.Sprintf(BeaconBlockSubnetTopicFormat, digest) + validProtocolSuffix,
topic: fmt.Sprintf(BlockSubnetTopicFormat, digest) + validProtocolSuffix,
want: true,
},
{
name: "block topic on unknown fork",
topic: fmt.Sprintf(BeaconBlockSubnetTopicFormat, [4]byte{0xFF, 0xEE, 0x56, 0x21}) + validProtocolSuffix,
topic: fmt.Sprintf(BlockSubnetTopicFormat, [4]byte{0xFF, 0xEE, 0x56, 0x21}) + validProtocolSuffix,
want: false,
},
{
name: "block topic missing protocol suffix",
topic: fmt.Sprintf(BeaconBlockSubnetTopicFormat, currentFork),
topic: fmt.Sprintf(BlockSubnetTopicFormat, currentFork),
want: false,
},
{
name: "block topic wrong protocol suffix",
topic: fmt.Sprintf(BeaconBlockSubnetTopicFormat, currentFork) + "/foobar",
topic: fmt.Sprintf(BlockSubnetTopicFormat, currentFork) + "/foobar",
want: false,
},
{
@@ -75,12 +75,12 @@ func TestService_CanSubscribe(t *testing.T) {
},
{
name: "att subnet topic on current fork",
topic: fmt.Sprintf(BeaconAttestationSubnetTopicFormat, digest, 55 /*subnet*/) + validProtocolSuffix,
topic: fmt.Sprintf(AttestationSubnetTopicFormat, digest, 55 /*subnet*/) + validProtocolSuffix,
want: true,
},
{
name: "att subnet topic on unknown fork",
topic: fmt.Sprintf(BeaconAttestationSubnetTopicFormat, [4]byte{0xCC, 0xBB, 0xAA, 0xA1} /*fork digest*/, 54 /*subnet*/) + validProtocolSuffix,
topic: fmt.Sprintf(AttestationSubnetTopicFormat, [4]byte{0xCC, 0xBB, 0xAA, 0xA1} /*fork digest*/, 54 /*subnet*/) + validProtocolSuffix,
want: false,
},
}
@@ -90,7 +90,7 @@ func TestService_CanSubscribe(t *testing.T) {
formatting := []interface{}{digest}
// Special case for attestation subnets which have a second formatting placeholder.
if topic == BeaconAttestationSubnetTopicFormat || topic == SyncCommitteeSubnetTopicFormat || topic == BlobSubnetTopicFormat || topic == DataColumnSubnetTopicFormat {
if topic == AttestationSubnetTopicFormat || topic == SyncCommitteeSubnetTopicFormat || topic == BlobSubnetTopicFormat || topic == DataColumnSubnetTopicFormat {
formatting = append(formatting, 0 /* some subnet ID */)
}
@@ -252,7 +252,7 @@ func TestService_FilterIncomingSubscriptions(t *testing.T) {
return &b
}(),
Topicid: func() *string {
s := fmt.Sprintf(BeaconBlockSubnetTopicFormat, digest) + validProtocolSuffix
s := fmt.Sprintf(BlockSubnetTopicFormat, digest) + validProtocolSuffix
return &s
}(),
},
@@ -266,7 +266,7 @@ func TestService_FilterIncomingSubscriptions(t *testing.T) {
return &b
}(),
Topicid: func() *string {
s := fmt.Sprintf(BeaconBlockSubnetTopicFormat, digest) + validProtocolSuffix
s := fmt.Sprintf(BlockSubnetTopicFormat, digest) + validProtocolSuffix
return &s
}(),
},
@@ -282,7 +282,7 @@ func TestService_FilterIncomingSubscriptions(t *testing.T) {
return &b
}(),
Topicid: func() *string {
s := fmt.Sprintf(BeaconBlockSubnetTopicFormat, digest) + validProtocolSuffix
s := fmt.Sprintf(BlockSubnetTopicFormat, digest) + validProtocolSuffix
return &s
}(),
},
@@ -292,7 +292,7 @@ func TestService_FilterIncomingSubscriptions(t *testing.T) {
return &b
}(),
Topicid: func() *string {
s := fmt.Sprintf(BeaconBlockSubnetTopicFormat, digest) + validProtocolSuffix
s := fmt.Sprintf(BlockSubnetTopicFormat, digest) + validProtocolSuffix
return &s
}(),
},
@@ -306,7 +306,7 @@ func TestService_FilterIncomingSubscriptions(t *testing.T) {
return &b
}(),
Topicid: func() *string {
s := fmt.Sprintf(BeaconBlockSubnetTopicFormat, digest) + validProtocolSuffix
s := fmt.Sprintf(BlockSubnetTopicFormat, digest) + validProtocolSuffix
return &s
}(),
},

View File

@@ -11,7 +11,7 @@ import (
)
func FuzzMsgID(f *testing.F) {
validTopic := fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f, 0x2a}) + "/" + encoder.ProtocolSuffixSSZSnappy
validTopic := fmt.Sprintf(p2p.BlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f, 0x2a}) + "/" + encoder.ProtocolSuffixSSZSnappy
f.Add(validTopic)
f.Fuzz(func(t *testing.T, topic string) {

View File

@@ -40,7 +40,7 @@ func TestService_PublishToTopicConcurrentMapWrite(t *testing.T) {
s.host = p0.BHost
s.pubsub = p0.PubSub()
topic := fmt.Sprintf(BeaconBlockSubnetTopicFormat, fd) + "/" + encoder.ProtocolSuffixSSZSnappy
topic := fmt.Sprintf(BlockSubnetTopicFormat, fd) + "/" + encoder.ProtocolSuffixSSZSnappy
// Establish the remote peer to be subscribed to the outgoing topic.
_, err = p1.SubscribeToTopic(topic)
@@ -95,7 +95,7 @@ func TestExtractGossipDigest(t *testing.T) {
},
{
name: "short digest",
topic: fmt.Sprintf(BeaconBlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f}) + "/" + encoder.ProtocolSuffixSSZSnappy,
topic: fmt.Sprintf(BlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f}) + "/" + encoder.ProtocolSuffixSSZSnappy,
want: [4]byte{},
wantErr: true,
error: errors.New("invalid digest length wanted"),
@@ -109,7 +109,7 @@ func TestExtractGossipDigest(t *testing.T) {
},
{
name: "valid topic",
topic: fmt.Sprintf(BeaconBlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f, 0x2a}) + "/" + encoder.ProtocolSuffixSSZSnappy,
topic: fmt.Sprintf(BlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f, 0x2a}) + "/" + encoder.ProtocolSuffixSSZSnappy,
want: [4]byte{0xb5, 0x30, 0x3f, 0x2a},
wantErr: false,
error: nil,
@@ -128,7 +128,7 @@ func TestExtractGossipDigest(t *testing.T) {
}
func BenchmarkExtractGossipDigest(b *testing.B) {
topic := fmt.Sprintf(BeaconBlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f, 0x2a}) + "/" + encoder.ProtocolSuffixSSZSnappy
topic := fmt.Sprintf(BlockSubnetTopicFormat, []byte{0xb5, 0x30, 0x3f, 0x2a}) + "/" + encoder.ProtocolSuffixSSZSnappy
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := ExtractGossipDigest(topic)

View File

@@ -43,6 +43,10 @@ var _ runtime.Service = (*Service)(nil)
// defined below.
var pollingPeriod = 6 * time.Second
// When looking for new nodes, if not enough nodes are found,
// we stop after this amount of iterations.
var batchSize = 2_000
// Refresh rate of ENR set at twice per slot.
var refreshRate = slots.DivideSlotBy(2)
@@ -477,8 +481,8 @@ func (s *Service) connectWithPeer(ctx context.Context, info peer.AddrInfo) error
if info.ID == s.host.ID() {
return nil
}
if s.Peers().IsBad(info.ID) {
return errors.New("refused to connect to bad peer")
if err := s.Peers().IsBad(info.ID); err != nil {
return errors.Wrap(err, "refused to connect to bad peer")
}
ctx, cancel := context.WithTimeout(ctx, maxDialTimeout)
defer cancel()

View File

@@ -201,11 +201,11 @@ func TestListenForNewNodes(t *testing.T) {
require.NoError(t, err)
defer bootListener.Close()
// Use shorter period for testing.
currentPeriod := pollingPeriod
pollingPeriod = 1 * time.Second
// Use shorter batch size for testing.
currentBatchSize := batchSize
batchSize = 5
defer func() {
pollingPeriod = currentPeriod
batchSize = currentBatchSize
}()
bootNode := bootListener.Self()
@@ -313,7 +313,7 @@ func TestService_JoinLeaveTopic(t *testing.T) {
assert.Equal(t, 0, len(s.joinedTopics))
topic := fmt.Sprintf(BeaconAttestationSubnetTopicFormat, fd, 42) + "/" + encoder.ProtocolSuffixSSZSnappy
topic := fmt.Sprintf(AttestationSubnetTopicFormat, fd, 42) + "/" + encoder.ProtocolSuffixSSZSnappy
topicHandle, err := s.JoinTopic(topic)
assert.NoError(t, err)
assert.Equal(t, 1, len(s.joinedTopics))

View File

@@ -76,11 +76,11 @@ func (s *Service) nodeFilter(topic string, index uint64) (func(node *enode.Node)
func searchForPeers(
iterator enode.Iterator,
batchSize int,
peersToFindCount int,
peersToFindCount uint,
filter func(node *enode.Node) bool,
) []*enode.Node {
nodeFromNodeID := make(map[enode.ID]*enode.Node, batchSize)
for i := 0; i < batchSize && len(nodeFromNodeID) <= peersToFindCount && iterator.Next(); i++ {
for i := 0; i < batchSize && uint(len(nodeFromNodeID)) <= peersToFindCount && iterator.Next(); i++ {
node := iterator.Node()
// Filter out nodes that do not meet the criteria.
@@ -141,7 +141,7 @@ func (s *Service) FindPeersWithSubnet(
index uint64,
threshold int,
) (bool, error) {
const batchSize = 2000
const minLogInterval = 1 * time.Minute
ctx, span := trace.StartSpan(ctx, "p2p.FindPeersWithSubnet")
defer span.End()
@@ -180,19 +180,17 @@ func (s *Service) FindPeersWithSubnet(
return true, nil
}
log.WithFields(logrus.Fields{
"topic": topic,
"currentPeerCount": peerCountForTopic,
"targetPeerCount": threshold,
}).Debug("Searching for new peers in the network - Start")
log := log.WithFields(logrus.Fields{
"topic": topic,
"targetPeerCount": threshold,
})
log.WithField("currentPeerCount", peerCountForTopic).Debug("Searching for new peers for a subnet - start")
lastLogTime := time.Now()
wg := new(sync.WaitGroup)
for {
// If we have enough peers, we can exit the loop. This is the happy path.
if missingPeerCountForTopic == 0 {
break
}
// If the context is done, we can exit the loop. This is the unhappy path.
if err := ctx.Err(); err != nil {
return false, errors.Errorf(
@@ -202,7 +200,7 @@ func (s *Service) FindPeersWithSubnet(
}
// Search for new peers in the network.
nodes := searchForPeers(iterator, batchSize, missingPeerCountForTopic, filter)
nodes := searchForPeers(iterator, batchSize, uint(missingPeerCountForTopic), filter)
// Restrict dials if limit is applied.
maxConcurrentDials := math.MaxInt
@@ -221,10 +219,20 @@ func (s *Service) FindPeersWithSubnet(
wg.Wait()
}
_, missingPeerCountForTopic = peersSummary(topic, threshold)
peerCountForTopic, missingPeerCountForTopic := peersSummary(topic, threshold)
// If we have enough peers, we can exit the loop. This is the happy path.
if missingPeerCountForTopic == 0 {
break
}
if time.Since(lastLogTime) > minLogInterval {
lastLogTime = time.Now()
log.WithField("currentPeerCount", peerCountForTopic).Debug("Searching for new peers for a subnet - continue")
}
}
log.WithField("topic", topic).Debug("Searching for new peers in the network - Success")
log.WithField("currentPeerCount", threshold).Debug("Searching for new peers for a subnet - success")
return true, nil
}
@@ -346,13 +354,12 @@ func (s *Service) updateSubnetRecordWithMetadataV3(
localNode.Set(custodySubnetCountEntry)
newSeqNumber := s.metaData.SequenceNumber() + 1
cscBytes := []byte{uint8(custodySubnetCount)}
s.metaData = wrapper.WrappedMetadataV2(&pb.MetaDataV2{
SeqNumber: newSeqNumber,
Attnets: bitVAtt,
Syncnets: bitVSync,
CustodySubnetCount: cscBytes,
CustodySubnetCount: custodySubnetCount,
})
}
@@ -370,22 +377,29 @@ func initializePersistentSubnets(id enode.ID, epoch primitives.Epoch) error {
return nil
}
// initializePersistentColumnSubnets initialize persisten column subnets
func initializePersistentColumnSubnets(id enode.ID) error {
// Check if the column subnets are already cached.
_, ok, expTime := cache.ColumnSubnetIDs.GetColumnSubnets()
if ok && expTime.After(time.Now()) {
return nil
}
subsMap, err := peerdas.CustodyColumnSubnets(id, peerdas.CustodySubnetCount())
// Retrieve the subnets we should be subscribed to.
subnetSamplingSize := peerdas.SubnetSamplingSize()
subnetsMap, err := peerdas.CustodyColumnSubnets(id, subnetSamplingSize)
if err != nil {
return err
return errors.Wrap(err, "custody column subnets")
}
subs := make([]uint64, 0, len(subsMap))
for sub := range subsMap {
subs = append(subs, sub)
subnets := make([]uint64, 0, len(subnetsMap))
for subnet := range subnetsMap {
subnets = append(subnets, subnet)
}
cache.ColumnSubnetIDs.AddColumnSubnets(subs)
// Add the subnets to the cache.
cache.ColumnSubnetIDs.AddColumnSubnets(subnets)
return nil
}

View File

@@ -113,7 +113,7 @@ func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
service.dv5Listener.LocalNode().Set(entry)
// Join and subscribe to the subnet, needed by libp2p.
topic, err := service.pubsub.Join(fmt.Sprintf(BeaconAttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + "/ssz_snappy")
topic, err := service.pubsub.Join(fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet) + "/ssz_snappy")
require.NoError(t, err)
_, err = topic.Subscribe()
@@ -155,7 +155,7 @@ func TestStartDiscV5_FindPeersWithSubnet(t *testing.T) {
exists := make([]bool, 0, 3)
for i := 1; i <= 3; i++ {
subnet := uint64(i)
topic := fmt.Sprintf(BeaconAttestationSubnetTopicFormat, bootNodeForkDigest, subnet)
topic := fmt.Sprintf(AttestationSubnetTopicFormat, bootNodeForkDigest, subnet)
exist := false

View File

@@ -35,22 +35,22 @@ const (
// Topic Formats
//
// BeaconAttestationSubnetTopicFormat is the topic format for the attestation subnet.
BeaconAttestationSubnetTopicFormat = GossipProtocolAndDigest + GossipAttestationMessage + "_%d"
// AttestationSubnetTopicFormat is the topic format for the attestation subnet.
AttestationSubnetTopicFormat = GossipProtocolAndDigest + GossipAttestationMessage + "_%d"
// SyncCommitteeSubnetTopicFormat is the topic format for the sync committee subnet.
SyncCommitteeSubnetTopicFormat = GossipProtocolAndDigest + GossipSyncCommitteeMessage + "_%d"
// BeaconBlockSubnetTopicFormat is the topic format for the block subnet.
BeaconBlockSubnetTopicFormat = GossipProtocolAndDigest + GossipBlockMessage
// VoluntaryExitSubnetTopicFormat is the topic format for the voluntary exit subnet.
VoluntaryExitSubnetTopicFormat = GossipProtocolAndDigest + GossipExitMessage
// BlockSubnetTopicFormat is the topic format for the block subnet.
BlockSubnetTopicFormat = GossipProtocolAndDigest + GossipBlockMessage
// ExitSubnetTopicFormat is the topic format for the voluntary exit subnet.
ExitSubnetTopicFormat = GossipProtocolAndDigest + GossipExitMessage
// ProposerSlashingSubnetTopicFormat is the topic format for the proposer slashing subnet.
ProposerSlashingSubnetTopicFormat = GossipProtocolAndDigest + GossipProposerSlashingMessage
// AttesterSlashingSubnetTopicFormat is the topic format for the attester slashing subnet.
AttesterSlashingSubnetTopicFormat = GossipProtocolAndDigest + GossipAttesterSlashingMessage
// BeaconAggregateAndProofSubnetTopicFormat is the topic format for the aggregate and proof subnet.
BeaconAggregateAndProofSubnetTopicFormat = GossipProtocolAndDigest + GossipAggregateAndProofMessage
// SyncCommitteeContributionAndProofSubnetTopicFormat is the topic format for the sync aggregate and proof subnet.
SyncCommitteeContributionAndProofSubnetTopicFormat = GossipProtocolAndDigest + GossipContributionAndProofMessage
// AggregateAndProofSubnetTopicFormat is the topic format for the aggregate and proof subnet.
AggregateAndProofSubnetTopicFormat = GossipProtocolAndDigest + GossipAggregateAndProofMessage
// SyncContributionAndProofSubnetTopicFormat is the topic format for the sync aggregate and proof subnet.
SyncContributionAndProofSubnetTopicFormat = GossipProtocolAndDigest + GossipContributionAndProofMessage
// BlsToExecutionChangeSubnetTopicFormat is the topic format for the bls to execution change subnet.
BlsToExecutionChangeSubnetTopicFormat = GossipProtocolAndDigest + GossipBlsToExecutionChangeMessage
// BlobSubnetTopicFormat is the topic format for the blob subnet.

View File

@@ -61,28 +61,31 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
if err != nil {
log.WithError(err).Error("Could not convert slot to time")
}
log.WithFields(logrus.Fields{
"slot": req.Slot,
"sinceSlotStartTime": time.Since(t),
}).Info("Begin building block")
log := log.WithField("slot", req.Slot)
log.WithField("sinceSlotStartTime", time.Since(t)).Info("Begin building block")
// A syncing validator should not produce a block.
if vs.SyncChecker.Syncing() {
log.Error("Fail to build block: node is syncing")
return nil, status.Error(codes.Unavailable, "Syncing to latest head, not ready to respond")
}
// An optimistic validator MUST NOT produce a block (i.e., sign across the DOMAIN_BEACON_PROPOSER domain).
if slots.ToEpoch(req.Slot) >= params.BeaconConfig().BellatrixForkEpoch {
if err := vs.optimisticStatus(ctx); err != nil {
log.WithError(err).Error("Fail to build block: node is optimistic")
return nil, status.Errorf(codes.Unavailable, "Validator is not ready to propose: %v", err)
}
}
head, parentRoot, err := vs.getParentState(ctx, req.Slot)
if err != nil {
log.WithError(err).Error("Fail to build block: could not get parent state")
return nil, err
}
sBlk, err := getEmptyBlock(req.Slot)
if err != nil {
log.WithError(err).Error("Fail to build block: could not get empty block")
return nil, status.Errorf(codes.Internal, "Could not prepare block: %v", err)
}
// Set slot, graffiti, randao reveal, and parent root.
@@ -94,6 +97,7 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
// Set proposer index.
idx, err := helpers.BeaconProposerIndex(ctx, head)
if err != nil {
log.WithError(err).Error("Fail to build block: could not calculate proposer index")
return nil, fmt.Errorf("could not calculate proposer index %w", err)
}
sBlk.SetProposerIndex(idx)
@@ -104,14 +108,17 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
}
resp, err := vs.BuildBlockParallel(ctx, sBlk, head, req.SkipMevBoost, builderBoostFactor)
log.WithFields(logrus.Fields{
"slot": req.Slot,
"sinceSlotStartTime": time.Since(t),
"validator": sBlk.Block().ProposerIndex(),
}).Info("Finished building block")
if err != nil {
log.WithError(err).Error("Fail to build block: could not build block in parallel")
return nil, errors.Wrap(err, "could not build block in parallel")
}
log.WithFields(logrus.Fields{
"sinceSlotStartTime": time.Since(t),
"validator": sBlk.Block().ProposerIndex(),
"parentRoot": fmt.Sprintf("%#x", parentRoot),
}).Info("Finished building block")
return resp, nil
}

View File

@@ -41,7 +41,7 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.Attestation{})]
// Given that both sync message related subnets have the same message name, we have to
// differentiate them below.
case strings.Contains(topic, p2p.GossipSyncCommitteeMessage) && !strings.Contains(topic, p2p.SyncCommitteeContributionAndProofSubnetTopicFormat):
case strings.Contains(topic, p2p.GossipSyncCommitteeMessage) && !strings.Contains(topic, p2p.SyncContributionAndProofSubnetTopicFormat):
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.SyncCommitteeMessage{})]
case strings.Contains(topic, p2p.GossipBlobSidecarMessage):
topic = p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.BlobSidecar{})]
@@ -83,11 +83,11 @@ func (*Service) replaceForkDigest(topic string) (string, error) {
func extractValidDataTypeFromTopic(topic string, digest []byte, clock *startup.Clock) (ssz.Unmarshaler, error) {
switch topic {
case p2p.BeaconBlockSubnetTopicFormat:
case p2p.BlockSubnetTopicFormat:
return extractDataTypeFromTypeMap(types.BlockMap, digest, clock)
case p2p.BeaconAttestationSubnetTopicFormat:
case p2p.AttestationSubnetTopicFormat:
return extractDataTypeFromTypeMap(types.AttestationMap, digest, clock)
case p2p.BeaconAggregateAndProofSubnetTopicFormat:
case p2p.AggregateAndProofSubnetTopicFormat:
return extractDataTypeFromTypeMap(types.AggregateAttestationMap, digest, clock)
}
return nil, nil

View File

@@ -306,55 +306,120 @@ func (s *Service) sendBatchRootRequest(ctx context.Context, roots [][32]byte, ra
ctx, span := trace.StartSpan(ctx, "sendBatchRootRequest")
defer span.End()
roots = dedupRoots(roots)
s.pendingQueueLock.RLock()
for i := len(roots) - 1; i >= 0; i-- {
r := roots[i]
if s.seenPendingBlocks[r] || s.cfg.chain.BlockBeingSynced(r) {
roots = append(roots[:i], roots[i+1:]...)
} else {
log.WithField("blockRoot", fmt.Sprintf("%#x", r)).Debug("Requesting block by root")
}
}
s.pendingQueueLock.RUnlock()
// Exit early if there are no roots to request.
if len(roots) == 0 {
return nil
}
bestPeers := s.getBestPeers()
if len(bestPeers) == 0 {
// Remove duplicates (if any) from the list of roots.
roots = dedupRoots(roots)
// Reversly iterate through the list of roots to request blocks, and filter out roots that are already
// seen in pending blocks or being synced.
func() {
s.pendingQueueLock.RLock()
defer s.pendingQueueLock.RUnlock()
for i := len(roots) - 1; i >= 0; i-- {
r := roots[i]
if s.seenPendingBlocks[r] || s.cfg.chain.BlockBeingSynced(r) {
roots = append(roots[:i], roots[i+1:]...)
continue
}
log.WithField("blockRoot", fmt.Sprintf("%#x", r)).Debug("Requesting block by root")
}
}()
// Nothing to do, exit early.
if len(roots) == 0 {
return nil
}
// Randomly choose a peer to query from our best peers. If that peer cannot return
// all the requested blocks, we randomly select another peer.
pid := bestPeers[randGen.Int()%len(bestPeers)]
for i := 0; i < numOfTries; i++ {
// Fetch best peers to request blocks from.
bestPeers := s.getBestPeers()
// Filter out peers that do not custody a superset of our columns.
// (Very likely, keep only supernode peers)
// TODO: Change this to be able to fetch from all peers.
headSlot := s.cfg.chain.HeadSlot()
peerDASIsActive := coreTime.PeerDASIsActive(headSlot)
if peerDASIsActive {
var err error
bestPeers, err = s.cfg.p2p.GetValidCustodyPeers(bestPeers)
if err != nil {
return errors.Wrap(err, "get valid custody peers")
}
}
// No suitable peer, exit early.
if len(bestPeers) == 0 {
log.WithField("roots", fmt.Sprintf("%#x", roots)).Debug("Send batch root request: No suitable peers")
return nil
}
// Randomly choose a peer to query from our best peers.
// If that peer cannot return all the requested blocks,
// we randomly select another peer.
randomIndex := randGen.Int() % len(bestPeers)
pid := bestPeers[randomIndex]
for range numOfTries {
req := p2ptypes.BeaconBlockByRootsReq(roots)
currentEpoch := slots.ToEpoch(s.cfg.clock.CurrentSlot())
// Get the current epoch.
currentSlot := s.cfg.clock.CurrentSlot()
currentEpoch := slots.ToEpoch(currentSlot)
// Trim the request to the maximum number of blocks we can request if needed.
maxReqBlock := params.MaxRequestBlock(currentEpoch)
if uint64(len(roots)) > maxReqBlock {
rootCount := uint64(len(roots))
if rootCount > maxReqBlock {
req = roots[:maxReqBlock]
}
// Send the request to the peer.
if err := s.sendRecentBeaconBlocksRequest(ctx, &req, pid); err != nil {
tracing.AnnotateError(span, err)
log.WithError(err).Debug("Could not send recent block request")
}
newRoots := make([][32]byte, 0, len(roots))
s.pendingQueueLock.RLock()
for _, rt := range roots {
if !s.seenPendingBlocks[rt] {
newRoots = append(newRoots, rt)
// Filter out roots that are already seen in pending blocks.
newRoots := make([][32]byte, 0, rootCount)
func() {
s.pendingQueueLock.RLock()
defer s.pendingQueueLock.RUnlock()
for _, rt := range roots {
if !s.seenPendingBlocks[rt] {
newRoots = append(newRoots, rt)
}
}
}
s.pendingQueueLock.RUnlock()
}()
// Exit early if all roots have been seen.
// This is the happy path.
if len(newRoots) == 0 {
break
return nil
}
// Choosing a new peer with the leftover set of
// roots to request.
// There is still some roots that have not been seen.
// Choosing a new peer with the leftover set of oots to request.
roots = newRoots
pid = bestPeers[randGen.Int()%len(bestPeers)]
// Choose a new peer to query.
randomIndex = randGen.Int() % len(bestPeers)
pid = bestPeers[randomIndex]
}
// Some roots are still missing after all allowed tries.
// This is the unhappy path.
log.WithFields(logrus.Fields{
"roots": fmt.Sprintf("%#x", roots),
"tries": numOfTries,
}).Debug("Send batch root request: Some roots are still missing after all allowed tries")
return nil
}

View File

@@ -97,7 +97,7 @@ func TestRateLimiter_ExceedRawCapacity(t *testing.T) {
for i := 0; i < defaultBurstLimit; i++ {
assert.ErrorContains(t, p2ptypes.ErrRateLimited.Error(), rlimiter.validateRawRpcRequest(stream))
}
assert.Equal(t, true, p1.Peers().IsBad(p2.PeerID()), "peer is not marked as a bad peer")
assert.NotNil(t, p1.Peers().IsBad(p2.PeerID()), "peer is not marked as a bad peer")
require.NoError(t, stream.Close(), "could not close stream")
if util.WaitTimeout(&wg, 1*time.Second) {

View File

@@ -164,6 +164,9 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
ctx, cancel := context.WithTimeout(s.ctx, ttfbTimeout)
defer cancel()
conn := stream.Conn()
remotePeer := conn.RemotePeer()
// Resetting after closing is a no-op so defer a reset in case something goes wrong.
// It's up to the handler to Close the stream (send an EOF) if
// it successfully writes a response. We don't blindly call
@@ -183,12 +186,13 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) {
ctx, span := trace.StartSpan(ctx, "sync.rpc")
defer span.End()
span.AddAttributes(trace.StringAttribute("topic", topic))
span.AddAttributes(trace.StringAttribute("peer", stream.Conn().RemotePeer().String()))
span.AddAttributes(trace.StringAttribute("peer", remotePeer.String()))
log := log.WithField("peer", stream.Conn().RemotePeer().String()).WithField("topic", string(stream.Protocol()))
// Check before hand that peer is valid.
if s.cfg.p2p.Peers().IsBad(stream.Conn().RemotePeer()) {
if err := s.sendGoodByeAndDisconnect(ctx, p2ptypes.GoodbyeCodeBanned, stream.Conn().RemotePeer()); err != nil {
if err := s.cfg.p2p.Peers().IsBad(remotePeer); err != nil {
log.WithError(err).WithField("peer", remotePeer).Debug("Sending goodbye and disconnect")
if err := s.sendGoodByeAndDisconnect(ctx, p2ptypes.GoodbyeCodeBanned, remotePeer); err != nil {
log.WithError(err).Debug("Could not disconnect from peer")
}
return

View File

@@ -16,6 +16,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"github.com/sirupsen/logrus"
)
// beaconBlocksByRangeRPCHandler looks up the request blocks from the database from a given start block.
@@ -26,15 +27,23 @@ func (s *Service) beaconBlocksByRangeRPCHandler(ctx context.Context, msg interfa
defer cancel()
SetRPCStreamDeadlines(stream)
remotePeer := stream.Conn().RemotePeer()
m, ok := msg.(*pb.BeaconBlocksByRangeRequest)
if !ok {
return errors.New("message is not type *pb.BeaconBlockByRangeRequest")
}
log.WithField("startSlot", m.StartSlot).WithField("count", m.Count).Debug("Serving block by range request")
log.WithFields(logrus.Fields{
"startSlot": m.StartSlot,
"count": m.Count,
"peer": remotePeer,
}).Debug("Serving block by range request")
rp, err := validateRangeRequest(m, s.cfg.clock.CurrentSlot())
if err != nil {
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(remotePeer)
tracing.AnnotateError(span, err)
return err
}
@@ -50,12 +59,12 @@ func (s *Service) beaconBlocksByRangeRPCHandler(ctx context.Context, msg interfa
if err != nil {
return err
}
remainingBucketCapacity := blockLimiter.Remaining(stream.Conn().RemotePeer().String())
remainingBucketCapacity := blockLimiter.Remaining(remotePeer.String())
span.AddAttributes(
trace.Int64Attribute("start", int64(rp.start)), // lint:ignore uintcast -- This conversion is OK for tracing.
trace.Int64Attribute("end", int64(rp.end)), // lint:ignore uintcast -- This conversion is OK for tracing.
trace.Int64Attribute("count", int64(m.Count)),
trace.StringAttribute("peer", stream.Conn().RemotePeer().String()),
trace.StringAttribute("peer", remotePeer.String()),
trace.Int64Attribute("remaining_capacity", remainingBucketCapacity),
)
@@ -82,12 +91,21 @@ func (s *Service) beaconBlocksByRangeRPCHandler(ctx context.Context, msg interfa
}
rpcBlocksByRangeResponseLatency.Observe(float64(time.Since(batchStart).Milliseconds()))
}
if err := batch.error(); err != nil {
log.WithError(err).Debug("error in BlocksByRange batch")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
log.WithError(err).Debug("Serving block by range request - BlocksByRange batch")
// If we hit a rate limit, the error response has already been written, and the stream is already closed.
if !errors.Is(err, p2ptypes.ErrRateLimited) {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
}
tracing.AnnotateError(span, err)
return err
}
log.Debug("Serving block by range request")
closeStream(stream, log)
return nil
}

View File

@@ -99,6 +99,7 @@ func (s *Service) blobSidecarsByRangeRPCHandler(ctx context.Context, msg interfa
}
var batch blockBatch
wQuota := params.BeaconConfig().MaxRequestBlobSidecars
for batch, ok = batcher.next(ctx, stream); ok; batch, ok = batcher.next(ctx, stream) {
batchStart := time.Now()
@@ -114,7 +115,12 @@ func (s *Service) blobSidecarsByRangeRPCHandler(ctx context.Context, msg interfa
}
if err := batch.error(); err != nil {
log.WithError(err).Debug("error in BlobSidecarsByRange batch")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
// If we hit a rate limit, the error response has already been written, and the stream is already closed.
if !errors.Is(err, p2ptypes.ErrRateLimited) {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
}
tracing.AnnotateError(span, err)
return err
}

View File

@@ -118,7 +118,12 @@ func (s *Service) dataColumnSidecarsByRangeRPCHandler(ctx context.Context, msg i
}
if err := batch.error(); err != nil {
log.WithError(err).Debug("error in DataColumnSidecarsByRange batch")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
// If we hit a rate limit, the error response has already been written, and the stream is already closed.
if !errors.Is(err, p2ptypes.ErrRateLimited) {
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
}
tracing.AnnotateError(span, err)
return err
}

View File

@@ -4,6 +4,7 @@ import (
"context"
"fmt"
"math"
"slices"
"sort"
"time"
@@ -26,6 +27,17 @@ import (
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
// uint64MapToSortedSlice produces a sorted uint64 slice from a map.
func uint64MapToSortedSlice(input map[uint64]bool) []uint64 {
output := make([]uint64, 0, len(input))
for idx := range input {
output = append(output, idx)
}
slices.Sort[[]uint64](output)
return output
}
func (s *Service) dataColumnSidecarByRootRPCHandler(ctx context.Context, msg interface{}, stream libp2pcore.Stream) error {
ctx, span := trace.StartSpan(ctx, "sync.dataColumnSidecarByRootRPCHandler")
defer span.End()
@@ -44,6 +56,8 @@ func (s *Service) dataColumnSidecarByRootRPCHandler(ctx context.Context, msg int
}
requestedColumnIdents := *ref
requestedColumnsCount := uint64(len(requestedColumnIdents))
if err := validateDataColumnsByRootRequest(requestedColumnIdents); err != nil {
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
@@ -79,29 +93,35 @@ func (s *Service) dataColumnSidecarByRootRPCHandler(ctx context.Context, msg int
return errors.Wrap(err, "custody columns")
}
numberOfColumns := params.BeaconConfig().NumberOfColumns
var (
custodied interface{} = "all"
requested interface{} = "all"
)
custodiedColumnsCount := uint64(len(custodiedColumns))
if custodiedColumnsCount != numberOfColumns {
custodied = uint64MapToSortedSlice(custodiedColumns)
}
if requestedColumnsCount != numberOfColumns {
requested = requestedColumnsList
}
custodiedColumnsList := make([]uint64, 0, len(custodiedColumns))
for column := range custodiedColumns {
custodiedColumnsList = append(custodiedColumnsList, column)
}
// Sort the custodied columns by index.
sort.Slice(custodiedColumnsList, func(i, j int) bool {
return custodiedColumnsList[i] < custodiedColumnsList[j]
})
slices.Sort[[]uint64](custodiedColumnsList)
fields := logrus.Fields{
"requested": requestedColumnsList,
"custodiedCount": len(custodiedColumnsList),
"requestedCount": len(requestedColumnsList),
}
if uint64(len(custodiedColumnsList)) == params.BeaconConfig().NumberOfColumns {
fields["custodied"] = "all"
} else {
fields["custodied"] = custodiedColumnsList
}
log.WithFields(fields).Debug("Data column sidecar by root request received")
log.WithFields(logrus.Fields{
"custodied": custodied,
"requested": requested,
}).Debug("Data column sidecar by root request received")
// Subscribe to the data column feed.
rootIndexChan := make(chan filesystem.RootIndexPair)

View File

@@ -55,10 +55,7 @@ func (s *Service) goodbyeRPCHandler(_ context.Context, msg interface{}, stream l
// disconnectBadPeer checks whether peer is considered bad by some scorer, and tries to disconnect
// the peer, if that is the case. Additionally, disconnection reason is obtained from scorer.
func (s *Service) disconnectBadPeer(ctx context.Context, id peer.ID) {
if !s.cfg.p2p.Peers().IsBad(id) {
return
}
func (s *Service) disconnectBadPeer(ctx context.Context, id peer.ID, badPeerErr error) {
err := s.cfg.p2p.Peers().Scorers().ValidationError(id)
goodbyeCode := p2ptypes.ErrToGoodbyeCode(err)
if err == nil {
@@ -67,6 +64,8 @@ func (s *Service) disconnectBadPeer(ctx context.Context, id peer.ID) {
if err := s.sendGoodByeAndDisconnect(ctx, goodbyeCode, id); err != nil {
log.WithError(err).Debug("Error when disconnecting with bad peer")
}
log.WithError(badPeerErr).WithField("peerID", id).Debug("Initiate peer disconnection")
}
// A custom goodbye method that is used by our connection handler, in the

View File

@@ -104,7 +104,7 @@ func (s *Service) metaDataHandler(_ context.Context, _ interface{}, stream libp2
Attnets: metadata.AttnetsBitfield(),
SeqNumber: metadata.SequenceNumber(),
Syncnets: bitfield.Bitvector4{byte(0x00)},
CustodySubnetCount: []byte{0},
CustodySubnetCount: 0,
})
case version.Altair:
metadata = wrapper.WrappedMetadataV2(
@@ -112,7 +112,7 @@ func (s *Service) metaDataHandler(_ context.Context, _ interface{}, stream libp2
Attnets: metadata.AttnetsBitfield(),
SeqNumber: metadata.SequenceNumber(),
Syncnets: metadata.SyncnetsBitfield(),
CustodySubnetCount: []byte{0},
CustodySubnetCount: 0,
})
}
}

View File

@@ -153,7 +153,7 @@ func TestMetadataRPCHandler_SendMetadataRequest(t *testing.T) {
SeqNumber: seqNumber,
Attnets: attnets,
Syncnets: syncnets,
CustodySubnetCount: []byte{custodySubnetCount},
CustodySubnetCount: custodySubnetCount,
}),
expected: wrapper.WrappedMetadataV0(&pb.MetaDataV0{
SeqNumber: seqNumber,
@@ -200,7 +200,7 @@ func TestMetadataRPCHandler_SendMetadataRequest(t *testing.T) {
SeqNumber: seqNumber,
Attnets: attnets,
Syncnets: syncnets,
CustodySubnetCount: []byte{custodySubnetCount},
CustodySubnetCount: custodySubnetCount,
}),
expected: wrapper.WrappedMetadataV1(&pb.MetaDataV1{
SeqNumber: seqNumber,
@@ -221,7 +221,7 @@ func TestMetadataRPCHandler_SendMetadataRequest(t *testing.T) {
SeqNumber: seqNumber,
Attnets: attnets,
Syncnets: bitfield.Bitvector4{byte(0x00)},
CustodySubnetCount: []byte{0},
CustodySubnetCount: 0,
}),
},
{
@@ -238,7 +238,7 @@ func TestMetadataRPCHandler_SendMetadataRequest(t *testing.T) {
SeqNumber: seqNumber,
Attnets: attnets,
Syncnets: syncnets,
CustodySubnetCount: []byte{0},
CustodySubnetCount: 0,
}),
},
{
@@ -250,13 +250,13 @@ func TestMetadataRPCHandler_SendMetadataRequest(t *testing.T) {
SeqNumber: seqNumber,
Attnets: attnets,
Syncnets: syncnets,
CustodySubnetCount: []byte{custodySubnetCount},
CustodySubnetCount: custodySubnetCount,
}),
expected: wrapper.WrappedMetadataV2(&pb.MetaDataV2{
SeqNumber: seqNumber,
Attnets: attnets,
Syncnets: syncnets,
CustodySubnetCount: []byte{custodySubnetCount},
CustodySubnetCount: custodySubnetCount,
}),
},
}

View File

@@ -25,7 +25,7 @@ import (
"github.com/sirupsen/logrus"
)
// maintainPeerStatuses by infrequently polling peers for their latest status.
// maintainPeerStatuses maintain peer statuses by polling peers for their latest status twice per epoch.
func (s *Service) maintainPeerStatuses() {
// Run twice per epoch.
interval := time.Duration(params.BeaconConfig().SlotsPerEpoch.Div(2).Mul(params.BeaconConfig().SecondsPerSlot)) * time.Second
@@ -43,11 +43,15 @@ func (s *Service) maintainPeerStatuses() {
log.WithError(err).Debug("Error when disconnecting with peer")
}
s.cfg.p2p.Peers().SetConnectionState(id, peers.PeerDisconnected)
log.WithFields(logrus.Fields{
"peer": id,
"reason": "maintain peer statuses - peer is not connected",
}).Debug("Initiate peer disconnection")
return
}
// Disconnect from peers that are considered bad by any of the registered scorers.
if s.cfg.p2p.Peers().IsBad(id) {
s.disconnectBadPeer(s.ctx, id)
if err := s.cfg.p2p.Peers().IsBad(id); err != nil {
s.disconnectBadPeer(s.ctx, id, err)
return
}
// If the status hasn't been updated in the recent interval time.
@@ -73,6 +77,11 @@ func (s *Service) maintainPeerStatuses() {
if err := s.sendGoodByeAndDisconnect(s.ctx, p2ptypes.GoodbyeCodeTooManyPeers, id); err != nil {
log.WithField("peer", id).WithError(err).Debug("Could not disconnect with peer")
}
log.WithFields(logrus.Fields{
"peer": id,
"reason": "to be pruned",
}).Debug("Initiate peer disconnection")
}
})
}
@@ -169,8 +178,8 @@ func (s *Service) sendRPCStatusRequest(ctx context.Context, id peer.ID) error {
// If validation fails, validation error is logged, and peer status scorer will mark peer as bad.
err = s.validateStatusMessage(ctx, msg)
s.cfg.p2p.Peers().Scorers().PeerStatusScorer().SetPeerStatus(id, msg, err)
if s.cfg.p2p.Peers().IsBad(id) {
s.disconnectBadPeer(s.ctx, id)
if err := s.cfg.p2p.Peers().IsBad(id); err != nil {
s.disconnectBadPeer(s.ctx, id, err)
}
return err
}

View File

@@ -877,7 +877,7 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
require.NoError(t, cw.SetClock(startup.NewClock(chain.Genesis, chain.ValidatorsRoot)))
assert.Equal(t, false, p1.Peers().Scorers().IsBadPeer(p2.PeerID()), "Peer is marked as bad")
assert.NoError(t, p1.Peers().Scorers().IsBadPeer(p2.PeerID()), "Peer is marked as bad")
p1.Connect(p2)
if util.WaitTimeout(&wg, time.Second) {
@@ -889,7 +889,7 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
require.NoError(t, err, "Could not obtain peer connection state")
assert.Equal(t, peers.PeerDisconnected, connectionState, "Expected peer to be disconnected")
assert.Equal(t, true, p1.Peers().Scorers().IsBadPeer(p2.PeerID()), "Peer is not marked as bad")
assert.NotNil(t, p1.Peers().Scorers().IsBadPeer(p2.PeerID()), "Peer is not marked as bad")
}
func TestStatusRPC_ValidGenesisMessage(t *testing.T) {

View File

@@ -170,7 +170,6 @@ type Service struct {
receivedDataColumnsFromRoot map[[fieldparams.RootLength]byte]map[uint64]bool
receivedDataColumnsFromRootLock sync.RWMutex
ctxMap ContextByteVersions
sampler DataColumnSampler
}
// NewService initializes new regular sync service.
@@ -360,12 +359,6 @@ func (s *Service) startTasksPostInitialSync() {
// Start the fork watcher.
go s.forkWatcher()
// Start data columns sampling if peerDAS is enabled.
if params.PeerDASEnabled() {
s.sampler = newDataColumnSampler1D(s.cfg.p2p, s.cfg.clock, s.ctxMap, s.cfg.stateNotifier, s.newColumnVerifier)
go s.sampler.Run(s.ctx)
}
case <-s.ctx.Done():
log.Debug("Context closed, exiting goroutine")
}

View File

@@ -54,173 +54,117 @@ func (s *Service) noopValidator(_ context.Context, _ peer.ID, msg *pubsub.Messag
return pubsub.ValidationAccept, nil
}
// registerSubscribersFromGenesis registers subscribers for subnets needed from genesis.
// It includes: beacon block, beacon aggregate and proof, voluntary exit, proposer slashing, attester slashing and beacon attestation.
func (s *Service) registerSubscribersFromGenesis(digest [4]byte) {
// Beacon block
// Register PubSub subscribers
func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
s.subscribe(
p2p.BeaconBlockSubnetTopicFormat,
s.validateBeaconBlockPubSubMsg,
p2p.BlockSubnetTopicFormat,
s.validateBeaconBlockPubSub,
s.beaconBlockSubscriber,
digest,
)
// Beacon aggregate and proof
s.subscribe(
p2p.BeaconAggregateAndProofSubnetTopicFormat,
s.validateBeaconAggregateAndProofPubSubMsg,
s.beaconAggregateAndProofSubscriber,
p2p.AggregateAndProofSubnetTopicFormat,
s.validateAggregateAndProof,
s.beaconAggregateProofSubscriber,
digest,
)
// Voluntary exit
s.subscribe(
p2p.VoluntaryExitSubnetTopicFormat,
s.validateVoluntaryExitPubSubMsg,
p2p.ExitSubnetTopicFormat,
s.validateVoluntaryExit,
s.voluntaryExitSubscriber,
digest,
)
// Proposer slashing
s.subscribe(
p2p.ProposerSlashingSubnetTopicFormat,
s.validateProposerSlashingPubSubMsg,
s.validateProposerSlashing,
s.proposerSlashingSubscriber,
digest,
)
// Attester slashing
s.subscribe(
p2p.AttesterSlashingSubnetTopicFormat,
s.validateAttesterSlashingPubSubMsg,
s.validateAttesterSlashing,
s.attesterSlashingSubscriber,
digest,
)
// Beacon attestation -- subscribe to all subnets.
if flags.Get().SubscribeToAllSubnets {
s.subscribeStaticWithSubnets(
p2p.BeaconAttestationSubnetTopicFormat,
s.validateBeaconAttestationPubSubMsg,
s.beaconAttestationSubscriber,
p2p.AttestationSubnetTopicFormat,
s.validateCommitteeIndexBeaconAttestation, /* validator */
s.committeeIndexBeaconAttestationSubscriber, /* message handler */
digest,
params.BeaconConfig().AttestationSubnetCount,
)
return
}
// Beacon attestation -- subscribe to required subnets.
s.subscribeDynamicWithSubnets(
p2p.BeaconAttestationSubnetTopicFormat,
s.validateBeaconAttestationPubSubMsg,
s.beaconAttestationSubscriber,
digest,
)
}
// registerSubscribersFromAltair registers subscribers for subnets needed from the Altair hard fork.
// It includes: sync committee contribution and proof and sync committee.
func (s *Service) registerSubscribersFromAltair(digest [4]byte) {
// Sync committee contribution and proof
s.subscribe(
p2p.SyncCommitteeContributionAndProofSubnetTopicFormat,
s.validateSyncCommitteeContributionAndProofPubSubMsg,
s.syncCommitteeContributionAndProofSubscriber,
digest,
)
// Sync committee -- subscribe to all subnets.
if flags.Get().SubscribeToAllSubnets {
s.subscribeStaticWithSyncSubnets(
p2p.SyncCommitteeSubnetTopicFormat,
s.validateSyncCommitteeMessage,
s.syncCommitteeMessageSubscriber,
} else {
s.subscribeDynamicWithSubnets(
p2p.AttestationSubnetTopicFormat,
s.validateCommitteeIndexBeaconAttestation, /* validator */
s.committeeIndexBeaconAttestationSubscriber, /* message handler */
digest,
)
return
}
// Sync committee -- subscribe to required subnets.
s.subscribeDynamicWithSyncSubnets(
p2p.SyncCommitteeSubnetTopicFormat,
s.validateSyncCommitteeMessage,
s.syncCommitteeMessageSubscriber,
digest,
)
}
// registerSubscribersFromCapella registers subscribers for subnets needed from the altair fork.
// It includes: BLS to execution change
func (s *Service) registerSubscribersFromCapella(digest [4]byte) {
// BLS to execution change
s.subscribe(
p2p.BlsToExecutionChangeSubnetTopicFormat,
s.validateBlsToExecutionChange,
s.blsToExecutionChangeSubscriber,
digest,
)
}
// registerSubscribersFromDeneb registers subscribers for subnets needed from the deneb fork.
// It includes: Blob sidecar and data column sidecar (depending of the peerDAS status).
func (s *Service) registerSubscribersFromDeneb(epoch primitives.Epoch, digest [4]byte) {
peerDasIsActive := coreTime.PeerDASIsActive(slots.UnsafeEpochStart(epoch))
if !peerDasIsActive {
// PeerDAS is not yet active, blob sidecar
s.subscribeStaticWithSubnets(
p2p.BlobSubnetTopicFormat,
s.validateBlob,
s.blobSubscriber,
digest,
params.BeaconConfig().BlobsidecarSubnetCount,
)
return
}
// PeerDAS is active, data columns sidecar -- subscribe to all subnets.
if flags.Get().SubscribeToAllSubnets {
s.subscribeStaticWithSubnets(
p2p.DataColumnSubnetTopicFormat,
s.validateDataColumn,
s.dataColumnSubscriber,
digest,
params.BeaconConfig().DataColumnSidecarSubnetCount,
)
return
}
// PeerDAS is active, data columns sidecar -- subscribe to required subnets.
s.subscribeDynamicWithColumnSubnets(
p2p.DataColumnSubnetTopicFormat,
s.validateDataColumn,
s.dataColumnSubscriber,
digest,
)
}
// registerSubscribers registers subscribers
func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
// From genesis
s.registerSubscribersFromGenesis(digest)
// From Altair hard fork
// Altair Fork Version
if epoch >= params.BeaconConfig().AltairForkEpoch {
s.registerSubscribersFromAltair(digest)
s.subscribe(
p2p.SyncContributionAndProofSubnetTopicFormat,
s.validateSyncContributionAndProof,
s.syncContributionAndProofSubscriber,
digest,
)
if flags.Get().SubscribeToAllSubnets {
s.subscribeStaticWithSyncSubnets(
p2p.SyncCommitteeSubnetTopicFormat,
s.validateSyncCommitteeMessage, /* validator */
s.syncCommitteeMessageSubscriber, /* message handler */
digest,
)
} else {
s.subscribeDynamicWithSyncSubnets(
p2p.SyncCommitteeSubnetTopicFormat,
s.validateSyncCommitteeMessage, /* validator */
s.syncCommitteeMessageSubscriber, /* message handler */
digest,
)
}
}
// From Capella hard fork
// New Gossip Topic in Capella
if epoch >= params.BeaconConfig().CapellaForkEpoch {
s.registerSubscribersFromCapella(digest)
s.subscribe(
p2p.BlsToExecutionChangeSubnetTopicFormat,
s.validateBlsToExecutionChange,
s.blsToExecutionChangeSubscriber,
digest,
)
}
// From Debeb hard fork
// New Gossip Topic in Deneb
if epoch >= params.BeaconConfig().DenebForkEpoch {
s.registerSubscribersFromDeneb(epoch, digest)
if coreTime.PeerDASIsActive(slots.UnsafeEpochStart(epoch)) {
if flags.Get().SubscribeToAllSubnets {
s.subscribeStaticWithSubnets(
p2p.DataColumnSubnetTopicFormat,
s.validateDataColumn, /* validator */
s.dataColumnSubscriber, /* message handler */
digest,
params.BeaconConfig().DataColumnSidecarSubnetCount,
)
} else {
s.subscribeDynamicWithColumnSubnets(
p2p.DataColumnSubnetTopicFormat,
s.validateDataColumn, /* validator */
s.dataColumnSubscriber, /* message handler */
digest,
)
}
} else {
s.subscribeStaticWithSubnets(
p2p.BlobSubnetTopicFormat,
s.validateBlob, /* validator */
s.blobSubscriber, /* message handler */
digest,
params.BeaconConfig().BlobsidecarSubnetCount,
)
}
}
}
@@ -447,7 +391,7 @@ func (s *Service) subscribeStaticWithSubnets(topic string, validator wrappedVal,
}
// Check every slot that there are enough peers
for i := uint64(0); i < subnetCount; i++ {
if !s.validPeersExist(s.addDigestAndIndexToTopic(topic, digest, i)) {
if !s.enoughPeersAreConnected(s.addDigestAndIndexToTopic(topic, digest, i)) {
_, err := s.cfg.p2p.FindPeersWithSubnet(
s.ctx,
s.addDigestAndIndexToTopic(topic, digest, i),
@@ -569,7 +513,7 @@ func (s *Service) subscribeAggregatorSubnet(
if _, exists := subscriptions[idx]; !exists {
subscriptions[idx] = s.subscribeWithBase(subnetTopic, validate, handle)
}
if !s.validPeersExist(subnetTopic) {
if !s.enoughPeersAreConnected(subnetTopic) {
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
if err != nil {
log.WithError(err).Debug("Could not search for peers")
@@ -622,7 +566,7 @@ func (s *Service) subscribeStaticWithSyncSubnets(topic string, validator wrapped
}
// Check every slot that there are enough peers
for i := uint64(0); i < params.BeaconConfig().SyncCommitteeSubnetCount; i++ {
if !s.validPeersExist(s.addDigestAndIndexToTopic(topic, digest, i)) {
if !s.enoughPeersAreConnected(s.addDigestAndIndexToTopic(topic, digest, i)) {
_, err := s.cfg.p2p.FindPeersWithSubnet(
s.ctx,
s.addDigestAndIndexToTopic(topic, digest, i),
@@ -702,7 +646,7 @@ func (s *Service) subscribeToSyncSubnets(
subnetTopic := fmt.Sprintf(topic, digest, subnetIndex)
// Check if we have enough peers in the subnet. Skip if we do.
if s.validPeersExist(subnetTopic) {
if s.enoughPeersAreConnected(subnetTopic) {
continue
}
@@ -733,13 +677,13 @@ func (s *Service) subscribeDynamicWithSyncSubnets(
genesisValidatorsRoot := s.cfg.clock.GenesisValidatorsRoot()
// Retrieve the epoch of the fork corresponding to the digest.
_, e, err := forks.RetrieveForkDataFromDigest(digest, genesisValidatorsRoot[:])
_, epoch, err := forks.RetrieveForkDataFromDigest(digest, genesisValidatorsRoot[:])
if err != nil {
panic(err)
}
// Retrieve the base protobuf message.
base := p2p.GossipTopicMappings(topicFormat, e)
base := p2p.GossipTopicMappings(topicFormat, epoch)
if base == nil {
panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topicFormat))
}
@@ -796,7 +740,7 @@ func (s *Service) subscribeColumnSubnet(
minimumPeersPerSubnet := flags.Get().MinimumPeersPerSubnet
if !s.validPeersExist(subnetTopic) {
if !s.enoughPeersAreConnected(subnetTopic) {
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, minimumPeersPerSubnet)
if err != nil {
log.WithError(err).Debug("Could not search for peers")
@@ -867,7 +811,7 @@ func (s *Service) subscribeDynamicWithColumnSubnets(
func (s *Service) lookupAttesterSubnets(digest [4]byte, idx uint64) {
topic := p2p.GossipTypeMapping[reflect.TypeOf(&ethpb.Attestation{})]
subnetTopic := fmt.Sprintf(topic, digest, idx)
if !s.validPeersExist(subnetTopic) {
if !s.enoughPeersAreConnected(subnetTopic) {
// perform a search for peers with the desired committee index.
_, err := s.cfg.p2p.FindPeersWithSubnet(s.ctx, subnetTopic, idx, flags.Get().MinimumPeersPerSubnet)
if err != nil {
@@ -891,8 +835,8 @@ func (s *Service) unSubscribeFromTopic(topic string) {
}
}
// find if we have peers who are subscribed to the same subnet
func (s *Service) validPeersExist(subnetTopic string) bool {
// enoughPeersAreConnected checks if we have enough peers which are subscribed to the same subnet.
func (s *Service) enoughPeersAreConnected(subnetTopic string) bool {
topic := subnetTopic + s.cfg.p2p.Encoding().ProtocolSuffix()
threshold := flags.Get().MinimumPeersPerSubnet

View File

@@ -10,9 +10,9 @@ import (
"google.golang.org/protobuf/proto"
)
// beaconAggregateAndProofSubscriber forwards the incoming validated aggregated attestation and proof to the
// beaconAggregateProofSubscriber forwards the incoming validated aggregated attestation and proof to the
// attestation pool for processing.
func (s *Service) beaconAggregateAndProofSubscriber(_ context.Context, msg proto.Message) error {
func (s *Service) beaconAggregateProofSubscriber(_ context.Context, msg proto.Message) error {
a, ok := msg.(ethpb.SignedAggregateAttAndProof)
if !ok {
return fmt.Errorf("message was not type ethpb.SignedAggregateAttAndProof, type=%T", msg)

View File

@@ -33,7 +33,7 @@ func TestBeaconAggregateProofSubscriber_CanSaveAggregatedAttestation(t *testing.
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
}
require.NoError(t, r.beaconAggregateAndProofSubscriber(context.Background(), a))
require.NoError(t, r.beaconAggregateProofSubscriber(context.Background(), a))
assert.DeepSSZEqual(t, []ethpb.Att{a.Message.Aggregate}, r.cfg.attPool.AggregatedAttestations(), "Did not save aggregated attestation")
}
@@ -55,7 +55,7 @@ func TestBeaconAggregateProofSubscriber_CanSaveUnaggregatedAttestation(t *testin
AggregatorIndex: 100,
},
}
require.NoError(t, r.beaconAggregateAndProofSubscriber(context.Background(), a))
require.NoError(t, r.beaconAggregateProofSubscriber(context.Background(), a))
atts, err := r.cfg.attPool.UnaggregatedAttestations()
require.NoError(t, err)

View File

@@ -14,7 +14,7 @@ import (
"google.golang.org/protobuf/proto"
)
func (s *Service) beaconAttestationSubscriber(_ context.Context, msg proto.Message) error {
func (s *Service) committeeIndexBeaconAttestationSubscriber(_ context.Context, msg proto.Message) error {
a, ok := msg.(eth.Att)
if !ok {
return fmt.Errorf("message was not type eth.Att, type=%T", msg)

View File

@@ -9,10 +9,10 @@ import (
"google.golang.org/protobuf/proto"
)
// syncCommitteeContributionAndProofSubscriber forwards the incoming validated sync contributions and proof to the
// syncContributionAndProofSubscriber forwards the incoming validated sync contributions and proof to the
// contribution pool for processing.
// skipcq: SCC-U1000
func (s *Service) syncCommitteeContributionAndProofSubscriber(_ context.Context, msg proto.Message) error {
func (s *Service) syncContributionAndProofSubscriber(_ context.Context, msg proto.Message) error {
sContr, ok := msg.(*ethpb.SignedContributionAndProof)
if !ok {
return fmt.Errorf("message was not type *ethpb.SignedContributionAndProof, type=%T", msg)

View File

@@ -350,7 +350,7 @@ func Test_wrapAndReportValidation(t *testing.T) {
}
fd, err := forks.CreateForkDigest(mChain.GenesisTime(), mChain.ValidatorsRoot[:])
assert.NoError(t, err)
mockTopic := fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, fd) + encoder.SszNetworkEncoder{}.ProtocolSuffix()
mockTopic := fmt.Sprintf(p2p.BlockSubnetTopicFormat, fd) + encoder.SszNetworkEncoder{}.ProtocolSuffix()
type args struct {
topic string
v wrappedVal

View File

@@ -23,7 +23,7 @@ func TestSubTopicHandler_CRUD(t *testing.T) {
enc := encoder.SszNetworkEncoder{}
// Valid topic added in.
topic := fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic := fmt.Sprintf(p2p.BlockSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.addTopic(topic, new(pubsub.Subscription))
assert.Equal(t, true, h.topicExists(topic))
assert.Equal(t, true, h.digestExists(digest))
@@ -36,11 +36,11 @@ func TestSubTopicHandler_CRUD(t *testing.T) {
h = newSubTopicHandler()
// Multiple Topics added in.
topic = fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.BlockSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.addTopic(topic, new(pubsub.Subscription))
assert.Equal(t, true, h.topicExists(topic))
topic = fmt.Sprintf(p2p.VoluntaryExitSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.ExitSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.addTopic(topic, new(pubsub.Subscription))
assert.Equal(t, true, h.topicExists(topic))
@@ -52,11 +52,11 @@ func TestSubTopicHandler_CRUD(t *testing.T) {
h.addTopic(topic, new(pubsub.Subscription))
assert.Equal(t, true, h.topicExists(topic))
topic = fmt.Sprintf(p2p.BeaconAggregateAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.AggregateAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.addTopic(topic, new(pubsub.Subscription))
assert.Equal(t, true, h.topicExists(topic))
topic = fmt.Sprintf(p2p.SyncCommitteeContributionAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.SyncContributionAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.addTopic(topic, new(pubsub.Subscription))
assert.Equal(t, true, h.topicExists(topic))
@@ -71,7 +71,7 @@ func TestSubTopicHandler_CRUD(t *testing.T) {
h.removeTopic(topic)
assert.Equal(t, false, h.topicExists(topic))
topic = fmt.Sprintf(p2p.VoluntaryExitSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.ExitSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.removeTopic(topic)
assert.Equal(t, false, h.topicExists(topic))
@@ -83,15 +83,15 @@ func TestSubTopicHandler_CRUD(t *testing.T) {
assert.Equal(t, 4, len(h.allTopics()))
// Remove remaining topics.
topic = fmt.Sprintf(p2p.BeaconBlockSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.BlockSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.removeTopic(topic)
assert.Equal(t, false, h.topicExists(topic))
topic = fmt.Sprintf(p2p.BeaconAggregateAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.AggregateAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.removeTopic(topic)
assert.Equal(t, false, h.topicExists(topic))
topic = fmt.Sprintf(p2p.SyncCommitteeContributionAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
topic = fmt.Sprintf(p2p.SyncContributionAndProofSubnetTopicFormat, digest) + enc.ProtocolSuffix()
h.removeTopic(topic)
assert.Equal(t, false, h.topicExists(topic))

View File

@@ -108,7 +108,7 @@ func FuzzValidateBeaconBlockPubSub_Phase0(f *testing.F) {
Topic: &strTop,
},
}
_, err := r.validateBeaconBlockPubSubMsg(ctx, peer.ID(pid), msg)
_, err := r.validateBeaconBlockPubSub(ctx, peer.ID(pid), msg)
_ = err
})
}
@@ -190,7 +190,7 @@ func FuzzValidateBeaconBlockPubSub_Altair(f *testing.F) {
Topic: &strTop,
},
}
_, err := r.validateBeaconBlockPubSubMsg(ctx, peer.ID(pid), msg)
_, err := r.validateBeaconBlockPubSub(ctx, peer.ID(pid), msg)
_ = err
})
}
@@ -272,7 +272,7 @@ func FuzzValidateBeaconBlockPubSub_Bellatrix(f *testing.F) {
Topic: &strTop,
},
}
_, err := r.validateBeaconBlockPubSubMsg(ctx, peer.ID(pid), msg)
_, err := r.validateBeaconBlockPubSub(ctx, peer.ID(pid), msg)
_ = err
})
}

View File

@@ -26,9 +26,9 @@ import (
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
// validateBeaconAggregateAndProofPubSubMsg verifies the aggregated signature and the selection proof is valid before forwarding to the
// validateAggregateAndProof verifies the aggregated signature and the selection proof is valid before forwarding to the
// network and downstream services.
func (s *Service) validateBeaconAggregateAndProofPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateAggregateAndProof(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
receivedTime := prysmTime.Now()
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil

View File

@@ -185,7 +185,7 @@ func TestValidateAggregateAndProof_NoBlock(t *testing.T) {
},
}
if res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg); res == pubsub.ValidationAccept {
if res, err := r.validateAggregateAndProof(context.Background(), "", msg); res == pubsub.ValidationAccept {
_ = err
t.Error("Expected validate to fail")
}
@@ -255,7 +255,7 @@ func TestValidateAggregateAndProof_NotWithinSlotRange(t *testing.T) {
},
}
if res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg); res == pubsub.ValidationAccept {
if res, err := r.validateAggregateAndProof(context.Background(), "", msg); res == pubsub.ValidationAccept {
_ = err
t.Error("Expected validate to fail")
}
@@ -272,7 +272,7 @@ func TestValidateAggregateAndProof_NotWithinSlotRange(t *testing.T) {
Topic: &topic,
},
}
if res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg); res == pubsub.ValidationAccept {
if res, err := r.validateAggregateAndProof(context.Background(), "", msg); res == pubsub.ValidationAccept {
_ = err
t.Error("Expected validate to fail")
}
@@ -338,7 +338,7 @@ func TestValidateAggregateAndProof_ExistedInPool(t *testing.T) {
}
require.NoError(t, r.cfg.attPool.SaveBlockAttestation(att))
if res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg); res == pubsub.ValidationAccept {
if res, err := r.validateAggregateAndProof(context.Background(), "", msg); res == pubsub.ValidationAccept {
_ = err
t.Error("Expected validate to fail")
}
@@ -442,7 +442,7 @@ func TestValidateAggregateAndProof_CanValidate(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg)
res, err := r.validateAggregateAndProof(context.Background(), "", msg)
assert.NoError(t, err)
assert.Equal(t, pubsub.ValidationAccept, res, "Validated status is false")
assert.NotNil(t, msg.ValidatorData, "Did not set validator data")
@@ -545,7 +545,7 @@ func TestVerifyIndexInCommittee_SeenAggregatorEpoch(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg)
res, err := r.validateAggregateAndProof(context.Background(), "", msg)
assert.NoError(t, err)
require.Equal(t, pubsub.ValidationAccept, res, "Validated status is false")
@@ -562,7 +562,7 @@ func TestVerifyIndexInCommittee_SeenAggregatorEpoch(t *testing.T) {
}
time.Sleep(10 * time.Millisecond) // Wait for cached value to pass through buffers.
if res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg); res == pubsub.ValidationAccept {
if res, err := r.validateAggregateAndProof(context.Background(), "", msg); res == pubsub.ValidationAccept {
_ = err
t.Fatal("Validated status is true")
}
@@ -654,7 +654,7 @@ func TestValidateAggregateAndProof_BadBlock(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg)
res, err := r.validateAggregateAndProof(context.Background(), "", msg)
assert.NotNil(t, err)
assert.Equal(t, pubsub.ValidationReject, res, "Validated status is true")
}
@@ -744,7 +744,7 @@ func TestValidateAggregateAndProof_RejectWhenAttEpochDoesntEqualTargetEpoch(t *t
Topic: &topic,
},
}
res, err := r.validateBeaconAggregateAndProofPubSubMsg(context.Background(), "", msg)
res, err := r.validateAggregateAndProof(context.Background(), "", msg)
assert.NotNil(t, err)
assert.Equal(t, pubsub.ValidationReject, res)
}

View File

@@ -20,7 +20,7 @@ import (
// Clients who receive an attester slashing on this topic MUST validate the conditions within VerifyAttesterSlashing before
// forwarding it across the network.
func (s *Service) validateAttesterSlashingPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateAttesterSlashing(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {

View File

@@ -108,7 +108,7 @@ func TestValidateAttesterSlashing_ValidSlashing(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err := r.validateAttesterSlashing(ctx, "foobar", msg)
assert.NoError(t, err)
valid := res == pubsub.ValidationAccept
@@ -153,7 +153,7 @@ func TestValidateAttesterSlashing_ValidOldSlashing(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err := r.validateAttesterSlashing(ctx, "foobar", msg)
assert.ErrorContains(t, "validators were previously slashed", err)
valid := res == pubsub.ValidationIgnore
@@ -198,7 +198,7 @@ func TestValidateAttesterSlashing_InvalidSlashing_WithdrawableEpoch(t *testing.T
Topic: &topic,
},
}
res, err := r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err := r.validateAttesterSlashing(ctx, "foobar", msg)
assert.NoError(t, err)
valid := res == pubsub.ValidationAccept
@@ -211,7 +211,7 @@ func TestValidateAttesterSlashing_InvalidSlashing_WithdrawableEpoch(t *testing.T
}
require.NoError(t, s.SetValidators(vals))
res, err = r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err = r.validateAttesterSlashing(ctx, "foobar", msg)
assert.ErrorContains(t, "none of the validators are slashable", err)
invalid := res == pubsub.ValidationReject
@@ -257,7 +257,7 @@ func TestValidateAttesterSlashing_CanFilter(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err := r.validateAttesterSlashing(ctx, "foobar", msg)
_ = err
ignored := res == pubsub.ValidationIgnore
assert.Equal(t, true, ignored)
@@ -278,7 +278,7 @@ func TestValidateAttesterSlashing_CanFilter(t *testing.T) {
Topic: &topic,
},
}
res, err = r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err = r.validateAttesterSlashing(ctx, "foobar", msg)
_ = err
ignored = res == pubsub.ValidationIgnore
assert.Equal(t, true, ignored)
@@ -315,7 +315,7 @@ func TestValidateAttesterSlashing_ContextTimeout(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err := r.validateAttesterSlashing(ctx, "foobar", msg)
_ = err
valid := res == pubsub.ValidationAccept
assert.Equal(t, false, valid, "slashing from the far distant future should have timed out and returned false")
@@ -346,7 +346,7 @@ func TestValidateAttesterSlashing_Syncing(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateAttesterSlashingPubSubMsg(ctx, "foobar", msg)
res, err := r.validateAttesterSlashing(ctx, "foobar", msg)
_ = err
valid := res == pubsub.ValidationAccept
assert.Equal(t, false, valid, "Passed validation")

View File

@@ -35,7 +35,7 @@ import (
// - The attestation is unaggregated -- that is, it has exactly one participating validator (len(get_attesting_indices(state, attestation.data, attestation.aggregation_bits)) == 1).
// - attestation.data.slot is within the last ATTESTATION_PROPAGATION_SLOT_RANGE slots (attestation.data.slot + ATTESTATION_PROPAGATION_SLOT_RANGE >= current_slot >= attestation.data.slot).
// - The signature of attestation is valid.
func (s *Service) validateBeaconAttestationPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateCommitteeIndexBeaconAttestation(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}

View File

@@ -290,7 +290,7 @@ func TestService_validateCommitteeIndexBeaconAttestation(t *testing.T) {
m.Message.Topic = nil
}
res, err := s.validateBeaconAttestationPubSubMsg(ctx, "", m)
res, err := s.validateCommitteeIndexBeaconAttestation(ctx, "", m)
received := res == pubsub.ValidationAccept
if received != tt.want {
t.Fatalf("Did not received wanted validation. Got %v, wanted %v", !tt.want, tt.want)

View File

@@ -35,10 +35,10 @@ var (
errRejectCommitmentLen = errors.New("[REJECT] The length of KZG commitments is less than or equal to the limitation defined in Consensus Layer")
)
// validateBeaconBlockPubSubMsg checks that the incoming block has a valid BLS signature.
// validateBeaconBlockPubSub checks that the incoming block has a valid BLS signature.
// Blocks that have already been seen are ignored. If the BLS signature is any valid signature,
// this method rebroadcasts the message.
func (s *Service) validateBeaconBlockPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
receivedTime := prysmTime.Now()
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.

View File

@@ -101,7 +101,7 @@ func TestValidateBeaconBlockPubSub_InvalidSignature(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.ErrorIs(t, err, signing.ErrSigFailedToVerify)
result := res == pubsub.ValidationReject
assert.Equal(t, true, result)
@@ -145,7 +145,7 @@ func TestValidateBeaconBlockPubSub_BlockAlreadyPresentInDB(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
assert.Equal(t, res, pubsub.ValidationIgnore, "block present in DB should be ignored")
}
@@ -208,7 +208,7 @@ func TestValidateBeaconBlockPubSub_CanRecoverStateSummary(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
result := res == pubsub.ValidationAccept
assert.Equal(t, true, result)
@@ -274,7 +274,7 @@ func TestValidateBeaconBlockPubSub_IsInCache(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
result := res == pubsub.ValidationAccept
assert.Equal(t, true, result)
@@ -340,7 +340,7 @@ func TestValidateBeaconBlockPubSub_ValidProposerSignature(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
result := res == pubsub.ValidationAccept
assert.Equal(t, true, result)
@@ -409,7 +409,7 @@ func TestValidateBeaconBlockPubSub_WithLookahead(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
result := res == pubsub.ValidationAccept
assert.Equal(t, true, result)
@@ -477,7 +477,7 @@ func TestValidateBeaconBlockPubSub_AdvanceEpochsForState(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
result := res == pubsub.ValidationAccept
assert.Equal(t, true, result)
@@ -520,7 +520,7 @@ func TestValidateBeaconBlockPubSub_Syncing(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
assert.Equal(t, res, pubsub.ValidationIgnore, "block is ignored until fully synced")
}
@@ -586,7 +586,7 @@ func TestValidateBeaconBlockPubSub_IgnoreAndQueueBlocksFromNearFuture(t *testing
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.ErrorContains(t, "early block, with current slot", err)
assert.Equal(t, res, pubsub.ValidationIgnore, "early block should be ignored and queued")
@@ -637,7 +637,7 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromFuture(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
assert.Equal(t, res, pubsub.ValidationIgnore, "block from the future should be ignored")
}
@@ -688,7 +688,7 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromThePast(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.ErrorContains(t, "greater or equal to block slot", err)
assert.Equal(t, res, pubsub.ValidationIgnore, "block from the past should be ignored")
}
@@ -750,7 +750,7 @@ func TestValidateBeaconBlockPubSub_SeenProposerSlot(t *testing.T) {
}
r.setSeenBlockIndexSlot(msg.Block.Slot, msg.Block.ProposerIndex)
time.Sleep(10 * time.Millisecond) // Wait for cached value to pass through buffers.
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.NoError(t, err)
assert.Equal(t, res, pubsub.ValidationIgnore, "seen proposer block should be ignored")
}
@@ -801,7 +801,7 @@ func TestValidateBeaconBlockPubSub_FilterByFinalizedEpoch(t *testing.T) {
},
}
res, err := r.validateBeaconBlockPubSubMsg(context.Background(), "", m)
res, err := r.validateBeaconBlockPubSub(context.Background(), "", m)
_ = err
assert.Equal(t, pubsub.ValidationIgnore, res)
@@ -817,7 +817,7 @@ func TestValidateBeaconBlockPubSub_FilterByFinalizedEpoch(t *testing.T) {
},
}
res, err = r.validateBeaconBlockPubSubMsg(context.Background(), "", m)
res, err = r.validateBeaconBlockPubSub(context.Background(), "", m)
assert.NoError(t, err)
assert.Equal(t, pubsub.ValidationIgnore, res)
}
@@ -884,7 +884,7 @@ func TestValidateBeaconBlockPubSub_ParentNotFinalizedDescendant(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.Equal(t, pubsub.ValidationReject, res, "Wrong validation result returned")
require.ErrorContains(t, "not descendant of finalized checkpoint", err)
}
@@ -950,7 +950,7 @@ func TestValidateBeaconBlockPubSub_InvalidParentBlock(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.ErrorContains(t, "could not unmarshal bytes into signature", err)
assert.Equal(t, res, pubsub.ValidationReject, "block with invalid signature should be rejected")
@@ -982,7 +982,7 @@ func TestValidateBeaconBlockPubSub_InvalidParentBlock(t *testing.T) {
r.cfg.chain = chainService
r.cfg.clock = startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)
res, err = r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err = r.validateBeaconBlockPubSub(ctx, "", m)
require.ErrorContains(t, "has an invalid parent", err)
// Expect block with bad parent to fail too
assert.Equal(t, res, pubsub.ValidationReject, "block with invalid parent should be ignored")
@@ -1044,7 +1044,7 @@ func TestValidateBeaconBlockPubSub_InsertValidPendingBlock(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.ErrorContains(t, "unknown parent for block", err)
assert.Equal(t, res, pubsub.ValidationIgnore, "block with unknown parent should be ignored")
bRoot, err = msg.Block.HashTreeRoot()
@@ -1129,7 +1129,7 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromBadParent(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
assert.ErrorContains(t, "invalid parent", err)
assert.Equal(t, res, pubsub.ValidationReject)
}
@@ -1230,7 +1230,7 @@ func TestValidateBeaconBlockPubSub_ValidExecutionPayload(t *testing.T) {
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.NoError(t, err)
result := res == pubsub.ValidationAccept
require.Equal(t, true, result)
@@ -1301,7 +1301,7 @@ func TestValidateBeaconBlockPubSub_InvalidPayloadTimestamp(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.NotNil(t, err)
result := res == pubsub.ValidationReject
assert.Equal(t, true, result)
@@ -1462,7 +1462,7 @@ func Test_validateBeaconBlockProcessingWhenParentIsOptimistic(t *testing.T) {
},
}
res, err := r.validateBeaconBlockPubSubMsg(ctx, "", m)
res, err := r.validateBeaconBlockPubSub(ctx, "", m)
require.NoError(t, err)
result := res == pubsub.ValidationAccept
assert.Equal(t, true, result)

View File

@@ -9,6 +9,7 @@ import (
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
@@ -18,6 +19,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/runtime/logging"
prysmTime "github.com/prysmaticlabs/prysm/v5/time"
"github.com/prysmaticlabs/prysm/v5/time/slots"
"github.com/sirupsen/logrus"
)
// https://github.com/ethereum/consensus-specs/blob/dev/specs/_features/eip7594/p2p-interface.md#the-gossip-domain-gossipsub
@@ -58,6 +60,18 @@ func (s *Service) validateDataColumn(ctx context.Context, pid peer.ID, msg *pubs
return pubsub.ValidationReject, errors.Wrap(err, "roDataColumn conversion failure")
}
dataColumnsRejectSlotMultiple := features.Get().DataColumnsRejectSlotMultiple
blockSlot := uint64(ds.SignedBlockHeader.Header.Slot)
if dataColumnsRejectSlotMultiple != 0 && blockSlot%dataColumnsRejectSlotMultiple == 0 {
log.WithFields(logrus.Fields{
"slot": blockSlot,
"topic": msg.Topic,
}).Warning("Voluntary ignore data column sidecar gossip")
return pubsub.ValidationIgnore, err
}
verifier := s.newColumnVerifier(ds, verification.GossipColumnSidecarRequirements)
if err := verifier.DataColumnIndexInBounds(); err != nil {
@@ -130,13 +144,20 @@ func (s *Service) validateDataColumn(ctx context.Context, pid peer.ID, msg *pubs
msg.ValidatorData = verifiedRODataColumn
fields := logging.DataColumnFields(ds)
sinceSlotStartTime := receivedTime.Sub(startTime)
validationTime := s.cfg.clock.Now().Sub(receivedTime)
fields["sinceSlotStartTime"] = sinceSlotStartTime
fields["validationTime"] = validationTime
log.WithFields(fields).Debug("Accepted data column sidecar gossip")
peerGossipScore := s.cfg.p2p.Peers().Scorers().GossipScorer().Score(pid)
log.
WithFields(logging.DataColumnFields(ds)).
WithFields(logrus.Fields{
"sinceSlotStartTime": sinceSlotStartTime,
"validationTime": validationTime,
"peer": pid[len(pid)-6:],
"peerGossipScore": peerGossipScore,
}).
Debug("Accepted data column sidecar gossip")
return pubsub.ValidationAccept, nil
}

View File

@@ -17,7 +17,7 @@ import (
// Clients who receive a proposer slashing on this topic MUST validate the conditions within VerifyProposerSlashing before
// forwarding it across the network.
func (s *Service) validateProposerSlashingPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateProposerSlashing(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {

View File

@@ -140,7 +140,7 @@ func TestValidateProposerSlashing_ValidSlashing(t *testing.T) {
},
}
res, err := r.validateProposerSlashingPubSubMsg(ctx, "", m)
res, err := r.validateProposerSlashing(ctx, "", m)
assert.NoError(t, err)
valid := res == pubsub.ValidationAccept
assert.Equal(t, true, valid, "Failed validation")
@@ -183,7 +183,7 @@ func TestValidateProposerSlashing_ValidOldSlashing(t *testing.T) {
},
}
res, err := r.validateProposerSlashingPubSubMsg(ctx, "", m)
res, err := r.validateProposerSlashing(ctx, "", m)
assert.ErrorContains(t, "proposer is already slashed", err)
valid := res == pubsub.ValidationIgnore
assert.Equal(t, true, valid, "Failed validation")
@@ -220,7 +220,7 @@ func TestValidateProposerSlashing_ContextTimeout(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateProposerSlashingPubSubMsg(ctx, "", m)
res, err := r.validateProposerSlashing(ctx, "", m)
assert.NotNil(t, err)
valid := res == pubsub.ValidationAccept
assert.Equal(t, false, valid, "Slashing from the far distant future should have timed out and returned false")
@@ -250,7 +250,7 @@ func TestValidateProposerSlashing_Syncing(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateProposerSlashingPubSubMsg(ctx, "", m)
res, err := r.validateProposerSlashing(ctx, "", m)
_ = err
valid := res == pubsub.ValidationAccept
assert.Equal(t, false, valid, "Did not fail validation")

View File

@@ -20,7 +20,7 @@ import (
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
// validateSyncCommitteeContributionAndProofPubSubMsg verifies the aggregated signature and the selection proof is valid before forwarding to the
// validateSyncContributionAndProof verifies the aggregated signature and the selection proof is valid before forwarding to the
// network and downstream services.
// Gossip Validation Conditions:
// [IGNORE] The contribution's slot is for the current slot (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance), i.e. contribution.slot == current_slot.
@@ -38,7 +38,7 @@ import (
// [REJECT] The aggregator signature, signed_contribution_and_proof.signature, is valid.
// [REJECT] The aggregate signature is valid for the message beacon_block_root and aggregate pubkey derived from the participation
// info in aggregation_bits for the subcommittee specified by the contribution.subcommittee_index.
func (s *Service) validateSyncCommitteeContributionAndProofPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateSyncContributionAndProof(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
ctx, span := trace.StartSpan(ctx, "sync.validateSyncContributionAndProof")
defer span.End()

View File

@@ -44,7 +44,7 @@ import (
func TestService_ValidateSyncContributionAndProof(t *testing.T) {
database := testingdb.SetupDB(t)
headRoot, keys := fillUpBlocksAndState(context.Background(), t, database)
defaultTopic := p2p.SyncCommitteeContributionAndProofSubnetTopicFormat
defaultTopic := p2p.SyncContributionAndProofSubnetTopicFormat
defaultTopic = fmt.Sprintf(defaultTopic, []byte{0xAB, 0x00, 0xCC, 0x9E})
defaultTopic = defaultTopic + "/" + encoder.ProtocolSuffixSSZSnappy
chainService := &mockChain.ChainService{
@@ -878,7 +878,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
}
}
require.Equal(t, true, svc.chainIsStarted())
if got, err := svc.validateSyncCommitteeContributionAndProofPubSubMsg(ctx, tt.args.pid, msg); got != tt.want {
if got, err := svc.validateSyncContributionAndProof(ctx, tt.args.pid, msg); got != tt.want {
_ = err
t.Errorf("validateSyncContributionAndProof() = %v, want %v", got, tt.want)
}
@@ -890,7 +890,7 @@ func TestValidateSyncContributionAndProof(t *testing.T) {
ctx := context.Background()
database := testingdb.SetupDB(t)
headRoot, keys := fillUpBlocksAndState(ctx, t, database)
defaultTopic := p2p.SyncCommitteeContributionAndProofSubnetTopicFormat
defaultTopic := p2p.SyncContributionAndProofSubnetTopicFormat
defaultTopic = fmt.Sprintf(defaultTopic, []byte{0xAB, 0x00, 0xCC, 0x9E})
defaultTopic = defaultTopic + "/" + encoder.ProtocolSuffixSSZSnappy
var emptySig [96]byte
@@ -1004,7 +1004,7 @@ func TestValidateSyncContributionAndProof(t *testing.T) {
opSub := s.cfg.operationNotifier.OperationFeed().Subscribe(opChannel)
defer opSub.Unsubscribe()
_, err = s.validateSyncCommitteeContributionAndProofPubSubMsg(ctx, pid, pubsubMsg)
_, err = s.validateSyncContributionAndProof(ctx, pid, pubsubMsg)
require.NoError(t, err)
// Ensure the state notification was broadcast.

View File

@@ -17,7 +17,7 @@ import (
// Clients who receive a voluntary exit on this topic MUST validate the conditions within process_voluntary_exit before
// forwarding it across the network.
func (s *Service) validateVoluntaryExitPubSubMsg(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
func (s *Service) validateVoluntaryExit(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == s.cfg.p2p.PeerID() {

View File

@@ -118,7 +118,7 @@ func TestValidateVoluntaryExit_ValidExit(t *testing.T) {
opSub := r.cfg.operationNotifier.OperationFeed().Subscribe(opChannel)
defer opSub.Unsubscribe()
res, err := r.validateVoluntaryExitPubSubMsg(ctx, "", m)
res, err := r.validateVoluntaryExit(ctx, "", m)
require.NoError(t, err)
require.Equal(t, pubsub.ValidationAccept, res, "Failed validation")
require.NotNil(t, m.ValidatorData, "Decoded message was not set on the message validator data")
@@ -166,7 +166,7 @@ func TestValidateVoluntaryExit_InvalidExitSlot(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateVoluntaryExitPubSubMsg(ctx, "", m)
res, err := r.validateVoluntaryExit(ctx, "", m)
_ = err
valid := res == pubsub.ValidationAccept
assert.Equal(t, false, valid, "passed validation")
@@ -197,7 +197,7 @@ func TestValidateVoluntaryExit_ValidExit_Syncing(t *testing.T) {
Topic: &topic,
},
}
res, err := r.validateVoluntaryExitPubSubMsg(ctx, "", m)
res, err := r.validateVoluntaryExit(ctx, "", m)
_ = err
valid := res == pubsub.ValidationAccept
assert.Equal(t, false, valid, "Validation should have failed")

View File

@@ -82,9 +82,12 @@ type Flags struct {
// changed on disk. This feature is for advanced use cases only.
KeystoreImportDebounceInterval time.Duration
// DataColumnsWithholdCount specifies the likelihood of withholding a data column sidecar when proposing a block (percentage)
// DataColumnsWithholdCount specifies the number of data columns that should be withheld when proposing a block.
DataColumnsWithholdCount uint64
// DataColumnsRejectSlotMultiple specifies the multiple of slot number where data columns should be rejected.
DataColumnsRejectSlotMultiple uint64
// AggregateIntervals specifies the time durations at which we aggregate attestations preparing for forkchoice.
AggregateIntervals [3]time.Duration
}
@@ -269,6 +272,11 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
cfg.DataColumnsWithholdCount = ctx.Uint64(DataColumnsWithholdCount.Name)
}
if ctx.IsSet(DataColumnsRejectSlotMultiple.Name) {
logEnabled(DataColumnsRejectSlotMultiple)
cfg.DataColumnsRejectSlotMultiple = ctx.Uint64(DataColumnsRejectSlotMultiple.Name)
}
cfg.AggregateIntervals = [3]time.Duration{aggregateFirstInterval.Value, aggregateSecondInterval.Value, aggregateThirdInterval.Value}
Init(cfg)
return nil

View File

@@ -182,6 +182,13 @@ var (
Value: 0,
Hidden: true,
}
// DataColumnsWithholdCount is a flag for withholding data columns when proposing a block.
DataColumnsRejectSlotMultiple = &cli.Uint64Flag{
Name: "data-columns-reject-slot-multiple",
Usage: "Reject all data columns for slots that are a multiple of this value. DO NOT USE IN PRODUCTION.",
Value: 0,
Hidden: true,
}
)
// devModeFlags holds list of flags that are set when development mode is on.
@@ -242,6 +249,7 @@ var BeaconChainFlags = append(deprecatedBeaconFlags, append(deprecatedFlags, []c
EnableCommitteeAwarePacking,
EnablePeerDAS,
DataColumnsWithholdCount,
DataColumnsRejectSlotMultiple,
}...)...)
// E2EBeaconChainFlags contains a list of the beacon chain feature flags to be tested in E2E.

View File

@@ -6,7 +6,6 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v5/consensus-types/wrapper",
visibility = ["//visibility:public"],
deps = [
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/metadata:go_default_library",
"//runtime/version:go_default_library",

View File

@@ -2,7 +2,6 @@ package wrapper
import (
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1/metadata"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
@@ -38,7 +37,7 @@ func (m MetadataV0) SyncnetsBitfield() bitfield.Bitvector4 {
}
// CustodySubnetCount returns custody subnet count from the metadata.
func (m MetadataV0) CustodySubnetCount() uint8 {
func (m MetadataV0) CustodySubnetCount() uint64 {
return 0
}
@@ -132,7 +131,7 @@ func (m MetadataV1) SyncnetsBitfield() bitfield.Bitvector4 {
}
// CustodySubnetCount returns custody subnet count from the metadata.
func (m MetadataV1) CustodySubnetCount() uint8 {
func (m MetadataV1) CustodySubnetCount() uint64 {
return 0
}
@@ -226,8 +225,8 @@ func (m MetadataV2) SyncnetsBitfield() bitfield.Bitvector4 {
}
// CustodySubnetCount returns custody subnet count from the metadata.
func (m MetadataV2) CustodySubnetCount() uint8 {
return bytesutil.FromBytes1(m.md.CustodySubnetCount)
func (m MetadataV2) CustodySubnetCount() uint64 {
return m.md.CustodySubnetCount
}
// InnerObject returns the underlying metadata protobuf structure.

View File

@@ -66,14 +66,6 @@ func Bytes32(x uint64) []byte {
return bytes
}
// FromBytes1 returns an integer from a byte slice with a size of 1.
func FromBytes1(x []byte) uint8 {
if len(x) < 1 {
return 0
}
return x[0]
}
// FromBytes2 returns an integer which is stored in the little-endian format(2, 'little')
// from a byte array.
func FromBytes2(x []byte) uint16 {

View File

@@ -11,7 +11,7 @@ type Metadata interface {
SequenceNumber() uint64
AttnetsBitfield() bitfield.Bitvector64
SyncnetsBitfield() bitfield.Bitvector4
CustodySubnetCount() uint8
CustodySubnetCount() uint64
InnerObject() interface{}
IsNil() bool
Copy() Metadata

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 2b6089e02fcaeb5244f5bd8568b0d90eca6cccd827a21f19affac4a07bb8d355
// Hash: ac6fbb6f912d4e5a3374b91a3dc73c1d5ac1e5b4a9c32513b3d5ef9de7885be8
package eth
import (
@@ -578,11 +578,7 @@ func (m *MetaDataV2) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = append(dst, m.Syncnets...)
// Field (3) 'CustodySubnetCount'
if size := len(m.CustodySubnetCount); size != 1 {
err = ssz.ErrBytesLengthFn("--.CustodySubnetCount", size, 1)
return
}
dst = append(dst, m.CustodySubnetCount...)
dst = ssz.MarshalUint64(dst, m.CustodySubnetCount)
return
}
@@ -591,7 +587,7 @@ func (m *MetaDataV2) MarshalSSZTo(buf []byte) (dst []byte, err error) {
func (m *MetaDataV2) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size != 18 {
if size != 25 {
return ssz.ErrSize
}
@@ -611,17 +607,14 @@ func (m *MetaDataV2) UnmarshalSSZ(buf []byte) error {
m.Syncnets = append(m.Syncnets, buf[16:17]...)
// Field (3) 'CustodySubnetCount'
if cap(m.CustodySubnetCount) == 0 {
m.CustodySubnetCount = make([]byte, 0, len(buf[17:18]))
}
m.CustodySubnetCount = append(m.CustodySubnetCount, buf[17:18]...)
m.CustodySubnetCount = ssz.UnmarshallUint64(buf[17:25])
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the MetaDataV2 object
func (m *MetaDataV2) SizeSSZ() (size int) {
size = 18
size = 25
return
}
@@ -652,11 +645,7 @@ func (m *MetaDataV2) HashTreeRootWith(hh *ssz.Hasher) (err error) {
hh.PutBytes(m.Syncnets)
// Field (3) 'CustodySubnetCount'
if size := len(m.CustodySubnetCount); size != 1 {
err = ssz.ErrBytesLengthFn("--.CustodySubnetCount", size, 1)
return
}
hh.PutBytes(m.CustodySubnetCount)
hh.PutUint64(m.CustodySubnetCount)
hh.Merkleize(indx)
return

View File

@@ -356,7 +356,7 @@ type MetaDataV2 struct {
SeqNumber uint64 `protobuf:"varint,1,opt,name=seq_number,json=seqNumber,proto3" json:"seq_number,omitempty"`
Attnets github_com_prysmaticlabs_go_bitfield.Bitvector64 `protobuf:"bytes,2,opt,name=attnets,proto3" json:"attnets,omitempty" cast-type:"github.com/prysmaticlabs/go-bitfield.Bitvector64" ssz-size:"8"`
Syncnets github_com_prysmaticlabs_go_bitfield.Bitvector4 `protobuf:"bytes,3,opt,name=syncnets,proto3" json:"syncnets,omitempty" cast-type:"github.com/prysmaticlabs/go-bitfield.Bitvector4" ssz-size:"1"`
CustodySubnetCount []byte `protobuf:"bytes,4,opt,name=custody_subnet_count,json=custodySubnetCount,proto3" json:"custody_subnet_count,omitempty" ssz-size:"1"`
CustodySubnetCount uint64 `protobuf:"varint,4,opt,name=custody_subnet_count,json=custodySubnetCount,proto3" json:"custody_subnet_count,omitempty"`
}
func (x *MetaDataV2) Reset() {
@@ -412,11 +412,11 @@ func (x *MetaDataV2) GetSyncnets() github_com_prysmaticlabs_go_bitfield.Bitvecto
return github_com_prysmaticlabs_go_bitfield.Bitvector4(nil)
}
func (x *MetaDataV2) GetCustodySubnetCount() []byte {
func (x *MetaDataV2) GetCustodySubnetCount() uint64 {
if x != nil {
return x.CustodySubnetCount
}
return nil
return 0
}
type BlobSidecarsByRangeRequest struct {
@@ -616,7 +616,7 @@ var file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc = []byte{
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66,
0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69, 0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x34, 0x8a,
0xb5, 0x18, 0x01, 0x31, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x22, 0x8f,
0xb5, 0x18, 0x01, 0x31, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x22, 0x88,
0x02, 0x0a, 0x0a, 0x4d, 0x65, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x56, 0x32, 0x12, 0x1d, 0x0a,
0x0a, 0x73, 0x65, 0x71, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
0x04, 0x52, 0x09, 0x73, 0x65, 0x71, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x07,
@@ -630,43 +630,42 @@ var file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc = []byte{
0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62,
0x73, 0x2f, 0x67, 0x6f, 0x2d, 0x62, 0x69, 0x74, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x2e, 0x42, 0x69,
0x74, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x31, 0x52, 0x08, 0x73,
0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x12, 0x37, 0x0a, 0x14, 0x63, 0x75, 0x73, 0x74, 0x6f,
0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x63, 0x75, 0x73, 0x74, 0x6f,
0x64, 0x79, 0x5f, 0x73, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x05, 0x8a, 0xb5, 0x18, 0x01, 0x31, 0x52, 0x12, 0x63, 0x75,
0x73, 0x74, 0x6f, 0x64, 0x79, 0x53, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74,
0x22, 0x98, 0x01, 0x0a, 0x1a, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72,
0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x64, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20,
0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62,
0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65,
0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69,
0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72,
0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02,
0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xc1, 0x01, 0x0a, 0x20,
0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61,
0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x12, 0x64, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01,
0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61,
0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73,
0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d,
0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61,
0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x07,
0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x07, 0x92,
0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x42,
0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, 0x50,
0x32, 0x50, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50,
0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72,
0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x64, 0x79, 0x53, 0x75,
0x62, 0x6e, 0x65, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x98, 0x01, 0x0a, 0x1a, 0x42, 0x6c,
0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e, 0x67,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x64, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72,
0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5,
0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79,
0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x53,
0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x14,
0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x63,
0x6f, 0x75, 0x6e, 0x74, 0x22, 0xc1, 0x01, 0x0a, 0x20, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c,
0x75, 0x6d, 0x6e, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x42, 0x79, 0x52, 0x61, 0x6e,
0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x64, 0x0a, 0x0a, 0x73, 0x74, 0x61,
0x72, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82,
0xb5, 0x18, 0x41, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72,
0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d,
0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15,
0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x61,
0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x33,
0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74,
0x79, 0x70, 0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e,
0x53, 0x6c, 0x6f, 0x74, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x6c, 0x6f, 0x74, 0x12,
0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05,
0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
0x18, 0x03, 0x20, 0x03, 0x28, 0x04, 0x42, 0x07, 0x92, 0xb5, 0x18, 0x03, 0x31, 0x32, 0x38, 0x52,
0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x42, 0x9b, 0x01, 0x0a, 0x19, 0x6f, 0x72, 0x67,
0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x42, 0x10, 0x50, 0x32, 0x50, 0x4d, 0x65, 0x73, 0x73, 0x61,
0x67, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68,
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63,
0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68,
0x61, 0x31, 0x3b, 0x65, 0x74, 0x68, 0xaa, 0x02, 0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x45, 0x74, 0x68, 0x2e, 0x56, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0xca, 0x02,
0x15, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x74, 0x68, 0x5c, 0x76, 0x31,
0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@@ -74,7 +74,7 @@ message MetaDataV2 {
uint64 seq_number = 1;
bytes attnets = 2 [(ethereum.eth.ext.ssz_size) = "8", (ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/go-bitfield.Bitvector64"];
bytes syncnets = 3 [(ethereum.eth.ext.ssz_size) = "1", (ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/go-bitfield.Bitvector4"];
bytes custody_subnet_count = 4 [(ethereum.eth.ext.ssz_size) = "1"];
uint64 custody_subnet_count = 4;
}
/*

View File

@@ -10,18 +10,15 @@ import (
// DataColumnFields extracts a standard set of fields from a DataColumnSidecar into a logrus.Fields struct
// which can be passed to log.WithFields.
func DataColumnFields(column blocks.RODataColumn) logrus.Fields {
kzgCommitmentsShort := make([][]byte, 0, len(column.KzgCommitments))
for _, kzgCommitment := range column.KzgCommitments {
kzgCommitmentsShort = append(kzgCommitmentsShort, kzgCommitment[:3])
}
kzgCommitmentCount := len(column.KzgCommitments)
return logrus.Fields{
"slot": column.Slot(),
"propIdx": column.ProposerIndex(),
"blockRoot": fmt.Sprintf("%#x", column.BlockRoot())[:8],
"parentRoot": fmt.Sprintf("%#x", column.ParentRoot())[:8],
"kzgCommitments": fmt.Sprintf("%#x", kzgCommitmentsShort),
"colIdx": column.ColumnIndex,
"slot": column.Slot(),
"propIdx": column.ProposerIndex(),
"blockRoot": fmt.Sprintf("%#x", column.BlockRoot())[:8],
"parentRoot": fmt.Sprintf("%#x", column.ParentRoot())[:8],
"kzgCommitmentCount": kzgCommitmentCount,
"colIdx": column.ColumnIndex,
}
}

View File

@@ -69,14 +69,14 @@ const (
var metricComparisonTests = []comparisonTest{
{
name: "beacon aggregate and proof",
topic1: fmt.Sprintf(p2pFailValidationTopic, p2p.BeaconAggregateAndProofSubnetTopicFormat),
topic2: fmt.Sprintf(p2pReceivedTotalTopic, p2p.BeaconAggregateAndProofSubnetTopicFormat),
topic1: fmt.Sprintf(p2pFailValidationTopic, p2p.AggregateAndProofSubnetTopicFormat),
topic2: fmt.Sprintf(p2pReceivedTotalTopic, p2p.AggregateAndProofSubnetTopicFormat),
expectedComparison: 0.8,
},
{
name: "committee index beacon attestations",
topic1: fmt.Sprintf(p2pFailValidationTopic, formatTopic(p2p.BeaconAttestationSubnetTopicFormat)),
topic2: fmt.Sprintf(p2pReceivedTotalTopic, formatTopic(p2p.BeaconAttestationSubnetTopicFormat)),
topic1: fmt.Sprintf(p2pFailValidationTopic, formatTopic(p2p.AttestationSubnetTopicFormat)),
topic2: fmt.Sprintf(p2pReceivedTotalTopic, formatTopic(p2p.AttestationSubnetTopicFormat)),
expectedComparison: 0.15,
},
{

View File

@@ -186,7 +186,7 @@ func (v *validator) ProposeBlock(ctx context.Context, slot primitives.Slot, pubK
log = log.WithFields(logrus.Fields{
"payloadHash": fmt.Sprintf("%#x", bytesutil.Trunc(p.BlockHash())),
"parentHash": fmt.Sprintf("%#x", bytesutil.Trunc(p.ParentHash())),
"blockNumber": p.BlockNumber,
"blockNumber": p.BlockNumber(),
})
if !blk.IsBlinded() {
txs, err := p.Transactions()