mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-11 06:18:05 -05:00
Compare commits
15 Commits
peerDAS-re
...
lost-data-
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
f27404e829 | ||
|
|
d97faab4e7 | ||
|
|
d0b232b86c | ||
|
|
536fec55c4 | ||
|
|
eebee6e0fe | ||
|
|
b1f66b6882 | ||
|
|
ee349da18f | ||
|
|
f40b142d31 | ||
|
|
b2de1a4026 | ||
|
|
b3c95f1ea3 | ||
|
|
560b1951ed | ||
|
|
411339f546 | ||
|
|
e8273791c0 | ||
|
|
56df2d000b | ||
|
|
ec35840722 |
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -18,7 +18,6 @@ go_library(
|
||||
"handshake.go",
|
||||
"info.go",
|
||||
"interfaces.go",
|
||||
"iterator.go",
|
||||
"log.go",
|
||||
"message_id.go",
|
||||
"monitoring.go",
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -103,8 +103,8 @@ func TestService_Broadcast_ReturnsErr_TopicNotMapped(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestService_Attestation_Subnet(t *testing.T) {
|
||||
if gtm := GossipTypeMapping[reflect.TypeOf(ðpb.Attestation{})]; gtm != BeaconAttestationSubnetTopicFormat {
|
||||
t.Errorf("Constant is out of date. Wanted %s, got %s", BeaconAttestationSubnetTopicFormat, gtm)
|
||||
if gtm := GossipTypeMapping[reflect.TypeOf(ðpb.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(ðpb.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(ðpb.Attestation{AggregationBits: bitfield.NewBitlist(7)})
|
||||
topic := BeaconAttestationSubnetTopicFormat
|
||||
topic := AttestationSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(msg)] = topic
|
||||
digest, err := p.currentForkDigest()
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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 ðpb.SignedBeaconBlock{} },
|
||||
BeaconAttestationSubnetTopicFormat: func() proto.Message { return ðpb.Attestation{} },
|
||||
VoluntaryExitSubnetTopicFormat: func() proto.Message { return ðpb.SignedVoluntaryExit{} },
|
||||
ProposerSlashingSubnetTopicFormat: func() proto.Message { return ðpb.ProposerSlashing{} },
|
||||
AttesterSlashingSubnetTopicFormat: func() proto.Message { return ðpb.AttesterSlashing{} },
|
||||
BeaconAggregateAndProofSubnetTopicFormat: func() proto.Message { return ðpb.SignedAggregateAttestationAndProof{} },
|
||||
SyncCommitteeContributionAndProofSubnetTopicFormat: func() proto.Message { return ðpb.SignedContributionAndProof{} },
|
||||
SyncCommitteeSubnetTopicFormat: func() proto.Message { return ðpb.SyncCommitteeMessage{} },
|
||||
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return ðpb.SignedBLSToExecutionChange{} },
|
||||
BlobSubnetTopicFormat: func() proto.Message { return ðpb.BlobSidecar{} },
|
||||
DataColumnSubnetTopicFormat: func() proto.Message { return ðpb.DataColumnSidecar{} },
|
||||
BlockSubnetTopicFormat: func() proto.Message { return ðpb.SignedBeaconBlock{} },
|
||||
AttestationSubnetTopicFormat: func() proto.Message { return ðpb.Attestation{} },
|
||||
ExitSubnetTopicFormat: func() proto.Message { return ðpb.SignedVoluntaryExit{} },
|
||||
ProposerSlashingSubnetTopicFormat: func() proto.Message { return ðpb.ProposerSlashing{} },
|
||||
AttesterSlashingSubnetTopicFormat: func() proto.Message { return ðpb.AttesterSlashing{} },
|
||||
AggregateAndProofSubnetTopicFormat: func() proto.Message { return ðpb.SignedAggregateAttestationAndProof{} },
|
||||
SyncContributionAndProofSubnetTopicFormat: func() proto.Message { return ðpb.SignedContributionAndProof{} },
|
||||
SyncCommitteeSubnetTopicFormat: func() proto.Message { return ðpb.SyncCommitteeMessage{} },
|
||||
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return ðpb.SignedBLSToExecutionChange{} },
|
||||
BlobSubnetTopicFormat: func() proto.Message { return ðpb.BlobSidecar{} },
|
||||
DataColumnSubnetTopicFormat: func() proto.Message { return ðpb.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 ðpb.SignedBeaconBlockElectra{}
|
||||
}
|
||||
@@ -46,7 +46,7 @@ func GossipTopicMappings(topic string, epoch primitives.Epoch) proto.Message {
|
||||
return ðpb.SignedBeaconBlockAltair{}
|
||||
}
|
||||
return gossipMessage(topic)
|
||||
case BeaconAttestationSubnetTopicFormat:
|
||||
case AttestationSubnetTopicFormat:
|
||||
if epoch >= params.BeaconConfig().ElectraForkEpoch {
|
||||
return ðpb.AttestationElectra{}
|
||||
}
|
||||
@@ -56,7 +56,7 @@ func GossipTopicMappings(topic string, epoch primitives.Epoch) proto.Message {
|
||||
return ðpb.AttesterSlashingElectra{}
|
||||
}
|
||||
return gossipMessage(topic)
|
||||
case BeaconAggregateAndProofSubnetTopicFormat:
|
||||
case AggregateAndProofSubnetTopicFormat:
|
||||
if epoch >= params.BeaconConfig().ElectraForkEpoch {
|
||||
return ðpb.SignedAggregateAttestationAndProofElectra{}
|
||||
}
|
||||
@@ -93,16 +93,16 @@ func init() {
|
||||
GossipTypeMapping[reflect.TypeOf(v())] = k
|
||||
}
|
||||
// Specially handle Altair objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockAltair{})] = BeaconBlockSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockAltair{})] = BlockSubnetTopicFormat
|
||||
// Specially handle Bellatrix objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockBellatrix{})] = BeaconBlockSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockBellatrix{})] = BlockSubnetTopicFormat
|
||||
// Specially handle Capella objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockCapella{})] = BeaconBlockSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockCapella{})] = BlockSubnetTopicFormat
|
||||
// Specially handle Deneb objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockDeneb{})] = BeaconBlockSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockDeneb{})] = BlockSubnetTopicFormat
|
||||
// Specially handle Electra objects.
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockElectra{})] = BeaconBlockSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.AttestationElectra{})] = BeaconAttestationSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedBeaconBlockElectra{})] = BlockSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.AttestationElectra{})] = AttestationSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.AttesterSlashingElectra{})] = AttesterSlashingSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedAggregateAttestationAndProofElectra{})] = BeaconAggregateAndProofSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedAggregateAttestationAndProofElectra{})] = AggregateAndProofSubnetTopicFormat
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
}()
|
||||
},
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -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'}
|
||||
|
||||
@@ -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",
|
||||
],
|
||||
)
|
||||
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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()))
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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()))
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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()))
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
}(),
|
||||
},
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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))
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -41,7 +41,7 @@ func (s *Service) decodePubsubMessage(msg *pubsub.Message) (ssz.Unmarshaler, err
|
||||
topic = p2p.GossipTypeMapping[reflect.TypeOf(ðpb.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(ðpb.SyncCommitteeMessage{})]
|
||||
case strings.Contains(topic, p2p.GossipBlobSidecarMessage):
|
||||
topic = p2p.GossipTypeMapping[reflect.TypeOf(ðpb.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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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,
|
||||
}),
|
||||
},
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
@@ -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(ðpb.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
|
||||
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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))
|
||||
|
||||
|
||||
@@ -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
|
||||
})
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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() {
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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() {
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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()
|
||||
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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() {
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
77
proto/prysm/v1alpha1/p2p_messages.pb.go
generated
77
proto/prysm/v1alpha1/p2p_messages.pb.go
generated
@@ -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 (
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
/*
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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,
|
||||
},
|
||||
{
|
||||
|
||||
@@ -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()
|
||||
|
||||
Reference in New Issue
Block a user