mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-08 21:08:10 -05:00
**What type of PR is this?** Feature What does this PR do? Why is it needed? This PR takes @MarcoPolo 's PR at https://github.com/OffchainLabs/prysm/pull/16130 to completion with tests. The description on his PR: """ a relatively small change to optimize network send order. Without this, network writes tend to prioritize sending data for one column to all peers before sending data for later columns (e.g for two columns and 4 peers per column it would send A,A,A,A,B,B,B,B). With batch publishing we can change the write order to round robin across columns (e.g. A,B,A,B,A,B,A,B). In cases where the process is sending at a rate over the network limit, this approach allows at least some copies of the column to propagate through the network. In early simulations with bandwidth limits of 50mbps for the publisher, this improved dissemination by ~20-30%. """ See the issue for some more context. **Which issues(s) does this PR fix?** Fixes https://github.com/OffchainLabs/prysm/issues/16129 Other notes for review Acknowledgements - [x] I have read [CONTRIBUTING.md](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md). - [ ] I have included a uniquely named [changelog fragment file](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md#maintaining-changelogmd). - [x] I have added a description with sufficient context for reviewers to understand this PR. - [ ] I have tested that my changes work as expected and I added a testing plan to the PR description (if applicable). --------- Co-authored-by: Marco Munizaga <git@marcopolo.io> Co-authored-by: Kasey Kirkham <kasey@users.noreply.github.com> Co-authored-by: kasey <489222+kasey@users.noreply.github.com> Co-authored-by: Preston Van Loon <pvanloon@offchainlabs.com> Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>
979 lines
31 KiB
Go
979 lines
31 KiB
Go
package p2p
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"net"
|
|
"reflect"
|
|
"sync"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/OffchainLabs/go-bitfield"
|
|
"github.com/OffchainLabs/prysm/v7/beacon-chain/blockchain/kzg"
|
|
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
|
|
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
|
|
testDB "github.com/OffchainLabs/prysm/v7/beacon-chain/db/testing"
|
|
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/peers"
|
|
"github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/peers/scorers"
|
|
p2ptest "github.com/OffchainLabs/prysm/v7/beacon-chain/p2p/testing"
|
|
"github.com/OffchainLabs/prysm/v7/cmd/beacon-chain/flags"
|
|
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
|
|
"github.com/OffchainLabs/prysm/v7/config/params"
|
|
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
|
|
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
|
|
"github.com/OffchainLabs/prysm/v7/consensus-types/wrapper"
|
|
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
|
|
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
|
|
testpb "github.com/OffchainLabs/prysm/v7/proto/testing"
|
|
"github.com/OffchainLabs/prysm/v7/testing/assert"
|
|
"github.com/OffchainLabs/prysm/v7/testing/require"
|
|
"github.com/OffchainLabs/prysm/v7/testing/util"
|
|
"github.com/OffchainLabs/prysm/v7/time/slots"
|
|
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
|
"github.com/libp2p/go-libp2p/core/host"
|
|
"github.com/libp2p/go-libp2p/core/peer"
|
|
"github.com/libp2p/go-libp2p/core/protocol"
|
|
"google.golang.org/protobuf/proto"
|
|
)
|
|
|
|
func TestService_Broadcast(t *testing.T) {
|
|
p1 := p2ptest.NewTestP2P(t)
|
|
p2 := p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
if len(p1.BHost.Network().Peers()) == 0 {
|
|
t.Fatal("No peers")
|
|
}
|
|
|
|
p := &Service{
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{},
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
}
|
|
|
|
msg := ðpb.Fork{
|
|
Epoch: 55,
|
|
CurrentVersion: []byte("fooo"),
|
|
PreviousVersion: []byte("barr"),
|
|
}
|
|
|
|
topic := "/eth2/%x/testing"
|
|
// Set a test gossip mapping for testpb.TestSimpleMessage.
|
|
GossipTypeMapping[reflect.TypeFor[*ethpb.Fork]()] = topic
|
|
digest, err := p.currentForkDigest()
|
|
require.NoError(t, err)
|
|
topic = fmt.Sprintf(topic, digest)
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(50 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 1*time.Second)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
result := ðpb.Fork{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
if !proto.Equal(result, msg) {
|
|
tt.Errorf("Did not receive expected message, got %+v, wanted %+v", result, msg)
|
|
}
|
|
}(t)
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.Broadcast(t.Context(), msg))
|
|
if util.WaitTimeout(&wg, 1*time.Second) {
|
|
t.Error("Failed to receive pubsub within 1s")
|
|
}
|
|
}
|
|
|
|
func TestService_Broadcast_ReturnsErr_TopicNotMapped(t *testing.T) {
|
|
p := Service{
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
}
|
|
assert.ErrorContains(t, ErrMessageNotMapped.Error(), p.Broadcast(t.Context(), &testpb.AddressBook{}))
|
|
}
|
|
|
|
func TestService_Attestation_Subnet(t *testing.T) {
|
|
if gtm := GossipTypeMapping[reflect.TypeFor[*ethpb.Attestation]()]; gtm != AttestationSubnetTopicFormat {
|
|
t.Errorf("Constant is out of date. Wanted %s, got %s", AttestationSubnetTopicFormat, gtm)
|
|
}
|
|
|
|
tests := []struct {
|
|
att *ethpb.Attestation
|
|
topic string
|
|
}{
|
|
{
|
|
att: ðpb.Attestation{
|
|
Data: ðpb.AttestationData{
|
|
CommitteeIndex: 0,
|
|
Slot: 2,
|
|
},
|
|
},
|
|
topic: "/eth2/00000000/beacon_attestation_2",
|
|
},
|
|
{
|
|
att: ðpb.Attestation{
|
|
Data: ðpb.AttestationData{
|
|
CommitteeIndex: 11,
|
|
Slot: 10,
|
|
},
|
|
},
|
|
topic: "/eth2/00000000/beacon_attestation_21",
|
|
},
|
|
{
|
|
att: ðpb.Attestation{
|
|
Data: ðpb.AttestationData{
|
|
CommitteeIndex: 55,
|
|
Slot: 529,
|
|
},
|
|
},
|
|
topic: "/eth2/00000000/beacon_attestation_8",
|
|
},
|
|
}
|
|
for _, tt := range tests {
|
|
subnet := helpers.ComputeSubnetFromCommitteeAndSlot(100, tt.att.Data.CommitteeIndex, tt.att.Data.Slot)
|
|
assert.Equal(t, tt.topic, attestationToTopic(subnet, [4]byte{} /* fork digest */), "Wrong topic")
|
|
}
|
|
}
|
|
|
|
func TestService_BroadcastAttestation(t *testing.T) {
|
|
p1 := p2ptest.NewTestP2P(t)
|
|
p2 := p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
if len(p1.BHost.Network().Peers()) == 0 {
|
|
t.Fatal("No peers")
|
|
}
|
|
|
|
p := &Service{
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{},
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
|
|
msg := util.HydrateAttestation(ðpb.Attestation{AggregationBits: bitfield.NewBitlist(7)})
|
|
subnet := uint64(5)
|
|
|
|
topic := AttestationSubnetTopicFormat
|
|
GossipTypeMapping[reflect.TypeFor[*ethpb.Attestation]()] = topic
|
|
digest, err := p.currentForkDigest()
|
|
require.NoError(t, err)
|
|
topic = fmt.Sprintf(topic, digest, subnet)
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(50 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 1*time.Second)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
result := ðpb.Attestation{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
if !proto.Equal(result, msg) {
|
|
tt.Errorf("Did not receive expected message, got %+v, wanted %+v", result, msg)
|
|
}
|
|
}(t)
|
|
|
|
// Attempt to broadcast nil object should fail.
|
|
ctx := t.Context()
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastAttestation(ctx, subnet, nil))
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.BroadcastAttestation(ctx, subnet, msg))
|
|
if util.WaitTimeout(&wg, 1*time.Second) {
|
|
t.Error("Failed to receive pubsub within 1s")
|
|
}
|
|
}
|
|
|
|
func TestService_BroadcastAttestationWithDiscoveryAttempts(t *testing.T) {
|
|
const port = uint(2000)
|
|
|
|
// The DB has to be shared in all peers to avoid the
|
|
// duplicate metrics collector registration attempted.
|
|
// However, we don't care for this test.
|
|
db := testDB.SetupDB(t)
|
|
|
|
// Setup bootnode.
|
|
cfg := &Config{PingInterval: testPingInterval, DB: db}
|
|
cfg.UDPPort = uint(port)
|
|
_, pkey := createAddrAndPrivKey(t)
|
|
ipAddr := net.ParseIP("127.0.0.1")
|
|
genesisTime := time.Now()
|
|
genesisValidatorsRoot := make([]byte, 32)
|
|
|
|
s := &Service{
|
|
cfg: cfg,
|
|
genesisTime: genesisTime,
|
|
genesisValidatorsRoot: genesisValidatorsRoot,
|
|
custodyInfo: &custodyInfo{},
|
|
ctx: t.Context(),
|
|
custodyInfoSet: make(chan struct{}),
|
|
}
|
|
|
|
close(s.custodyInfoSet)
|
|
|
|
bootListener, err := s.createListener(ipAddr, pkey)
|
|
require.NoError(t, err)
|
|
defer bootListener.Close()
|
|
|
|
bootNode := bootListener.Self()
|
|
subnet := uint64(5)
|
|
|
|
var listeners []*listenerWrapper
|
|
var hosts []host.Host
|
|
// setup other nodes.
|
|
cfg = &Config{
|
|
Discv5BootStrapAddrs: []string{bootNode.String()},
|
|
MaxPeers: 2,
|
|
PingInterval: testPingInterval,
|
|
DB: db,
|
|
}
|
|
// Setup 2 different hosts
|
|
for i := uint(1); i <= 2; i++ {
|
|
h, pkey, ipAddr := createHost(t, port+i)
|
|
cfg.UDPPort = uint(port + i)
|
|
cfg.TCPPort = uint(port + i)
|
|
if len(listeners) > 0 {
|
|
cfg.Discv5BootStrapAddrs = append(cfg.Discv5BootStrapAddrs, listeners[len(listeners)-1].Self().String())
|
|
}
|
|
s := &Service{
|
|
cfg: cfg,
|
|
genesisTime: genesisTime,
|
|
genesisValidatorsRoot: genesisValidatorsRoot,
|
|
custodyInfo: &custodyInfo{},
|
|
ctx: t.Context(),
|
|
custodyInfoSet: make(chan struct{}),
|
|
}
|
|
|
|
close(s.custodyInfoSet)
|
|
|
|
listener, err := s.startDiscoveryV5(ipAddr, pkey)
|
|
// Set for 2nd peer
|
|
if i == 2 {
|
|
s.dv5Listener = listener
|
|
s.metaData = wrapper.WrappedMetadataV0(new(ethpb.MetaDataV0))
|
|
bitV := bitfield.NewBitvector64()
|
|
bitV.SetBitAt(subnet, true)
|
|
err := s.updateSubnetRecordWithMetadata(bitV)
|
|
require.NoError(t, err)
|
|
}
|
|
assert.NoError(t, err, "Could not start discovery for node")
|
|
listeners = append(listeners, listener)
|
|
hosts = append(hosts, h)
|
|
}
|
|
defer func() {
|
|
// Close down all peers.
|
|
for _, listener := range listeners {
|
|
listener.Close()
|
|
}
|
|
}()
|
|
|
|
// close peers upon exit of test
|
|
defer func() {
|
|
for _, h := range hosts {
|
|
if err := h.Close(); err != nil {
|
|
t.Log(err)
|
|
}
|
|
}
|
|
}()
|
|
|
|
ps1, err := pubsub.NewGossipSub(t.Context(), hosts[0],
|
|
pubsub.WithMessageSigning(false),
|
|
pubsub.WithStrictSignatureVerification(false),
|
|
)
|
|
require.NoError(t, err)
|
|
|
|
ps2, err := pubsub.NewGossipSub(t.Context(), hosts[1],
|
|
pubsub.WithMessageSigning(false),
|
|
pubsub.WithStrictSignatureVerification(false),
|
|
)
|
|
require.NoError(t, err)
|
|
p := &Service{
|
|
host: hosts[0],
|
|
ctx: t.Context(),
|
|
pubsub: ps1,
|
|
dv5Listener: listeners[0],
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: cfg,
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
|
|
p2 := &Service{
|
|
host: hosts[1],
|
|
ctx: t.Context(),
|
|
pubsub: ps2,
|
|
dv5Listener: listeners[1],
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: cfg,
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
go p.listenForNewNodes()
|
|
go p2.listenForNewNodes()
|
|
|
|
msg := util.HydrateAttestation(ðpb.Attestation{AggregationBits: bitfield.NewBitlist(7)})
|
|
topic := AttestationSubnetTopicFormat
|
|
GossipTypeMapping[reflect.TypeFor[*ethpb.Attestation]()] = topic
|
|
digest, err := p.currentForkDigest()
|
|
require.NoError(t, err)
|
|
topic = fmt.Sprintf(topic, digest, subnet)
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
// We don't use our internal subscribe method
|
|
// due to using floodsub over here.
|
|
tpHandle, err := p2.JoinTopic(topic)
|
|
require.NoError(t, err)
|
|
sub, err := tpHandle.Subscribe()
|
|
require.NoError(t, err)
|
|
|
|
tpHandle, err = p.JoinTopic(topic)
|
|
require.NoError(t, err)
|
|
_, err = tpHandle.Subscribe()
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(500 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
nodePeers := p.pubsub.ListPeers(topic)
|
|
nodePeers2 := p2.pubsub.ListPeers(topic)
|
|
|
|
assert.Equal(t, 1, len(nodePeers))
|
|
assert.Equal(t, 1, len(nodePeers2))
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 4*time.Second)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
result := ðpb.Attestation{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
if !proto.Equal(result, msg) {
|
|
tt.Errorf("Did not receive expected message, got %+v, wanted %+v", result, msg)
|
|
}
|
|
}(t)
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.BroadcastAttestation(t.Context(), subnet, msg))
|
|
if util.WaitTimeout(&wg, 4*time.Second) {
|
|
t.Error("Failed to receive pubsub within 4s")
|
|
}
|
|
}
|
|
|
|
func TestService_BroadcastSyncCommittee(t *testing.T) {
|
|
p1 := p2ptest.NewTestP2P(t)
|
|
p2 := p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
if len(p1.BHost.Network().Peers()) == 0 {
|
|
t.Fatal("No peers")
|
|
}
|
|
|
|
p := &Service{
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{},
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
|
|
msg := util.HydrateSyncCommittee(ðpb.SyncCommitteeMessage{})
|
|
subnet := uint64(5)
|
|
|
|
topic := SyncCommitteeSubnetTopicFormat
|
|
GossipTypeMapping[reflect.TypeFor[*ethpb.SyncCommitteeMessage]()] = topic
|
|
digest, err := p.currentForkDigest()
|
|
require.NoError(t, err)
|
|
topic = fmt.Sprintf(topic, digest, subnet)
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(50 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 1*time.Second)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
result := ðpb.SyncCommitteeMessage{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
if !proto.Equal(result, msg) {
|
|
tt.Errorf("Did not receive expected message, got %+v, wanted %+v", result, msg)
|
|
}
|
|
}(t)
|
|
|
|
// Broadcasting nil should fail.
|
|
ctx := t.Context()
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastSyncCommitteeMessage(ctx, subnet, nil))
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.BroadcastSyncCommitteeMessage(ctx, subnet, msg))
|
|
if util.WaitTimeout(&wg, 1*time.Second) {
|
|
t.Error("Failed to receive pubsub within 1s")
|
|
}
|
|
}
|
|
|
|
func TestService_BroadcastBlob(t *testing.T) {
|
|
p1 := p2ptest.NewTestP2P(t)
|
|
p2 := p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
require.NotEqual(t, 0, len(p1.BHost.Network().Peers()), "No peers")
|
|
|
|
p := &Service{
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{},
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
|
|
header := util.HydrateSignedBeaconHeader(ðpb.SignedBeaconBlockHeader{})
|
|
commitmentInclusionProof := make([][]byte, 17)
|
|
for i := range commitmentInclusionProof {
|
|
commitmentInclusionProof[i] = bytesutil.PadTo([]byte{}, 32)
|
|
}
|
|
blobSidecar := ðpb.BlobSidecar{
|
|
Index: 1,
|
|
Blob: bytesutil.PadTo([]byte{'C'}, fieldparams.BlobLength),
|
|
KzgCommitment: bytesutil.PadTo([]byte{'D'}, fieldparams.BLSPubkeyLength),
|
|
KzgProof: bytesutil.PadTo([]byte{'E'}, fieldparams.BLSPubkeyLength),
|
|
SignedBlockHeader: header,
|
|
CommitmentInclusionProof: commitmentInclusionProof,
|
|
}
|
|
subnet := uint64(0)
|
|
|
|
topic := BlobSubnetTopicFormat
|
|
GossipTypeMapping[reflect.TypeFor[*ethpb.BlobSidecar]()] = topic
|
|
digest, err := p.currentForkDigest()
|
|
require.NoError(t, err)
|
|
topic = fmt.Sprintf(topic, digest, subnet)
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(50 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 1*time.Second)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
result := ðpb.BlobSidecar{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
require.DeepEqual(t, result, blobSidecar)
|
|
}(t)
|
|
|
|
// Attempt to broadcast nil object should fail.
|
|
ctx := t.Context()
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastBlob(ctx, subnet, nil))
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.BroadcastBlob(ctx, subnet, blobSidecar))
|
|
require.Equal(t, false, util.WaitTimeout(&wg, 1*time.Second), "Failed to receive pubsub within 1s")
|
|
}
|
|
|
|
func TestService_BroadcastLightClientOptimisticUpdate(t *testing.T) {
|
|
params.SetupTestConfigCleanup(t)
|
|
config := params.BeaconConfig().Copy()
|
|
config.SyncMessageDueBPS = 60 // ~72 millisecond
|
|
params.OverrideBeaconConfig(config)
|
|
|
|
p1 := p2ptest.NewTestP2P(t)
|
|
p2 := p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
require.NotEqual(t, 0, len(p1.BHost.Network().Peers()))
|
|
|
|
p := &Service{
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{},
|
|
genesisTime: time.Now().Add(-33 * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second), // the signature slot of the mock update is 33
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
|
|
msg, err := util.MockOptimisticUpdate()
|
|
require.NoError(t, err)
|
|
|
|
GossipTypeMapping[reflect.TypeOf(msg)] = LightClientOptimisticUpdateTopicFormat
|
|
topic := fmt.Sprintf(LightClientOptimisticUpdateTopicFormat, params.ForkDigest(slots.ToEpoch(msg.AttestedHeader().Beacon().Slot)))
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(50 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 150*time.Millisecond)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
slotStartTime, err := slots.StartTime(p.genesisTime, msg.SignatureSlot())
|
|
require.NoError(t, err)
|
|
expectedDelay := params.BeaconConfig().SlotComponentDuration(params.BeaconConfig().SyncMessageDueBPS)
|
|
if time.Now().Before(slotStartTime.Add(expectedDelay)) {
|
|
tt.Errorf("Message received too early, now %v, expected at least %v", time.Now(), slotStartTime.Add(expectedDelay))
|
|
}
|
|
|
|
result := ðpb.LightClientOptimisticUpdateAltair{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
if !proto.Equal(result, msg.Proto()) {
|
|
tt.Errorf("Did not receive expected message, got %+v, wanted %+v", result, msg)
|
|
}
|
|
}(t)
|
|
|
|
// Broadcasting nil should fail.
|
|
ctx := t.Context()
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastLightClientOptimisticUpdate(ctx, nil))
|
|
var nilUpdate interfaces.LightClientOptimisticUpdate
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastLightClientOptimisticUpdate(ctx, nilUpdate))
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.BroadcastLightClientOptimisticUpdate(ctx, msg))
|
|
if util.WaitTimeout(&wg, 1*time.Second) {
|
|
t.Error("Failed to receive pubsub within 1s")
|
|
}
|
|
}
|
|
|
|
func TestService_BroadcastLightClientFinalityUpdate(t *testing.T) {
|
|
params.SetupTestConfigCleanup(t)
|
|
config := params.BeaconConfig().Copy()
|
|
config.SyncMessageDueBPS = 60 // ~72 millisecond
|
|
params.OverrideBeaconConfig(config)
|
|
|
|
p1 := p2ptest.NewTestP2P(t)
|
|
p2 := p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
require.NotEqual(t, 0, len(p1.BHost.Network().Peers()))
|
|
|
|
p := &Service{
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{},
|
|
genesisTime: time.Now().Add(-33 * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second), // the signature slot of the mock update is 33
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(t.Context(), &peers.StatusConfig{
|
|
ScorerParams: &scorers.Config{},
|
|
}),
|
|
}
|
|
|
|
msg, err := util.MockFinalityUpdate()
|
|
require.NoError(t, err)
|
|
|
|
GossipTypeMapping[reflect.TypeOf(msg)] = LightClientFinalityUpdateTopicFormat
|
|
topic := fmt.Sprintf(LightClientFinalityUpdateTopicFormat, params.ForkDigest(slots.ToEpoch(msg.AttestedHeader().Beacon().Slot)))
|
|
|
|
// External peer subscribes to the topic.
|
|
topic += p.Encoding().ProtocolSuffix()
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
time.Sleep(50 * time.Millisecond) // libp2p fails without this delay...
|
|
|
|
// Async listen for the pubsub, must be before the broadcast.
|
|
var wg sync.WaitGroup
|
|
wg.Add(1)
|
|
go func(tt *testing.T) {
|
|
defer wg.Done()
|
|
ctx, cancel := context.WithTimeout(t.Context(), 150*time.Millisecond)
|
|
defer cancel()
|
|
|
|
incomingMessage, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
slotStartTime, err := slots.StartTime(p.genesisTime, msg.SignatureSlot())
|
|
require.NoError(t, err)
|
|
expectedDelay := params.BeaconConfig().SlotComponentDuration(params.BeaconConfig().SyncMessageDueBPS)
|
|
if time.Now().Before(slotStartTime.Add(expectedDelay)) {
|
|
tt.Errorf("Message received too early, now %v, expected at least %v", time.Now(), slotStartTime.Add(expectedDelay))
|
|
}
|
|
|
|
result := ðpb.LightClientFinalityUpdateAltair{}
|
|
require.NoError(t, p.Encoding().DecodeGossip(incomingMessage.Data, result))
|
|
if !proto.Equal(result, msg.Proto()) {
|
|
tt.Errorf("Did not receive expected message, got %+v, wanted %+v", result, msg)
|
|
}
|
|
}(t)
|
|
|
|
// Broadcasting nil should fail.
|
|
ctx := t.Context()
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastLightClientFinalityUpdate(ctx, nil))
|
|
var nilUpdate interfaces.LightClientFinalityUpdate
|
|
require.ErrorContains(t, "attempted to broadcast nil", p.BroadcastLightClientFinalityUpdate(ctx, nilUpdate))
|
|
|
|
// Broadcast to peers and wait.
|
|
require.NoError(t, p.BroadcastLightClientFinalityUpdate(ctx, msg))
|
|
if util.WaitTimeout(&wg, 1*time.Second) {
|
|
t.Error("Failed to receive pubsub within 1s")
|
|
}
|
|
}
|
|
|
|
func TestService_BroadcastDataColumn(t *testing.T) {
|
|
const (
|
|
port = 2000
|
|
columnIndex = 12
|
|
topicFormat = DataColumnSubnetTopicFormat
|
|
)
|
|
|
|
ctx := t.Context()
|
|
|
|
// Load the KZG trust setup.
|
|
err := kzg.Start()
|
|
require.NoError(t, err)
|
|
|
|
gFlags := new(flags.GlobalFlags)
|
|
gFlags.MinimumPeersPerSubnet = 1
|
|
flags.Init(gFlags)
|
|
|
|
// Reset config.
|
|
defer flags.Init(new(flags.GlobalFlags))
|
|
|
|
// Create two peers and connect them.
|
|
p1, p2 := p2ptest.NewTestP2P(t), p2ptest.NewTestP2P(t)
|
|
p1.Connect(p2)
|
|
|
|
// Test the peers are connected.
|
|
require.NotEqual(t, 0, len(p1.BHost.Network().Peers()), "No peers")
|
|
|
|
// Create a host.
|
|
_, pkey, ipAddr := createHost(t, port)
|
|
|
|
// Create a shared DB for the service
|
|
db := testDB.SetupDB(t)
|
|
|
|
// Create and close the custody info channel immediately since custodyInfo is already set
|
|
custodyInfoSet := make(chan struct{})
|
|
close(custodyInfoSet)
|
|
|
|
service := &Service{
|
|
ctx: ctx,
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{DB: db},
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(ctx, &peers.StatusConfig{ScorerParams: &scorers.Config{}}),
|
|
custodyInfo: &custodyInfo{},
|
|
custodyInfoSet: custodyInfoSet,
|
|
}
|
|
|
|
// Create a listener.
|
|
listener, err := service.startDiscoveryV5(ipAddr, pkey)
|
|
require.NoError(t, err)
|
|
|
|
service.dv5Listener = listener
|
|
|
|
digest, err := service.currentForkDigest()
|
|
require.NoError(t, err)
|
|
|
|
subnet := peerdas.ComputeSubnetForDataColumnSidecar(columnIndex)
|
|
topic := fmt.Sprintf(topicFormat, digest, subnet) + service.Encoding().ProtocolSuffix()
|
|
|
|
_, verifiedRoSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, []util.DataColumnParam{{Index: columnIndex}})
|
|
verifiedRoSidecar := verifiedRoSidecars[0]
|
|
|
|
// Subscribe to the topic.
|
|
sub, err := p2.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
|
|
// libp2p fails without this delay
|
|
time.Sleep(50 * time.Millisecond)
|
|
|
|
// Broadcast to peers and wait.
|
|
err = service.BroadcastDataColumnSidecars(ctx, []blocks.VerifiedRODataColumn{verifiedRoSidecar})
|
|
require.NoError(t, err)
|
|
|
|
// Receive the message.
|
|
ctx, cancel := context.WithTimeout(t.Context(), 5*time.Second)
|
|
defer cancel()
|
|
|
|
msg, err := sub.Next(ctx)
|
|
require.NoError(t, err)
|
|
|
|
var result ethpb.DataColumnSidecar
|
|
require.NoError(t, service.Encoding().DecodeGossip(msg.Data, &result))
|
|
require.DeepEqual(t, &result, verifiedRoSidecar)
|
|
}
|
|
|
|
type topicInvoked struct {
|
|
topic string
|
|
pid peer.ID
|
|
}
|
|
|
|
// rpcOrderTracer is a RawTracer implementation that captures the order of SendRPC calls.
|
|
// It records the topics of messages sent via pubsub to verify round-robin ordering.
|
|
type rpcOrderTracer struct {
|
|
mu sync.Mutex
|
|
invoked []*topicInvoked
|
|
byTopic map[string][]peer.ID
|
|
}
|
|
|
|
func (t *rpcOrderTracer) SendRPC(rpc *pubsub.RPC, pid peer.ID) {
|
|
t.mu.Lock()
|
|
defer t.mu.Unlock()
|
|
for _, msg := range rpc.GetPublish() {
|
|
invoked := &topicInvoked{topic: msg.GetTopic(), pid: pid}
|
|
t.invoked = append(t.invoked, invoked)
|
|
t.byTopic[invoked.topic] = append(t.byTopic[invoked.topic], invoked.pid)
|
|
}
|
|
}
|
|
|
|
func newRpcOrderTracer() *rpcOrderTracer {
|
|
return &rpcOrderTracer{byTopic: make(map[string][]peer.ID)}
|
|
}
|
|
|
|
func (t *rpcOrderTracer) getTopics() []string {
|
|
t.mu.Lock()
|
|
defer t.mu.Unlock()
|
|
result := make([]string, len(t.invoked))
|
|
for i := range t.invoked {
|
|
result[i] = t.invoked[i].topic
|
|
}
|
|
return result
|
|
}
|
|
|
|
// No-op implementations for other RawTracer methods.
|
|
func (*rpcOrderTracer) AddPeer(peer.ID, protocol.ID) {}
|
|
func (*rpcOrderTracer) RemovePeer(peer.ID) {}
|
|
func (*rpcOrderTracer) Join(string) {}
|
|
func (*rpcOrderTracer) Leave(string) {}
|
|
func (*rpcOrderTracer) Graft(peer.ID, string) {}
|
|
func (*rpcOrderTracer) Prune(peer.ID, string) {}
|
|
func (*rpcOrderTracer) ValidateMessage(*pubsub.Message) {}
|
|
func (*rpcOrderTracer) DeliverMessage(*pubsub.Message) {}
|
|
func (*rpcOrderTracer) RejectMessage(*pubsub.Message, string) {}
|
|
func (*rpcOrderTracer) DuplicateMessage(*pubsub.Message) {}
|
|
func (*rpcOrderTracer) ThrottlePeer(peer.ID) {}
|
|
func (*rpcOrderTracer) RecvRPC(*pubsub.RPC) {}
|
|
func (*rpcOrderTracer) DropRPC(*pubsub.RPC, peer.ID) {}
|
|
func (*rpcOrderTracer) UndeliverableMessage(*pubsub.Message) {}
|
|
|
|
// TestService_BroadcastDataColumnRoundRobin verifies that when broadcasting multiple
|
|
// data column sidecars, messages are interleaved in round-robin order by column index
|
|
// rather than sending all copies of one column before the next.
|
|
//
|
|
// Without batch publishing: A,A,A,A,B,B,B,B (all peers for column A, then all for column B)
|
|
// With batch publishing: A,B,A,B,A,B,A,B (interleaved by message ID)
|
|
func TestService_BroadcastDataColumnRoundRobin(t *testing.T) {
|
|
const (
|
|
port = 2100
|
|
topicFormat = DataColumnSubnetTopicFormat
|
|
)
|
|
|
|
ctx := t.Context()
|
|
|
|
// Load the KZG trust setup.
|
|
err := kzg.Start()
|
|
require.NoError(t, err)
|
|
|
|
gFlags := new(flags.GlobalFlags)
|
|
gFlags.MinimumPeersPerSubnet = 1
|
|
flags.Init(gFlags)
|
|
defer flags.Init(new(flags.GlobalFlags))
|
|
|
|
// Create a tracer to capture the order of SendRPC calls.
|
|
tracer := newRpcOrderTracer()
|
|
|
|
// Create the publisher node with the tracer injected.
|
|
p1 := p2ptest.NewTestP2PWithPubsubOptions(t, []pubsub.Option{pubsub.WithRawTracer(tracer)})
|
|
|
|
// Create subscriber peers.
|
|
expectedPeers := []*p2ptest.TestP2P{
|
|
p2ptest.NewTestP2P(t),
|
|
p2ptest.NewTestP2P(t),
|
|
}
|
|
|
|
// Connect peers.
|
|
for _, p := range expectedPeers {
|
|
p1.Connect(p)
|
|
}
|
|
require.NotEqual(t, 0, len(p1.BHost.Network().Peers()), "No peers")
|
|
|
|
// Create a host for discovery.
|
|
_, pkey, ipAddr := createHost(t, port)
|
|
|
|
// Create a shared DB for the service.
|
|
db := testDB.SetupDB(t)
|
|
|
|
// Create and close the custody info channel immediately since custodyInfo is already set.
|
|
custodyInfoSet := make(chan struct{})
|
|
close(custodyInfoSet)
|
|
|
|
service := &Service{
|
|
ctx: ctx,
|
|
host: p1.BHost,
|
|
pubsub: p1.PubSub(),
|
|
joinedTopics: map[string]*pubsub.Topic{},
|
|
cfg: &Config{DB: db},
|
|
genesisTime: time.Now(),
|
|
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
|
|
subnetsLock: make(map[uint64]*sync.RWMutex),
|
|
subnetsLockLock: sync.Mutex{},
|
|
peers: peers.NewStatus(ctx, &peers.StatusConfig{ScorerParams: &scorers.Config{}}),
|
|
custodyInfo: &custodyInfo{},
|
|
custodyInfoSet: custodyInfoSet,
|
|
}
|
|
|
|
// Create a listener for discovery.
|
|
listener, err := service.startDiscoveryV5(ipAddr, pkey)
|
|
require.NoError(t, err)
|
|
service.dv5Listener = listener
|
|
|
|
digest, err := service.currentForkDigest()
|
|
require.NoError(t, err)
|
|
|
|
// Create multiple data column sidecars with different column indices.
|
|
// Use indices that map to different subnets: 0, 32, 64 (assuming 128 columns and 64 subnets).
|
|
columnIndices := []uint64{0, 32, 64}
|
|
params := make([]util.DataColumnParam, len(columnIndices))
|
|
for i, idx := range columnIndices {
|
|
params[i] = util.DataColumnParam{Index: idx}
|
|
}
|
|
_, verifiedRoSidecars := util.CreateTestVerifiedRoDataColumnSidecars(t, params)
|
|
|
|
expectedTopics := make(map[string]bool)
|
|
// Subscribe peers to the relevant topics.
|
|
for _, idx := range columnIndices {
|
|
subnet := peerdas.ComputeSubnetForDataColumnSidecar(idx)
|
|
topic := fmt.Sprintf(topicFormat, digest, subnet) + service.Encoding().ProtocolSuffix()
|
|
for _, p := range expectedPeers {
|
|
_, err = p.SubscribeToTopic(topic)
|
|
require.NoError(t, err)
|
|
}
|
|
expectedTopics[topic] = true
|
|
}
|
|
// libp2p needs some time to establish mesh connections.
|
|
time.Sleep(100 * time.Millisecond)
|
|
|
|
// Broadcast all sidecars.
|
|
err = service.BroadcastDataColumnSidecars(ctx, verifiedRoSidecars)
|
|
require.NoError(t, err)
|
|
// Give some time for messages to be sent.
|
|
time.Sleep(100 * time.Millisecond)
|
|
|
|
topics := tracer.getTopics()
|
|
if len(topics) == 0 {
|
|
t.Fatal("Expected at least one message for each topic to be sent to each peer")
|
|
}
|
|
|
|
unseen := make(map[string]bool)
|
|
for k := range expectedTopics {
|
|
unseen[k] = true
|
|
}
|
|
// Verify round-robin invariant: before all message IDs are seen, no message ID may be repeated.
|
|
// In round-robin order, we should see each topic once before any topic repeats.
|
|
for _, topic := range topics {
|
|
if !expectedTopics[topic] {
|
|
continue
|
|
}
|
|
if !unseen[topic] {
|
|
t.Errorf("Topic %s repeated before all topics were seen once. This violates round-robin ordering.", topic)
|
|
}
|
|
delete(unseen, topic)
|
|
if len(unseen) == 0 {
|
|
break // all have been seen
|
|
}
|
|
}
|
|
require.Equal(t, 0, len(unseen))
|
|
|
|
// Verify that we actually saw all expected topics.
|
|
for topic := range expectedTopics {
|
|
require.Equal(t, len(expectedPeers), len(tracer.byTopic[topic]))
|
|
}
|
|
}
|