From 4829de4cf35a0111fbff4944ea453c2610572b43 Mon Sep 17 00:00:00 2001 From: Manu NALEPA Date: Mon, 5 Aug 2024 09:44:49 +0200 Subject: [PATCH] PeerDAS: Add `MetadataV3` with `custody_subnet_count` (#14274) * `sendPingRequest`: Add some comments. * `sendPingRequest`: Replace `stream.Conn().RemotePeer()` by `peerID`. * `pingHandler`: Add comments. * `sendMetaDataRequest`: Add comments and implement an unique test. * Gather `SchemaVersion`s in the same `const` definition. * Define `SchemaVersionV3`. * `MetaDataV1`: Fix comment. * Proto: Define `MetaDataV2`. * `MetaDataV2`: Generate SSZ. * `newColumnSubnetIDs`: Use smaller lines. * `metaDataHandler` and `sendMetaDataRequest`: Manage `MetaDataV2`. * `RefreshPersistentSubnets`: Refactor tests (no functional change). * `RefreshPersistentSubnets`: Refactor and add comments (no functional change). * `RefreshPersistentSubnets`: Compare cache with both ENR & metadata. * `RefreshPersistentSubnets`: Manage peerDAS. * `registerRPCHandlersPeerDAS`: Register `RPCMetaDataTopicV3`. * `CustodyCountFromRemotePeer`: Retrieve the count from metadata. Then default to ENR, then default to the default value. * Update beacon-chain/sync/rpc_metadata.go Co-authored-by: Nishant Das * Fix duplicate case. * Remove version testing. * `debug.proto`: Stop breaking ordering. --------- Co-authored-by: Nishant Das --- beacon-chain/cache/column_subnet_ids.go | 9 +- .../core/helpers/sync_committee_test.go | 2 + beacon-chain/p2p/BUILD.bazel | 3 +- beacon-chain/p2p/custody.go | 17 +- beacon-chain/p2p/custody_test.go | 55 ++- beacon-chain/p2p/discovery.go | 133 +++-- beacon-chain/p2p/discovery_test.go | 453 ++++++++++++------ beacon-chain/p2p/rpc_topic_mappings.go | 35 +- beacon-chain/p2p/sender.go | 2 +- beacon-chain/p2p/subnets.go | 30 ++ beacon-chain/p2p/testing/p2p.go | 11 +- beacon-chain/p2p/types/object_mapping.go | 4 +- beacon-chain/rpc/prysm/v1alpha1/debug/p2p.go | 2 + beacon-chain/sync/BUILD.bazel | 1 - beacon-chain/sync/data_columns_sampling.go | 15 +- beacon-chain/sync/fork_watcher_test.go | 1 + beacon-chain/sync/rate_limiter.go | 3 +- beacon-chain/sync/rate_limiter_test.go | 2 +- beacon-chain/sync/rpc.go | 6 +- beacon-chain/sync/rpc_metadata.go | 142 ++++-- beacon-chain/sync/rpc_metadata_test.go | 343 ++++++++----- beacon-chain/sync/rpc_ping.go | 140 ++++-- cmd/prysmctl/p2p/client.go | 2 +- cmd/prysmctl/p2p/handler.go | 2 +- config/params/mainnet_config.go | 1 + config/params/network_config.go | 7 +- consensus-types/wrapper/metadata.go | 140 +++++- proto/prysm/v1alpha1/BUILD.bazel | 1 + proto/prysm/v1alpha1/debug.pb.go | 238 ++++----- proto/prysm/v1alpha1/debug.proto | 5 +- .../v1alpha1/metadata/metadata_interfaces.go | 3 + proto/prysm/v1alpha1/non-core.ssz.go | 102 +++- proto/prysm/v1alpha1/p2p_messages.pb.go | 190 ++++++-- proto/prysm/v1alpha1/p2p_messages.proto | 18 + 34 files changed, 1528 insertions(+), 590 deletions(-) diff --git a/beacon-chain/cache/column_subnet_ids.go b/beacon-chain/cache/column_subnet_ids.go index 2762148806ab..79de06f092a6 100644 --- a/beacon-chain/cache/column_subnet_ids.go +++ b/beacon-chain/cache/column_subnet_ids.go @@ -19,9 +19,14 @@ var ColumnSubnetIDs = newColumnSubnetIDs() const columnKey = "columns" func newColumnSubnetIDs() *columnSubnetIDs { - epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot)) + secondsPerSlot := params.BeaconConfig().SecondsPerSlot + slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch + epochDuration := time.Duration(slotsPerEpoch.Mul(secondsPerSlot)) + // Set the default duration of a column subnet subscription as the column expiry period. - subLength := epochDuration * time.Duration(params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest) + minEpochsForDataColumnSidecarsRequest := time.Duration(params.BeaconConfig().MinEpochsForDataColumnSidecarsRequest) + subLength := epochDuration * minEpochsForDataColumnSidecarsRequest + persistentCache := cache.New(subLength*time.Second, epochDuration*time.Second) return &columnSubnetIDs{colSubCache: persistentCache} } diff --git a/beacon-chain/core/helpers/sync_committee_test.go b/beacon-chain/core/helpers/sync_committee_test.go index 60612947726d..16218f1f1ebb 100644 --- a/beacon-chain/core/helpers/sync_committee_test.go +++ b/beacon-chain/core/helpers/sync_committee_test.go @@ -78,6 +78,7 @@ func TestIsCurrentEpochSyncCommittee_UsingCommittee(t *testing.T) { func TestIsCurrentEpochSyncCommittee_DoesNotExist(t *testing.T) { helpers.ClearCache() + params.SetupTestConfigCleanup(t) validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize) syncCommittee := ðpb.SyncCommittee{ @@ -264,6 +265,7 @@ func TestCurrentEpochSyncSubcommitteeIndices_UsingCommittee(t *testing.T) { } func TestCurrentEpochSyncSubcommitteeIndices_DoesNotExist(t *testing.T) { + params.SetupTestConfigCleanup(t) helpers.ClearCache() validators := make([]*ethpb.Validator, params.BeaconConfig().SyncCommitteeSize) diff --git a/beacon-chain/p2p/BUILD.bazel b/beacon-chain/p2p/BUILD.bazel index a8de38a2a8ab..4603604cece7 100644 --- a/beacon-chain/p2p/BUILD.bazel +++ b/beacon-chain/p2p/BUILD.bazel @@ -170,13 +170,12 @@ go_test( "//network/forks:go_default_library", "//proto/eth/v1:go_default_library", "//proto/prysm/v1alpha1:go_default_library", + "//proto/prysm/v1alpha1/metadata:go_default_library", "//proto/testing:go_default_library", - "//runtime/version:go_default_library", "//testing/assert:go_default_library", "//testing/require:go_default_library", "//testing/util:go_default_library", "//time:go_default_library", - "//time/slots:go_default_library", "@com_github_ethereum_go_ethereum//crypto:go_default_library", "@com_github_ethereum_go_ethereum//p2p/discover:go_default_library", "@com_github_ethereum_go_ethereum//p2p/enode:go_default_library", diff --git a/beacon-chain/p2p/custody.go b/beacon-chain/p2p/custody.go index 9becc1128c56..6fbeb28e20ba 100644 --- a/beacon-chain/p2p/custody.go +++ b/beacon-chain/p2p/custody.go @@ -77,13 +77,28 @@ func (s *Service) CustodyCountFromRemotePeer(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 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 { log.WithError(err).WithFields(logrus.Fields{ "peerID": pid, "defaultValue": custodyRequirement, - }).Error("Failed to retrieve ENR for peer, defaulting to the default value") + }).Debug("Failed to retrieve ENR for peer, defaulting to the default value") return custodyRequirement } diff --git a/beacon-chain/p2p/custody_test.go b/beacon-chain/p2p/custody_test.go index 450f2538918c..9c94c3a2d8c7 100644 --- a/beacon-chain/p2p/custody_test.go +++ b/beacon-chain/p2p/custody_test.go @@ -16,8 +16,11 @@ import ( "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers" "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers/scorers" "github.com/prysmaticlabs/prysm/v5/config/params" + "github.com/prysmaticlabs/prysm/v5/consensus-types/wrapper" ecdsaprysm "github.com/prysmaticlabs/prysm/v5/crypto/ecdsa" prysmNetwork "github.com/prysmaticlabs/prysm/v5/network" + pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" + "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1/metadata" "github.com/prysmaticlabs/prysm/v5/testing/require" ) @@ -99,11 +102,12 @@ func TestGetValidCustodyPeers(t *testing.T) { func TestCustodyCountFromRemotePeer(t *testing.T) { const ( - expected uint64 = 7 - pid = "test-id" + expectedENR uint64 = 7 + expectedMetadata uint64 = 8 + pid = "test-id" ) - csc := peerdas.Csc(expected) + csc := peerdas.Csc(expectedENR) // Define a nil record var nilRecord *enr.Record = nil @@ -115,26 +119,49 @@ func TestCustodyCountFromRemotePeer(t *testing.T) { nominalRecord := &enr.Record{} nominalRecord.Set(csc) + // Define a metadata with zero custody. + zeroMetadata := wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + CustodySubnetCount: 0, + }) + + // Define a nominal metadata. + nominalMetadata := wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + CustodySubnetCount: expectedMetadata, + }) + testCases := []struct { name string record *enr.Record + metadata metadata.Metadata expected uint64 }{ { - name: "nominal", - record: nominalRecord, - expected: expected, - }, - { - name: "nil", + name: "No metadata - No ENR", record: nilRecord, expected: params.BeaconConfig().CustodyRequirement, }, { - name: "empty", + name: "No metadata - Empty ENR", record: emptyRecord, expected: params.BeaconConfig().CustodyRequirement, }, + { + name: "No Metadata - ENR", + record: nominalRecord, + expected: expectedENR, + }, + { + name: "Metadata with 0 value - ENR", + record: nominalRecord, + metadata: zeroMetadata, + expected: expectedENR, + }, + { + name: "Metadata - ENR", + record: nominalRecord, + metadata: nominalMetadata, + expected: expectedMetadata, + }, } for _, tc := range testCases { @@ -144,12 +171,18 @@ func TestCustodyCountFromRemotePeer(t *testing.T) { ScorerParams: &scorers.Config{}, }) + // Set the metadata. + if tc.metadata != nil { + peers.SetMetadata(pid, tc.metadata) + } + // Add a new peer with the record. peers.Add(tc.record, pid, nil, network.DirOutbound) // Create a new service. service := &Service{ - peers: peers, + peers: peers, + metaData: tc.metadata, } // Retrieve the custody count from the remote peer. diff --git a/beacon-chain/p2p/discovery.go b/beacon-chain/p2p/discovery.go index d9e1b9fee7b6..3eb816db04b3 100644 --- a/beacon-chain/p2p/discovery.go +++ b/beacon-chain/p2p/discovery.go @@ -55,62 +55,137 @@ func (quicProtocol) ENRKey() string { return quickProtocolEnrKey } // This routine checks for our attestation, sync committee and data column subnets and updates them if they have // been rotated. func (s *Service) RefreshPersistentSubnets() { - // return early if discv5 isnt running + // Return early if discv5 service isn't running. if s.dv5Listener == nil || !s.isInitialized() { return } - currEpoch := slots.ToEpoch(slots.CurrentSlot(uint64(s.genesisTime.Unix()))) - if err := initializePersistentSubnets(s.dv5Listener.LocalNode().ID(), currEpoch); err != nil { + + // Get the current epoch. + currentSlot := slots.CurrentSlot(uint64(s.genesisTime.Unix())) + currentEpoch := slots.ToEpoch(currentSlot) + + // Get our node ID. + nodeID := s.dv5Listener.LocalNode().ID() + + // Get our node record. + record := s.dv5Listener.Self().Record() + + // Get the version of our metadata. + metadataVersion := s.Metadata().Version() + + // Initialize persistent subnets. + if err := initializePersistentSubnets(nodeID, currentEpoch); err != nil { log.WithError(err).Error("Could not initialize persistent subnets") return } - if err := initializePersistentColumnSubnets(s.dv5Listener.LocalNode().ID()); err != nil { + + // Initialize persistent column subnets. + if err := initializePersistentColumnSubnets(nodeID); err != nil { log.WithError(err).Error("Could not initialize persistent column subnets") return } + // Get the current attestation subnet bitfield. bitV := bitfield.NewBitvector64() - committees := cache.SubnetIDs.GetAllSubnets() - for _, idx := range committees { + attestationCommittees := cache.SubnetIDs.GetAllSubnets() + for _, idx := range attestationCommittees { bitV.SetBitAt(idx, true) } - currentBitV, err := attBitvector(s.dv5Listener.Self().Record()) + + // Get the attestation subnet bitfield we store in our record. + inRecordBitV, err := attBitvector(record) if err != nil { log.WithError(err).Error("Could not retrieve att bitfield") return } - // Compare current epoch with our fork epochs + // Get the attestation subnet bitfield in our metadata. + inMetadataBitV := s.Metadata().AttnetsBitfield() + + // Is our attestation bitvector record up to date? + isBitVUpToDate := bytes.Equal(bitV, inRecordBitV) && bytes.Equal(bitV, inMetadataBitV) + + // Compare current epoch with Altair fork epoch altairForkEpoch := params.BeaconConfig().AltairForkEpoch - switch { - case currEpoch < altairForkEpoch: + + if currentEpoch < altairForkEpoch { // Phase 0 behaviour. - if bytes.Equal(bitV, currentBitV) { - // return early if bitfield hasn't changed + if isBitVUpToDate { + // Return early if bitfield hasn't changed. return } + + // Some data changed. Update the record and the metadata. s.updateSubnetRecordWithMetadata(bitV) - default: - // Retrieve sync subnets from application level - // cache. - bitS := bitfield.Bitvector4{byte(0x00)} - committees = cache.SyncSubnetIDs.GetAllSubnets(currEpoch) - for _, idx := range committees { - bitS.SetBitAt(idx, true) - } - currentBitS, err := syncBitvector(s.dv5Listener.Self().Record()) - if err != nil { - log.WithError(err).Error("Could not retrieve sync bitfield") - return - } - if bytes.Equal(bitV, currentBitV) && bytes.Equal(bitS, currentBitS) && - s.Metadata().Version() == version.Altair { - // return early if bitfields haven't changed + + // Ping all peers. + s.pingPeers() + + return + } + + // Get the current sync subnet bitfield. + bitS := bitfield.Bitvector4{byte(0x00)} + syncCommittees := cache.SyncSubnetIDs.GetAllSubnets(currentEpoch) + for _, idx := range syncCommittees { + bitS.SetBitAt(idx, true) + } + + // Get the sync subnet bitfield we store in our record. + inRecordBitS, err := syncBitvector(record) + if err != nil { + log.WithError(err).Error("Could not retrieve sync bitfield") + return + } + + // Get the sync subnet bitfield in our metadata. + currentBitSInMetadata := s.Metadata().SyncnetsBitfield() + + isBitSUpToDate := bytes.Equal(bitS, inRecordBitS) && bytes.Equal(bitS, currentBitSInMetadata) + + // Compare current epoch with EIP-7594 fork epoch. + eip7594ForkEpoch := params.BeaconConfig().Eip7594ForkEpoch + + if currentEpoch < eip7594ForkEpoch { + // Altair behaviour. + if metadataVersion == version.Altair && isBitVUpToDate && isBitSUpToDate { + // Nothing to do, return early. return } + + // Some data have changed, update our record and metadata. s.updateSubnetRecordWithMetadataV2(bitV, bitS) + + // Ping all peers to inform them of new metadata + s.pingPeers() + + return } - // ping all peers to inform them of new metadata + + // Get the current custody subnet count. + custodySubnetCount := peerdas.CustodySubnetCount() + + // Get the custody subnet count we store in our record. + inRecordCustodySubnetCount, err := peerdas.CustodyCountFromRecord(record) + if err != nil { + log.WithError(err).Error("Could not retrieve custody subnet count") + return + } + + // Get the custody subnet count in our metadata. + inMetadataCustodySubnetCount := s.Metadata().CustodySubnetCount() + + isCustodySubnetCountUpToDate := (custodySubnetCount == inRecordCustodySubnetCount && custodySubnetCount == inMetadataCustodySubnetCount) + + if isBitVUpToDate && isBitSUpToDate && isCustodySubnetCountUpToDate { + // Nothing to do, return early. + return + } + + // Some data changed. Update the record and the metadata. + s.updateSubnetRecordWithMetadataV3(bitV, bitS, custodySubnetCount) + + // Ping all peers. s.pingPeers() } diff --git a/beacon-chain/p2p/discovery_test.go b/beacon-chain/p2p/discovery_test.go index 30b8988d4f41..ac9c7f1bbd3f 100644 --- a/beacon-chain/p2p/discovery_test.go +++ b/beacon-chain/p2p/discovery_test.go @@ -16,12 +16,12 @@ import ( "github.com/ethereum/go-ethereum/p2p/discover" "github.com/ethereum/go-ethereum/p2p/enode" "github.com/ethereum/go-ethereum/p2p/enr" + "github.com/libp2p/go-libp2p/core/crypto" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" + swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" "github.com/prysmaticlabs/go-bitfield" - logTest "github.com/sirupsen/logrus/hooks/test" - mock "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing" "github.com/prysmaticlabs/prysm/v5/beacon-chain/cache" "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/peerdas" @@ -33,13 +33,13 @@ import ( "github.com/prysmaticlabs/prysm/v5/config/params" "github.com/prysmaticlabs/prysm/v5/consensus-types/wrapper" leakybucket "github.com/prysmaticlabs/prysm/v5/container/leaky-bucket" + ecdsaprysm "github.com/prysmaticlabs/prysm/v5/crypto/ecdsa" "github.com/prysmaticlabs/prysm/v5/encoding/bytesutil" prysmNetwork "github.com/prysmaticlabs/prysm/v5/network" ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" - "github.com/prysmaticlabs/prysm/v5/runtime/version" "github.com/prysmaticlabs/prysm/v5/testing/assert" "github.com/prysmaticlabs/prysm/v5/testing/require" - "github.com/prysmaticlabs/prysm/v5/time/slots" + logTest "github.com/sirupsen/logrus/hooks/test" ) var discoveryWaitTime = 1 * time.Second @@ -446,177 +446,314 @@ func addPeer(t *testing.T, p *peers.Status, state peerdata.PeerConnectionState) return id } -func TestRefreshPersistentSubnets_ForkBoundaries(t *testing.T) { +func createAndConnectPeer(t *testing.T, p2pService *testp2p.TestP2P, offset int) { + // Create the private key. + privateKeyBytes := make([]byte, 32) + for i := 0; i < 32; i++ { + privateKeyBytes[i] = byte(offset + i) + } + + privateKey, err := crypto.UnmarshalSecp256k1PrivateKey(privateKeyBytes) + require.NoError(t, err) + + // Create the peer. + peer := testp2p.NewTestP2P(t, swarmt.OptPeerPrivateKey(privateKey)) + + // Add the peer and connect it. + p2pService.Peers().Add(&enr.Record{}, peer.PeerID(), nil, network.DirOutbound) + p2pService.Peers().SetConnectionState(peer.PeerID(), peers.PeerConnected) + p2pService.Connect(peer) +} + +// Define the ping count. +var actualPingCount int + +type check struct { + pingCount int + metadataSequenceNumber uint64 + attestationSubnets []uint64 + syncSubnets []uint64 + custodySubnetCount *uint64 +} + +func checkPingCountCacheMetadataRecord( + t *testing.T, + service *Service, + expected check, +) { + // Check the ping count. + require.Equal(t, expected.pingCount, actualPingCount) + + // Check the attestation subnets in the cache. + actualAttestationSubnets := cache.SubnetIDs.GetAllSubnets() + require.DeepSSZEqual(t, expected.attestationSubnets, actualAttestationSubnets) + + // Check the metadata sequence number. + actualMetadataSequenceNumber := service.metaData.SequenceNumber() + require.Equal(t, expected.metadataSequenceNumber, actualMetadataSequenceNumber) + + // Compute expected attestation subnets bits. + expectedBitV := bitfield.NewBitvector64() + exists := false + + for _, idx := range expected.attestationSubnets { + exists = true + expectedBitV.SetBitAt(idx, true) + } + + // Check attnets in ENR. + var actualBitVENR bitfield.Bitvector64 + err := service.dv5Listener.LocalNode().Node().Record().Load(enr.WithEntry(attSubnetEnrKey, &actualBitVENR)) + require.NoError(t, err) + require.DeepSSZEqual(t, expectedBitV, actualBitVENR) + + // Check attnets in metadata. + if !exists { + expectedBitV = nil + } + + actualBitVMetadata := service.metaData.AttnetsBitfield() + require.DeepSSZEqual(t, expectedBitV, actualBitVMetadata) + + if expected.syncSubnets != nil { + // Compute expected sync subnets bits. + expectedBitS := bitfield.NewBitvector4() + exists = false + + for _, idx := range expected.syncSubnets { + exists = true + expectedBitS.SetBitAt(idx, true) + } + + // Check syncnets in ENR. + var actualBitSENR bitfield.Bitvector4 + err := service.dv5Listener.LocalNode().Node().Record().Load(enr.WithEntry(syncCommsSubnetEnrKey, &actualBitSENR)) + require.NoError(t, err) + require.DeepSSZEqual(t, expectedBitS, actualBitSENR) + + // Check syncnets in metadata. + if !exists { + expectedBitS = nil + } + + actualBitSMetadata := service.metaData.SyncnetsBitfield() + require.DeepSSZEqual(t, expectedBitS, actualBitSMetadata) + } + + if expected.custodySubnetCount != nil { + // Check custody subnet count in ENR. + 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) + + // Check custody subnet count in metadata. + actualCustodySubnetCountMetadata := service.metaData.CustodySubnetCount() + require.Equal(t, *expected.custodySubnetCount, actualCustodySubnetCountMetadata) + } +} + +func TestRefreshPersistentSubnets(t *testing.T) { params.SetupTestConfigCleanup(t) + // Clean up caches after usage. defer cache.SubnetIDs.EmptyAllCaches() + defer cache.SyncSubnetIDs.EmptyAllCaches() + + const ( + altairForkEpoch = 5 + eip7594ForkEpoch = 10 + ) + + custodySubnetCount := uint64(1) - tests := []struct { - name string - svcBuilder func(t *testing.T) *Service - postValidation func(t *testing.T, s *Service) + // Set up epochs. + defaultCfg := params.BeaconConfig() + cfg := defaultCfg.Copy() + cfg.AltairForkEpoch = altairForkEpoch + cfg.Eip7594ForkEpoch = eip7594ForkEpoch + params.OverrideBeaconConfig(cfg) + + // Compute the number of seconds per epoch. + secondsPerSlot := params.BeaconConfig().SecondsPerSlot + slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch + secondsPerEpoch := secondsPerSlot * uint64(slotsPerEpoch) + + testCases := []struct { + name string + epochSinceGenesis uint64 + checks []check }{ { - name: "metadata no change", - svcBuilder: func(t *testing.T) *Service { - port := 2000 - ipAddr, pkey := createAddrAndPrivKey(t) - s := &Service{ - genesisTime: time.Now(), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - cfg: &Config{UDPPort: uint(port)}, - } - listener, err := s.createListener(ipAddr, pkey) - assert.NoError(t, err) - s.dv5Listener = listener - s.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0)) - s.updateSubnetRecordWithMetadata([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}) - return s - }, - postValidation: func(t *testing.T, s *Service) { - currEpoch := slots.ToEpoch(slots.CurrentSlot(uint64(s.genesisTime.Unix()))) - subs, err := computeSubscribedSubnets(s.dv5Listener.LocalNode().ID(), currEpoch) - assert.NoError(t, err) - - bitV := bitfield.NewBitvector64() - for _, idx := range subs { - bitV.SetBitAt(idx, true) - } - assert.DeepEqual(t, bitV, s.metaData.AttnetsBitfield()) + name: "Phase0", + epochSinceGenesis: 0, + checks: []check{ + { + pingCount: 0, + metadataSequenceNumber: 0, + attestationSubnets: []uint64{}, + }, + { + pingCount: 1, + metadataSequenceNumber: 1, + attestationSubnets: []uint64{40, 41}, + }, + { + pingCount: 1, + metadataSequenceNumber: 1, + attestationSubnets: []uint64{40, 41}, + }, + { + pingCount: 1, + metadataSequenceNumber: 1, + attestationSubnets: []uint64{40, 41}, + }, }, }, { - name: "metadata updated", - svcBuilder: func(t *testing.T) *Service { - port := 2000 - ipAddr, pkey := createAddrAndPrivKey(t) - s := &Service{ - genesisTime: time.Now(), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - cfg: &Config{UDPPort: uint(port)}, - } - listener, err := s.createListener(ipAddr, pkey) - assert.NoError(t, err) - s.dv5Listener = listener - s.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0)) - s.updateSubnetRecordWithMetadata([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01}) - cache.SubnetIDs.AddPersistentCommittee([]uint64{1, 2, 3, 23}, 0) - return s - }, - postValidation: func(t *testing.T, s *Service) { - assert.DeepEqual(t, bitfield.Bitvector64{0xe, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0}, s.metaData.AttnetsBitfield()) - }, - }, - { - name: "metadata updated at fork epoch", - svcBuilder: func(t *testing.T) *Service { - port := 2000 - ipAddr, pkey := createAddrAndPrivKey(t) - s := &Service{ - genesisTime: time.Now().Add(-5 * oneEpochDuration()), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - cfg: &Config{UDPPort: uint(port)}, - } - listener, err := s.createListener(ipAddr, pkey) - assert.NoError(t, err) - - // Update params - cfg := params.BeaconConfig().Copy() - cfg.AltairForkEpoch = 5 - params.OverrideBeaconConfig(cfg) - params.BeaconConfig().InitializeForkSchedule() - - s.dv5Listener = listener - s.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0)) - s.updateSubnetRecordWithMetadata([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01}) - cache.SubnetIDs.AddPersistentCommittee([]uint64{1, 2, 3, 23}, 0) - return s - }, - postValidation: func(t *testing.T, s *Service) { - assert.Equal(t, version.Altair, s.metaData.Version()) - assert.DeepEqual(t, bitfield.Bitvector4{0x00}, s.metaData.MetadataObjV1().Syncnets) - assert.DeepEqual(t, bitfield.Bitvector64{0xe, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0}, s.metaData.AttnetsBitfield()) + name: "Altair", + epochSinceGenesis: altairForkEpoch, + checks: []check{ + { + pingCount: 0, + metadataSequenceNumber: 0, + attestationSubnets: []uint64{}, + syncSubnets: nil, + }, + { + pingCount: 1, + metadataSequenceNumber: 1, + attestationSubnets: []uint64{40, 41}, + syncSubnets: nil, + }, + { + pingCount: 2, + metadataSequenceNumber: 2, + attestationSubnets: []uint64{40, 41}, + syncSubnets: []uint64{1, 2}, + }, + { + pingCount: 2, + metadataSequenceNumber: 2, + attestationSubnets: []uint64{40, 41}, + syncSubnets: []uint64{1, 2}, + }, }, }, { - name: "metadata updated at fork epoch with no bitfield", - svcBuilder: func(t *testing.T) *Service { - port := 2000 - ipAddr, pkey := createAddrAndPrivKey(t) - s := &Service{ - genesisTime: time.Now().Add(-5 * oneEpochDuration()), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - cfg: &Config{UDPPort: uint(port)}, - } - listener, err := s.createListener(ipAddr, pkey) - assert.NoError(t, err) - - // Update params - cfg := params.BeaconConfig().Copy() - cfg.AltairForkEpoch = 5 - params.OverrideBeaconConfig(cfg) - params.BeaconConfig().InitializeForkSchedule() - - s.dv5Listener = listener - s.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0)) - s.updateSubnetRecordWithMetadata([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}) - return s - }, - postValidation: func(t *testing.T, s *Service) { - assert.Equal(t, version.Altair, s.metaData.Version()) - assert.DeepEqual(t, bitfield.Bitvector4{0x00}, s.metaData.MetadataObjV1().Syncnets) - currEpoch := slots.ToEpoch(slots.CurrentSlot(uint64(s.genesisTime.Unix()))) - subs, err := computeSubscribedSubnets(s.dv5Listener.LocalNode().ID(), currEpoch) - assert.NoError(t, err) - - bitV := bitfield.NewBitvector64() - for _, idx := range subs { - bitV.SetBitAt(idx, true) - } - assert.DeepEqual(t, bitV, s.metaData.AttnetsBitfield()) - }, - }, - { - name: "metadata updated past fork epoch with bitfields", - svcBuilder: func(t *testing.T) *Service { - port := 2000 - ipAddr, pkey := createAddrAndPrivKey(t) - s := &Service{ - genesisTime: time.Now().Add(-6 * oneEpochDuration()), - genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), - cfg: &Config{UDPPort: uint(port)}, - } - listener, err := s.createListener(ipAddr, pkey) - assert.NoError(t, err) - - // Update params - cfg := params.BeaconConfig().Copy() - cfg.AltairForkEpoch = 5 - params.OverrideBeaconConfig(cfg) - params.BeaconConfig().InitializeForkSchedule() - - s.dv5Listener = listener - s.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0)) - s.updateSubnetRecordWithMetadata([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}) - cache.SubnetIDs.AddPersistentCommittee([]uint64{1, 2, 3, 23}, 0) - cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte{'A'}, 0, []uint64{0, 1}, 0) - return s - }, - postValidation: func(t *testing.T, s *Service) { - assert.Equal(t, version.Altair, s.metaData.Version()) - assert.DeepEqual(t, bitfield.Bitvector4{0x03}, s.metaData.MetadataObjV1().Syncnets) - assert.DeepEqual(t, bitfield.Bitvector64{0xe, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0}, s.metaData.AttnetsBitfield()) + name: "PeerDAS", + epochSinceGenesis: eip7594ForkEpoch, + checks: []check{ + { + pingCount: 0, + metadataSequenceNumber: 0, + attestationSubnets: []uint64{}, + syncSubnets: nil, + }, + { + pingCount: 1, + metadataSequenceNumber: 1, + attestationSubnets: []uint64{40, 41}, + syncSubnets: nil, + custodySubnetCount: &custodySubnetCount, + }, + { + pingCount: 2, + metadataSequenceNumber: 2, + attestationSubnets: []uint64{40, 41}, + syncSubnets: []uint64{1, 2}, + custodySubnetCount: &custodySubnetCount, + }, + { + pingCount: 2, + metadataSequenceNumber: 2, + attestationSubnets: []uint64{40, 41}, + syncSubnets: []uint64{1, 2}, + custodySubnetCount: &custodySubnetCount, + }, }, }, } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - s := tt.svcBuilder(t) - s.RefreshPersistentSubnets() - tt.postValidation(t, s) - s.dv5Listener.Close() + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + const peerOffset = 1 + + // Initialize the ping count. + actualPingCount = 0 + + // Create the private key. + privateKeyBytes := make([]byte, 32) + for i := 0; i < 32; i++ { + privateKeyBytes[i] = byte(i) + } + + unmarshalledPrivateKey, err := crypto.UnmarshalSecp256k1PrivateKey(privateKeyBytes) + require.NoError(t, err) + + privateKey, err := ecdsaprysm.ConvertFromInterfacePrivKey(unmarshalledPrivateKey) + require.NoError(t, err) + + // Create a p2p service. + p2p := testp2p.NewTestP2P(t) + + // Create and connect a peer. + createAndConnectPeer(t, p2p, peerOffset) + + // Create a service. + service := &Service{ + pingMethod: func(_ context.Context, _ peer.ID) error { + actualPingCount++ + return nil + }, + cfg: &Config{UDPPort: 2000}, + peers: p2p.Peers(), + genesisTime: time.Now().Add(-time.Duration(tc.epochSinceGenesis*secondsPerEpoch) * time.Second), + genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32), + } + + // Create a listener. + listener, err := service.createListener(nil, privateKey) + require.NoError(t, err) + + // Set the listener and the metadata. + service.dv5Listener = listener + service.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0)) + + // Run a check. + checkPingCountCacheMetadataRecord(t, service, tc.checks[0]) + + // Refresh the persistent subnets. + service.RefreshPersistentSubnets() + time.Sleep(10 * time.Millisecond) + + // Run a check. + checkPingCountCacheMetadataRecord(t, service, tc.checks[1]) + + // Add a sync committee subnet. + cache.SyncSubnetIDs.AddSyncCommitteeSubnets([]byte{'a'}, altairForkEpoch, []uint64{1, 2}, 1*time.Hour) + + // Refresh the persistent subnets. + service.RefreshPersistentSubnets() + time.Sleep(10 * time.Millisecond) + + // Run a check. + checkPingCountCacheMetadataRecord(t, service, tc.checks[2]) + + // Refresh the persistent subnets. + service.RefreshPersistentSubnets() + time.Sleep(10 * time.Millisecond) + + // Run a check. + checkPingCountCacheMetadataRecord(t, service, tc.checks[3]) + + // Clean the test. + service.dv5Listener.Close() cache.SubnetIDs.EmptyAllCaches() cache.SyncSubnetIDs.EmptyAllCaches() }) } + + // Reset the config. + params.OverrideBeaconConfig(defaultCfg) } diff --git a/beacon-chain/p2p/rpc_topic_mappings.go b/beacon-chain/p2p/rpc_topic_mappings.go index 0b4f6688d95f..901d497a7f1a 100644 --- a/beacon-chain/p2p/rpc_topic_mappings.go +++ b/beacon-chain/p2p/rpc_topic_mappings.go @@ -10,11 +10,16 @@ import ( pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" ) -// SchemaVersionV1 specifies the schema version for our rpc protocol ID. -const SchemaVersionV1 = "/1" +const ( + // SchemaVersionV1 specifies the schema version for our rpc protocol ID. + SchemaVersionV1 = "/1" + + // SchemaVersionV2 specifies the next schema version for our rpc protocol ID. + SchemaVersionV2 = "/2" -// SchemaVersionV2 specifies the next schema version for our rpc protocol ID. -const SchemaVersionV2 = "/2" + // SchemaVersionV3 specifies the next schema version for our rpc protocol ID. + SchemaVersionV3 = "/3" +) // Specifies the protocol prefix for all our Req/Resp topics. const protocolPrefix = "/eth2/beacon_chain/req" @@ -85,6 +90,9 @@ const ( RPCBlocksByRootTopicV2 = protocolPrefix + BeaconBlocksByRootsMessageName + SchemaVersionV2 // RPCMetaDataTopicV2 defines the v2 topic for the metadata rpc method. RPCMetaDataTopicV2 = protocolPrefix + MetadataMessageName + SchemaVersionV2 + + // V3 RPC Topics + RPCMetaDataTopicV3 = protocolPrefix + MetadataMessageName + SchemaVersionV3 ) // RPC errors for topic parsing. @@ -109,6 +117,7 @@ var RPCTopicMappings = map[string]interface{}{ // RPC Metadata Message RPCMetaDataTopicV1: new(interface{}), RPCMetaDataTopicV2: new(interface{}), + RPCMetaDataTopicV3: new(interface{}), // BlobSidecarsByRange v1 Message RPCBlobSidecarsByRangeTopicV1: new(pb.BlobSidecarsByRangeRequest), // BlobSidecarsByRoot v1 Message @@ -146,9 +155,15 @@ var altairMapping = map[string]bool{ MetadataMessageName: true, } +// Maps all the RPC messages which are to updated with peerDAS fork epoch. +var peerDASMapping = map[string]bool{ + MetadataMessageName: true, +} + var versionMapping = map[string]bool{ SchemaVersionV1: true, SchemaVersionV2: true, + SchemaVersionV3: true, } // OmitContextBytesV1 keeps track of which RPC methods do not write context bytes in their v1 incarnations. @@ -276,13 +291,25 @@ func (r RPCTopic) Version() string { // TopicFromMessage constructs the rpc topic from the provided message // type and epoch. func TopicFromMessage(msg string, epoch primitives.Epoch) (string, error) { + // Check if the topic is known. if !messageMapping[msg] { return "", errors.Errorf("%s: %s", invalidRPCMessageType, msg) } + + // Base version is version 1. version := SchemaVersionV1 + + // Check if the message is to be updated in altair. isAltair := epoch >= params.BeaconConfig().AltairForkEpoch if isAltair && altairMapping[msg] { version = SchemaVersionV2 } + + // Check if the message is to be updated in peerDAS. + isPeerDAS := epoch >= params.BeaconConfig().Eip7594ForkEpoch + if isPeerDAS && peerDASMapping[msg] { + version = SchemaVersionV3 + } + return protocolPrefix + msg + version, nil } diff --git a/beacon-chain/p2p/sender.go b/beacon-chain/p2p/sender.go index cacf5da124ff..d8455eef7a55 100644 --- a/beacon-chain/p2p/sender.go +++ b/beacon-chain/p2p/sender.go @@ -42,7 +42,7 @@ func (s *Service) Send(ctx context.Context, message interface{}, baseTopic strin return nil, err } // do not encode anything if we are sending a metadata request - if baseTopic != RPCMetaDataTopicV1 && baseTopic != RPCMetaDataTopicV2 { + if baseTopic != RPCMetaDataTopicV1 && baseTopic != RPCMetaDataTopicV2 && baseTopic != RPCMetaDataTopicV3 { castedMsg, ok := message.(ssz.Marshaler) if !ok { return nil, errors.Errorf("%T does not support the ssz marshaller interface", message) diff --git a/beacon-chain/p2p/subnets.go b/beacon-chain/p2p/subnets.go index cd9cc1eeadd1..e33fb14d6091 100644 --- a/beacon-chain/p2p/subnets.go +++ b/beacon-chain/p2p/subnets.go @@ -31,6 +31,7 @@ var syncCommsSubnetCount = params.BeaconConfig().SyncCommitteeSubnetCount var attSubnetEnrKey = params.BeaconNetworkConfig().AttSubnetKey var syncCommsSubnetEnrKey = params.BeaconNetworkConfig().SyncCommsSubnetKey +var custodySubnetCountEnrKey = params.BeaconNetworkConfig().CustodySubnetCountKey // The value used with the subnet, in order // to create an appropriate key to retrieve @@ -219,6 +220,35 @@ func (s *Service) updateSubnetRecordWithMetadataV2(bitVAtt bitfield.Bitvector64, }) } +// updateSubnetRecordWithMetadataV3 updates: +// - attestation subnet tracked, +// - sync subnets tracked, and +// - custody subnet count +// both in the node's record and in the node's metadata. +func (s *Service) updateSubnetRecordWithMetadataV3( + bitVAtt bitfield.Bitvector64, + bitVSync bitfield.Bitvector4, + custodySubnetCount uint64, +) { + attSubnetsEntry := enr.WithEntry(attSubnetEnrKey, &bitVAtt) + syncSubnetsEntry := enr.WithEntry(syncCommsSubnetEnrKey, &bitVSync) + custodySubnetCountEntry := enr.WithEntry(custodySubnetCountEnrKey, custodySubnetCount) + + localNode := s.dv5Listener.LocalNode() + localNode.Set(attSubnetsEntry) + localNode.Set(syncSubnetsEntry) + localNode.Set(custodySubnetCountEntry) + + newSeqNumber := s.metaData.SequenceNumber() + 1 + + s.metaData = wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: newSeqNumber, + Attnets: bitVAtt, + Syncnets: bitVSync, + CustodySubnetCount: custodySubnetCount, + }) +} + func initializePersistentSubnets(id enode.ID, epoch primitives.Epoch) error { _, ok, expTime := cache.SubnetIDs.GetPersistentSubnets() if ok && expTime.After(time.Now()) { diff --git a/beacon-chain/p2p/testing/p2p.go b/beacon-chain/p2p/testing/p2p.go index ad96e3a43cde..9b1e49f8ac10 100644 --- a/beacon-chain/p2p/testing/p2p.go +++ b/beacon-chain/p2p/testing/p2p.go @@ -36,8 +36,11 @@ import ( // We have to declare this again here to prevent a circular dependency // with the main p2p package. -const metatadataV1Topic = "/eth2/beacon_chain/req/metadata/1" -const metatadataV2Topic = "/eth2/beacon_chain/req/metadata/2" +const ( + metadataV1Topic = "/eth2/beacon_chain/req/metadata/1" + metadataV2Topic = "/eth2/beacon_chain/req/metadata/2" + metadataV3Topic = "/eth2/beacon_chain/req/metadata/3" +) // TestP2P represents a p2p implementation that can be used for testing. type TestP2P struct { @@ -325,6 +328,8 @@ func (p *TestP2P) AddDisconnectionHandler(f func(ctx context.Context, id peer.ID // Send a message to a specific peer. func (p *TestP2P) Send(ctx context.Context, msg interface{}, topic string, pid peer.ID) (network.Stream, error) { + metadataTopics := map[string]bool{metadataV1Topic: true, metadataV2Topic: true, metadataV3Topic: true} + t := topic if t == "" { return nil, fmt.Errorf("protocol doesn't exist for proto message: %v", msg) @@ -334,7 +339,7 @@ func (p *TestP2P) Send(ctx context.Context, msg interface{}, topic string, pid p return nil, err } - if topic != metatadataV1Topic && topic != metatadataV2Topic { + if !metadataTopics[topic] { castedMsg, ok := msg.(ssz.Marshaler) if !ok { p.t.Fatalf("%T doesn't support ssz marshaler", msg) diff --git a/beacon-chain/p2p/types/object_mapping.go b/beacon-chain/p2p/types/object_mapping.go index e8646b34ee7a..4d693775ebf6 100644 --- a/beacon-chain/p2p/types/object_mapping.go +++ b/beacon-chain/p2p/types/object_mapping.go @@ -87,10 +87,10 @@ func InitializeDataMaps() { return wrapper.WrappedMetadataV1(ðpb.MetaDataV1{}), nil }, bytesutil.ToBytes4(params.BeaconConfig().DenebForkVersion): func() (metadata.Metadata, error) { - return wrapper.WrappedMetadataV1(ðpb.MetaDataV1{}), nil + return wrapper.WrappedMetadataV2(ðpb.MetaDataV2{}), nil }, bytesutil.ToBytes4(params.BeaconConfig().ElectraForkVersion): func() (metadata.Metadata, error) { - return wrapper.WrappedMetadataV1(ðpb.MetaDataV1{}), nil + return wrapper.WrappedMetadataV2(ðpb.MetaDataV2{}), nil }, } diff --git a/beacon-chain/rpc/prysm/v1alpha1/debug/p2p.go b/beacon-chain/rpc/prysm/v1alpha1/debug/p2p.go index 386ef08427e9..c7d6d6034bbb 100644 --- a/beacon-chain/rpc/prysm/v1alpha1/debug/p2p.go +++ b/beacon-chain/rpc/prysm/v1alpha1/debug/p2p.go @@ -105,6 +105,8 @@ func (ds *Server) getPeer(pid peer.ID) (*ethpb.DebugPeerResponse, error) { peerInfo.MetadataV0 = metadata.MetadataObjV0() case metadata.MetadataObjV1() != nil: peerInfo.MetadataV1 = metadata.MetadataObjV1() + case metadata.MetadataObjV2() != nil: + peerInfo.MetadataV2 = metadata.MetadataObjV2() } } addresses := peerStore.Addrs(pid) diff --git a/beacon-chain/sync/BUILD.bazel b/beacon-chain/sync/BUILD.bazel index c2f25ab41272..027be6745f5a 100644 --- a/beacon-chain/sync/BUILD.bazel +++ b/beacon-chain/sync/BUILD.bazel @@ -249,7 +249,6 @@ go_test( "//crypto/bls:go_default_library", "//crypto/rand:go_default_library", "//encoding/bytesutil:go_default_library", - "//encoding/ssz/equality:go_default_library", "//network/forks:go_default_library", "//proto/engine/v1:go_default_library", "//proto/eth/v2:go_default_library", diff --git a/beacon-chain/sync/data_columns_sampling.go b/beacon-chain/sync/data_columns_sampling.go index 1bb0aaf58e6e..3595d5299f98 100644 --- a/beacon-chain/sync/data_columns_sampling.go +++ b/beacon-chain/sync/data_columns_sampling.go @@ -139,6 +139,9 @@ func (d *dataColumnSampler1D) samplingRoutine(ctx context.Context) { // Refresh peer information. func (d *dataColumnSampler1D) refreshPeerInfo() { + dataColumnSidecarSubnetCount := params.BeaconConfig().DataColumnSidecarSubnetCount + columnsPerSubnet := fieldparams.NumberOfColumns / dataColumnSidecarSubnetCount + d.Lock() defer d.Unlock() @@ -146,19 +149,23 @@ func (d *dataColumnSampler1D) refreshPeerInfo() { d.prunePeerInfo(activePeers) for _, pid := range activePeers { - if _, ok := d.columnFromPeer[pid]; ok { - // TODO: need to update peer info here after validator custody. + csc := d.p2p.CustodyCountFromRemotePeer(pid) + + columns, ok := d.columnFromPeer[pid] + columnsCount := uint64(len(columns)) + + if ok && columnsCount == csc*columnsPerSubnet { + // No change for this peer. continue } - csc := d.p2p.CustodyCountFromRemotePeer(pid) nid, err := p2p.ConvertPeerIDToNodeID(pid) if err != nil { log.WithError(err).WithField("peerID", pid).Error("Failed to convert peer ID to node ID") continue } - columns, err := peerdas.CustodyColumns(nid, csc) + columns, err = peerdas.CustodyColumns(nid, csc) if err != nil { log.WithError(err).WithField("peerID", pid).Error("Failed to determine peer custody columns") continue diff --git a/beacon-chain/sync/fork_watcher_test.go b/beacon-chain/sync/fork_watcher_test.go index 3f97fbd86eac..d3b4ea3bd980 100644 --- a/beacon-chain/sync/fork_watcher_test.go +++ b/beacon-chain/sync/fork_watcher_test.go @@ -388,6 +388,7 @@ func TestService_CheckForPreviousEpochFork(t *testing.T) { } } +// oneEpoch returns the duration of one epoch. func oneEpoch() time.Duration { return time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot)) * time.Second } diff --git a/beacon-chain/sync/rate_limiter.go b/beacon-chain/sync/rate_limiter.go index bf0ea3847d4b..0ca92ae875dc 100644 --- a/beacon-chain/sync/rate_limiter.go +++ b/beacon-chain/sync/rate_limiter.go @@ -56,9 +56,10 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter { topicMap := make(map[string]*leakybucket.Collector, len(p2p.RPCTopicMappings)) // Goodbye Message topicMap[addEncoding(p2p.RPCGoodByeTopicV1)] = leakybucket.NewCollector(1, 1, leakyBucketPeriod, false /* deleteEmptyBuckets */) - // MetadataV0 Message + // Metadata Message topicMap[addEncoding(p2p.RPCMetaDataTopicV1)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */) topicMap[addEncoding(p2p.RPCMetaDataTopicV2)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */) + topicMap[addEncoding(p2p.RPCMetaDataTopicV3)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */) // Ping Message topicMap[addEncoding(p2p.RPCPingTopicV1)] = leakybucket.NewCollector(1, defaultBurstLimit, leakyBucketPeriod, false /* deleteEmptyBuckets */) // Status Message diff --git a/beacon-chain/sync/rate_limiter_test.go b/beacon-chain/sync/rate_limiter_test.go index 7805cdbb656c..c05b0f18ea66 100644 --- a/beacon-chain/sync/rate_limiter_test.go +++ b/beacon-chain/sync/rate_limiter_test.go @@ -18,7 +18,7 @@ import ( func TestNewRateLimiter(t *testing.T) { rlimiter := newRateLimiter(mockp2p.NewTestP2P(t)) - assert.Equal(t, 14, len(rlimiter.limiterMap), "correct number of topics not registered") + assert.Equal(t, 15, len(rlimiter.limiterMap), "correct number of topics not registered") } func TestNewRateLimiter_FreeCorrectly(t *testing.T) { diff --git a/beacon-chain/sync/rpc.go b/beacon-chain/sync/rpc.go index 7a53e63a2c07..27fee0dca608 100644 --- a/beacon-chain/sync/rpc.go +++ b/beacon-chain/sync/rpc.go @@ -121,6 +121,10 @@ func (s *Service) registerRPCHandlersPeerDAS() { p2p.RPCDataColumnSidecarsByRangeTopicV1, s.dataColumnSidecarsByRangeRPCHandler, ) + s.registerRPC( + p2p.RPCMetaDataTopicV3, + s.metaDataHandler, + ) } // Remove all v1 Stream handlers that are no longer supported @@ -215,7 +219,7 @@ func (s *Service) registerRPC(baseTopic string, handle rpcHandler) { // since metadata requests do not have any data in the payload, we // do not decode anything. - if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 { + if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 || baseTopic == p2p.RPCMetaDataTopicV3 { if err := handle(ctx, base, stream); err != nil { messageFailedProcessingCounter.WithLabelValues(topic).Inc() if !errors.Is(err, p2ptypes.ErrWrongForkDigestVersion) { diff --git a/beacon-chain/sync/rpc_metadata.go b/beacon-chain/sync/rpc_metadata.go index 65fb0003d896..5b0e72ce7f2c 100644 --- a/beacon-chain/sync/rpc_metadata.go +++ b/beacon-chain/sync/rpc_metadata.go @@ -21,97 +21,168 @@ import ( func (s *Service) metaDataHandler(_ context.Context, _ interface{}, stream libp2pcore.Stream) error { SetRPCStreamDeadlines(stream) + // Validate the incoming request regarding rate limiting. if err := s.rateLimiter.validateRequest(stream, 1); err != nil { - return err + return errors.Wrap(err, "validate request") } + s.rateLimiter.add(stream, 1) - if s.cfg.p2p.Metadata() == nil || s.cfg.p2p.Metadata().IsNil() { + // Retrieve our metadata. + metadata := s.cfg.p2p.Metadata() + + // Handle the case our metadata is nil. + if metadata == nil || metadata.IsNil() { nilErr := errors.New("nil metadata stored for host") + resp, err := s.generateErrorResponse(responseCodeServerError, types.ErrGeneric.Error()) if err != nil { log.WithError(err).Debug("Could not generate a response error") - } else if _, err := stream.Write(resp); err != nil { + return nilErr + } + + if _, err := stream.Write(resp); err != nil { log.WithError(err).Debug("Could not write to stream") } + return nilErr } + + // Get the stream version from the protocol. _, _, streamVersion, err := p2p.TopicDeconstructor(string(stream.Protocol())) if err != nil { + wrappedErr := errors.Wrap(err, "topic deconstructor") + resp, genErr := s.generateErrorResponse(responseCodeServerError, types.ErrGeneric.Error()) if genErr != nil { log.WithError(genErr).Debug("Could not generate a response error") - } else if _, wErr := stream.Write(resp); wErr != nil { + return wrappedErr + } + + if _, wErr := stream.Write(resp); wErr != nil { log.WithError(wErr).Debug("Could not write to stream") } - return err + return wrappedErr } - currMd := s.cfg.p2p.Metadata() + + // Handle the case where the stream version is not recognized. + metadataVersion := metadata.Version() switch streamVersion { case p2p.SchemaVersionV1: - // We have a v1 metadata object saved locally, so we - // convert it back to a v0 metadata object. - if currMd.Version() != version.Phase0 { - currMd = wrapper.WrappedMetadataV0( + switch metadataVersion { + case version.Altair, version.Deneb: + metadata = wrapper.WrappedMetadataV0( &pb.MetaDataV0{ - Attnets: currMd.AttnetsBitfield(), - SeqNumber: currMd.SequenceNumber(), + Attnets: metadata.AttnetsBitfield(), + SeqNumber: metadata.SequenceNumber(), }) } + case p2p.SchemaVersionV2: - // We have a v0 metadata object saved locally, so we - // convert it to a v1 metadata object. - if currMd.Version() != version.Altair { - currMd = wrapper.WrappedMetadataV1( + switch metadataVersion { + case version.Phase0: + metadata = wrapper.WrappedMetadataV1( &pb.MetaDataV1{ - Attnets: currMd.AttnetsBitfield(), - SeqNumber: currMd.SequenceNumber(), + Attnets: metadata.AttnetsBitfield(), + SeqNumber: metadata.SequenceNumber(), Syncnets: bitfield.Bitvector4{byte(0x00)}, }) + case version.Deneb: + metadata = wrapper.WrappedMetadataV1( + &pb.MetaDataV1{ + Attnets: metadata.AttnetsBitfield(), + SeqNumber: metadata.SequenceNumber(), + Syncnets: metadata.SyncnetsBitfield(), + }) + } + + case p2p.SchemaVersionV3: + switch metadataVersion { + case version.Phase0: + metadata = wrapper.WrappedMetadataV2( + &pb.MetaDataV2{ + Attnets: metadata.AttnetsBitfield(), + SeqNumber: metadata.SequenceNumber(), + Syncnets: bitfield.Bitvector4{byte(0x00)}, + CustodySubnetCount: 0, + }) + case version.Altair: + metadata = wrapper.WrappedMetadataV2( + &pb.MetaDataV2{ + Attnets: metadata.AttnetsBitfield(), + SeqNumber: metadata.SequenceNumber(), + Syncnets: metadata.SyncnetsBitfield(), + CustodySubnetCount: 0, + }) } } + + // Write the METADATA response into the stream. if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil { - return err + return errors.Wrap(err, "write metadata response") } - _, err = s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, currMd) + + // Encode the metadata and write it to the stream. + _, err = s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, metadata) if err != nil { - return err + return errors.Wrap(err, "encode metadata") } + closeStream(stream, log) return nil } -func (s *Service) sendMetaDataRequest(ctx context.Context, id peer.ID) (metadata.Metadata, error) { +// sendMetaDataRequest sends a METADATA request to the peer and return the response. +func (s *Service) sendMetaDataRequest(ctx context.Context, peerID peer.ID) (metadata.Metadata, error) { ctx, cancel := context.WithTimeout(ctx, respTimeout) defer cancel() - topic, err := p2p.TopicFromMessage(p2p.MetadataMessageName, slots.ToEpoch(s.cfg.clock.CurrentSlot())) + // Compute the current epoch. + currentSlot := s.cfg.clock.CurrentSlot() + currentEpoch := slots.ToEpoch(currentSlot) + + // Compute the topic for the metadata request regarding the current epoch. + topic, err := p2p.TopicFromMessage(p2p.MetadataMessageName, currentEpoch) if err != nil { - return nil, err + return nil, errors.Wrap(err, "topic from message") } - stream, err := s.cfg.p2p.Send(ctx, new(interface{}), topic, id) + + // Send the METADATA request to the peer. + message := new(interface{}) + stream, err := s.cfg.p2p.Send(ctx, message, topic, peerID) if err != nil { - return nil, err + return nil, errors.Wrap(err, "send metadata request") } + defer closeStream(stream, log) + + // Read the METADATA response from the peer. code, errMsg, err := ReadStatusCode(stream, s.cfg.p2p.Encoding()) if err != nil { - s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer()) - return nil, err + s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID) + return nil, errors.Wrap(err, "read status code") } + if code != 0 { - s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer()) + s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID) return nil, errors.New(errMsg) } + + // Get the genesis validators root. valRoot := s.cfg.clock.GenesisValidatorsRoot() - rpcCtx, err := forks.ForkDigestFromEpoch(slots.ToEpoch(s.cfg.clock.CurrentSlot()), valRoot[:]) + + // Get the fork digest from the current epoch and the genesis validators root. + rpcCtx, err := forks.ForkDigestFromEpoch(currentEpoch, valRoot[:]) if err != nil { - return nil, err + return nil, errors.Wrap(err, "fork digest from epoch") } + + // Instantiate zero value of the metadata. msg, err := extractDataTypeFromTypeMap(types.MetaDataMap, rpcCtx[:], s.cfg.clock) if err != nil { - return nil, err + return nil, errors.Wrap(err, "extract data type from type map") } + // Defensive check to ensure valid objects are being sent. topicVersion := "" switch msg.Version() { @@ -119,13 +190,20 @@ func (s *Service) sendMetaDataRequest(ctx context.Context, id peer.ID) (metadata topicVersion = p2p.SchemaVersionV1 case version.Altair: topicVersion = p2p.SchemaVersionV2 + case version.Deneb: + topicVersion = p2p.SchemaVersionV3 } + + // Validate the version of the topic. if err := validateVersion(topicVersion, stream); err != nil { return nil, err } + + // Decode the metadata from the peer. if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil { s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer()) return nil, err } + return msg, nil } diff --git a/beacon-chain/sync/rpc_metadata_test.go b/beacon-chain/sync/rpc_metadata_test.go index 23eb74041f0b..005269c3005d 100644 --- a/beacon-chain/sync/rpc_metadata_test.go +++ b/beacon-chain/sync/rpc_metadata_test.go @@ -8,6 +8,7 @@ import ( "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/protocol" + "github.com/prysmaticlabs/go-bitfield" mock "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing" db "github.com/prysmaticlabs/prysm/v5/beacon-chain/db/testing" "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p" @@ -16,8 +17,8 @@ import ( "github.com/prysmaticlabs/prysm/v5/config/params" "github.com/prysmaticlabs/prysm/v5/consensus-types/wrapper" leakybucket "github.com/prysmaticlabs/prysm/v5/container/leaky-bucket" - "github.com/prysmaticlabs/prysm/v5/encoding/ssz/equality" pb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1" + "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1/metadata" "github.com/prysmaticlabs/prysm/v5/testing/assert" "github.com/prysmaticlabs/prysm/v5/testing/require" "github.com/prysmaticlabs/prysm/v5/testing/util" @@ -75,157 +76,239 @@ func TestMetaDataRPCHandler_ReceivesMetadata(t *testing.T) { } } -func TestMetadataRPCHandler_SendsMetadata(t *testing.T) { - p1 := p2ptest.NewTestP2P(t) - p2 := p2ptest.NewTestP2P(t) - p1.Connect(p2) - assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected") - bitfield := [8]byte{'A', 'B'} - p2.LocalMetadata = wrapper.WrappedMetadataV0(&pb.MetaDataV0{ - SeqNumber: 2, - Attnets: bitfield[:], - }) - - // Set up a head state in the database with data we expect. - chain := &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}} - d := db.SetupDB(t) - r := &Service{ +func createService(peer p2p.P2P, chain *mock.ChainService) *Service { + return &Service{ cfg: &config{ - beaconDB: d, - p2p: p1, - chain: chain, - clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot), - }, - rateLimiter: newRateLimiter(p1), - } - - r2 := &Service{ - cfg: &config{ - beaconDB: d, - p2p: p2, - chain: &mock.ChainService{Genesis: time.Now(), ValidatorsRoot: [32]byte{}}, + p2p: peer, + chain: chain, + clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot), }, - rateLimiter: newRateLimiter(p2), - } - - // Setup streams - pcl := protocol.ID(p2p.RPCMetaDataTopicV1 + r.cfg.p2p.Encoding().ProtocolSuffix()) - topic := string(pcl) - r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(1, 1, time.Second, false) - r2.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(1, 1, time.Second, false) - - var wg sync.WaitGroup - wg.Add(1) - p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) { - defer wg.Done() - assert.NoError(t, r2.metaDataHandler(context.Background(), new(interface{}), stream)) - }) - - md, err := r.sendMetaDataRequest(context.Background(), p2.BHost.ID()) - assert.NoError(t, err) - - if !equality.DeepEqual(md.InnerObject(), p2.LocalMetadata.InnerObject()) { - t.Fatalf("MetadataV0 unequal, received %v but wanted %v", md, p2.LocalMetadata) + rateLimiter: newRateLimiter(peer), } +} - if util.WaitTimeout(&wg, 1*time.Second) { - t.Fatal("Did not receive stream within 1 sec") - } +func TestMetadataRPCHandler_SendMetadataRequest(t *testing.T) { + const ( + requestTimeout = 1 * time.Second + seqNumber = 2 + custodySubnetCount = 4 + ) - conns := p1.BHost.Network().ConnsToPeer(p2.BHost.ID()) - if len(conns) == 0 { - t.Error("Peer is disconnected despite receiving a valid ping") - } -} + attnets := []byte{'A', 'B', 'C', 'D', 'E', 'F', 'G', 'H'} + syncnets := []byte{0x4} -func TestMetadataRPCHandler_SendsMetadataAltair(t *testing.T) { + // Configure the test beacon chain. params.SetupTestConfigCleanup(t) - bCfg := params.BeaconConfig().Copy() - bCfg.AltairForkEpoch = 5 - params.OverrideBeaconConfig(bCfg) + beaconChainConfig := params.BeaconConfig().Copy() + beaconChainConfig.AltairForkEpoch = 5 + beaconChainConfig.DenebForkEpoch = 10 + beaconChainConfig.Eip7594ForkEpoch = 10 + params.OverrideBeaconConfig(beaconChainConfig) params.BeaconConfig().InitializeForkSchedule() - p1 := p2ptest.NewTestP2P(t) - p2 := p2ptest.NewTestP2P(t) - p1.Connect(p2) - assert.Equal(t, 1, len(p1.BHost.Network().Peers()), "Expected peers to be connected") - bitfield := [8]byte{'A', 'B'} - p2.LocalMetadata = wrapper.WrappedMetadataV0(&pb.MetaDataV0{ - SeqNumber: 2, - Attnets: bitfield[:], - }) + // Compute the number of seconds in an epoch. + secondsPerEpoch := oneEpoch() - // Set up a head state in the database with data we expect. - d := db.SetupDB(t) - chain := &mock.ChainService{Genesis: time.Now().Add(-5 * oneEpoch()), ValidatorsRoot: [32]byte{}} - r := &Service{ - cfg: &config{ - beaconDB: d, - p2p: p1, - chain: chain, - clock: startup.NewClock(chain.Genesis, chain.ValidatorsRoot), + testCases := []struct { + name string + topic string + epochsSinceGenesisPeer1, epochsSinceGenesisPeer2 int + metadataPeer2, expected metadata.Metadata + }{ + { + name: "Phase0-Phase0", + topic: p2p.RPCMetaDataTopicV1, + epochsSinceGenesisPeer1: 0, + epochsSinceGenesisPeer2: 0, + metadataPeer2: wrapper.WrappedMetadataV0(&pb.MetaDataV0{ + SeqNumber: seqNumber, + Attnets: attnets, + }), + expected: wrapper.WrappedMetadataV0(&pb.MetaDataV0{ + SeqNumber: seqNumber, + Attnets: attnets, + }), }, - rateLimiter: newRateLimiter(p1), - } - - chain2 := &mock.ChainService{Genesis: time.Now().Add(-5 * oneEpoch()), ValidatorsRoot: [32]byte{}} - r2 := &Service{ - cfg: &config{ - beaconDB: d, - p2p: p2, - chain: chain2, - clock: startup.NewClock(chain2.Genesis, chain2.ValidatorsRoot), + { + name: "Phase0-Altair", + topic: p2p.RPCMetaDataTopicV1, + epochsSinceGenesisPeer1: 0, + epochsSinceGenesisPeer2: 5, + metadataPeer2: wrapper.WrappedMetadataV1(&pb.MetaDataV1{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + }), + expected: wrapper.WrappedMetadataV0(&pb.MetaDataV0{ + SeqNumber: seqNumber, + Attnets: attnets, + }), + }, + { + name: "Phase0-PeerDAS", + topic: p2p.RPCMetaDataTopicV1, + epochsSinceGenesisPeer1: 0, + epochsSinceGenesisPeer2: 10, + metadataPeer2: wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + CustodySubnetCount: custodySubnetCount, + }), + expected: wrapper.WrappedMetadataV0(&pb.MetaDataV0{ + SeqNumber: seqNumber, + Attnets: attnets, + }), + }, + { + name: "Altair-Phase0", + topic: p2p.RPCMetaDataTopicV2, + epochsSinceGenesisPeer1: 5, + epochsSinceGenesisPeer2: 0, + metadataPeer2: wrapper.WrappedMetadataV0(&pb.MetaDataV0{ + SeqNumber: seqNumber, + Attnets: attnets, + }), + expected: wrapper.WrappedMetadataV1(&pb.MetaDataV1{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: bitfield.Bitvector4{byte(0x00)}, + }), + }, + { + name: "Altair-Altair", + topic: p2p.RPCMetaDataTopicV2, + epochsSinceGenesisPeer1: 5, + epochsSinceGenesisPeer2: 5, + metadataPeer2: wrapper.WrappedMetadataV1(&pb.MetaDataV1{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + }), + expected: wrapper.WrappedMetadataV1(&pb.MetaDataV1{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + }), + }, + { + name: "Altair-PeerDAS", + topic: p2p.RPCMetaDataTopicV2, + epochsSinceGenesisPeer1: 5, + epochsSinceGenesisPeer2: 10, + metadataPeer2: wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + CustodySubnetCount: custodySubnetCount, + }), + expected: wrapper.WrappedMetadataV1(&pb.MetaDataV1{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + }), + }, + { + name: "PeerDAS-Phase0", + topic: p2p.RPCMetaDataTopicV3, + epochsSinceGenesisPeer1: 10, + epochsSinceGenesisPeer2: 0, + metadataPeer2: wrapper.WrappedMetadataV0(&pb.MetaDataV0{ + SeqNumber: seqNumber, + Attnets: attnets, + }), + expected: wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: bitfield.Bitvector4{byte(0x00)}, + CustodySubnetCount: 0, + }), + }, + { + name: "PeerDAS-Altail", + topic: p2p.RPCMetaDataTopicV3, + epochsSinceGenesisPeer1: 10, + epochsSinceGenesisPeer2: 5, + metadataPeer2: wrapper.WrappedMetadataV1(&pb.MetaDataV1{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + }), + expected: wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + CustodySubnetCount: 0, + }), + }, + { + name: "PeerDAS-PeerDAS", + topic: p2p.RPCMetaDataTopicV3, + epochsSinceGenesisPeer1: 10, + epochsSinceGenesisPeer2: 10, + metadataPeer2: wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + CustodySubnetCount: custodySubnetCount, + }), + expected: wrapper.WrappedMetadataV2(&pb.MetaDataV2{ + SeqNumber: seqNumber, + Attnets: attnets, + Syncnets: syncnets, + CustodySubnetCount: custodySubnetCount, + }), }, - rateLimiter: newRateLimiter(p2), } - // Setup streams - pcl := protocol.ID(p2p.RPCMetaDataTopicV2 + r.cfg.p2p.Encoding().ProtocolSuffix()) - topic := string(pcl) - r.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(2, 2, time.Second, false) - r2.rateLimiter.limiterMap[topic] = leakybucket.NewCollector(2, 2, time.Second, false) + for _, tc := range testCases { + var wg sync.WaitGroup - var wg sync.WaitGroup - wg.Add(1) - p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) { - defer wg.Done() - err := r2.metaDataHandler(context.Background(), new(interface{}), stream) - assert.NoError(t, err) - }) + ctx := context.Background() - _, err := r.sendMetaDataRequest(context.Background(), p2.BHost.ID()) - assert.NoError(t, err) + // Setup and connect peers. + peer1, peer2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t) + peer1.Connect(peer2) - if util.WaitTimeout(&wg, 1*time.Second) { - t.Fatal("Did not receive stream within 1 sec") - } + // Ensure the peers are connected. + peersCount := len(peer1.BHost.Network().Peers()) + assert.Equal(t, 1, peersCount, "Expected peers to be connected") - // Fix up peer with the correct metadata. - p2.LocalMetadata = wrapper.WrappedMetadataV1(&pb.MetaDataV1{ - SeqNumber: 2, - Attnets: bitfield[:], - Syncnets: []byte{0x0}, - }) + // Setup sync services. + genesisPeer1 := time.Now().Add(-time.Duration(tc.epochsSinceGenesisPeer1) * secondsPerEpoch) + genesisPeer2 := time.Now().Add(-time.Duration(tc.epochsSinceGenesisPeer2) * secondsPerEpoch) - wg.Add(1) - p2.BHost.SetStreamHandler(pcl, func(stream network.Stream) { - defer wg.Done() - assert.NoError(t, r2.metaDataHandler(context.Background(), new(interface{}), stream)) - }) + chainPeer1 := &mock.ChainService{Genesis: genesisPeer1, ValidatorsRoot: [32]byte{}} + chainPeer2 := &mock.ChainService{Genesis: genesisPeer2, ValidatorsRoot: [32]byte{}} - md, err := r.sendMetaDataRequest(context.Background(), p2.BHost.ID()) - assert.NoError(t, err) + servicePeer1 := createService(peer1, chainPeer1) + servicePeer2 := createService(peer2, chainPeer2) - if !equality.DeepEqual(md.InnerObject(), p2.LocalMetadata.InnerObject()) { - t.Fatalf("MetadataV1 unequal, received %v but wanted %v", md, p2.LocalMetadata) - } + // Define the behavior of peer2 when receiving a METADATA request. + protocolSuffix := servicePeer2.cfg.p2p.Encoding().ProtocolSuffix() + protocolID := protocol.ID(tc.topic + protocolSuffix) + peer2.LocalMetadata = tc.metadataPeer2 - if util.WaitTimeout(&wg, 1*time.Second) { - t.Fatal("Did not receive stream within 1 sec") - } + wg.Add(1) + peer2.BHost.SetStreamHandler(protocolID, func(stream network.Stream) { + defer wg.Done() + err := servicePeer2.metaDataHandler(ctx, new(interface{}), stream) + assert.NoError(t, err) + }) - conns := p1.BHost.Network().ConnsToPeer(p2.BHost.ID()) - if len(conns) == 0 { - t.Error("Peer is disconnected despite receiving a valid ping") + // Send a METADATA request from peer1 to peer2. + actual, err := servicePeer1.sendMetaDataRequest(ctx, peer2.BHost.ID()) + assert.NoError(t, err) + + // Wait until the METADATA request is received by peer2 or timeout. + timeOutReached := util.WaitTimeout(&wg, requestTimeout) + require.Equal(t, false, timeOutReached, "Did not receive METADATA request within timeout") + + // Compare the received METADATA object with the expected METADATA object. + require.DeepSSZEqual(t, tc.expected.InnerObject(), actual.InnerObject(), "Metadata unequal") + + // Ensure the peers are still connected. + peersCount = len(peer1.BHost.Network().Peers()) + assert.Equal(t, 1, peersCount, "Expected peers to be connected") } } diff --git a/beacon-chain/sync/rpc_ping.go b/beacon-chain/sync/rpc_ping.go index e082b5cbab63..872f4a25e391 100644 --- a/beacon-chain/sync/rpc_ping.go +++ b/beacon-chain/sync/rpc_ping.go @@ -2,12 +2,12 @@ package sync import ( "context" - "errors" "fmt" "strings" libp2pcore "github.com/libp2p/go-libp2p/core" "github.com/libp2p/go-libp2p/core/peer" + "github.com/pkg/errors" "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p" p2ptypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types" "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives" @@ -16,127 +16,191 @@ import ( ) // pingHandler reads the incoming ping rpc message from the peer. +// If the peer's sequence number is higher than the one we have in our store for it, +// we send a METADATA request to the peer to get its latest metadata, and store it. +// TODO: This function is actually poorly named, since it does more than just reading a ping message. func (s *Service) pingHandler(_ context.Context, msg interface{}, stream libp2pcore.Stream) error { SetRPCStreamDeadlines(stream) + // Convert the message to SSW Uint64 type. m, ok := msg.(*primitives.SSZUint64) if !ok { return fmt.Errorf("wrong message type for ping, got %T, wanted *uint64", msg) } + + // Validate the incoming request regarding rate limiting. if err := s.rateLimiter.validateRequest(stream, 1); err != nil { - return err + return errors.Wrap(err, "validate request") } + s.rateLimiter.add(stream, 1) - valid, err := s.validateSequenceNum(*m, stream.Conn().RemotePeer()) + + // Retrieve the peer ID. + peerID := stream.Conn().RemotePeer() + + // Check if the peer sequence number is higher than the one we have in our store. + valid, err := s.validateSequenceNum(*m, peerID) if err != nil { // Descore peer for giving us a bad sequence number. if errors.Is(err, p2ptypes.ErrInvalidSequenceNum) { - s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer()) + s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID) s.writeErrorResponseToStream(responseCodeInvalidRequest, p2ptypes.ErrInvalidSequenceNum.Error(), stream) } - return err + + return errors.Wrap(err, "validate sequence number") } + + // We can already prepare a success response to the peer. if _, err := stream.Write([]byte{responseCodeSuccess}); err != nil { - return err + return errors.Wrap(err, "write response") } - sq := primitives.SSZUint64(s.cfg.p2p.MetadataSeq()) - if _, err := s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, &sq); err != nil { + + // Retrieve our own sequence number. + seqNumber := s.cfg.p2p.MetadataSeq() + + // SSZ encode our sequence number. + seqNumberSSZ := primitives.SSZUint64(seqNumber) + + // Send our sequence number back to the peer. + if _, err := s.cfg.p2p.Encoding().EncodeWithMaxLength(stream, &seqNumberSSZ); err != nil { return err } closeStream(stream, log) if valid { - // If the sequence number was valid we're done. + // If the peer's sequence numberwas valid we're done. return nil } - // The sequence number was not valid. Start our own ping back to the peer. + // The peer's sequence number was not valid. We ask the peer for its metadata. go func() { - // New context so the calling function doesn't cancel on us. + // Define a new context so the calling function doesn't cancel on us. ctx, cancel := context.WithTimeout(context.Background(), ttfbTimeout) defer cancel() - md, err := s.sendMetaDataRequest(ctx, stream.Conn().RemotePeer()) + + // Send a METADATA request to the peer. + peerMetadata, err := s.sendMetaDataRequest(ctx, peerID) if err != nil { // We cannot compare errors directly as the stream muxer error // type isn't compatible with the error we have, so a direct // equality checks fails. if !strings.Contains(err.Error(), p2ptypes.ErrIODeadline.Error()) { - log.WithField("peer", stream.Conn().RemotePeer()).WithError(err).Debug("Could not send metadata request") + log.WithField("peer", peerID).WithError(err).Debug("Could not send metadata request") } + return } - // update metadata if there is no error - s.cfg.p2p.Peers().SetMetadata(stream.Conn().RemotePeer(), md) + + // Update peer's metadata. + s.cfg.p2p.Peers().SetMetadata(peerID, peerMetadata) }() return nil } -func (s *Service) sendPingRequest(ctx context.Context, id peer.ID) error { +// sendPingRequest first sends a PING request to the peer. +// If the peer responds with a sequence number higher than latest one for it we have in our store, +// then this function sends a METADATA request to the peer, and stores the metadata received. +// TODO: This function is actually poorly named, since it does more than just sending a ping request. +func (s *Service) sendPingRequest(ctx context.Context, peerID peer.ID) error { ctx, cancel := context.WithTimeout(ctx, respTimeout) defer cancel() - metadataSeq := primitives.SSZUint64(s.cfg.p2p.MetadataSeq()) - topic, err := p2p.TopicFromMessage(p2p.PingMessageName, slots.ToEpoch(s.cfg.clock.CurrentSlot())) + // Get the current epoch. + currentSlot := s.cfg.clock.CurrentSlot() + currentEpoch := slots.ToEpoch(currentSlot) + + // SSZ encode our metadata sequence number. + metadataSeq := s.cfg.p2p.MetadataSeq() + encodedMetadataSeq := primitives.SSZUint64(metadataSeq) + + // Get the PING topic for the current epoch. + topic, err := p2p.TopicFromMessage(p2p.PingMessageName, currentEpoch) if err != nil { - return err + return errors.Wrap(err, "topic from message") } - stream, err := s.cfg.p2p.Send(ctx, &metadataSeq, topic, id) + + // Send the PING request to the peer. + stream, err := s.cfg.p2p.Send(ctx, &encodedMetadataSeq, topic, peerID) if err != nil { - return err + return errors.Wrap(err, "send ping request") } - currentTime := time.Now() defer closeStream(stream, log) + startTime := time.Now() + + // Read the response from the peer. code, errMsg, err := ReadStatusCode(stream, s.cfg.p2p.Encoding()) if err != nil { - return err + return errors.Wrap(err, "read status code") } - // Records the latency of the ping request for that peer. - s.cfg.p2p.Host().Peerstore().RecordLatency(id, time.Now().Sub(currentTime)) + // Record the latency of the ping request for that peer. + s.cfg.p2p.Host().Peerstore().RecordLatency(peerID, time.Now().Sub(startTime)) + + // If the peer responded with an error, increment the bad responses scorer. if code != 0 { - s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer()) + s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID) return errors.New(errMsg) } + + // Decode the sequence number from the peer. msg := new(primitives.SSZUint64) if err := s.cfg.p2p.Encoding().DecodeWithMaxLength(stream, msg); err != nil { - return err + return errors.Wrap(err, "decode sequence number") } - valid, err := s.validateSequenceNum(*msg, stream.Conn().RemotePeer()) + + // Determine if the peer's sequence number returned by the peer is higher than the one we have in our store. + valid, err := s.validateSequenceNum(*msg, peerID) if err != nil { // Descore peer for giving us a bad sequence number. if errors.Is(err, p2ptypes.ErrInvalidSequenceNum) { - s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer()) + s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(peerID) } - return err + + return errors.Wrap(err, "validate sequence number") } + + // The sequence number have in our store for this peer is the same as the one returned by the peer, all good. if valid { return nil } - md, err := s.sendMetaDataRequest(ctx, stream.Conn().RemotePeer()) + + // We need to send a METADATA request to the peer to get its latest metadata. + md, err := s.sendMetaDataRequest(ctx, peerID) if err != nil { - // do not increment bad responses, as its - // already done in the request method. - return err + // do not increment bad responses, as its already done in the request method. + return errors.Wrap(err, "send metadata request") } - s.cfg.p2p.Peers().SetMetadata(stream.Conn().RemotePeer(), md) + + // Update the metadata for the peer. + s.cfg.p2p.Peers().SetMetadata(peerID, md) + return nil } -// validates the peer's sequence number. +// validateSequenceNum validates the peer's sequence number. +// - If the peer's sequence number is greater than the sequence number we have in our store for the peer, return false. +// - If the peer's sequence number is equal to the sequence number we have in our store for the peer, return true. +// - If the peer's sequence number is less than the sequence number we have in our store for the peer, return an error. func (s *Service) validateSequenceNum(seq primitives.SSZUint64, id peer.ID) (bool, error) { + // Retrieve the metadata for the peer we got in our store. md, err := s.cfg.p2p.Peers().Metadata(id) if err != nil { - return false, err + return false, errors.Wrap(err, "get metadata") } + + // If we have no metadata for the peer, return false. if md == nil || md.IsNil() { return false, nil } - // Return error on invalid sequence number. + + // The peer's sequence number must be less than or equal to the sequence number we have in our store. if md.SequenceNumber() > uint64(seq) { return false, p2ptypes.ErrInvalidSequenceNum } + + // Return true if the peer's sequence number is equal to the sequence number we have in our store. return md.SequenceNumber() == uint64(seq), nil } diff --git a/cmd/prysmctl/p2p/client.go b/cmd/prysmctl/p2p/client.go index 146e7f0ef365..ba308f6466e9 100644 --- a/cmd/prysmctl/p2p/client.go +++ b/cmd/prysmctl/p2p/client.go @@ -126,7 +126,7 @@ func (c *client) Send( return nil, errors.Wrap(err, "could not open new stream") } // do not encode anything if we are sending a metadata request - if baseTopic != p2p.RPCMetaDataTopicV1 && baseTopic != p2p.RPCMetaDataTopicV2 { + if baseTopic != p2p.RPCMetaDataTopicV1 && baseTopic != p2p.RPCMetaDataTopicV2 && baseTopic != p2p.RPCMetaDataTopicV3 { castedMsg, ok := message.(ssz.Marshaler) if !ok { return nil, errors.Errorf("%T does not support the ssz marshaller interface", message) diff --git a/cmd/prysmctl/p2p/handler.go b/cmd/prysmctl/p2p/handler.go index c0aa289e0a56..b9763b72cd2c 100644 --- a/cmd/prysmctl/p2p/handler.go +++ b/cmd/prysmctl/p2p/handler.go @@ -50,7 +50,7 @@ func (c *client) registerRPCHandler(baseTopic string, handle rpcHandler) { // since metadata requests do not have any data in the payload, we // do not decode anything. - if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 { + if baseTopic == p2p.RPCMetaDataTopicV1 || baseTopic == p2p.RPCMetaDataTopicV2 || baseTopic == p2p.RPCMetaDataTopicV3 { if err := handle(context.Background(), base, stream); err != nil { if !errors.Is(err, p2ptypes.ErrWrongForkDigestVersion) { log.WithError(err).Debug("Could not handle p2p RPC") diff --git a/config/params/mainnet_config.go b/config/params/mainnet_config.go index 14b2d0af47d3..fd52f3b17c52 100644 --- a/config/params/mainnet_config.go +++ b/config/params/mainnet_config.go @@ -35,6 +35,7 @@ var mainnetNetworkConfig = &NetworkConfig{ ETH2Key: "eth2", AttSubnetKey: "attnets", SyncCommsSubnetKey: "syncnets", + CustodySubnetCountKey: "csc", MinimumPeersInSubnetSearch: 20, ContractDeploymentBlock: 11184524, // Note: contract was deployed in block 11052984 but no transactions were sent until 11184524. BootstrapNodes: []string{ diff --git a/config/params/network_config.go b/config/params/network_config.go index 1d619417dcd9..a46cc8c13cd5 100644 --- a/config/params/network_config.go +++ b/config/params/network_config.go @@ -8,9 +8,10 @@ import ( // NetworkConfig defines the spec based network parameters. type NetworkConfig struct { // DiscoveryV5 Config - ETH2Key string // ETH2Key is the ENR key of the Ethereum consensus object in an enr. - AttSubnetKey string // AttSubnetKey is the ENR key of the subnet bitfield in the enr. - SyncCommsSubnetKey string // SyncCommsSubnetKey is the ENR key of the sync committee subnet bitfield in the enr. + ETH2Key string // ETH2Key is the ENR key of the Ethereum consensus object. + AttSubnetKey string // AttSubnetKey is the ENR key of the subnet bitfield. + SyncCommsSubnetKey string // SyncCommsSubnetKey is the ENR key of the sync committee subnet bitfield. + CustodySubnetCountKey string // CustodySubnetCountKey is the ENR key of the custody subnet count. MinimumPeersInSubnetSearch uint64 // PeersInSubnetSearch is the required amount of peers that we need to be able to lookup in a subnet search. // Chain Network Config diff --git a/consensus-types/wrapper/metadata.go b/consensus-types/wrapper/metadata.go index 824ffa6e95aa..adbbc81e4c50 100644 --- a/consensus-types/wrapper/metadata.go +++ b/consensus-types/wrapper/metadata.go @@ -8,6 +8,9 @@ import ( "google.golang.org/protobuf/proto" ) +// MetadataV0 +// ---------- + // MetadataV0 is a convenience wrapper around our metadata protobuf object. type MetadataV0 struct { md *pb.MetaDataV0 @@ -28,6 +31,16 @@ func (m MetadataV0) AttnetsBitfield() bitfield.Bitvector64 { return m.md.Attnets } +// SyncnetsBitfield returns the bitfield stored in the metadata. +func (m MetadataV0) SyncnetsBitfield() bitfield.Bitvector4 { + return bitfield.Bitvector4{0} +} + +// CustodySubnetCount returns custody subnet count from the metadata. +func (m MetadataV0) CustodySubnetCount() uint64 { + return 0 +} + // InnerObject returns the underlying metadata protobuf structure. func (m MetadataV0) InnerObject() interface{} { return m.md @@ -74,15 +87,24 @@ func (m MetadataV0) MetadataObjV0() *pb.MetaDataV0 { // MetadataObjV1 returns the inner metadata object in its type // specified form. If it doesn't exist then we return nothing. -func (_ MetadataV0) MetadataObjV1() *pb.MetaDataV1 { +func (MetadataV0) MetadataObjV1() *pb.MetaDataV1 { + return nil +} + +// MetadataObjV2 returns the inner metadata object in its type +// specified form. If it doesn't exist then we return nothing. +func (MetadataV0) MetadataObjV2() *pb.MetaDataV2 { return nil } // Version returns the fork version of the underlying object. -func (_ MetadataV0) Version() int { +func (MetadataV0) Version() int { return version.Phase0 } +// MetadataV1 +// ---------- + // MetadataV1 is a convenience wrapper around our metadata v2 protobuf object. type MetadataV1 struct { md *pb.MetaDataV1 @@ -103,6 +125,16 @@ func (m MetadataV1) AttnetsBitfield() bitfield.Bitvector64 { return m.md.Attnets } +// SyncnetsBitfield returns the bitfield stored in the metadata. +func (m MetadataV1) SyncnetsBitfield() bitfield.Bitvector4 { + return m.md.Syncnets +} + +// CustodySubnetCount returns custody subnet count from the metadata. +func (m MetadataV1) CustodySubnetCount() uint64 { + return 0 +} + // InnerObject returns the underlying metadata protobuf structure. func (m MetadataV1) InnerObject() interface{} { return m.md @@ -143,7 +175,7 @@ func (m MetadataV1) UnmarshalSSZ(buf []byte) error { // MetadataObjV0 returns the inner metadata object in its type // specified form. If it doesn't exist then we return nothing. -func (_ MetadataV1) MetadataObjV0() *pb.MetaDataV0 { +func (MetadataV1) MetadataObjV0() *pb.MetaDataV0 { return nil } @@ -153,7 +185,107 @@ func (m MetadataV1) MetadataObjV1() *pb.MetaDataV1 { return m.md } +// MetadataObjV2 returns the inner metadata object in its type +// specified form. If it doesn't exist then we return nothing. +func (m MetadataV1) MetadataObjV2() *pb.MetaDataV2 { + return nil +} + // Version returns the fork version of the underlying object. -func (_ MetadataV1) Version() int { +func (MetadataV1) Version() int { return version.Altair } + +// MetadataV2 +// ---------- + +// MetadataV2 is a convenience wrapper around our metadata v3 protobuf object. +type MetadataV2 struct { + md *pb.MetaDataV2 +} + +// WrappedMetadataV2 wrappers around the provided protobuf object. +func WrappedMetadataV2(md *pb.MetaDataV2) MetadataV2 { + return MetadataV2{md: md} +} + +// SequenceNumber returns the sequence number from the metadata. +func (m MetadataV2) SequenceNumber() uint64 { + return m.md.SeqNumber +} + +// AttnetsBitfield returns the bitfield stored in the metadata. +func (m MetadataV2) AttnetsBitfield() bitfield.Bitvector64 { + return m.md.Attnets +} + +// SyncnetsBitfield returns the bitfield stored in the metadata. +func (m MetadataV2) SyncnetsBitfield() bitfield.Bitvector4 { + return m.md.Syncnets +} + +// CustodySubnetCount returns custody subnet count from the metadata. +func (m MetadataV2) CustodySubnetCount() uint64 { + return m.md.CustodySubnetCount +} + +// InnerObject returns the underlying metadata protobuf structure. +func (m MetadataV2) InnerObject() interface{} { + return m.md +} + +// IsNil checks for the nilness of the underlying object. +func (m MetadataV2) IsNil() bool { + return m.md == nil +} + +// Copy performs a full copy of the underlying metadata object. +func (m MetadataV2) Copy() metadata.Metadata { + return WrappedMetadataV2(proto.Clone(m.md).(*pb.MetaDataV2)) +} + +// MarshalSSZ marshals the underlying metadata object +// into its serialized form. +func (m MetadataV2) MarshalSSZ() ([]byte, error) { + return m.md.MarshalSSZ() +} + +// MarshalSSZTo marshals the underlying metadata object +// into its serialized form into the provided byte buffer. +func (m MetadataV2) MarshalSSZTo(dst []byte) ([]byte, error) { + return m.md.MarshalSSZTo(dst) +} + +// SizeSSZ returns the serialized size of the metadata object. +func (m MetadataV2) SizeSSZ() int { + return m.md.SizeSSZ() +} + +// UnmarshalSSZ unmarshals the provided byte buffer into +// the underlying metadata object. +func (m MetadataV2) UnmarshalSSZ(buf []byte) error { + return m.md.UnmarshalSSZ(buf) +} + +// MetadataObjV0 returns the inner metadata object in its type +// specified form. If it doesn't exist then we return nothing. +func (MetadataV2) MetadataObjV0() *pb.MetaDataV0 { + return nil +} + +// MetadataObjV1 returns the inner metadata object in its type +// specified form. If it doesn't exist then we return nothing. +func (m MetadataV2) MetadataObjV1() *pb.MetaDataV1 { + return nil +} + +// MetadataObjV2 returns the inner metadata object in its type +// specified form. If it doesn't exist then we return nothing. +func (m MetadataV2) MetadataObjV2() *pb.MetaDataV2 { + return m.md +} + +// Version returns the fork version of the underlying object. +func (MetadataV2) Version() int { + return version.Deneb +} diff --git a/proto/prysm/v1alpha1/BUILD.bazel b/proto/prysm/v1alpha1/BUILD.bazel index 412a4eab1f46..56732b5e3b98 100644 --- a/proto/prysm/v1alpha1/BUILD.bazel +++ b/proto/prysm/v1alpha1/BUILD.bazel @@ -249,6 +249,7 @@ ssz_gen_marshal( "DataColumnSidecarsByRangeRequest", "MetaDataV0", "MetaDataV1", + "MetaDataV2", "SignedValidatorRegistrationV1", "ValidatorRegistrationV1", "BuilderBid", diff --git a/proto/prysm/v1alpha1/debug.pb.go b/proto/prysm/v1alpha1/debug.pb.go index 1ae0501d8175..273bb9b3b8da 100755 --- a/proto/prysm/v1alpha1/debug.pb.go +++ b/proto/prysm/v1alpha1/debug.pb.go @@ -738,6 +738,7 @@ type DebugPeerResponse_PeerInfo struct { ProtocolVersion string `protobuf:"bytes,5,opt,name=protocol_version,json=protocolVersion,proto3" json:"protocol_version,omitempty"` AgentVersion string `protobuf:"bytes,6,opt,name=agent_version,json=agentVersion,proto3" json:"agent_version,omitempty"` PeerLatency uint64 `protobuf:"varint,7,opt,name=peer_latency,json=peerLatency,proto3" json:"peer_latency,omitempty"` + MetadataV2 *MetaDataV2 `protobuf:"bytes,8,opt,name=metadataV2,proto3" json:"metadataV2,omitempty"` } func (x *DebugPeerResponse_PeerInfo) Reset() { @@ -821,6 +822,13 @@ func (x *DebugPeerResponse_PeerInfo) GetPeerLatency() uint64 { return 0 } +func (x *DebugPeerResponse_PeerInfo) GetMetadataV2() *MetaDataV2 { + if x != nil { + return x.MetadataV2 + } + return nil +} + var File_proto_prysm_v1alpha1_debug_proto protoreflect.FileDescriptor var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{ @@ -883,7 +891,7 @@ var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{ 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x09, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x22, 0xbf, 0x06, + 0x73, 0x65, 0x52, 0x09, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x22, 0x82, 0x07, 0x0a, 0x11, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x13, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, @@ -915,7 +923,7 @@ var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{ 0x72, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, - 0x09, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0xc2, 0x02, 0x0a, 0x08, 0x50, + 0x09, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x85, 0x03, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x41, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, 0x30, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, @@ -935,100 +943,104 @@ var file_proto_prysm_v1alpha1_debug_proto_rawDesc = []byte{ 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x6c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0b, 0x70, 0x65, 0x65, 0x72, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x22, - 0xc9, 0x03, 0x0a, 0x09, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, - 0x0d, 0x6f, 0x76, 0x65, 0x72, 0x61, 0x6c, 0x6c, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x02, 0x52, 0x0c, 0x6f, 0x76, 0x65, 0x72, 0x61, 0x6c, 0x6c, 0x53, 0x63, 0x6f, - 0x72, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x5f, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x70, 0x72, - 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x12, 0x30, 0x0a, - 0x14, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, - 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x12, 0x62, 0x6c, 0x6f, - 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, - 0x54, 0x0a, 0x0c, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, - 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x63, - 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, - 0x72, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x53, - 0x63, 0x6f, 0x72, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x5f, - 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0b, 0x67, 0x6f, 0x73, - 0x73, 0x69, 0x70, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x62, 0x65, 0x68, 0x61, - 0x76, 0x69, 0x6f, 0x75, 0x72, 0x5f, 0x70, 0x65, 0x6e, 0x61, 0x6c, 0x74, 0x79, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x02, 0x52, 0x10, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x75, 0x72, 0x50, 0x65, - 0x6e, 0x61, 0x6c, 0x74, 0x79, 0x12, 0x29, 0x0a, 0x10, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x1a, 0x69, 0x0a, 0x10, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, - 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe6, 0x01, 0x0a, 0x12, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, - 0x6f, 0x74, 0x12, 0x20, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x69, 0x6e, 0x5f, 0x6d, 0x65, - 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x49, 0x6e, - 0x4d, 0x65, 0x73, 0x68, 0x12, 0x38, 0x0a, 0x18, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x36, - 0x0a, 0x17, 0x6d, 0x65, 0x73, 0x68, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, - 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, - 0x15, 0x6d, 0x65, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, - 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x1a, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, - 0x64, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, - 0x72, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x02, 0x52, 0x18, 0x69, 0x6e, 0x76, 0x61, - 0x6c, 0x69, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, - 0x72, 0x69, 0x65, 0x73, 0x32, 0xf4, 0x04, 0x0a, 0x05, 0x44, 0x65, 0x62, 0x75, 0x67, 0x12, 0x82, - 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, - 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x65, - 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x28, 0x04, 0x52, 0x0b, 0x70, 0x65, 0x65, 0x72, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79, 0x12, + 0x41, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, 0x32, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, + 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x61, + 0x44, 0x61, 0x74, 0x61, 0x56, 0x32, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x56, 0x32, 0x22, 0xc9, 0x03, 0x0a, 0x09, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x76, 0x65, 0x72, 0x61, 0x6c, 0x6c, 0x5f, 0x73, 0x63, 0x6f, 0x72, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0c, 0x6f, 0x76, 0x65, 0x72, 0x61, 0x6c, 0x6c, + 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x0f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x73, + 0x12, 0x30, 0x0a, 0x14, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, + 0x65, 0x72, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x12, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x53, 0x63, 0x6f, + 0x72, 0x65, 0x12, 0x54, 0x0a, 0x0c, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x73, 0x63, 0x6f, 0x72, + 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, + 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, + 0x2e, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x67, 0x6f, 0x73, 0x73, + 0x69, 0x70, 0x5f, 0x73, 0x63, 0x6f, 0x72, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0b, + 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x62, + 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x75, 0x72, 0x5f, 0x70, 0x65, 0x6e, 0x61, 0x6c, 0x74, 0x79, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, 0x10, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x75, + 0x72, 0x50, 0x65, 0x6e, 0x61, 0x6c, 0x74, 0x79, 0x12, 0x29, 0x0a, 0x10, 0x76, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x1a, 0x69, 0x0a, 0x10, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, + 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3f, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, + 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, + 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe6, + 0x01, 0x0a, 0x12, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x53, 0x63, 0x6f, 0x72, 0x65, 0x53, 0x6e, 0x61, + 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x20, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x69, 0x6e, + 0x5f, 0x6d, 0x65, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x74, 0x69, 0x6d, + 0x65, 0x49, 0x6e, 0x4d, 0x65, 0x73, 0x68, 0x12, 0x38, 0x0a, 0x18, 0x66, 0x69, 0x72, 0x73, 0x74, + 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, + 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x02, 0x52, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, + 0x73, 0x12, 0x36, 0x0a, 0x17, 0x6d, 0x65, 0x73, 0x68, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x02, 0x52, 0x15, 0x6d, 0x65, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, + 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x1a, 0x69, 0x6e, 0x76, + 0x61, 0x6c, 0x69, 0x64, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x65, 0x6c, + 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x02, 0x52, 0x18, 0x69, + 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x44, 0x65, 0x6c, + 0x69, 0x76, 0x65, 0x72, 0x69, 0x65, 0x73, 0x32, 0xf4, 0x04, 0x0a, 0x05, 0x44, 0x65, 0x62, 0x75, + 0x67, 0x12, 0x82, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, + 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x65, 0x61, + 0x63, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, + 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x21, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, + 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, + 0x2f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x7c, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x12, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, + 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x79, 0x52, 0x6f, 0x6f, 0x74, 0x1a, 0x22, 0x2e, + 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, + 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x21, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, + 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x7a, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x67, 0x69, + 0x6e, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, + 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, + 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x23, 0x82, 0xd3, 0xe4, + 0x93, 0x02, 0x1d, 0x22, 0x1b, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, + 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, + 0x12, 0x71, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x16, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, + 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, + 0x62, 0x75, 0x67, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x22, 0x21, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, - 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x12, 0x7c, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, - 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, - 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x42, 0x79, 0x52, 0x6f, 0x6f, 0x74, 0x1a, 0x22, 0x2e, 0x65, 0x74, 0x68, - 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, - 0x61, 0x31, 0x2e, 0x53, 0x53, 0x5a, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x21, - 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, - 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x12, 0x7a, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x4c, - 0x65, 0x76, 0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, - 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x4c, 0x6f, 0x67, - 0x67, 0x69, 0x6e, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x23, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1d, - 0x22, 0x1b, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, - 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x71, 0x0a, - 0x09, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x65, 0x65, 0x72, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x1a, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, + 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, + 0x65, 0x72, 0x73, 0x12, 0x79, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x22, + 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, + 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, - 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x22, 0x21, 0x82, - 0xd3, 0xe4, 0x93, 0x02, 0x1b, 0x12, 0x19, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, - 0x70, 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x12, 0x79, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x50, 0x65, 0x65, 0x72, 0x12, 0x22, 0x2e, 0x65, 0x74, - 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x61, 0x6c, 0x70, - 0x68, 0x61, 0x31, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x28, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, - 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, 0x02, - 0x1a, 0x12, 0x18, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, - 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x42, 0x95, 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, 0x0a, 0x44, 0x65, 0x62, 0x75, 0x67, - 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, + 0x50, 0x65, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x20, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x1a, 0x12, 0x18, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x2f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x42, 0x95, + 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, 0x0a, 0x44, 0x65, + 0x62, 0x75, 0x67, 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 ( @@ -1064,8 +1076,9 @@ var file_proto_prysm_v1alpha1_debug_proto_goTypes = []interface{}{ (*Status)(nil), // 15: ethereum.eth.v1alpha1.Status (*MetaDataV0)(nil), // 16: ethereum.eth.v1alpha1.MetaDataV0 (*MetaDataV1)(nil), // 17: ethereum.eth.v1alpha1.MetaDataV1 - (*emptypb.Empty)(nil), // 18: google.protobuf.Empty - (*PeerRequest)(nil), // 19: ethereum.eth.v1alpha1.PeerRequest + (*MetaDataV2)(nil), // 18: ethereum.eth.v1alpha1.MetaDataV2 + (*emptypb.Empty)(nil), // 19: google.protobuf.Empty + (*PeerRequest)(nil), // 20: ethereum.eth.v1alpha1.PeerRequest } var file_proto_prysm_v1alpha1_debug_proto_depIdxs = []int32{ 0, // 0: ethereum.eth.v1alpha1.LoggingLevelRequest.level:type_name -> ethereum.eth.v1alpha1.LoggingLevelRequest.Level @@ -1078,22 +1091,23 @@ var file_proto_prysm_v1alpha1_debug_proto_depIdxs = []int32{ 12, // 7: ethereum.eth.v1alpha1.ScoreInfo.topic_scores:type_name -> ethereum.eth.v1alpha1.ScoreInfo.TopicScoresEntry 16, // 8: ethereum.eth.v1alpha1.DebugPeerResponse.PeerInfo.metadataV0:type_name -> ethereum.eth.v1alpha1.MetaDataV0 17, // 9: ethereum.eth.v1alpha1.DebugPeerResponse.PeerInfo.metadataV1:type_name -> ethereum.eth.v1alpha1.MetaDataV1 - 10, // 10: ethereum.eth.v1alpha1.ScoreInfo.TopicScoresEntry.value:type_name -> ethereum.eth.v1alpha1.TopicScoreSnapshot - 3, // 11: ethereum.eth.v1alpha1.Debug.GetBeaconState:input_type -> ethereum.eth.v1alpha1.BeaconStateRequest - 4, // 12: ethereum.eth.v1alpha1.Debug.GetBlock:input_type -> ethereum.eth.v1alpha1.BlockRequestByRoot - 6, // 13: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:input_type -> ethereum.eth.v1alpha1.LoggingLevelRequest - 18, // 14: ethereum.eth.v1alpha1.Debug.ListPeers:input_type -> google.protobuf.Empty - 19, // 15: ethereum.eth.v1alpha1.Debug.GetPeer:input_type -> ethereum.eth.v1alpha1.PeerRequest - 5, // 16: ethereum.eth.v1alpha1.Debug.GetBeaconState:output_type -> ethereum.eth.v1alpha1.SSZResponse - 5, // 17: ethereum.eth.v1alpha1.Debug.GetBlock:output_type -> ethereum.eth.v1alpha1.SSZResponse - 18, // 18: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:output_type -> google.protobuf.Empty - 7, // 19: ethereum.eth.v1alpha1.Debug.ListPeers:output_type -> ethereum.eth.v1alpha1.DebugPeerResponses - 8, // 20: ethereum.eth.v1alpha1.Debug.GetPeer:output_type -> ethereum.eth.v1alpha1.DebugPeerResponse - 16, // [16:21] is the sub-list for method output_type - 11, // [11:16] is the sub-list for method input_type - 11, // [11:11] is the sub-list for extension type_name - 11, // [11:11] is the sub-list for extension extendee - 0, // [0:11] is the sub-list for field type_name + 18, // 10: ethereum.eth.v1alpha1.DebugPeerResponse.PeerInfo.metadataV2:type_name -> ethereum.eth.v1alpha1.MetaDataV2 + 10, // 11: ethereum.eth.v1alpha1.ScoreInfo.TopicScoresEntry.value:type_name -> ethereum.eth.v1alpha1.TopicScoreSnapshot + 3, // 12: ethereum.eth.v1alpha1.Debug.GetBeaconState:input_type -> ethereum.eth.v1alpha1.BeaconStateRequest + 4, // 13: ethereum.eth.v1alpha1.Debug.GetBlock:input_type -> ethereum.eth.v1alpha1.BlockRequestByRoot + 6, // 14: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:input_type -> ethereum.eth.v1alpha1.LoggingLevelRequest + 19, // 15: ethereum.eth.v1alpha1.Debug.ListPeers:input_type -> google.protobuf.Empty + 20, // 16: ethereum.eth.v1alpha1.Debug.GetPeer:input_type -> ethereum.eth.v1alpha1.PeerRequest + 5, // 17: ethereum.eth.v1alpha1.Debug.GetBeaconState:output_type -> ethereum.eth.v1alpha1.SSZResponse + 5, // 18: ethereum.eth.v1alpha1.Debug.GetBlock:output_type -> ethereum.eth.v1alpha1.SSZResponse + 19, // 19: ethereum.eth.v1alpha1.Debug.SetLoggingLevel:output_type -> google.protobuf.Empty + 7, // 20: ethereum.eth.v1alpha1.Debug.ListPeers:output_type -> ethereum.eth.v1alpha1.DebugPeerResponses + 8, // 21: ethereum.eth.v1alpha1.Debug.GetPeer:output_type -> ethereum.eth.v1alpha1.DebugPeerResponse + 17, // [17:22] is the sub-list for method output_type + 12, // [12:17] is the sub-list for method input_type + 12, // [12:12] is the sub-list for extension type_name + 12, // [12:12] is the sub-list for extension extendee + 0, // [0:12] is the sub-list for field type_name } func init() { file_proto_prysm_v1alpha1_debug_proto_init() } diff --git a/proto/prysm/v1alpha1/debug.proto b/proto/prysm/v1alpha1/debug.proto index 172d6f6a9793..23503ad4b4f5 100644 --- a/proto/prysm/v1alpha1/debug.proto +++ b/proto/prysm/v1alpha1/debug.proto @@ -98,8 +98,7 @@ message DebugPeerResponses { message DebugPeerResponse { // Peer related metadata that is useful for debugging. message PeerInfo { - // Metadata of the peer, containing their bitfield - // and sequence number. + // Metadata of the peer. MetaDataV0 metadataV0 = 1; MetaDataV1 metadataV1 = 2; // List of protocols the peer supports. @@ -112,6 +111,8 @@ message DebugPeerResponse { string agent_version = 6; // Latency of responses from peer(in ms). uint64 peer_latency = 7; + // Metadata of the peer. + MetaDataV2 metadataV2 = 8; } // Listening addresses know of the peer. repeated string listening_addresses = 1; diff --git a/proto/prysm/v1alpha1/metadata/metadata_interfaces.go b/proto/prysm/v1alpha1/metadata/metadata_interfaces.go index 1135e32c6a00..b57a8753ceb7 100644 --- a/proto/prysm/v1alpha1/metadata/metadata_interfaces.go +++ b/proto/prysm/v1alpha1/metadata/metadata_interfaces.go @@ -10,6 +10,8 @@ import ( type Metadata interface { SequenceNumber() uint64 AttnetsBitfield() bitfield.Bitvector64 + SyncnetsBitfield() bitfield.Bitvector4 + CustodySubnetCount() uint64 InnerObject() interface{} IsNil() bool Copy() Metadata @@ -17,5 +19,6 @@ type Metadata interface { ssz.Unmarshaler MetadataObjV0() *pb.MetaDataV0 MetadataObjV1() *pb.MetaDataV1 + MetadataObjV2() *pb.MetaDataV2 Version() int } diff --git a/proto/prysm/v1alpha1/non-core.ssz.go b/proto/prysm/v1alpha1/non-core.ssz.go index 374bc152e880..9d949949848c 100644 --- a/proto/prysm/v1alpha1/non-core.ssz.go +++ b/proto/prysm/v1alpha1/non-core.ssz.go @@ -1,5 +1,5 @@ // Code generated by fastssz. DO NOT EDIT. -// Hash: 2d03f202cb6eeae27637655515f76ce68af9b5021ea332e37cd10e7f15c25530 +// Hash: ac6fbb6f912d4e5a3374b91a3dc73c1d5ac1e5b4a9c32513b3d5ef9de7885be8 package eth import ( @@ -551,6 +551,106 @@ func (m *MetaDataV1) HashTreeRootWith(hh *ssz.Hasher) (err error) { return } +// MarshalSSZ ssz marshals the MetaDataV2 object +func (m *MetaDataV2) MarshalSSZ() ([]byte, error) { + return ssz.MarshalSSZ(m) +} + +// MarshalSSZTo ssz marshals the MetaDataV2 object to a target array +func (m *MetaDataV2) MarshalSSZTo(buf []byte) (dst []byte, err error) { + dst = buf + + // Field (0) 'SeqNumber' + dst = ssz.MarshalUint64(dst, m.SeqNumber) + + // Field (1) 'Attnets' + if size := len(m.Attnets); size != 8 { + err = ssz.ErrBytesLengthFn("--.Attnets", size, 8) + return + } + dst = append(dst, m.Attnets...) + + // Field (2) 'Syncnets' + if size := len(m.Syncnets); size != 1 { + err = ssz.ErrBytesLengthFn("--.Syncnets", size, 1) + return + } + dst = append(dst, m.Syncnets...) + + // Field (3) 'CustodySubnetCount' + dst = ssz.MarshalUint64(dst, m.CustodySubnetCount) + + return +} + +// UnmarshalSSZ ssz unmarshals the MetaDataV2 object +func (m *MetaDataV2) UnmarshalSSZ(buf []byte) error { + var err error + size := uint64(len(buf)) + if size != 25 { + return ssz.ErrSize + } + + // Field (0) 'SeqNumber' + m.SeqNumber = ssz.UnmarshallUint64(buf[0:8]) + + // Field (1) 'Attnets' + if cap(m.Attnets) == 0 { + m.Attnets = make([]byte, 0, len(buf[8:16])) + } + m.Attnets = append(m.Attnets, buf[8:16]...) + + // Field (2) 'Syncnets' + if cap(m.Syncnets) == 0 { + m.Syncnets = make([]byte, 0, len(buf[16:17])) + } + m.Syncnets = append(m.Syncnets, buf[16:17]...) + + // Field (3) 'CustodySubnetCount' + 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 = 25 + return +} + +// HashTreeRoot ssz hashes the MetaDataV2 object +func (m *MetaDataV2) HashTreeRoot() ([32]byte, error) { + return ssz.HashWithDefaultHasher(m) +} + +// HashTreeRootWith ssz hashes the MetaDataV2 object with a hasher +func (m *MetaDataV2) HashTreeRootWith(hh *ssz.Hasher) (err error) { + indx := hh.Index() + + // Field (0) 'SeqNumber' + hh.PutUint64(m.SeqNumber) + + // Field (1) 'Attnets' + if size := len(m.Attnets); size != 8 { + err = ssz.ErrBytesLengthFn("--.Attnets", size, 8) + return + } + hh.PutBytes(m.Attnets) + + // Field (2) 'Syncnets' + if size := len(m.Syncnets); size != 1 { + err = ssz.ErrBytesLengthFn("--.Syncnets", size, 1) + return + } + hh.PutBytes(m.Syncnets) + + // Field (3) 'CustodySubnetCount' + hh.PutUint64(m.CustodySubnetCount) + + hh.Merkleize(indx) + return +} + // MarshalSSZ ssz marshals the BlobSidecarsByRangeRequest object func (b *BlobSidecarsByRangeRequest) MarshalSSZ() ([]byte, error) { return ssz.MarshalSSZ(b) diff --git a/proto/prysm/v1alpha1/p2p_messages.pb.go b/proto/prysm/v1alpha1/p2p_messages.pb.go index b06a459ef7f0..796dc5d3c038 100755 --- a/proto/prysm/v1alpha1/p2p_messages.pb.go +++ b/proto/prysm/v1alpha1/p2p_messages.pb.go @@ -348,6 +348,77 @@ func (x *MetaDataV1) GetSyncnets() github_com_prysmaticlabs_go_bitfield.Bitvecto return github_com_prysmaticlabs_go_bitfield.Bitvector4(nil) } +type MetaDataV2 struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + 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 uint64 `protobuf:"varint,4,opt,name=custody_subnet_count,json=custodySubnetCount,proto3" json:"custody_subnet_count,omitempty"` +} + +func (x *MetaDataV2) Reset() { + *x = MetaDataV2{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MetaDataV2) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MetaDataV2) ProtoMessage() {} + +func (x *MetaDataV2) ProtoReflect() protoreflect.Message { + mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MetaDataV2.ProtoReflect.Descriptor instead. +func (*MetaDataV2) Descriptor() ([]byte, []int) { + return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{5} +} + +func (x *MetaDataV2) GetSeqNumber() uint64 { + if x != nil { + return x.SeqNumber + } + return 0 +} + +func (x *MetaDataV2) GetAttnets() github_com_prysmaticlabs_go_bitfield.Bitvector64 { + if x != nil { + return x.Attnets + } + return github_com_prysmaticlabs_go_bitfield.Bitvector64(nil) +} + +func (x *MetaDataV2) GetSyncnets() github_com_prysmaticlabs_go_bitfield.Bitvector4 { + if x != nil { + return x.Syncnets + } + return github_com_prysmaticlabs_go_bitfield.Bitvector4(nil) +} + +func (x *MetaDataV2) GetCustodySubnetCount() uint64 { + if x != nil { + return x.CustodySubnetCount + } + return 0 +} + type BlobSidecarsByRangeRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -360,7 +431,7 @@ type BlobSidecarsByRangeRequest struct { func (x *BlobSidecarsByRangeRequest) Reset() { *x = BlobSidecarsByRangeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5] + mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -373,7 +444,7 @@ func (x *BlobSidecarsByRangeRequest) String() string { func (*BlobSidecarsByRangeRequest) ProtoMessage() {} func (x *BlobSidecarsByRangeRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5] + mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -386,7 +457,7 @@ func (x *BlobSidecarsByRangeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BlobSidecarsByRangeRequest.ProtoReflect.Descriptor instead. func (*BlobSidecarsByRangeRequest) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{5} + return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{6} } func (x *BlobSidecarsByRangeRequest) GetStartSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { @@ -416,7 +487,7 @@ type DataColumnSidecarsByRangeRequest struct { func (x *DataColumnSidecarsByRangeRequest) Reset() { *x = DataColumnSidecarsByRangeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6] + mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -429,7 +500,7 @@ func (x *DataColumnSidecarsByRangeRequest) String() string { func (*DataColumnSidecarsByRangeRequest) ProtoMessage() {} func (x *DataColumnSidecarsByRangeRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6] + mi := &file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -442,7 +513,7 @@ func (x *DataColumnSidecarsByRangeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DataColumnSidecarsByRangeRequest.ProtoReflect.Descriptor instead. func (*DataColumnSidecarsByRangeRequest) Descriptor() ([]byte, []int) { - return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{6} + return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP(), []int{7} } func (x *DataColumnSidecarsByRangeRequest) GetStartSlot() github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Slot { @@ -545,40 +616,56 @@ 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, 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, 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, + 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, + 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x39, 0x82, + 0xb5, 0x18, 0x30, 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, 0x36, 0x34, 0x8a, 0xb5, 0x18, 0x01, 0x38, 0x52, 0x07, 0x61, 0x74, 0x74, 0x6e, 0x65, 0x74, + 0x73, 0x12, 0x54, 0x0a, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x6e, 0x65, 0x74, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0c, 0x42, 0x38, 0x82, 0xb5, 0x18, 0x2f, 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, 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, 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, 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 ( @@ -593,15 +680,16 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescGZIP() []byte { return file_proto_prysm_v1alpha1_p2p_messages_proto_rawDescData } -var file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 7) +var file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 8) var file_proto_prysm_v1alpha1_p2p_messages_proto_goTypes = []interface{}{ (*Status)(nil), // 0: ethereum.eth.v1alpha1.Status (*BeaconBlocksByRangeRequest)(nil), // 1: ethereum.eth.v1alpha1.BeaconBlocksByRangeRequest (*ENRForkID)(nil), // 2: ethereum.eth.v1alpha1.ENRForkID (*MetaDataV0)(nil), // 3: ethereum.eth.v1alpha1.MetaDataV0 (*MetaDataV1)(nil), // 4: ethereum.eth.v1alpha1.MetaDataV1 - (*BlobSidecarsByRangeRequest)(nil), // 5: ethereum.eth.v1alpha1.BlobSidecarsByRangeRequest - (*DataColumnSidecarsByRangeRequest)(nil), // 6: ethereum.eth.v1alpha1.DataColumnSidecarsByRangeRequest + (*MetaDataV2)(nil), // 5: ethereum.eth.v1alpha1.MetaDataV2 + (*BlobSidecarsByRangeRequest)(nil), // 6: ethereum.eth.v1alpha1.BlobSidecarsByRangeRequest + (*DataColumnSidecarsByRangeRequest)(nil), // 7: ethereum.eth.v1alpha1.DataColumnSidecarsByRangeRequest } var file_proto_prysm_v1alpha1_p2p_messages_proto_depIdxs = []int32{ 0, // [0:0] is the sub-list for method output_type @@ -678,7 +766,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() { } } file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BlobSidecarsByRangeRequest); i { + switch v := v.(*MetaDataV2); i { case 0: return &v.state case 1: @@ -690,6 +778,18 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() { } } file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BlobSidecarsByRangeRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_prysm_v1alpha1_p2p_messages_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DataColumnSidecarsByRangeRequest); i { case 0: return &v.state @@ -708,7 +808,7 @@ func file_proto_prysm_v1alpha1_p2p_messages_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_prysm_v1alpha1_p2p_messages_proto_rawDesc, NumEnums: 0, - NumMessages: 7, + NumMessages: 8, NumExtensions: 0, NumServices: 0, }, diff --git a/proto/prysm/v1alpha1/p2p_messages.proto b/proto/prysm/v1alpha1/p2p_messages.proto index 5113701785e3..0ea6a4772760 100644 --- a/proto/prysm/v1alpha1/p2p_messages.proto +++ b/proto/prysm/v1alpha1/p2p_messages.proto @@ -51,6 +51,7 @@ message MetaDataV0 { ( seq_number: uint64 attnets: Bitvector[ATTESTATION_SUBNET_COUNT] + syncnets: Bitvector[SYNC_COMMITTEE_SUBNET_COUNT] ) */ message MetaDataV1 { @@ -59,6 +60,23 @@ message MetaDataV1 { bytes syncnets = 3 [(ethereum.eth.ext.ssz_size) = "1", (ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/go-bitfield.Bitvector4"]; } +/* + Spec Definition: + MetaData + ( + seq_number: uint64 + attnets: Bitvector[ATTESTATION_SUBNET_COUNT] + syncnets: Bitvector[SYNC_COMMITTEE_SUBNET_COUNT] + custody_subnet_count: uint64 + ) +*/ +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"]; + uint64 custody_subnet_count = 4; +} + /* Spec Definition: (