From 3b4b810a1450305eb49d5f9328679d5892ec1957 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Tue, 15 Aug 2023 12:46:06 -0700 Subject: [PATCH 01/35] cleans up is connected test helper --- network/middleware.go | 2 - network/p2p/middleware/middleware.go | 10 ----- network/test/blob_service_test.go | 7 ++-- network/test/epochtransition_test.go | 61 ++++++++++++++++------------ 4 files changed, 40 insertions(+), 40 deletions(-) diff --git a/network/middleware.go b/network/middleware.go index d8e14ee82c1..be2c65281e7 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -57,8 +57,6 @@ type Middleware interface { // NewPingService creates a new PingService for the given ping protocol ID. NewPingService(pingProtocol protocol.ID, provider PingInfoProvider) PingService - - IsConnected(nodeID flow.Identifier) (bool, error) } // Overlay represents the interface that middleware uses to interact with the diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index c908e8d7f18..802bb00aa43 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -805,16 +805,6 @@ func (m *Middleware) Publish(msg *network.OutgoingMessageScope) error { return nil } -// IsConnected returns true if this node is connected to the node with id nodeID. -// All errors returned from this function can be considered benign. -func (m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { - peerID, err := m.idTranslator.GetPeerID(nodeID) - if err != nil { - return false, fmt.Errorf("could not find peer id for target id: %w", err) - } - return m.libP2PNode.IsConnected(peerID) -} - // unicastMaxMsgSize returns the max permissible size for a unicast message func unicastMaxMsgSize(messageType string) int { switch messageType { diff --git a/network/test/blob_service_test.go b/network/test/blob_service_test.go index 7e3c40f96d8..3a401ad4b0b 100644 --- a/network/test/blob_service_test.go +++ b/network/test/blob_service_test.go @@ -10,6 +10,7 @@ import ( "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/sync" blockstore "github.com/ipfs/go-ipfs-blockstore" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.uber.org/atomic" @@ -118,10 +119,10 @@ func (suite *BlobServiceTestSuite) SetupTest() { // let nodes connect to each other only after they are all listening on Bitswap topologyActive.Store(true) suite.Require().Eventually(func() bool { - for i, mw := range mws { + for i, libp2pNode := range nodes { for j := i + 1; j < suite.numNodes; j++ { - connected, err := mw.IsConnected(ids[j].NodeID) - suite.Require().NoError(err) + connected, err := libp2pNode.IsConnected(nodes[j].Host().ID()) + require.NoError(suite.T(), err) if !connected { return false } diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index 26169f91843..46eb1ae1b79 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -25,6 +25,7 @@ import ( "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/internal/testutils" "github.com/onflow/flow-go/network/mocknetwork" + "github.com/onflow/flow-go/network/p2p" mockprotocol "github.com/onflow/flow-go/state/protocol/mock" "github.com/onflow/flow-go/utils/unittest" ) @@ -48,10 +49,11 @@ type MutableIdentityTableSuite struct { // testNode encapsulates the node state which includes its identity, middleware, network, // mesh engine and the id refresher type testNode struct { - id *flow.Identity - mw network.Middleware - net network.Network - engine *testutils.MeshEngine + id *flow.Identity + libp2pNode p2p.LibP2PNode + mw network.Middleware + net network.Network + engine *testutils.MeshEngine } // testNodeList encapsulates a list of test node and @@ -120,6 +122,16 @@ func (t *testNodeList) networks() []network.Network { return nets } +func (t *testNodeList) libp2pNodes() []p2p.LibP2PNode { + t.RLock() + defer t.RUnlock() + nodes := make([]p2p.LibP2PNode, len(t.nodes)) + for i, node := range t.nodes { + nodes[i] = node.libp2pNode + } + return nodes +} + func TestMutableIdentityTable(t *testing.T) { unittest.SkipUnless(t, unittest.TEST_TODO, "broken test") suite.Run(t, new(MutableIdentityTableSuite)) @@ -200,10 +212,11 @@ func (suite *MutableIdentityTableSuite) addNodes(count int) { // create the test engines for i := 0; i < count; i++ { node := testNode{ - id: ids[i], - mw: mws[i], - net: nets[i], - engine: engines[i], + id: ids[i], + libp2pNode: nodes[i], + mw: mws[i], + net: nets[i], + engine: engines[i], } suite.testNodes.append(node) } @@ -226,7 +239,6 @@ func (suite *MutableIdentityTableSuite) TestNewNodeAdded() { newNode, err := suite.testNodes.lastAdded() require.NoError(suite.T(), err) newID := newNode.id - newMiddleware := newNode.mw suite.logger.Debug(). Str("new_node", newID.NodeID.String()). @@ -240,7 +252,7 @@ func (suite *MutableIdentityTableSuite) TestNewNodeAdded() { // check if the new node has sufficient connections with the existing nodes // if it does, then it has been inducted successfully in the network - suite.assertConnected(newMiddleware, ids.Filter(filter.Not(filter.HasNodeID(newID.NodeID)))) + suite.assertConnected(newNode.libp2pNode, suite.testNodes.libp2pNodes()) // check that all the engines on this new epoch can talk to each other using any of the three networking primitives suite.assertNetworkPrimitives(ids, engs, nil, nil) @@ -250,11 +262,9 @@ func (suite *MutableIdentityTableSuite) TestNewNodeAdded() { // list (ie. as a result of an ejection or transition into an epoch where that node // has un-staked) then it cannot connect to the network. func (suite *MutableIdentityTableSuite) TestNodeRemoved() { - // removed a node removedNode := suite.removeNode() removedID := removedNode.id - removedMiddleware := removedNode.mw removedEngine := removedNode.engine // update IDs for all the remaining nodes @@ -265,7 +275,7 @@ func (suite *MutableIdentityTableSuite) TestNodeRemoved() { remainingEngs := suite.testNodes.engines() // assert that the removed node has no connections with any of the other nodes - suite.assertDisconnected(removedMiddleware, remainingIDs) + suite.assertDisconnected(removedNode.libp2pNode, suite.testNodes.libp2pNodes()) // check that all remaining engines can still talk to each other while the ones removed can't // using any of the three networking primitives @@ -284,15 +294,12 @@ func (suite *MutableIdentityTableSuite) TestNodesAddedAndRemoved() { // remove a node removedNode := suite.removeNode() removedID := removedNode.id - removedMiddleware := removedNode.mw removedEngine := removedNode.engine // add a node suite.addNodes(1) newNode, err := suite.testNodes.lastAdded() require.NoError(suite.T(), err) - newID := newNode.id - newMiddleware := newNode.mw // update all current nodes suite.signalIdentityChanged() @@ -301,10 +308,10 @@ func (suite *MutableIdentityTableSuite) TestNodesAddedAndRemoved() { remainingEngs := suite.testNodes.engines() // check if the new node has sufficient connections with the existing nodes - suite.assertConnected(newMiddleware, remainingIDs.Filter(filter.Not(filter.HasNodeID(newID.NodeID)))) + suite.assertConnected(newNode.libp2pNode, suite.testNodes.libp2pNodes()) // assert that the removed node has no connections with any of the other nodes - suite.assertDisconnected(removedMiddleware, remainingIDs) + suite.assertDisconnected(removedNode.libp2pNode, suite.testNodes.libp2pNodes()) // check that all remaining engines can still talk to each other while the ones removed can't // using any of the three networking primitives @@ -317,13 +324,17 @@ func (suite *MutableIdentityTableSuite) TestNodesAddedAndRemoved() { // assertConnected checks that the middleware of a node is directly connected // to at least half of the other nodes. -func (suite *MutableIdentityTableSuite) assertConnected(mw network.Middleware, ids flow.IdentityList) { +func (suite *MutableIdentityTableSuite) assertConnected(thisNode p2p.LibP2PNode, allNodes []p2p.LibP2PNode) { t := suite.T() - threshold := len(ids) / 2 + threshold := len(allNodes) / 2 require.Eventuallyf(t, func() bool { connections := 0 - for _, id := range ids { - connected, err := mw.IsConnected(id.NodeID) + for _, node := range allNodes { + if node == thisNode { + // we don't want to check if a node is connected to itself + continue + } + connected, err := thisNode.IsConnected(node.Host().ID()) require.NoError(t, err) if connected { connections++ @@ -339,11 +350,11 @@ func (suite *MutableIdentityTableSuite) assertConnected(mw network.Middleware, i // assertDisconnected checks that the middleware of a node is not connected to any of the other nodes specified in the // ids list -func (suite *MutableIdentityTableSuite) assertDisconnected(mw network.Middleware, ids flow.IdentityList) { +func (suite *MutableIdentityTableSuite) assertDisconnected(thisNode p2p.LibP2PNode, allNodes []p2p.LibP2PNode) { t := suite.T() require.Eventuallyf(t, func() bool { - for _, id := range ids { - connected, err := mw.IsConnected(id.NodeID) + for _, node := range allNodes { + connected, err := thisNode.IsConnected(node.Host().ID()) require.NoError(t, err) if connected { return false From dbc14e5b3fe539dabd652f647b20b3159b385382 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Tue, 15 Aug 2023 14:03:18 -0700 Subject: [PATCH 02/35] adds spork id to libp2p node --- insecure/corruptlibp2p/p2p_node.go | 4 +++- network/p2p/builder.go | 2 +- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 12 +++--------- network/p2p/p2pnode/libp2pNode.go | 6 ++++-- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/insecure/corruptlibp2p/p2p_node.go b/insecure/corruptlibp2p/p2p_node.go index b3fbd0cb36d..02ce4e3a3f3 100644 --- a/insecure/corruptlibp2p/p2p_node.go +++ b/insecure/corruptlibp2p/p2p_node.go @@ -8,6 +8,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/rs/zerolog" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/codec/cbor" @@ -53,10 +54,11 @@ func (n *CorruptP2PNode) Subscribe(topic channels.Topic, _ p2p.TopicValidatorFun // NewCorruptLibP2PNode returns corrupted libP2PNode that will subscribe to topics using the AcceptAllTopicValidator. func NewCorruptLibP2PNode( logger zerolog.Logger, + sporkId flow.Identifier, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager, disallowListCacheCfg *p2p.DisallowListCacheConfig) p2p.LibP2PNode { - node := p2pnode.NewNode(logger, host, pCache, peerManager, disallowListCacheCfg) + node := p2pnode.NewNode(logger, sporkId, host, pCache, peerManager, disallowListCacheCfg) return &CorruptP2PNode{Node: node, logger: logger, codec: cbor.NewCodec()} } diff --git a/network/p2p/builder.go b/network/p2p/builder.go index b856f931a29..b530f4cc49c 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -23,7 +23,7 @@ import ( ) type GossipSubFactoryFunc func(context.Context, zerolog.Logger, host.Host, PubSubAdapterConfig, CollectionClusterChangesConsumer) (PubSubAdapter, error) -type CreateNodeFunc func(zerolog.Logger, host.Host, ProtocolPeerCache, PeerManager, *DisallowListCacheConfig) LibP2PNode +type CreateNodeFunc func(zerolog.Logger, flow.Identifier, host.Host, ProtocolPeerCache, PeerManager, *DisallowListCacheConfig) LibP2PNode type GossipSubAdapterConfigFunc func(*BasePubSubAdapterConfig) PubSubAdapterConfig // GossipSubBuilder provides a builder pattern for creating a GossipSub pubsub system. diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index ac3f30fd4d0..42e22a2fb02 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -44,13 +44,6 @@ import ( "github.com/onflow/flow-go/network/p2p/utils" ) -type GossipSubFactoryFunc func(context.Context, zerolog.Logger, host.Host, p2p.PubSubAdapterConfig) (p2p.PubSubAdapter, error) -type CreateNodeFunc func(logger zerolog.Logger, - host host.Host, - pCache *p2pnode.ProtocolPeerCache, - peerManager *connection.PeerManager) p2p.LibP2PNode -type GossipSubAdapterConfigFunc func(*p2p.BasePubSubAdapterConfig) p2p.PubSubAdapterConfig - type LibP2PNodeBuilder struct { gossipSubBuilder p2p.GossipSubBuilder sporkId flow.Identifier @@ -308,7 +301,7 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { } } - node := builder.createNode(builder.logger, h, pCache, peerManager, builder.disallowListCacheCfg) + node := builder.createNode(builder.logger, builder.sporkId, h, pCache, peerManager, builder.disallowListCacheCfg) if builder.connGater != nil { builder.connGater.SetDisallowListOracle(node) @@ -421,11 +414,12 @@ func defaultLibP2POptions(address string, key fcrypto.PrivateKey) ([]config.Opti // DefaultCreateNodeFunc returns new libP2P node. func DefaultCreateNodeFunc(logger zerolog.Logger, + sporkId flow.Identifier, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager, disallowListCacheCfg *p2p.DisallowListCacheConfig) p2p.LibP2PNode { - return p2pnode.NewNode(logger, host, pCache, peerManager, disallowListCacheCfg) + return p2pnode.NewNode(logger, sporkId, host, pCache, peerManager, disallowListCacheCfg) } // DefaultNodeBuilder returns a node builder. diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 38746521430..c20551f28c4 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -66,20 +66,22 @@ type Node struct { // Cache of temporary disallow-listed peers, when a peer is disallow-listed, the connections to that peer // are closed and further connections are not allowed till the peer is removed from the disallow-list. disallowListedCache p2p.DisallowListCache + sporkId flow.Identifier } // NewNode creates a new libp2p node and sets its parameters. func NewNode( logger zerolog.Logger, + sporkId flow.Identifier, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager, disallowLstCacheCfg *p2p.DisallowListCacheConfig, ) *Node { - lg := logger.With().Str("component", "libp2p-node").Logger() return &Node{ host: host, - logger: lg, + logger: logger.With().Str("component", "libp2p-node").Logger(), + sporkId: sporkId, topics: make(map[channels.Topic]p2p.Topic), subs: make(map[channels.Topic]p2p.Subscription), pCache: pCache, From 72f4d96468bf79401486ed94fd9ac0bc9c4d1bf5 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Tue, 15 Aug 2023 14:12:40 -0700 Subject: [PATCH 03/35] moves publish logic --- network/p2p/libp2pNode.go | 2 +- network/p2p/middleware/middleware.go | 31 +--------------------------- network/p2p/p2pnode/libp2pNode.go | 29 ++++++++++++++++++++++++-- 3 files changed, 29 insertions(+), 33 deletions(-) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 061e45a43ff..62555af935d 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -67,7 +67,7 @@ type LibP2PNode interface { // UnSubscribe cancels the subscriber and closes the topic. UnSubscribe(topic channels.Topic) error // Publish publishes the given payload on the topic. - Publish(ctx context.Context, topic channels.Topic, data []byte) error + Publish(ctx context.Context, msgScope *network.OutgoingMessageScope) error // Host returns pointer to host object of node. Host() host.Host // WithDefaultUnicastProtocol overrides the default handler of the unicast manager and registers all preferred protocols. diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 802bb00aa43..1eb55b0b591 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -30,7 +30,6 @@ import ( "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/blob" - "github.com/onflow/flow-go/network/p2p/p2pnode" "github.com/onflow/flow-go/network/p2p/ping" "github.com/onflow/flow-go/network/p2p/unicast/protocols" "github.com/onflow/flow-go/network/p2p/unicast/ratelimit" @@ -774,35 +773,7 @@ func (m *Middleware) processMessage(scope *network.IncomingMessageScope) { // // All errors returned from this function can be considered benign. func (m *Middleware) Publish(msg *network.OutgoingMessageScope) error { - m.log.Debug(). - Str("channel", msg.Channel().String()). - Interface("msg", msg.Proto()). - Str("type", msg.PayloadType()). - Int("msg_size", msg.Size()). - Msg("publishing new message") - - // convert the message to bytes to be put on the wire. - data, err := msg.Proto().Marshal() - if err != nil { - return fmt.Errorf("failed to marshal the message: %w", err) - } - - msgSize := len(data) - if msgSize > p2pnode.DefaultMaxPubSubMsgSize { - // libp2p pubsub will silently drop the message if its size is greater than the configured pubsub max message size - // hence return an error as this message is undeliverable - return fmt.Errorf("message size %d exceeds configured max message size %d", msgSize, p2pnode.DefaultMaxPubSubMsgSize) - } - - topic := channels.TopicFromChannel(msg.Channel(), m.rootBlockID) - - // publish the bytes on the topic - err = m.libP2PNode.Publish(m.ctx, topic, data) - if err != nil { - return fmt.Errorf("failed to publish the message: %w", err) - } - - return nil + return m.libP2PNode.Publish(m.ctx, msg) } // unicastMaxMsgSize returns the max permissible size for a unicast message diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index c20551f28c4..50216155b71 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -319,15 +319,40 @@ func (n *Node) UnSubscribe(topic channels.Topic) error { // Publish publishes the given payload on the topic. // All errors returned from this function can be considered benign. -func (n *Node) Publish(ctx context.Context, topic channels.Topic, data []byte) error { +func (n *Node) Publish(ctx context.Context, msgScope *flownet.OutgoingMessageScope) error { + lg := n.logger.With(). + Str("channel", msgScope.Channel().String()). + Interface("proto_message", msgScope.Proto()). + Str("payload_type", msgScope.PayloadType()). + Int("message_size", msgScope.Size()).Logger() + lg.Debug().Msg("received message to publish") + + // convert the message to bytes to be put on the wire. + data, err := msgScope.Proto().Marshal() + if err != nil { + return fmt.Errorf("failed to marshal the message: %w", err) + } + + msgSize := len(data) + if msgSize > DefaultMaxPubSubMsgSize { + // libp2p pubsub will silently drop the message if its size is greater than the configured pubsub max message size + // hence return an error as this message is undeliverable + return fmt.Errorf("message size %d exceeds configured max message size %d", msgSize, DefaultMaxPubSubMsgSize) + } + + topic := channels.TopicFromChannel(msgScope.Channel(), n.sporkId) + lg = lg.With().Str("topic", topic.String()).Logger() + ps, found := n.topics[topic] if !found { return fmt.Errorf("could not find topic (%s)", topic) } - err := ps.Publish(ctx, data) + err = ps.Publish(ctx, data) if err != nil { return fmt.Errorf("could not publish to topic (%s): %w", topic, err) } + + lg.Debug().Msg("published message to topic") return nil } From 68e225f9543a2a72a5c66cfe2d2d7291a46cef89 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Tue, 15 Aug 2023 15:16:02 -0700 Subject: [PATCH 04/35] fixes broken tests for publish migration --- network/p2p/dht/dht_test.go | 24 ++++----- network/p2p/libp2pNode.go | 4 +- network/p2p/middleware/middleware.go | 11 +--- network/p2p/p2pnode/libp2pNode.go | 30 +++++++++-- network/p2p/test/fixtures.go | 50 ++++++++++++++----- network/p2p/test/sporking_test.go | 2 +- .../p2p/tracer/gossipSubMeshTracer_test.go | 6 +-- 7 files changed, 82 insertions(+), 45 deletions(-) diff --git a/network/p2p/dht/dht_test.go b/network/p2p/dht/dht_test.go index 5ea0ee70e6a..a959a5a0496 100644 --- a/network/p2p/dht/dht_test.go +++ b/network/p2p/dht/dht_test.go @@ -15,6 +15,7 @@ import ( libp2pmsg "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module/irrecoverable" mockmodule "github.com/onflow/flow-go/module/mock" + flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" @@ -102,7 +103,6 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - topic := channels.Topic("/flow/" + unittest.IdentifierFixture().String()) count := 5 golog.SetAllLoggers(golog.LevelFatal) // change this to Debug if libp2p logs are needed @@ -155,18 +155,16 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { // hence expect count and not count - 1 messages to be received (one by each node, including the sender) ch := make(chan peer.ID, count) - codec := unittest.NetworkCodec() - - payload, _ := codec.Encode(&libp2pmsg.TestMessage{}) - msg := &message.Message{ - Payload: payload, - } - - data, err := msg.Marshal() + messageScope, err := flownet.NewOutgoingScope( + ids.NodeIDs(), + channels.TestNetworkChannel, + &libp2pmsg.TestMessage{}, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) require.NoError(t, err) logger := unittest.Logger() - + topic := channels.TopicFromChannel(channels.TestNetworkChannel, sporkId) topicValidator := flowpubsub.TopicValidator(logger, unittest.AllowAllPeerFilter()) for _, n := range nodes { s, err := n.Subscribe(topic, topicValidator) @@ -176,7 +174,7 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { msg, err := s.Next(ctx) require.NoError(t, err) require.NotNil(t, msg) - assert.Equal(t, data, msg.Data) + assert.Equal(t, messageScope.Proto().Payload, msg.Data) ch <- nodeID }(s, n.Host().ID()) } @@ -196,7 +194,7 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { require.Eventually(t, fullyConnectedGraph, time.Second*5, ticksForAssertEventually, "nodes failed to discover each other") // Step 4: publish a message to the topic - require.NoError(t, dhtServerNode.Publish(ctx, topic, data)) + require.NoError(t, dhtServerNode.Publish(ctx, messageScope)) // Step 5: By now, all peers would have been discovered and the message should have been successfully published // A hash set to keep track of the nodes who received the message @@ -221,6 +219,6 @@ loop: // Step 6: unsubscribes all nodes from the topic for _, n := range nodes { - assert.NoError(t, n.UnSubscribe(topic)) + assert.NoError(t, n.Unsubscribe(channels.TestNetworkChannel)) } } diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 62555af935d..898a9c6916e 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -64,8 +64,8 @@ type LibP2PNode interface { ListPeers(topic string) []peer.ID // Subscribe subscribes the node to the given topic and returns the subscription Subscribe(topic channels.Topic, topicValidator TopicValidatorFunc) (Subscription, error) - // UnSubscribe cancels the subscriber and closes the topic. - UnSubscribe(topic channels.Topic) error + // Unsubscribe cancels the subscriber and closes the topic corresponding to the given channel. + Unsubscribe(channel channels.Channel) error // Publish publishes the given payload on the topic. Publish(ctx context.Context, msgScope *network.OutgoingMessageScope) error // Host returns pointer to host object of node. diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 1eb55b0b591..4d3ae590247 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -624,16 +624,7 @@ func (m *Middleware) processPubSubMessages(msg *message.Message, peerID peer.ID) // // All errors returned from this function can be considered benign. func (m *Middleware) Unsubscribe(channel channels.Channel) error { - topic := channels.TopicFromChannel(channel, m.rootBlockID) - err := m.libP2PNode.UnSubscribe(topic) - if err != nil { - return fmt.Errorf("failed to unsubscribe from channel (%s): %w", channel, err) - } - - // update peers to remove nodes subscribed to channel - m.libP2PNode.RequestPeerUpdate() - - return nil + return m.libP2PNode.Unsubscribe(channel) } // processUnicastStreamMessage will decode, perform authorized sender validation and process a message diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 50216155b71..e2362e3fe69 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -106,7 +106,7 @@ func (n *Node) Stop() error { n.logger.Debug().Msg("unsubscribing from all topics") for t := range n.topics { - err := n.UnSubscribe(t) + err := n.unsubscribeTopic(t) // context cancelled errors are expected while unsubscribing from topics during shutdown if err != nil && !errors.Is(err, context.Canceled) { result = multierror.Append(result, err) @@ -280,11 +280,32 @@ func (n *Node) Subscribe(topic channels.Topic, topicValidator p2p.TopicValidator return s, err } -// UnSubscribe cancels the subscriber and closes the topic. +// Unsubscribe cancels the subscriber and closes the topic. // All errors returned from this function can be considered benign. -func (n *Node) UnSubscribe(topic channels.Topic) error { +func (n *Node) Unsubscribe(channel channels.Channel) error { + topic := channels.TopicFromChannel(channel, n.sporkId) + err := n.unsubscribeTopic(topic) + if err != nil { + return fmt.Errorf("failed to unsubscribe from topic: %w", err) + } + + n.RequestPeerUpdate() + + return nil +} + +// unsubscribeTopic cancels the subscriber and closes the topic. +// All errors returned from this function can be considered benign. +// Args: +// +// topic: topic to unsubscribe from +// +// Returns: +// error: error if any. +func (n *Node) unsubscribeTopic(topic channels.Topic) error { n.Lock() defer n.Unlock() + // Remove the Subscriber from the cache if s, found := n.subs[topic]; found { s.Cancel() @@ -314,7 +335,8 @@ func (n *Node) UnSubscribe(topic channels.Topic) error { n.logger.Debug(). Str("topic", topic.String()). Msg("unsubscribed from topic") - return err + + return nil } // Publish publishes the given payload on the topic. diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 50ecb5a568f..5be17d248eb 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -30,6 +30,7 @@ import ( flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2pfixtures" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/connection" p2pdht "github.com/onflow/flow-go/network/p2p/dht" @@ -582,19 +583,28 @@ func EnsurePubsubMessageExchange(t *testing.T, ctx context.Context, nodes []p2p. // let subscriptions propagate time.Sleep(1 * time.Second) + // TODO: remove this and use channel as an argument channel, ok := channels.ChannelFromTopic(topic) require.True(t, ok) for _, node := range nodes { for i := 0; i < count; i++ { // creates a unique message to be published by the node - msg := messageFactory() - data := p2pfixtures.MustEncodeEvent(t, msg, channel) - require.NoError(t, node.Publish(ctx, topic, data)) + payload := messageFactory() + outgoingMessageScope, err := flownet.NewOutgoingScope( + flow.IdentifierList{unittest.IdentifierFixture()}, + channel, + payload, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, node.Publish(ctx, outgoingMessageScope)) // wait for the message to be received by all nodes ctx, cancel := context.WithTimeout(ctx, 5*time.Second) - p2pfixtures.SubsMustReceiveMessage(t, ctx, data, subs) + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + p2pfixtures.SubsMustReceiveMessage(t, ctx, expectedReceivedData, subs) cancel() } } @@ -620,18 +630,27 @@ func EnsurePubsubMessageExchangeFromNode(t *testing.T, ctx context.Context, send // let subscriptions propagate time.Sleep(1 * time.Second) + // TODO: remove this and use channel as an argument channel, ok := channels.ChannelFromTopic(topic) require.True(t, ok) for i := 0; i < count; i++ { // creates a unique message to be published by the node - msg := messageFactory() - data := p2pfixtures.MustEncodeEvent(t, msg, channel) - require.NoError(t, sender.Publish(ctx, topic, data)) + payload := messageFactory() + outgoingMessageScope, err := flownet.NewOutgoingScope( + flow.IdentifierList{}, + channel, + payload, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, sender.Publish(ctx, outgoingMessageScope)) // wait for the message to be received by all nodes ctx, cancel := context.WithTimeout(ctx, 5*time.Second) - p2pfixtures.SubsMustReceiveMessage(t, ctx, data, []p2p.Subscription{toSub}) + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + p2pfixtures.SubsMustReceiveMessage(t, ctx, expectedReceivedData, []p2p.Subscription{toSub}) cancel() } } @@ -692,13 +711,20 @@ func EnsureNoPubsubMessageExchange(t *testing.T, ctx context.Context, from []p2p wg.Add(1) go func() { // creates a unique message to be published by the node. - msg := messageFactory() + // TODO: remove this and use channel as an argument channel, ok := channels.ChannelFromTopic(topic) require.True(t, ok) - data := p2pfixtures.MustEncodeEvent(t, msg, channel) - // ensure the message is NOT received by any of the nodes. - require.NoError(t, node.Publish(ctx, topic, data)) + payload := messageFactory() + outgoingMessageScope, err := flownet.NewOutgoingScope( + flow.IdentifierList{unittest.IdentifierFixture()}, + channel, + payload, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, node.Publish(ctx, outgoingMessageScope)) + ctx, cancel := context.WithTimeout(ctx, 5*time.Second) p2pfixtures.SubsMustNeverReceiveAnyMessage(t, ctx, subs) cancel() diff --git a/network/p2p/test/sporking_test.go b/network/p2p/test/sporking_test.go index 6f1b784fc72..b7aae10d32f 100644 --- a/network/p2p/test/sporking_test.go +++ b/network/p2p/test/sporking_test.go @@ -248,7 +248,7 @@ func TestOneToKCrosstalkPrevention(t *testing.T) { // mimic that node1 is now part of the new spork while node2 remains on the old spork // by unsubscribing node1 from 'topicBeforeSpork' and subscribing it to 'topicAfterSpork' // and keeping node2 subscribed to topic 'topicBeforeSpork' - err = node1.UnSubscribe(topicBeforeSpork) + err = node1.Unsubscribe(topicBeforeSpork) require.NoError(t, err) _, err = node1.Subscribe(topicAfterSpork, topicValidator) require.NoError(t, err) diff --git a/network/p2p/tracer/gossipSubMeshTracer_test.go b/network/p2p/tracer/gossipSubMeshTracer_test.go index c8a680df53e..407d03fb50d 100644 --- a/network/p2p/tracer/gossipSubMeshTracer_test.go +++ b/network/p2p/tracer/gossipSubMeshTracer_test.go @@ -176,9 +176,9 @@ func TestGossipSubMeshTracer(t *testing.T) { // all nodes except the tracerNode unsubscribe from the topic1, which triggers sending a PRUNE to the tracerNode for each unsubscription. // We expect the tracerNode to remove the otherNode1, otherNode2, and unknownNode from its mesh. - require.NoError(t, otherNode1.UnSubscribe(topic1)) - require.NoError(t, otherNode2.UnSubscribe(topic1)) - require.NoError(t, unknownNode.UnSubscribe(topic1)) + require.NoError(t, otherNode1.Unsubscribe(topic1)) + require.NoError(t, otherNode2.Unsubscribe(topic1)) + require.NoError(t, unknownNode.Unsubscribe(topic1)) assert.Eventually(t, func() bool { // eventually, the tracerNode should not have the other node in its mesh for topic1. From c5870f7dc1232772536cf89aaf097f2a0b5df921 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 11:45:21 -0700 Subject: [PATCH 05/35] fixes tests --- .../validation_inspector_test.go | 15 ++++++-- .../test/gossipsub/scoring/ihave_spam_test.go | 15 ++++++-- .../test/gossipsub/scoring/scoring_test.go | 31 ++++++++++++---- .../p2p/connection/connection_gater_test.go | 36 ++++++++++++++----- network/p2p/scoring/scoring_test.go | 15 ++++++-- network/p2p/test/fixtures.go | 33 +++++++++++++---- 6 files changed, 113 insertions(+), 32 deletions(-) diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go index 3dd873d0e7c..8d256bde9ba 100644 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go +++ b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go @@ -1228,9 +1228,18 @@ func TestGossipSubSpamMitigationIntegration(t *testing.T) { // now we expect the detection and mitigation to kick in and the victim node to disconnect from the spammer node. // so the spammer and victim nodes should not be able to exchange messages on the topic. - p2ptest.EnsureNoPubsubExchangeBetweenGroups(t, ctx, []p2p.LibP2PNode{victimNode}, []p2p.LibP2PNode{spammer.SpammerNode}, blockTopic, 1, func() interface{} { - return unittest.ProposalFixture() - }) + p2ptest.EnsureNoPubsubExchangeBetweenGroups( + t, + ctx, + []p2p.LibP2PNode{victimNode}, + flow.IdentifierList{victimId.NodeID}, + []p2p.LibP2PNode{spammer.SpammerNode}, + flow.IdentifierList{spammer.SpammerId.NodeID}, + blockTopic, + 1, + func() interface{} { + return unittest.ProposalFixture() + }) } // mockDistributorReadyDoneAware mocks the Ready and Done methods of the distributor to return a channel that is already closed, diff --git a/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go b/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go index 3342b3ac4dc..d4a1187aca0 100644 --- a/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go +++ b/insecure/integration/functional/test/gossipsub/scoring/ihave_spam_test.go @@ -302,9 +302,18 @@ func TestGossipSubIHaveBrokenPromises_Above_Threshold(t *testing.T) { require.Lessf(t, spammerScore, scoring.DefaultGraylistThreshold, "sanity check failed, the score of the spammer node must be less than graylist threshold: %f, actual: %f", scoring.DefaultGraylistThreshold, spammerScore) // since the spammer score is below the gossip, graylist and publish thresholds, it should not be able to exchange messages with victim anymore. - p2ptest.EnsureNoPubsubExchangeBetweenGroups(t, ctx, []p2p.LibP2PNode{spammer.SpammerNode}, []p2p.LibP2PNode{victimNode}, blockTopic, 1, func() interface{} { - return unittest.ProposalFixture() - }) + p2ptest.EnsureNoPubsubExchangeBetweenGroups( + t, + ctx, + []p2p.LibP2PNode{spammer.SpammerNode}, + flow.IdentifierList{spammer.SpammerId.NodeID}, + []p2p.LibP2PNode{victimNode}, + flow.IdentifierList{victimIdentity.NodeID}, + blockTopic, + 1, + func() interface{} { + return unittest.ProposalFixture() + }) } // spamIHaveBrokenPromises is a test utility function that is exclusive for the TestGossipSubIHaveBrokenPromises tests. diff --git a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go index d8baf4be735..2366678576e 100644 --- a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go +++ b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go @@ -173,9 +173,18 @@ func testGossipSubInvalidMessageDeliveryScoring(t *testing.T, spamMsgFactory fun // ensure that the topic snapshot of the spammer contains a record of at least (60%) of the spam messages sent. The 60% is to account for the messages that were delivered before the score was updated, after the spammer is PRUNED, as well as to account for decay. require.True(t, blkTopicSnapshot.InvalidMessageDeliveries > 0.6*float64(totalSpamMessages), "invalid message deliveries must be greater than %f. invalid message deliveries: %f", 0.9*float64(totalSpamMessages), blkTopicSnapshot.InvalidMessageDeliveries) - p2ptest.EnsureNoPubsubExchangeBetweenGroups(t, ctx, []p2p.LibP2PNode{victimNode}, []p2p.LibP2PNode{spammer.SpammerNode}, blockTopic, 1, func() interface{} { - return unittest.ProposalFixture() - }) + p2ptest.EnsureNoPubsubExchangeBetweenGroups( + t, + ctx, + []p2p.LibP2PNode{victimNode}, + flow.IdentifierList{victimIdentity.NodeID}, + []p2p.LibP2PNode{spammer.SpammerNode}, + flow.IdentifierList{spammer.SpammerId.NodeID}, + blockTopic, + 1, + func() interface{} { + return unittest.ProposalFixture() + }) } // TestGossipSubMeshDeliveryScoring_UnderDelivery_SingleTopic tests that when a peer is under-performing in a topic mesh, its score is (slightly) penalized. @@ -500,10 +509,18 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { // now the replaying node acts maliciously and just replays the same messages again. i = -1 - p2ptest.EnsureNoPubsubMessageExchange(t, ctx, []p2p.LibP2PNode{replayingNode}, []p2p.LibP2PNode{thisNode}, blockTopic, len(proposalList), func() interface{} { - i += 1 - return proposalList[i] - }) + p2ptest.EnsureNoPubsubMessageExchange( + t, + ctx, + []p2p.LibP2PNode{replayingNode}, + []p2p.LibP2PNode{thisNode}, + flow.IdentifierList{thisId.NodeID}, + blockTopic, + len(proposalList), + func() interface{} { + i += 1 + return proposalList[i] + }) // since the last decay interval, the replaying node has not delivered anything new, so its score should be penalized for under-performing. require.Eventually(t, func() bool { diff --git a/network/p2p/connection/connection_gater_test.go b/network/p2p/connection/connection_gater_test.go index cd240b5293b..672947adadc 100644 --- a/network/p2p/connection/connection_gater_test.go +++ b/network/p2p/connection/connection_gater_test.go @@ -235,6 +235,7 @@ func TestConnectionGater_InterceptUpgrade(t *testing.T) { count := 5 nodes := make([]p2p.LibP2PNode, 0, count) inbounds := make([]chan string, 0, count) + identities := make(flow.IdentityList, 0, count) disallowedPeerIds := unittest.NewProtectedMap[peer.ID, struct{}]() allPeerIds := make(peer.IDSlice, 0, count) @@ -266,6 +267,7 @@ func TestConnectionGater_InterceptUpgrade(t *testing.T) { p2ptest.WithConnectionGater(connectionGater)) idProvider.On("ByPeerID", node.Host().ID()).Return(&id, true).Maybe() nodes = append(nodes, node) + identities = append(identities, &id) allPeerIds = append(allPeerIds, node.Host().ID()) inbounds = append(inbounds, inbound) } @@ -301,7 +303,7 @@ func TestConnectionGater_InterceptUpgrade(t *testing.T) { require.False(t, disallowedPeerIds.Has(remote)) }).Return(true, control.DisconnectReason(0)) - ensureCommunicationSilenceAmongGroups(t, ctx, sporkId, nodes[:1], nodes[1:]) + ensureCommunicationSilenceAmongGroups(t, ctx, sporkId, nodes[:1], identities[:1].NodeIDs(), nodes[1:], identities[1:].NodeIDs()) ensureCommunicationOverAllProtocols(t, ctx, sporkId, nodes[1:], inbounds[1:]) } @@ -380,28 +382,44 @@ func TestConnectionGater_Disallow_Integration(t *testing.T) { // let peer manager prune the connections to the disallow-listed node. time.Sleep(1 * time.Second) // ensures no connection, unicast, or pubsub going to or coming from the disallow-listed node. - ensureCommunicationSilenceAmongGroups(t, ctx, sporkId, nodes[:count-1], nodes[count-1:]) + ensureCommunicationSilenceAmongGroups(t, ctx, sporkId, nodes[:count-1], ids[:count-1].NodeIDs(), nodes[count-1:], ids[count-1:].NodeIDs()) // now we add another node (the second last node) to the disallowed list. disallowedList.Add(ids[len(ids)-2], struct{}{}) // let peer manager prune the connections to the disallow-listed node. time.Sleep(1 * time.Second) // ensures no connection, unicast, or pubsub going to and coming from the disallow-listed nodes. - ensureCommunicationSilenceAmongGroups(t, ctx, sporkId, nodes[:count-2], nodes[count-2:]) + ensureCommunicationSilenceAmongGroups(t, ctx, sporkId, nodes[:count-2], ids[:count-2].NodeIDs(), nodes[count-2:], ids[count-2:].NodeIDs()) // ensures that all nodes are other non-disallow-listed nodes can exchange messages over the pubsub and unicast. ensureCommunicationOverAllProtocols(t, ctx, sporkId, nodes[:count-2], inbounds[:count-2]) } // ensureCommunicationSilenceAmongGroups ensures no connection, unicast, or pubsub going to or coming from between the two groups of nodes. -func ensureCommunicationSilenceAmongGroups(t *testing.T, ctx context.Context, sporkId flow.Identifier, groupA []p2p.LibP2PNode, groupB []p2p.LibP2PNode) { +func ensureCommunicationSilenceAmongGroups( + t *testing.T, + ctx context.Context, + sporkId flow.Identifier, + groupANodes []p2p.LibP2PNode, + groupAIdentifiers flow.IdentifierList, + groupBNodes []p2p.LibP2PNode, + groupBIdentifiers flow.IdentifierList) { // ensures no connection, unicast, or pubsub going to the disallow-listed nodes - p2ptest.EnsureNotConnectedBetweenGroups(t, ctx, groupA, groupB) + p2ptest.EnsureNotConnectedBetweenGroups(t, ctx, groupANodes, groupBNodes) blockTopic := channels.TopicFromChannel(channels.PushBlocks, sporkId) - p2ptest.EnsureNoPubsubExchangeBetweenGroups(t, ctx, groupA, groupB, blockTopic, 1, func() interface{} { - return unittest.ProposalFixture() - }) - p2pfixtures.EnsureNoStreamCreationBetweenGroups(t, ctx, groupA, groupB) + p2ptest.EnsureNoPubsubExchangeBetweenGroups( + t, + ctx, + groupANodes, + groupAIdentifiers, + groupBNodes, + groupBIdentifiers, + blockTopic, + 1, + func() interface{} { + return unittest.ProposalFixture() + }) + p2pfixtures.EnsureNoStreamCreationBetweenGroups(t, ctx, groupANodes, groupBNodes) } // ensureCommunicationOverAllProtocols ensures that all nodes are connected to each other, and they can exchange messages over the pubsub and unicast. diff --git a/network/p2p/scoring/scoring_test.go b/network/p2p/scoring/scoring_test.go index 47e6f27cb57..9aa6644005c 100644 --- a/network/p2p/scoring/scoring_test.go +++ b/network/p2p/scoring/scoring_test.go @@ -145,7 +145,16 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { } // checks no GossipSub message exchange should no longer happen between node1 and node2. - p2ptest.EnsureNoPubsubExchangeBetweenGroups(t, ctx, []p2p.LibP2PNode{node1}, []p2p.LibP2PNode{node2}, blockTopic, 1, func() interface{} { - return unittest.ProposalFixture() - }) + p2ptest.EnsureNoPubsubExchangeBetweenGroups( + t, + ctx, + []p2p.LibP2PNode{node1}, + flow.IdentifierList{id1.NodeID}, + []p2p.LibP2PNode{node2}, + flow.IdentifierList{id2.NodeID}, + blockTopic, + 1, + func() interface{} { + return unittest.ProposalFixture() + }) } diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 5be17d248eb..598bbe2693d 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -684,7 +684,15 @@ func EnsureNotConnectedBetweenGroups(t *testing.T, ctx context.Context, groupA [ // - topic: the topic to exchange messages on. // - count: the number of messages to exchange from each node. // - messageFactory: a function that creates a unique message to be published by the node. -func EnsureNoPubsubMessageExchange(t *testing.T, ctx context.Context, from []p2p.LibP2PNode, to []p2p.LibP2PNode, topic channels.Topic, count int, messageFactory func() interface{}) { +func EnsureNoPubsubMessageExchange( + t *testing.T, + ctx context.Context, + from []p2p.LibP2PNode, + to []p2p.LibP2PNode, + toIdentifiers flow.IdentifierList, + topic channels.Topic, + count int, + messageFactory func() interface{}) { subs := make([]p2p.Subscription, len(to)) tv := validator.TopicValidator( unittest.Logger(), @@ -717,7 +725,7 @@ func EnsureNoPubsubMessageExchange(t *testing.T, ctx context.Context, from []p2p payload := messageFactory() outgoingMessageScope, err := flownet.NewOutgoingScope( - flow.IdentifierList{unittest.IdentifierFixture()}, + toIdentifiers, channel, payload, unittest.NetworkCodec().Encode, @@ -742,16 +750,27 @@ func EnsureNoPubsubMessageExchange(t *testing.T, ctx context.Context, from []p2p // Args: // - t: *testing.T instance // - ctx: context.Context instance -// - groupA: first group of nodes- no message should be exchanged from any node of this group to the other group. -// - groupB: second group of nodes- no message should be exchanged from any node of this group to the other group. +// - groupANodes: first group of nodes- no message should be exchanged from any node of this group to the other group. +// - groupAIdentifiers: identifiers of the nodes in the first group. +// - groupBNodes: second group of nodes- no message should be exchanged from any node of this group to the other group. +// - groupBIdentifiers: identifiers of the nodes in the second group. // - topic: pubsub topic- no message should be exchanged on this topic. // - count: number of messages to be exchanged- no message should be exchanged. // - messageFactory: function to create a unique message to be published by the node. -func EnsureNoPubsubExchangeBetweenGroups(t *testing.T, ctx context.Context, groupA []p2p.LibP2PNode, groupB []p2p.LibP2PNode, topic channels.Topic, count int, messageFactory func() interface{}) { +func EnsureNoPubsubExchangeBetweenGroups( + t *testing.T, + ctx context.Context, + groupANodes []p2p.LibP2PNode, + groupAIdentifiers flow.IdentifierList, + groupBNodes []p2p.LibP2PNode, + groupBIdentifiers flow.IdentifierList, + topic channels.Topic, + count int, + messageFactory func() interface{}) { // ensure no message exchange from group A to group B - EnsureNoPubsubMessageExchange(t, ctx, groupA, groupB, topic, count, messageFactory) + EnsureNoPubsubMessageExchange(t, ctx, groupANodes, groupBNodes, groupBIdentifiers, topic, count, messageFactory) // ensure no message exchange from group B to group A - EnsureNoPubsubMessageExchange(t, ctx, groupB, groupA, topic, count, messageFactory) + EnsureNoPubsubMessageExchange(t, ctx, groupBNodes, groupANodes, groupAIdentifiers, topic, count, messageFactory) } // PeerIdSliceFixture returns a slice of random peer IDs for testing. From 2de77ca0241b24016e43178e9ecf08b5be5e5fff Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 11:54:36 -0700 Subject: [PATCH 06/35] updates mocks --- network/mocknetwork/middleware.go | 26 -------------------------- network/p2p/mock/create_node_func.go | 12 +++++++----- network/p2p/mock/lib_p2_p_node.go | 20 ++++++++++---------- 3 files changed, 17 insertions(+), 41 deletions(-) diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index 18cdaed21b0..a61deb80268 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -6,8 +6,6 @@ import ( datastore "github.com/ipfs/go-datastore" channels "github.com/onflow/flow-go/network/channels" - flow "github.com/onflow/flow-go/model/flow" - irrecoverable "github.com/onflow/flow-go/module/irrecoverable" mock "github.com/stretchr/testify/mock" @@ -38,30 +36,6 @@ func (_m *Middleware) Done() <-chan struct{} { return r0 } -// IsConnected provides a mock function with given fields: nodeID -func (_m *Middleware) IsConnected(nodeID flow.Identifier) (bool, error) { - ret := _m.Called(nodeID) - - var r0 bool - var r1 error - if rf, ok := ret.Get(0).(func(flow.Identifier) (bool, error)); ok { - return rf(nodeID) - } - if rf, ok := ret.Get(0).(func(flow.Identifier) bool); ok { - r0 = rf(nodeID) - } else { - r0 = ret.Get(0).(bool) - } - - if rf, ok := ret.Get(1).(func(flow.Identifier) error); ok { - r1 = rf(nodeID) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - // NewBlobService provides a mock function with given fields: channel, store, opts func (_m *Middleware) NewBlobService(channel channels.Channel, store datastore.Batching, opts ...network.BlobServiceOption) network.BlobService { _va := make([]interface{}, len(opts)) diff --git a/network/p2p/mock/create_node_func.go b/network/p2p/mock/create_node_func.go index 1a57772cbeb..b2db19f13ee 100644 --- a/network/p2p/mock/create_node_func.go +++ b/network/p2p/mock/create_node_func.go @@ -4,6 +4,8 @@ package mockp2p import ( host "github.com/libp2p/go-libp2p/core/host" + flow "github.com/onflow/flow-go/model/flow" + mock "github.com/stretchr/testify/mock" p2p "github.com/onflow/flow-go/network/p2p" @@ -16,13 +18,13 @@ type CreateNodeFunc struct { mock.Mock } -// Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4 -func (_m *CreateNodeFunc) Execute(_a0 zerolog.Logger, _a1 host.Host, _a2 p2p.ProtocolPeerCache, _a3 p2p.PeerManager, _a4 *p2p.DisallowListCacheConfig) p2p.LibP2PNode { - ret := _m.Called(_a0, _a1, _a2, _a3, _a4) +// Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4, _a5 +func (_m *CreateNodeFunc) Execute(_a0 zerolog.Logger, _a1 flow.Identifier, _a2 host.Host, _a3 p2p.ProtocolPeerCache, _a4 p2p.PeerManager, _a5 *p2p.DisallowListCacheConfig) p2p.LibP2PNode { + ret := _m.Called(_a0, _a1, _a2, _a3, _a4, _a5) var r0 p2p.LibP2PNode - if rf, ok := ret.Get(0).(func(zerolog.Logger, host.Host, p2p.ProtocolPeerCache, p2p.PeerManager, *p2p.DisallowListCacheConfig) p2p.LibP2PNode); ok { - r0 = rf(_a0, _a1, _a2, _a3, _a4) + if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, host.Host, p2p.ProtocolPeerCache, p2p.PeerManager, *p2p.DisallowListCacheConfig) p2p.LibP2PNode); ok { + r0 = rf(_a0, _a1, _a2, _a3, _a4, _a5) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(p2p.LibP2PNode) diff --git a/network/p2p/mock/lib_p2_p_node.go b/network/p2p/mock/lib_p2_p_node.go index 5813983110e..e67614d6212 100644 --- a/network/p2p/mock/lib_p2_p_node.go +++ b/network/p2p/mock/lib_p2_p_node.go @@ -284,13 +284,13 @@ func (_m *LibP2PNode) PeerScoreExposer() p2p.PeerScoreExposer { return r0 } -// Publish provides a mock function with given fields: ctx, topic, data -func (_m *LibP2PNode) Publish(ctx context.Context, topic channels.Topic, data []byte) error { - ret := _m.Called(ctx, topic, data) +// Publish provides a mock function with given fields: ctx, msgScope +func (_m *LibP2PNode) Publish(ctx context.Context, msgScope *flow_gonetwork.OutgoingMessageScope) error { + ret := _m.Called(ctx, msgScope) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, channels.Topic, []byte) error); ok { - r0 = rf(ctx, topic, data) + if rf, ok := ret.Get(0).(func(context.Context, *flow_gonetwork.OutgoingMessageScope) error); ok { + r0 = rf(ctx, msgScope) } else { r0 = ret.Error(0) } @@ -430,13 +430,13 @@ func (_m *LibP2PNode) Subscribe(topic channels.Topic, topicValidator p2p.TopicVa return r0, r1 } -// UnSubscribe provides a mock function with given fields: topic -func (_m *LibP2PNode) UnSubscribe(topic channels.Topic) error { - ret := _m.Called(topic) +// Unsubscribe provides a mock function with given fields: channel +func (_m *LibP2PNode) Unsubscribe(channel channels.Channel) error { + ret := _m.Called(channel) var r0 error - if rf, ok := ret.Get(0).(func(channels.Topic) error); ok { - r0 = rf(topic) + if rf, ok := ret.Get(0).(func(channels.Channel) error); ok { + r0 = rf(channel) } else { r0 = ret.Error(0) } From e47a3e04fb2f675fb65a5e0a50b9c767898329d0 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 12:51:23 -0700 Subject: [PATCH 07/35] increases timeout for disallowlisting test --- network/alsp/manager/manager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/alsp/manager/manager_test.go b/network/alsp/manager/manager_test.go index 09def778e51..8b0d925f642 100644 --- a/network/alsp/manager/manager_test.go +++ b/network/alsp/manager/manager_test.go @@ -278,7 +278,7 @@ func TestHandleReportedMisbehavior_And_DisallowListing_Integration(t *testing.T) unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "not all misbehavior reports have been processed") // ensures that the spammer is disallow-listed by the victim - p2ptest.RequireEventuallyNotConnected(t, []p2p.LibP2PNode{nodes[victimIndex]}, []p2p.LibP2PNode{nodes[spammerIndex]}, 100*time.Millisecond, 2*time.Second) + p2ptest.RequireEventuallyNotConnected(t, []p2p.LibP2PNode{nodes[victimIndex]}, []p2p.LibP2PNode{nodes[spammerIndex]}, 100*time.Millisecond, 5*time.Second) // despite disallow-listing spammer, ensure that (victim and honest) and (honest and spammer) are still connected. p2ptest.RequireConnectedEventually(t, []p2p.LibP2PNode{nodes[spammerIndex], nodes[honestIndex]}, 1*time.Millisecond, 100*time.Millisecond) From 43c72f5b5e3182f7904b3bb9af8323bd4286466f Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 13:15:19 -0700 Subject: [PATCH 08/35] fixes all scoring tests --- network/p2p/scoring/app_score_test.go | 38 ++++++++++--- .../scoring/subscription_validator_test.go | 56 ++++++++++++++----- 2 files changed, 70 insertions(+), 24 deletions(-) diff --git a/network/p2p/scoring/app_score_test.go b/network/p2p/scoring/app_score_test.go index 8e2a1ae1bb8..37cfd52a6b5 100644 --- a/network/p2p/scoring/app_score_test.go +++ b/network/p2p/scoring/app_score_test.go @@ -13,8 +13,10 @@ import ( "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/mock" + flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2pfixtures" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/scoring" p2ptest "github.com/onflow/flow-go/network/p2p/test" @@ -93,14 +95,22 @@ func TestFullGossipSubConnectivity(t *testing.T) { // checks end-to-end message delivery works // each node sends a distinct message to all and checks that all nodes receive it. for _, node := range nodes { - proposalMsg := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channels.PushBlocks) - require.NoError(t, node.Publish(ctx, blockTopic, proposalMsg)) + outgoingMessageScope, err := flownet.NewOutgoingScope( + ids.NodeIDs(), + channels.PushBlocks, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, node.Publish(ctx, outgoingMessageScope)) // checks that the message is received by all nodes. ctx1s, cancel1s := context.WithTimeout(ctx, 5*time.Second) - p2pfixtures.SubsMustReceiveMessage(t, ctx1s, proposalMsg, groupOneSubs) - p2pfixtures.SubsMustReceiveMessage(t, ctx1s, proposalMsg, accessNodeSubs) - p2pfixtures.SubsMustReceiveMessage(t, ctx1s, proposalMsg, groupTwoSubs) + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + p2pfixtures.SubsMustReceiveMessage(t, ctx1s, expectedReceivedData, groupOneSubs) + p2pfixtures.SubsMustReceiveMessage(t, ctx1s, expectedReceivedData, accessNodeSubs) + p2pfixtures.SubsMustReceiveMessage(t, ctx1s, expectedReceivedData, groupTwoSubs) cancel1s() } @@ -169,7 +179,7 @@ func testGossipSubMessageDeliveryUnderNetworkPartition(t *testing.T, honestPeerS ) allNodes := append([]p2p.LibP2PNode{con1Node, con2Node}, accessNodeGroup...) - allIds := append([]*flow.Identity{&con1Id, &con2Id}, accessNodeIds...) + allIds := append(flow.IdentityList{&con1Id, &con2Id}, accessNodeIds...) provider := id.NewFixedIdentityProvider(allIds) idProvider.On("ByPeerID", mocktestify.Anything).Return( @@ -206,8 +216,14 @@ func testGossipSubMessageDeliveryUnderNetworkPartition(t *testing.T, honestPeerS // let nodes reside on a full topology, hence no partition is caused by the topology. p2ptest.LetNodesDiscoverEachOther(t, ctx, allNodes, allIds) - proposalMsg := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channels.PushBlocks) - require.NoError(t, con1Node.Publish(ctx, blockTopic, proposalMsg)) + outgoingMessageScope, err := flownet.NewOutgoingScope( + allIds.NodeIDs(), + channels.PushBlocks, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, con1Node.Publish(ctx, outgoingMessageScope)) // we check that whether within a one-second window the message is received by the other honest consensus node. // the one-second window is important because it triggers the heartbeat of the con1Node to perform a lazy pull (iHave). @@ -218,7 +234,11 @@ func testGossipSubMessageDeliveryUnderNetworkPartition(t *testing.T, honestPeerS // If honest peer scoring is enabled, then con1Node and con2Node are certainly in the same mesh, and hence the message is delivered. ctx1s, cancel1s := context.WithTimeout(ctx, 1*time.Second) defer cancel1s() - return p2pfixtures.HasSubReceivedMessage(t, ctx1s, proposalMsg, con2Sub) + + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + + return p2pfixtures.HasSubReceivedMessage(t, ctx1s, expectedReceivedData, con2Sub) } // maliciousAppSpecificScore returns a malicious app specific penalty function that rewards the malicious node and diff --git a/network/p2p/scoring/subscription_validator_test.go b/network/p2p/scoring/subscription_validator_test.go index 549006b3bde..795165ac9cc 100644 --- a/network/p2p/scoring/subscription_validator_test.go +++ b/network/p2p/scoring/subscription_validator_test.go @@ -7,6 +7,8 @@ import ( "testing" "time" + flownet "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" p2ptest "github.com/onflow/flow-go/network/p2p/test" flowpubsub "github.com/onflow/flow-go/network/validator/pubsub" @@ -130,8 +132,8 @@ func TestSubscriptionValidator_InvalidSubscriptions(t *testing.T) { for _, role := range flow.Roles() { peerId := p2pfixtures.PeerIdFixture(t) unauthorizedChannels := channels.Channels(). // all channels - ExcludeChannels(channels.ChannelsByRole(role)). // excluding the channels for the role - ExcludePattern(regexp.MustCompile("^(test).*")) // excluding the test channels. + ExcludeChannels(channels.ChannelsByRole(role)). // excluding the channels for the role + ExcludePattern(regexp.MustCompile("^(test).*")) // excluding the test channels. sporkID := unittest.IdentifierFixture() unauthorizedTopics := make([]string, 0, len(unauthorizedChannels)) for _, channel := range unauthorizedChannels { @@ -237,14 +239,23 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // let the subscriptions be established time.Sleep(2 * time.Second) - proposalMsg := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channels.PushBlocks) - // consensus node publishes a proposal - require.NoError(t, conNode.Publish(ctx, blockTopic, proposalMsg)) + outgoingMessageScope, err := flownet.NewOutgoingScope( + ids.NodeIDs(), + channels.PushBlocks, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, conNode.Publish(ctx, outgoingMessageScope)) // checks that the message is received by all nodes. ctx1s, cancel1s := context.WithTimeout(ctx, 1*time.Second) defer cancel1s() - p2pfixtures.SubsMustReceiveMessage(t, ctx1s, proposalMsg, []p2p.Subscription{conSub, ver1SubBlocks, ver2SubBlocks}) + + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + + p2pfixtures.SubsMustReceiveMessage(t, ctx1s, expectedReceivedData, []p2p.Subscription{conSub, ver1SubBlocks, ver2SubBlocks}) // now consensus node is doing something very bad! // it is subscribing to a channel that it is not supposed to subscribe to. @@ -257,9 +268,14 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // consensus node publishes another proposal, but this time, it should not reach verification node. // since upon an unauthorized subscription, verification node should have slashed consensus node on // the GossipSub scoring protocol. - proposalMsg = p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channels.PushBlocks) - // publishes a message to the topic. - require.NoError(t, conNode.Publish(ctx, blockTopic, proposalMsg)) + outgoingMessageScope, err = flownet.NewOutgoingScope( + ids.NodeIDs(), + channels.PushBlocks, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, conNode.Publish(ctx, outgoingMessageScope)) ctx5s, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() @@ -267,15 +283,25 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // moreover, a verification node publishing a message to the request chunk topic should not reach consensus node. // however, both verification nodes should receive the message. - chunkDataPackRequestMsg := p2pfixtures.MustEncodeEvent(t, &messages.ChunkDataRequest{ - ChunkID: unittest.IdentifierFixture(), - Nonce: rand.Uint64(), - }, channels.RequestChunks) - require.NoError(t, verNode1.Publish(ctx, channels.TopicFromChannel(channels.RequestChunks, sporkId), chunkDataPackRequestMsg)) + outgoingMessageScope, err = flownet.NewOutgoingScope( + ids.NodeIDs(), + channels.RequestChunks, + &messages.ChunkDataRequest{ + ChunkID: unittest.IdentifierFixture(), + Nonce: rand.Uint64(), + }, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + require.NoError(t, verNode1.Publish(ctx, outgoingMessageScope)) ctx1s, cancel1s = context.WithTimeout(ctx, 1*time.Second) defer cancel1s() - p2pfixtures.SubsMustReceiveMessage(t, ctx1s, chunkDataPackRequestMsg, []p2p.Subscription{ver1SubChunks, ver2SubChunks}) + + expectedReceivedData, err = outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + + p2pfixtures.SubsMustReceiveMessage(t, ctx1s, expectedReceivedData, []p2p.Subscription{ver1SubChunks, ver2SubChunks}) ctx5s, cancel5s = context.WithTimeout(ctx, 5*time.Second) defer cancel5s() From fd68f3ce168b6d84622a328612fd29feaeb33049 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 13:21:45 -0700 Subject: [PATCH 09/35] fixes tracer tests --- network/p2p/tracer/gossipSubMeshTracer_test.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/network/p2p/tracer/gossipSubMeshTracer_test.go b/network/p2p/tracer/gossipSubMeshTracer_test.go index 407d03fb50d..4384ffcf9f1 100644 --- a/network/p2p/tracer/gossipSubMeshTracer_test.go +++ b/network/p2p/tracer/gossipSubMeshTracer_test.go @@ -39,8 +39,10 @@ func TestGossipSubMeshTracer(t *testing.T) { idProvider := mockmodule.NewIdentityProvider(t) defer cancel() - topic1 := channels.TopicFromChannel(channels.PushBlocks, sporkId) - topic2 := channels.TopicFromChannel(channels.PushReceipts, sporkId) + channel1 := channels.PushBlocks + topic1 := channels.TopicFromChannel(channel1, sporkId) + channel2 := channels.PushReceipts + topic2 := channels.TopicFromChannel(channel2, sporkId) loggerCycle := atomic.NewInt32(0) warnLoggerCycle := atomic.NewInt32(0) @@ -176,9 +178,9 @@ func TestGossipSubMeshTracer(t *testing.T) { // all nodes except the tracerNode unsubscribe from the topic1, which triggers sending a PRUNE to the tracerNode for each unsubscription. // We expect the tracerNode to remove the otherNode1, otherNode2, and unknownNode from its mesh. - require.NoError(t, otherNode1.Unsubscribe(topic1)) - require.NoError(t, otherNode2.Unsubscribe(topic1)) - require.NoError(t, unknownNode.Unsubscribe(topic1)) + require.NoError(t, otherNode1.Unsubscribe(channel1)) + require.NoError(t, otherNode2.Unsubscribe(channel1)) + require.NoError(t, unknownNode.Unsubscribe(channel1)) assert.Eventually(t, func() bool { // eventually, the tracerNode should not have the other node in its mesh for topic1. From 6c65145af5420331d547527dfa128b4a04970b4c Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 13:25:58 -0700 Subject: [PATCH 10/35] fixes subscription package tests --- .../subscription/subscription_filter_test.go | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/network/p2p/subscription/subscription_filter_test.go b/network/p2p/subscription/subscription_filter_test.go index 0c3d1d8b88c..fdb098ea012 100644 --- a/network/p2p/subscription/subscription_filter_test.go +++ b/network/p2p/subscription/subscription_filter_test.go @@ -14,8 +14,10 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/irrecoverable" + flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2pfixtures" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/subscription" p2ptest "github.com/onflow/flow-go/network/p2p/test" @@ -78,16 +80,26 @@ func TestFilterSubscribe(t *testing.T) { wg.Add(2) testPublish := func(wg *sync.WaitGroup, from p2p.LibP2PNode, sub p2p.Subscription) { - data := []byte("hello") - err := from.Publish(context.TODO(), badTopic, data) + outgoingMessageScope, err := flownet.NewOutgoingScope( + ids.NodeIDs(), + channels.SyncCommittee, + []byte("hello"), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + + err = from.Publish(context.TODO(), outgoingMessageScope) require.NoError(t, err) ctx, cancel := context.WithTimeout(context.Background(), time.Second) msg, err := sub.Next(ctx) cancel() require.NoError(t, err) - require.Equal(t, msg.Data, data) + + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() + require.NoError(t, err) + require.Equal(t, msg.Data, expectedReceivedData) ctx, cancel = context.WithTimeout(context.Background(), time.Second) _, err = unstakedSub.Next(ctx) From da8f4751b3bb57471d3118f83045f37251c66ace Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 15:26:39 -0700 Subject: [PATCH 11/35] moves spork id to network builder --- cmd/access/node_builder/access_node_builder.go | 1 + cmd/observer/node_builder/observer_builder.go | 1 + cmd/scaffold.go | 1 + follower/follower_builder.go | 1 + network/alsp/manager/manager_test.go | 9 +++++---- network/internal/testutils/testUtil.go | 5 ++++- network/p2p/p2pnode/libp2pNode.go | 3 ++- network/test/blob_service_test.go | 2 +- network/test/echoengine_test.go | 2 +- network/test/epochtransition_test.go | 2 +- network/test/meshengine_test.go | 2 +- 11 files changed, 19 insertions(+), 10 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 55fd48ace8e..5d33051cf96 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -802,6 +802,7 @@ func (builder *FlowAccessNodeBuilder) initNetwork(nodeID module.Local, IdentityProvider: builder.IdentityProvider, ReceiveCache: receiveCache, ConduitFactory: conduit.NewDefaultConduitFactory(), + SporkId: builder.SporkID, AlspCfg: &alspmgr.MisbehaviorReportManagerConfig{ Logger: builder.Logger, SpamRecordCacheSize: builder.FlowConfig.NetworkConfig.AlspConfig.SpamRecordCacheSize, diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index a8e7abdd4b6..207e6c167be 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -493,6 +493,7 @@ func (builder *ObserverServiceBuilder) initNetwork(nodeID module.Local, IdentityProvider: builder.IdentityProvider, ReceiveCache: receiveCache, ConduitFactory: conduit.NewDefaultConduitFactory(), + SporkId: builder.SporkID, AlspCfg: &alspmgr.MisbehaviorReportManagerConfig{ Logger: builder.Logger, SpamRecordCacheSize: builder.FlowConfig.NetworkConfig.AlspConfig.SpamRecordCacheSize, diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 07e2d63011e..e7ba460a53d 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -465,6 +465,7 @@ func (fnb *FlowNodeBuilder) InitFlowNetworkWithConduitFactory( Logger: fnb.Logger, Codec: fnb.CodecFactory(), Me: fnb.Me, + SporkId: fnb.SporkID, MiddlewareFactory: func() (network.Middleware, error) { return fnb.Middleware, nil }, Topology: topology.NewFullyConnectedTopology(), SubscriptionManager: subscriptionManager, diff --git a/follower/follower_builder.go b/follower/follower_builder.go index 27e69ce039c..d03ccde45c6 100644 --- a/follower/follower_builder.go +++ b/follower/follower_builder.go @@ -380,6 +380,7 @@ func (builder *FollowerServiceBuilder) initNetwork(nodeID module.Local, IdentityProvider: builder.IdentityProvider, ReceiveCache: receiveCache, ConduitFactory: conduit.NewDefaultConduitFactory(), + SporkId: builder.SporkID, AlspCfg: &alspmgr.MisbehaviorReportManagerConfig{ Logger: builder.Logger, SpamRecordCacheSize: builder.FlowConfig.NetworkConfig.AlspConfig.SpamRecordCacheSize, diff --git a/network/alsp/manager/manager_test.go b/network/alsp/manager/manager_test.go index 8b0d925f642..9ffd384386f 100644 --- a/network/alsp/manager/manager_test.go +++ b/network/alsp/manager/manager_test.go @@ -63,7 +63,7 @@ func TestNetworkPassesReportedMisbehavior(t *testing.T) { testutils.MiddlewareConfigFixture(t, sporkId), mocknetwork.NewViolationsConsumer(t)) - networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, mws[0]) + networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, sporkId, mws[0]) net, err := p2p.NewNetwork(networkCfg, p2p.WithAlspManager(misbehaviorReportManger)) require.NoError(t, err) @@ -126,7 +126,7 @@ func TestHandleReportedMisbehavior_Cache_Integration(t *testing.T) { nodes, testutils.MiddlewareConfigFixture(t, sporkId), mocknetwork.NewViolationsConsumer(t)) - networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, mws[0], p2p.WithAlspConfig(cfg)) + networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, sporkId, mws[0], p2p.WithAlspConfig(cfg)) net, err := p2p.NewNetwork(networkCfg) require.NoError(t, err) @@ -230,7 +230,7 @@ func TestHandleReportedMisbehavior_And_DisallowListing_Integration(t *testing.T) nodes, testutils.MiddlewareConfigFixture(t, sporkId), mocknetwork.NewViolationsConsumer(t)) - networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, mws[0], p2p.WithAlspConfig(cfg)) + networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, sporkId, mws[0], p2p.WithAlspConfig(cfg)) victimNetwork, err := p2p.NewNetwork(networkCfg) require.NoError(t, err) @@ -313,6 +313,7 @@ func TestHandleReportedMisbehavior_And_SlashingViolationsConsumer_Integration(t t, *ids[0], ids, + sporkId, mws[0], p2p.WithAlspConfig(managerCfgFixture(t))) victimNetwork, err := p2p.NewNetwork(networkCfg) @@ -406,7 +407,7 @@ func TestMisbehaviorReportMetrics(t *testing.T) { sporkId := unittest.IdentifierFixture() ids, nodes := testutils.LibP2PNodeForMiddlewareFixture(t, sporkId, 1) mws, _ := testutils.MiddlewareFixtures(t, ids, nodes, testutils.MiddlewareConfigFixture(t, sporkId), mocknetwork.NewViolationsConsumer(t)) - networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, mws[0], p2p.WithAlspConfig(cfg)) + networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, sporkId, mws[0], p2p.WithAlspConfig(cfg)) net, err := p2p.NewNetwork(networkCfg) require.NoError(t, err) diff --git a/network/internal/testutils/testUtil.go b/network/internal/testutils/testUtil.go index bc4da0ea972..ddfc19f186c 100644 --- a/network/internal/testutils/testUtil.go +++ b/network/internal/testutils/testUtil.go @@ -191,6 +191,7 @@ func MiddlewareFixtures(t *testing.T, identities flow.IdentityList, libP2PNodes // NetworksFixture generates the network for the given middlewares func NetworksFixture(t *testing.T, + sporkId flow.Identifier, ids flow.IdentityList, mws []network.Middleware) []network.Network { @@ -199,7 +200,7 @@ func NetworksFixture(t *testing.T, for i := 0; i < count; i++ { - params := NetworkConfigFixture(t, *ids[i], ids, mws[i]) + params := NetworkConfigFixture(t, *ids[i], ids, sporkId, mws[i]) net, err := p2p.NewNetwork(params) require.NoError(t, err) @@ -213,6 +214,7 @@ func NetworkConfigFixture( t *testing.T, myId flow.Identity, allIds flow.IdentityList, + sporkId flow.Identifier, mw network.Middleware, opts ...p2p.NetworkConfigOption) *p2p.NetworkConfig { @@ -240,6 +242,7 @@ func NetworkConfigFixture( IdentityProvider: id.NewFixedIdentityProvider(allIds), ReceiveCache: receiveCache, ConduitFactory: conduit.NewDefaultConduitFactory(), + SporkId: sporkId, AlspCfg: &alspmgr.MisbehaviorReportManagerConfig{ Logger: unittest.Logger(), SpamRecordCacheSize: defaultFlowConfig.NetworkConfig.AlspConfig.SpamRecordCacheSize, diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index e2362e3fe69..18a1e89f5be 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -344,6 +344,7 @@ func (n *Node) unsubscribeTopic(topic channels.Topic) error { func (n *Node) Publish(ctx context.Context, msgScope *flownet.OutgoingMessageScope) error { lg := n.logger.With(). Str("channel", msgScope.Channel().String()). + Hex("spork_id", logging.ID(msgScope.SporkId())). Interface("proto_message", msgScope.Proto()). Str("payload_type", msgScope.PayloadType()). Int("message_size", msgScope.Size()).Logger() @@ -362,7 +363,7 @@ func (n *Node) Publish(ctx context.Context, msgScope *flownet.OutgoingMessageSco return fmt.Errorf("message size %d exceeds configured max message size %d", msgSize, DefaultMaxPubSubMsgSize) } - topic := channels.TopicFromChannel(msgScope.Channel(), n.sporkId) + topic := channels.TopicFromChannel(msgScope.Channel(), msgScope.SporkId()) lg = lg.With().Str("topic", topic.String()).Logger() ps, found := n.topics[topic] diff --git a/network/test/blob_service_test.go b/network/test/blob_service_test.go index 3a401ad4b0b..f42d74ab140 100644 --- a/network/test/blob_service_test.go +++ b/network/test/blob_service_test.go @@ -99,7 +99,7 @@ func (suite *BlobServiceTestSuite) SetupTest() { nodes, testutils.MiddlewareConfigFixture(suite.T(), sporkId), mocknetwork.NewViolationsConsumer(suite.T())) - suite.networks = testutils.NetworksFixture(suite.T(), ids, mws) + suite.networks = testutils.NetworksFixture(suite.T(), sporkId, ids, mws) testutils.StartNodesAndNetworks(signalerCtx, suite.T(), nodes, suite.networks, 100*time.Millisecond) blobExchangeChannel := channels.Channel("blob-exchange") diff --git a/network/test/echoengine_test.go b/network/test/echoengine_test.go index 3c544279b6f..e329a83e780 100644 --- a/network/test/echoengine_test.go +++ b/network/test/echoengine_test.go @@ -62,7 +62,7 @@ func (suite *EchoEngineTestSuite) SetupTest() { nodes, testutils.MiddlewareConfigFixture(suite.T(), sporkId), mocknetwork.NewViolationsConsumer(suite.T())) - suite.nets = testutils.NetworksFixture(suite.T(), suite.ids, suite.mws) + suite.nets = testutils.NetworksFixture(suite.T(), sporkId, suite.ids, suite.mws) testutils.StartNodesAndNetworks(signalerCtx, suite.T(), nodes, suite.nets, 100*time.Millisecond) } diff --git a/network/test/epochtransition_test.go b/network/test/epochtransition_test.go index 46eb1ae1b79..0632f2a9a44 100644 --- a/network/test/epochtransition_test.go +++ b/network/test/epochtransition_test.go @@ -201,7 +201,7 @@ func (suite *MutableIdentityTableSuite) addNodes(count int) { nodes, testutils.MiddlewareConfigFixture(suite.T(), sporkId), mocknetwork.NewViolationsConsumer(suite.T())) - nets := testutils.NetworksFixture(suite.T(), ids, mws) + nets := testutils.NetworksFixture(suite.T(), sporkId, ids, mws) suite.cancels = append(suite.cancels, cancel) testutils.StartNodesAndNetworks(signalerCtx, suite.T(), nodes, nets, 100*time.Millisecond) diff --git a/network/test/meshengine_test.go b/network/test/meshengine_test.go index e7a5684f2f9..c2e2311a5af 100644 --- a/network/test/meshengine_test.go +++ b/network/test/meshengine_test.go @@ -108,7 +108,7 @@ func (suite *MeshEngineTestSuite) SetupTest() { libP2PNodes, testutils.MiddlewareConfigFixture(suite.T(), sporkId), mocknetwork.NewViolationsConsumer(suite.T())) - suite.nets = testutils.NetworksFixture(suite.T(), suite.ids, suite.mws) + suite.nets = testutils.NetworksFixture(suite.T(), sporkId, suite.ids, suite.mws) testutils.StartNodesAndNetworks(signalerCtx, suite.T(), libP2PNodes, suite.nets, 100*time.Millisecond) for _, observableConnMgr := range tagObservables { From f01cf4110100d2c4e15d914410303191513b2a85 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 15:27:07 -0700 Subject: [PATCH 12/35] replaces channel with topic on outgoing message scope --- network/message_scope.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/network/message_scope.go b/network/message_scope.go index 3db13a1b2bd..18831a1f662 100644 --- a/network/message_scope.go +++ b/network/message_scope.go @@ -89,7 +89,7 @@ func (m IncomingMessageScope) PayloadType() string { // OutgoingMessageScope captures the context around an outgoing message that is about to be sent. type OutgoingMessageScope struct { targetIds flow.IdentifierList // the target node IDs. - channelId channels.Channel // the channel ID. + topic channels.Topic // the topic, i.e., channel-id/spork-id. payload interface{} // the payload to be sent. encoder func(interface{}) ([]byte, error) // the encoder to encode the payload. msg *message.Message // raw proto message sent on wire. @@ -103,13 +103,13 @@ type OutgoingMessageScope struct { // should have exactly one target ID, while pubsub messages should have at least one target ID). func NewOutgoingScope( targetIds flow.IdentifierList, - channelId channels.Channel, + topic channels.Topic, payload interface{}, encoder func(interface{}) ([]byte, error), protocolType message.ProtocolType) (*OutgoingMessageScope, error) { scope := &OutgoingMessageScope{ targetIds: targetIds, - channelId: channelId, + topic: topic, payload: payload, encoder: encoder, protocol: protocolType, @@ -148,8 +148,8 @@ func (o OutgoingMessageScope) PayloadType() string { return MessageType(o.payload) } -func (o OutgoingMessageScope) Channel() channels.Channel { - return o.channelId +func (o OutgoingMessageScope) Topic() channels.Topic { + return o.topic } // buildMessage builds the raw proto message to be sent on the wire. @@ -165,9 +165,14 @@ func (o OutgoingMessageScope) buildMessage() (*message.Message, error) { emTargets = append(emTargets, tempID[:]) } + channel, ok := channels.ChannelFromTopic(o.topic) + if !ok { + return nil, fmt.Errorf("could not convert topic to channel: %s", o.topic) + } + return &message.Message{ TargetIDs: emTargets, - ChannelID: o.channelId.String(), + ChannelID: channel.String(), Payload: payload, }, nil } From 2e21367d68582dc43447bc216449d2fe3d9f7e61 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 15:28:06 -0700 Subject: [PATCH 13/35] fixes errors in test fixtures --- network/p2p/test/fixtures.go | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 598bbe2693d..c5684a18480 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -583,17 +583,13 @@ func EnsurePubsubMessageExchange(t *testing.T, ctx context.Context, nodes []p2p. // let subscriptions propagate time.Sleep(1 * time.Second) - // TODO: remove this and use channel as an argument - channel, ok := channels.ChannelFromTopic(topic) - require.True(t, ok) - for _, node := range nodes { for i := 0; i < count; i++ { // creates a unique message to be published by the node payload := messageFactory() outgoingMessageScope, err := flownet.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, - channel, + topic, payload, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) @@ -630,16 +626,12 @@ func EnsurePubsubMessageExchangeFromNode(t *testing.T, ctx context.Context, send // let subscriptions propagate time.Sleep(1 * time.Second) - // TODO: remove this and use channel as an argument - channel, ok := channels.ChannelFromTopic(topic) - require.True(t, ok) - for i := 0; i < count; i++ { // creates a unique message to be published by the node payload := messageFactory() outgoingMessageScope, err := flownet.NewOutgoingScope( flow.IdentifierList{}, - channel, + topic, payload, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) @@ -719,14 +711,11 @@ func EnsureNoPubsubMessageExchange( wg.Add(1) go func() { // creates a unique message to be published by the node. - // TODO: remove this and use channel as an argument - channel, ok := channels.ChannelFromTopic(topic) - require.True(t, ok) payload := messageFactory() outgoingMessageScope, err := flownet.NewOutgoingScope( toIdentifiers, - channel, + topic, payload, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) From 9382f64b849b9712e0724d0b0fc1c85757f33d12 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 16 Aug 2023 15:54:32 -0700 Subject: [PATCH 14/35] fixes tests --- network/p2p/network.go | 20 ++- network/p2p/p2pnode/libp2pNode.go | 12 +- network/p2p/test/sporking_test.go | 109 +++++++-------- network/p2p/test/topic_validator_test.go | 161 +++++++++++++++++------ 4 files changed, 185 insertions(+), 117 deletions(-) diff --git a/network/p2p/network.go b/network/p2p/network.go index a288b92c7d7..f564aa0340d 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -40,6 +40,7 @@ var NotEjectedFilter = filter.Not(filter.Ejected) type Network struct { sync.RWMutex *component.ComponentManager + sporkId flow.Identifier identityProvider module.IdentityProvider logger zerolog.Logger codec network.Codec @@ -99,6 +100,7 @@ type NetworkConfig struct { ReceiveCache *netcache.ReceiveCache ConduitFactory network.ConduitFactory AlspCfg *alspmgr.MisbehaviorReportManagerConfig + SporkId flow.Identifier } // NetworkConfigOption is a function that can be used to override network config parmeters. @@ -167,6 +169,7 @@ func NewNetwork(param *NetworkConfig, opts ...NetworkOption) (*Network, error) { registerEngineRequests: make(chan *registerEngineRequest), registerBlobServiceRequests: make(chan *registerBlobServiceRequest), misbehaviorReportManager: misbehaviorMngr, + sporkId: param.SporkId, } for _, opt := range opts { @@ -428,7 +431,7 @@ func (n *Network) UnicastOnChannel(channel channels.Channel, payload interface{} msg, err := network.NewOutgoingScope( flow.IdentifierList{targetID}, - channel, + channels.TopicFromChannel(channel, n.sporkId), payload, n.codec.Encode, message.ProtocolTypeUnicast) @@ -436,14 +439,14 @@ func (n *Network) UnicastOnChannel(channel channels.Channel, payload interface{} return fmt.Errorf("could not generate outgoing message scope for unicast: %w", err) } - n.metrics.UnicastMessageSendingStarted(msg.Channel().String()) - defer n.metrics.UnicastMessageSendingCompleted(msg.Channel().String()) + n.metrics.UnicastMessageSendingStarted(channel.String()) + defer n.metrics.UnicastMessageSendingCompleted(channel.String()) err = n.mw.SendDirect(msg) if err != nil { return fmt.Errorf("failed to send message to %x: %w", targetID, err) } - n.metrics.OutboundMessageSent(msg.Size(), msg.Channel().String(), message.ProtocolTypeUnicast.String(), msg.PayloadType()) + n.metrics.OutboundMessageSent(msg.Size(), channel.String(), message.ProtocolTypeUnicast.String(), msg.PayloadType()) return nil } @@ -506,7 +509,12 @@ func (n *Network) sendOnChannel(channel channels.Channel, msg interface{}, targe Msg("sending new message on channel") // generate network message (encoding) based on list of recipients - scope, err := network.NewOutgoingScope(targetIDs, channel, msg, n.codec.Encode, message.ProtocolTypePubSub) + scope, err := network.NewOutgoingScope( + targetIDs, + channels.TopicFromChannel(channel, n.sporkId), + msg, + n.codec.Encode, + message.ProtocolTypePubSub) if err != nil { return fmt.Errorf("failed to generate outgoing message scope %s: %w", channel, err) } @@ -518,7 +526,7 @@ func (n *Network) sendOnChannel(channel channels.Channel, msg interface{}, targe return fmt.Errorf("failed to send message on channel %s: %w", channel, err) } - n.metrics.OutboundMessageSent(scope.Size(), scope.Channel().String(), message.ProtocolTypePubSub.String(), scope.PayloadType()) + n.metrics.OutboundMessageSent(scope.Size(), channel.String(), message.ProtocolTypePubSub.String(), scope.PayloadType()) return nil } diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 18a1e89f5be..ed98a658b69 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -343,8 +343,7 @@ func (n *Node) unsubscribeTopic(topic channels.Topic) error { // All errors returned from this function can be considered benign. func (n *Node) Publish(ctx context.Context, msgScope *flownet.OutgoingMessageScope) error { lg := n.logger.With(). - Str("channel", msgScope.Channel().String()). - Hex("spork_id", logging.ID(msgScope.SporkId())). + Str("topic", msgScope.Topic().String()). Interface("proto_message", msgScope.Proto()). Str("payload_type", msgScope.PayloadType()). Int("message_size", msgScope.Size()).Logger() @@ -363,16 +362,13 @@ func (n *Node) Publish(ctx context.Context, msgScope *flownet.OutgoingMessageSco return fmt.Errorf("message size %d exceeds configured max message size %d", msgSize, DefaultMaxPubSubMsgSize) } - topic := channels.TopicFromChannel(msgScope.Channel(), msgScope.SporkId()) - lg = lg.With().Str("topic", topic.String()).Logger() - - ps, found := n.topics[topic] + ps, found := n.topics[msgScope.Topic()] if !found { - return fmt.Errorf("could not find topic (%s)", topic) + return fmt.Errorf("could not find topic (%s)", msgScope.Topic()) } err = ps.Publish(ctx, data) if err != nil { - return fmt.Errorf("could not publish to topic (%s): %w", topic, err) + return fmt.Errorf("could not publish to topic (%s): %w", msgScope.Topic(), err) } lg.Debug().Msg("published message to topic") diff --git a/network/p2p/test/sporking_test.go b/network/p2p/test/sporking_test.go index b7aae10d32f..d637114366e 100644 --- a/network/p2p/test/sporking_test.go +++ b/network/p2p/test/sporking_test.go @@ -237,54 +237,9 @@ func TestOneToKCrosstalkPrevention(t *testing.T) { time.Sleep(time.Second) // assert that node 1 can successfully send a message to node 2 via PubSub - testOneToKMessagingSucceeds(ctx, t, node1, sub2, topicBeforeSpork) - - // new root id after spork - rootIDAfterSpork := unittest.IdentifierFixture() - - // topic after the spork - topicAfterSpork := channels.TopicFromChannel(channels.TestNetworkChannel, rootIDAfterSpork) - - // mimic that node1 is now part of the new spork while node2 remains on the old spork - // by unsubscribing node1 from 'topicBeforeSpork' and subscribing it to 'topicAfterSpork' - // and keeping node2 subscribed to topic 'topicBeforeSpork' - err = node1.Unsubscribe(topicBeforeSpork) - require.NoError(t, err) - _, err = node1.Subscribe(topicAfterSpork, topicValidator) - require.NoError(t, err) - - // assert that node 1 can no longer send a message to node 2 via PubSub - testOneToKMessagingFails(ctx, t, node1, sub2, topicAfterSpork) -} - -func testOneToOneMessagingSucceeds(t *testing.T, sourceNode p2p.LibP2PNode, peerInfo peer.AddrInfo) { - // create stream from node 1 to node 2 - sourceNode.Host().Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) - s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) - // assert that stream creation succeeded - require.NoError(t, err) - assert.NotNil(t, s) -} - -func testOneToOneMessagingFails(t *testing.T, sourceNode p2p.LibP2PNode, peerInfo peer.AddrInfo) { - // create stream from source node to destination address - sourceNode.Host().Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) - _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) - // assert that stream creation failed - assert.Error(t, err) - // assert that it failed with the expected error - assert.Regexp(t, ".*failed to negotiate security protocol.*|.*protocols not supported.*", err) -} - -func testOneToKMessagingSucceeds(ctx context.Context, - t *testing.T, - sourceNode p2p.LibP2PNode, - dstnSub p2p.Subscription, - topic channels.Topic) { - - sentMsg, err := network.NewOutgoingScope( + outgoingMessageScope, err := network.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, - channels.TestNetworkChannel, + topicBeforeSpork, &libp2pmessage.TestMessage{ Text: string("hello"), }, @@ -292,32 +247,40 @@ func testOneToKMessagingSucceeds(ctx context.Context, message.ProtocolTypePubSub) require.NoError(t, err) - sentData, err := sentMsg.Proto().Marshal() + expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() require.NoError(t, err) // send a 1-k message from source node to destination node - err = sourceNode.Publish(ctx, topic, sentData) + err = node1.Publish(ctx, outgoingMessageScope) require.NoError(t, err) // assert that the message is received by the destination node unittest.AssertReturnsBefore(t, func() { - msg, err := dstnSub.Next(ctx) + msg, err := sub2.Next(ctx) require.NoError(t, err) - assert.Equal(t, sentData, msg.Data) + assert.Equal(t, expectedReceivedData, msg.Data) }, // libp2p hearbeats every second, so at most the message should take 1 second 2*time.Second) -} -func testOneToKMessagingFails(ctx context.Context, - t *testing.T, - sourceNode p2p.LibP2PNode, - dstnSub p2p.Subscription, - topic channels.Topic) { + // new root id after spork + rootIDAfterSpork := unittest.IdentifierFixture() - sentMsg, err := network.NewOutgoingScope( - flow.IdentifierList{unittest.IdentifierFixture()}, - channels.TestNetworkChannel, + // topic after the spork + topicAfterSpork := channels.TopicFromChannel(channels.TestNetworkChannel, rootIDAfterSpork) + + // mimic that node1 is now part of the new spork while node2 remains on the old spork + // by unsubscribing node1 from 'topicBeforeSpork' and subscribing it to 'topicAfterSpork' + // and keeping node2 subscribed to topic 'topicBeforeSpork' + err = node1.Unsubscribe(channels.TestNetworkChannel) + require.NoError(t, err) + _, err = node1.Subscribe(topicAfterSpork, topicValidator) + require.NoError(t, err) + + // assert that node 1 can no longer send a message to node 2 via PubSub + outgoingMessageScope, err = network.NewOutgoingScope( + flow.IdentifierList{id2.NodeID}, + topicAfterSpork, &libp2pmessage.TestMessage{ Text: string("hello"), }, @@ -325,18 +288,34 @@ func testOneToKMessagingFails(ctx context.Context, message.ProtocolTypePubSub) require.NoError(t, err) - sentData, err := sentMsg.Proto().Marshal() - require.NoError(t, err) - // send a 1-k message from source node to destination node - err = sourceNode.Publish(ctx, topic, sentData) + err = node1.Publish(ctx, outgoingMessageScope) require.NoError(t, err) // assert that the message is never received by the destination node _ = unittest.RequireNeverReturnBefore(t, func() { - _, _ = dstnSub.Next(ctx) + _, _ = sub2.Next(ctx) }, // libp2p hearbeats every second, so at most the message should take 1 second 2*time.Second, "nodes on different sporks were able to communicate") } + +func testOneToOneMessagingSucceeds(t *testing.T, sourceNode p2p.LibP2PNode, peerInfo peer.AddrInfo) { + // create stream from node 1 to node 2 + sourceNode.Host().Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) + s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) + // assert that stream creation succeeded + require.NoError(t, err) + assert.NotNil(t, s) +} + +func testOneToOneMessagingFails(t *testing.T, sourceNode p2p.LibP2PNode, peerInfo peer.AddrInfo) { + // create stream from source node to destination address + sourceNode.Host().Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) + _, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) + // assert that stream creation failed + assert.Error(t, err) + // assert that it failed with the expected error + assert.Regexp(t, ".*failed to negotiate security protocol.*|.*protocols not supported.*", err) +} diff --git a/network/p2p/test/topic_validator_test.go b/network/p2p/test/topic_validator_test.go index 5a7e402b141..e3d47c7c709 100644 --- a/network/p2p/test/topic_validator_test.go +++ b/network/p2p/test/topic_validator_test.go @@ -93,10 +93,16 @@ func TestTopicValidator_Unstaked(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - // create a dummy block proposal to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channel) - err = sn2.Publish(timedCtx, topic, data1) + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + + err = sn2.Publish(timedCtx, outgoingMessageScope1) require.NoError(t, err) // sn1 should not receive message from sn2 because sn2 is unstaked @@ -146,10 +152,16 @@ func TestTopicValidator_PublicChannel(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - // create a dummy sync request to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, &messages.SyncRequest{Nonce: 0, Height: 0}, channel) - err = sn2.Publish(timedCtx, topic, data1) + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + &messages.SyncRequest{Nonce: 0, Height: 0}, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + + err = sn2.Publish(timedCtx, outgoingMessageScope1) require.NoError(t, err) var wg sync.WaitGroup @@ -158,11 +170,14 @@ func TestTopicValidator_PublicChannel(t *testing.T) { timedCtx, cancel1s := context.WithTimeout(ctx, time.Second) defer cancel1s() + expectedReceivedData, err := outgoingMessageScope1.Proto().Marshal() + require.NoError(t, err) + // sn1 gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub1) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData, sub1) // sn2 also gets the message (as part of the libp2p loopback of published topic messages) - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub2) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData, sub2) unittest.RequireReturnsBefore(t, wg.Wait, 5*time.Second, "could not receive message on time") } @@ -208,11 +223,20 @@ func TestTopicValidator_TopicMismatch(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() + // create a dummy block proposal to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channels.Channel("invalid-channel")) + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) - err = sn2.Publish(timedCtx, topic, data1) + // intentionally overriding the channel id to be different from the topic + outgoingMessageScope1.Proto().ChannelID = channels.PublicSyncCommittee.String() + err = sn2.Publish(timedCtx, outgoingMessageScope1) // publish fails because the channel validation fails require.Error(t, err) @@ -238,7 +262,7 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { p2ptest.StartNodes(t, signalerCtx, nodes, 100*time.Millisecond) defer p2ptest.StopNodes(t, nodes, cancel, 100*time.Millisecond) - topic := channels.Topic("invalid-topic") + topic := channels.TopicFromChannel(channels.ConsensusCommittee, sporkId) pInfo2, err := utils.PeerAddressInfo(identity2) require.NoError(t, err) @@ -261,9 +285,19 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() // create a dummy block proposal to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channels.PushBlocks) + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + + // intentionally overriding the channel id to be a valid channel id (though the topic is invalid) + // hence imitating a message that was published to the wrong topic + outgoingMessageScope1.Proto().ChannelID = channels.PushBlocks.String() - err = sn2.Publish(timedCtx, topic, data1) + err = sn2.Publish(timedCtx, outgoingMessageScope1) // publish fails because the topic conversion fails require.Error(t, err) @@ -345,34 +379,52 @@ func TestAuthorizedSenderValidator_Unauthorized(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 60*time.Second) defer cancel5s() - // create a dummy block proposal to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channel) // sn2 publishes the block proposal, sn1 and an1 should receive the message because // SN nodes are authorized to send block proposals - err = sn2.Publish(timedCtx, topic, data1) + // create a dummy block proposal to publish from our SN node + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + err = sn2.Publish(timedCtx, outgoingMessageScope1) + require.NoError(t, err) + + expectedReceivedData1, err := outgoingMessageScope1.Proto().Marshal() require.NoError(t, err) // sn1 gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub1) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub1) // sn2 also gets the message (as part of the libp2p loopback of published topic messages) - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub2) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub2) // an1 also gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub3) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub3) timedCtx, cancel2s := context.WithTimeout(ctx, 2*time.Second) defer cancel2s() - data2 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channel) // the access node now publishes the block proposal message, AN are not authorized to publish block proposals // the message should be rejected by the topic validator on sn1 - err = an1.Publish(timedCtx, topic, data2) + outgoingMessageScope2, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + err = an1.Publish(timedCtx, outgoingMessageScope2) + require.NoError(t, err) + + expectedReceivedData2, err := outgoingMessageScope2.Proto().Marshal() require.NoError(t, err) // an1 receives its own message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data2, sub3) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData2, sub3) var wg sync.WaitGroup @@ -449,11 +501,17 @@ func TestAuthorizedSenderValidator_InvalidMsg(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - // create a dummy block proposal to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channel) + // create a dummy block proposal to publish from our SN node // sn2 publishes the block proposal on the sync committee channel - err = sn2.Publish(timedCtx, topic, data1) + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + err = sn2.Publish(timedCtx, outgoingMessageScope1) require.NoError(t, err) // sn1 should not receive message from sn2 @@ -532,29 +590,46 @@ func TestAuthorizedSenderValidator_Ejected(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - // create a dummy block proposal to publish from our SN node - data1 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channel) // sn2 publishes the block proposal, sn1 and an1 should receive the message because // SN nodes are authorized to send block proposals - err = sn2.Publish(timedCtx, topic, data1) + // create a dummy block proposal to publish from our SN node + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + err = sn2.Publish(timedCtx, outgoingMessageScope1) + require.NoError(t, err) + + expectedReceivedData1, err := outgoingMessageScope1.Proto().Marshal() require.NoError(t, err) // sn1 gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub1) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub1) // sn2 also gets the message (as part of the libp2p loopback of published topic messages) - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub2) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub2) // an1 also gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data1, sub3) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub3) // "eject" sn2 to ensure messages published by ejected nodes get rejected identity2.Ejected = true - data3 := p2pfixtures.MustEncodeEvent(t, unittest.ProposalFixture(), channel) + + outgoingMessageScope3, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + unittest.ProposalFixture(), + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) + timedCtx, cancel2s := context.WithTimeout(ctx, time.Second) defer cancel2s() - err = sn2.Publish(timedCtx, topic, data3) + err = sn2.Publish(timedCtx, outgoingMessageScope3) require.NoError(t, err) // sn1 should not receive rejected message from ejected sn2 @@ -627,19 +702,29 @@ func TestAuthorizedSenderValidator_ClusterChannel(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() + // create a dummy sync request to publish from our LN node - data := p2pfixtures.MustEncodeEvent(t, &messages.RangeRequest{}, channel) + outgoingMessageScope1, err := network.NewOutgoingScope( + flow.IdentifierList{identity1.NodeID, identity2.NodeID}, + topic, + &messages.RangeRequest{}, + unittest.NetworkCodec().Encode, + message.ProtocolTypePubSub) + require.NoError(t, err) // ln2 publishes the sync request on the cluster channel - err = ln2.Publish(timedCtx, topic, data) + err = ln2.Publish(timedCtx, outgoingMessageScope1) + require.NoError(t, err) + + expectedReceivedData1, err := outgoingMessageScope1.Proto().Marshal() require.NoError(t, err) // ln1 gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data, sub1) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub1) // ln2 also gets the message (as part of the libp2p loopback of published topic messages) - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data, sub2) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub2) // ln3 also gets the message - p2pfixtures.SubMustReceiveMessage(t, timedCtx, data, sub3) + p2pfixtures.SubMustReceiveMessage(t, timedCtx, expectedReceivedData1, sub3) } From e3dee97d7556659a7a4e240ac4d1f448845b36a1 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 09:43:01 -0700 Subject: [PATCH 15/35] adds message space interface --- network/message_scope.go | 52 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/network/message_scope.go b/network/message_scope.go index 18831a1f662..fe9d3acb291 100644 --- a/network/message_scope.go +++ b/network/message_scope.go @@ -204,3 +204,55 @@ func EventId(channel channels.Channel, payload []byte) (hash.Hash, error) { func MessageType(decodedPayload interface{}) string { return strings.TrimLeft(fmt.Sprintf("%T", decodedPayload), "*") } + +// IncomingMessageScoper defines the interface for handling incoming message scope. +type IncomingMessageScoper interface { + // OriginId returns the origin node ID. + OriginId() flow.Identifier + + // Proto returns the raw message received. + Proto() *message.Message + + // DecodedPayload returns the decoded payload of the message. + DecodedPayload() interface{} + + // Protocol returns the type of protocol used to receive the message. + Protocol() message.ProtocolType + + // Channel returns the channel of the message. + Channel() channels.Channel + + // Size returns the size of the message. + Size() int + + // TargetIDs returns the target node IDs, i.e., the intended recipients. + TargetIDs() flow.IdentifierList + + // EventID returns the hash of the payload and channel. + EventID() []byte + + // PayloadType returns the type of the decoded payload. + PayloadType() string +} + +// OutgoingMessageScoper defines the interface for handling outgoing message scope. +type OutgoingMessageScoper interface { + // TargetIds returns the target node IDs. + TargetIds() flow.IdentifierList + + // Size returns the size of the message. + Size() int + + // PayloadType returns the type of the payload to be sent. + PayloadType() string + + // Topic returns the topic, i.e., channel-id/spork-id. + Topic() channels.Topic + + // Proto returns the raw proto message sent on the wire. + Proto() *message.Message +} + +// Ensure structs implement the interfaces +var _ IncomingMessageScoper = &IncomingMessageScope{} +var _ OutgoingMessageScoper = &OutgoingMessageScope{} From 026f9e66c9ee56729e98719f211821165801482b Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 09:57:57 -0700 Subject: [PATCH 16/35] moves message scopes to implementation packages --- network/message/message_scope.go | 176 ++++++++++++++++++ network/message_scope.go | 170 +---------------- network/middleware.go | 7 +- network/mocknetwork/message_validator.go | 7 +- network/mocknetwork/middleware.go | 10 +- network/mocknetwork/overlay.go | 8 +- network/p2p/dht/dht_test.go | 3 +- network/p2p/libp2pNode.go | 3 +- network/p2p/middleware/middleware.go | 8 +- network/p2p/middleware/middleware_test.go | 3 +- network/p2p/mock/lib_p2_p_node.go | 5 +- network/p2p/network.go | 8 +- network/p2p/p2pnode/libp2pNode.go | 3 +- network/p2p/scoring/app_score_test.go | 5 +- .../scoring/subscription_validator_test.go | 11 +- .../subscription/subscription_filter_test.go | 3 +- network/p2p/test/fixtures.go | 6 +- network/p2p/test/sporking_test.go | 5 +- network/p2p/test/topic_validator_test.go | 20 +- .../ratelimit/bandwidth_rate_limiter_test.go | 5 +- network/test/middleware_test.go | 40 ++-- network/test/unicast_authorization_test.go | 22 +-- network/validator.go | 4 +- network/validator/any_validator.go | 3 +- network/validator/not_validator.go | 3 +- network/validator/origin_validator.go | 3 +- network/validator/sender_validator.go | 3 +- network/validator/target_validator.go | 3 +- 28 files changed, 282 insertions(+), 265 deletions(-) create mode 100644 network/message/message_scope.go diff --git a/network/message/message_scope.go b/network/message/message_scope.go new file mode 100644 index 00000000000..aa15c0112a2 --- /dev/null +++ b/network/message/message_scope.go @@ -0,0 +1,176 @@ +package message + +import ( + "fmt" + + "github.com/onflow/flow-go/crypto/hash" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/channels" +) + +// IncomingMessageScope captures the context around an incoming message that is received by the network layer. +type IncomingMessageScope struct { + originId flow.Identifier // the origin node ID. + targetIds flow.IdentifierList // the target node IDs (i.e., intended recipients). + eventId hash.Hash // hash of the payload and channel. + msg *Message // the raw message received. + decodedPayload interface{} // decoded payload of the message. + protocol ProtocolType // the type of protocol used to receive the message. +} + +// NewIncomingScope creates a new incoming message scope. +// All errors returned by this function are benign and should not cause the node to crash, especially that it is not +// safe to crash the node when receiving a message. +// It errors if event id (i.e., hash of the payload and channel) cannot be computed, or if it fails to +// convert the target IDs from bytes slice to a flow.IdentifierList. +func NewIncomingScope(originId flow.Identifier, protocol ProtocolType, msg *Message, decodedPayload interface{}) (*IncomingMessageScope, error) { + eventId, err := network.EventId(channels.Channel(msg.ChannelID), msg.Payload) + if err != nil { + return nil, fmt.Errorf("could not compute event id: %w", err) + } + + targetIds, err := flow.ByteSlicesToIds(msg.TargetIDs) + if err != nil { + return nil, fmt.Errorf("could not convert target ids: %w", err) + } + return &IncomingMessageScope{ + eventId: eventId, + originId: originId, + msg: msg, + decodedPayload: decodedPayload, + protocol: protocol, + targetIds: targetIds, + }, nil +} + +func (m IncomingMessageScope) OriginId() flow.Identifier { + return m.originId +} + +func (m IncomingMessageScope) Proto() *Message { + return m.msg +} + +func (m IncomingMessageScope) DecodedPayload() interface{} { + return m.decodedPayload +} + +func (m IncomingMessageScope) Protocol() ProtocolType { + return m.protocol +} + +func (m IncomingMessageScope) Channel() channels.Channel { + return channels.Channel(m.msg.ChannelID) +} + +func (m IncomingMessageScope) Size() int { + return m.msg.Size() +} + +func (m IncomingMessageScope) TargetIDs() flow.IdentifierList { + return m.targetIds +} + +func (m IncomingMessageScope) EventID() []byte { + return m.eventId[:] +} + +func (m IncomingMessageScope) PayloadType() string { + return network.MessageType(m.decodedPayload) +} + +// OutgoingMessageScope captures the context around an outgoing message that is about to be sent. +type OutgoingMessageScope struct { + targetIds flow.IdentifierList // the target node IDs. + topic channels.Topic // the topic, i.e., channel-id/spork-id. + payload interface{} // the payload to be sent. + encoder func(interface{}) ([]byte, error) // the encoder to encode the payload. + msg *Message // raw proto message sent on wire. + protocol ProtocolType // the type of protocol used to send the message. +} + +// NewOutgoingScope creates a new outgoing message scope. +// All errors returned by this function are benign and should not cause the node to crash. +// It errors if the encoder fails to encode the payload into a protobuf message, or +// if the number of target IDs does not match the protocol type (i.e., unicast messages +// should have exactly one target ID, while pubsub messages should have at least one target ID). +func NewOutgoingScope( + targetIds flow.IdentifierList, + topic channels.Topic, + payload interface{}, + encoder func(interface{}) ([]byte, error), + protocolType ProtocolType) (*OutgoingMessageScope, error) { + scope := &OutgoingMessageScope{ + targetIds: targetIds, + topic: topic, + payload: payload, + encoder: encoder, + protocol: protocolType, + } + + if protocolType == ProtocolTypeUnicast { + // for unicast messages, we should have exactly one target. + if len(targetIds) != 1 { + return nil, fmt.Errorf("expected exactly one target id for unicast message, got: %d", len(targetIds)) + } + } + if protocolType == ProtocolTypePubSub { + // for pubsub messages, we should have at least one target. + if len(targetIds) == 0 { + return nil, fmt.Errorf("expected at least one target id for pubsub message, got: %d", len(targetIds)) + } + } + + msg, err := scope.buildMessage() + if err != nil { + return nil, fmt.Errorf("could not build message: %w", err) + } + scope.msg = msg + return scope, nil +} + +func (o OutgoingMessageScope) TargetIds() flow.IdentifierList { + return o.targetIds +} + +func (o OutgoingMessageScope) Size() int { + return o.msg.Size() +} + +func (o OutgoingMessageScope) PayloadType() string { + return network.MessageType(o.payload) +} + +func (o OutgoingMessageScope) Topic() channels.Topic { + return o.topic +} + +// buildMessage builds the raw proto message to be sent on the wire. +func (o OutgoingMessageScope) buildMessage() (*Message, error) { + payload, err := o.encoder(o.payload) + if err != nil { + return nil, fmt.Errorf("could not encode payload: %w", err) + } + + emTargets := make([][]byte, 0) + for _, targetId := range o.targetIds { + tempID := targetId // avoid capturing loop variable + emTargets = append(emTargets, tempID[:]) + } + + channel, ok := channels.ChannelFromTopic(o.topic) + if !ok { + return nil, fmt.Errorf("could not convert topic to channel: %s", o.topic) + } + + return &Message{ + TargetIDs: emTargets, + ChannelID: channel.String(), + Payload: payload, + }, nil +} + +func (o OutgoingMessageScope) Proto() *Message { + return o.msg +} diff --git a/network/message_scope.go b/network/message_scope.go index fe9d3acb291..d39f7bc830f 100644 --- a/network/message_scope.go +++ b/network/message_scope.go @@ -15,172 +15,6 @@ const ( eventIDPackingPrefix = "libp2ppacking" ) -// IncomingMessageScope captures the context around an incoming message that is received by the network layer. -type IncomingMessageScope struct { - originId flow.Identifier // the origin node ID. - targetIds flow.IdentifierList // the target node IDs (i.e., intended recipients). - eventId hash.Hash // hash of the payload and channel. - msg *message.Message // the raw message received. - decodedPayload interface{} // decoded payload of the message. - protocol message.ProtocolType // the type of protocol used to receive the message. -} - -// NewIncomingScope creates a new incoming message scope. -// All errors returned by this function are benign and should not cause the node to crash, especially that it is not -// safe to crash the node when receiving a message. -// It errors if event id (i.e., hash of the payload and channel) cannot be computed, or if it fails to -// convert the target IDs from bytes slice to a flow.IdentifierList. -func NewIncomingScope(originId flow.Identifier, protocol message.ProtocolType, msg *message.Message, decodedPayload interface{}) (*IncomingMessageScope, error) { - eventId, err := EventId(channels.Channel(msg.ChannelID), msg.Payload) - if err != nil { - return nil, fmt.Errorf("could not compute event id: %w", err) - } - - targetIds, err := flow.ByteSlicesToIds(msg.TargetIDs) - if err != nil { - return nil, fmt.Errorf("could not convert target ids: %w", err) - } - return &IncomingMessageScope{ - eventId: eventId, - originId: originId, - msg: msg, - decodedPayload: decodedPayload, - protocol: protocol, - targetIds: targetIds, - }, nil -} - -func (m IncomingMessageScope) OriginId() flow.Identifier { - return m.originId -} - -func (m IncomingMessageScope) Proto() *message.Message { - return m.msg -} - -func (m IncomingMessageScope) DecodedPayload() interface{} { - return m.decodedPayload -} - -func (m IncomingMessageScope) Protocol() message.ProtocolType { - return m.protocol -} - -func (m IncomingMessageScope) Channel() channels.Channel { - return channels.Channel(m.msg.ChannelID) -} - -func (m IncomingMessageScope) Size() int { - return m.msg.Size() -} - -func (m IncomingMessageScope) TargetIDs() flow.IdentifierList { - return m.targetIds -} - -func (m IncomingMessageScope) EventID() []byte { - return m.eventId[:] -} - -func (m IncomingMessageScope) PayloadType() string { - return MessageType(m.decodedPayload) -} - -// OutgoingMessageScope captures the context around an outgoing message that is about to be sent. -type OutgoingMessageScope struct { - targetIds flow.IdentifierList // the target node IDs. - topic channels.Topic // the topic, i.e., channel-id/spork-id. - payload interface{} // the payload to be sent. - encoder func(interface{}) ([]byte, error) // the encoder to encode the payload. - msg *message.Message // raw proto message sent on wire. - protocol message.ProtocolType // the type of protocol used to send the message. -} - -// NewOutgoingScope creates a new outgoing message scope. -// All errors returned by this function are benign and should not cause the node to crash. -// It errors if the encoder fails to encode the payload into a protobuf message, or -// if the number of target IDs does not match the protocol type (i.e., unicast messages -// should have exactly one target ID, while pubsub messages should have at least one target ID). -func NewOutgoingScope( - targetIds flow.IdentifierList, - topic channels.Topic, - payload interface{}, - encoder func(interface{}) ([]byte, error), - protocolType message.ProtocolType) (*OutgoingMessageScope, error) { - scope := &OutgoingMessageScope{ - targetIds: targetIds, - topic: topic, - payload: payload, - encoder: encoder, - protocol: protocolType, - } - - if protocolType == message.ProtocolTypeUnicast { - // for unicast messages, we should have exactly one target. - if len(targetIds) != 1 { - return nil, fmt.Errorf("expected exactly one target id for unicast message, got: %d", len(targetIds)) - } - } - if protocolType == message.ProtocolTypePubSub { - // for pubsub messages, we should have at least one target. - if len(targetIds) == 0 { - return nil, fmt.Errorf("expected at least one target id for pubsub message, got: %d", len(targetIds)) - } - } - - msg, err := scope.buildMessage() - if err != nil { - return nil, fmt.Errorf("could not build message: %w", err) - } - scope.msg = msg - return scope, nil -} - -func (o OutgoingMessageScope) TargetIds() flow.IdentifierList { - return o.targetIds -} - -func (o OutgoingMessageScope) Size() int { - return o.msg.Size() -} - -func (o OutgoingMessageScope) PayloadType() string { - return MessageType(o.payload) -} - -func (o OutgoingMessageScope) Topic() channels.Topic { - return o.topic -} - -// buildMessage builds the raw proto message to be sent on the wire. -func (o OutgoingMessageScope) buildMessage() (*message.Message, error) { - payload, err := o.encoder(o.payload) - if err != nil { - return nil, fmt.Errorf("could not encode payload: %w", err) - } - - emTargets := make([][]byte, 0) - for _, targetId := range o.targetIds { - tempID := targetId // avoid capturing loop variable - emTargets = append(emTargets, tempID[:]) - } - - channel, ok := channels.ChannelFromTopic(o.topic) - if !ok { - return nil, fmt.Errorf("could not convert topic to channel: %s", o.topic) - } - - return &message.Message{ - TargetIDs: emTargets, - ChannelID: channel.String(), - Payload: payload, - }, nil -} - -func (o OutgoingMessageScope) Proto() *message.Message { - return o.msg -} - // EventId computes the event ID for a given channel and payload (i.e., the hash of the payload and channel). // All errors returned by this function are benign and should not cause the node to crash. // It errors if the hash function fails to hash the payload and channel. @@ -254,5 +88,5 @@ type OutgoingMessageScoper interface { } // Ensure structs implement the interfaces -var _ IncomingMessageScoper = &IncomingMessageScope{} -var _ OutgoingMessageScoper = &OutgoingMessageScope{} +var _ IncomingMessageScoper = &message.IncomingMessageScope{} +var _ OutgoingMessageScoper = &message.OutgoingMessageScope{} diff --git a/network/middleware.go b/network/middleware.go index be2c65281e7..f8bf3192d7b 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -10,6 +10,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/network/channels" + "github.com/onflow/flow-go/network/message" ) // Middleware represents the middleware layer, which manages the connections to @@ -32,13 +33,13 @@ type Middleware interface { // Dispatch should be used whenever guaranteed delivery to a specific target is required. Otherwise, Publish is // a more efficient candidate. // All errors returned from this function can be considered benign. - SendDirect(msg *OutgoingMessageScope) error + SendDirect(msg *message.OutgoingMessageScope) error // Publish publishes a message on the channel. It models a distributed broadcast where the message is meant for all or // a many nodes subscribing to the channel. It does not guarantee the delivery though, and operates on a best // effort. // All errors returned from this function can be considered benign. - Publish(msg *OutgoingMessageScope) error + Publish(msg *message.OutgoingMessageScope) error // Subscribe subscribes the middleware to a channel. // No errors are expected during normal operation. @@ -71,7 +72,7 @@ type Overlay interface { // Identity returns the Identity associated with the given peer ID, if it exists Identity(peer.ID) (*flow.Identity, bool) - Receive(*IncomingMessageScope) error + Receive(*message.IncomingMessageScope) error } // Connection represents an interface to read from & write to a connection. diff --git a/network/mocknetwork/message_validator.go b/network/mocknetwork/message_validator.go index f2c78f75d20..c0cad7f1fef 100644 --- a/network/mocknetwork/message_validator.go +++ b/network/mocknetwork/message_validator.go @@ -3,7 +3,8 @@ package mocknetwork import ( - network "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" + mock "github.com/stretchr/testify/mock" ) @@ -13,11 +14,11 @@ type MessageValidator struct { } // Validate provides a mock function with given fields: msg -func (_m *MessageValidator) Validate(msg network.IncomingMessageScope) bool { +func (_m *MessageValidator) Validate(msg message.IncomingMessageScope) bool { ret := _m.Called(msg) var r0 bool - if rf, ok := ret.Get(0).(func(network.IncomingMessageScope) bool); ok { + if rf, ok := ret.Get(0).(func(message.IncomingMessageScope) bool); ok { r0 = rf(msg) } else { r0 = ret.Get(0).(bool) diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index a61deb80268..6f5a0fab528 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -4,7 +4,9 @@ package mocknetwork import ( datastore "github.com/ipfs/go-datastore" + channels "github.com/onflow/flow-go/network/channels" + "github.com/onflow/flow-go/network/message" irrecoverable "github.com/onflow/flow-go/module/irrecoverable" @@ -86,11 +88,11 @@ func (_m *Middleware) OnDisallowListNotification(_a0 *network.DisallowListingUpd } // Publish provides a mock function with given fields: msg -func (_m *Middleware) Publish(msg *network.OutgoingMessageScope) error { +func (_m *Middleware) Publish(msg *message.OutgoingMessageScope) error { ret := _m.Called(msg) var r0 error - if rf, ok := ret.Get(0).(func(*network.OutgoingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(*message.OutgoingMessageScope) error); ok { r0 = rf(msg) } else { r0 = ret.Error(0) @@ -116,11 +118,11 @@ func (_m *Middleware) Ready() <-chan struct{} { } // SendDirect provides a mock function with given fields: msg -func (_m *Middleware) SendDirect(msg *network.OutgoingMessageScope) error { +func (_m *Middleware) SendDirect(msg *message.OutgoingMessageScope) error { ret := _m.Called(msg) var r0 error - if rf, ok := ret.Get(0).(func(*network.OutgoingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(*message.OutgoingMessageScope) error); ok { r0 = rf(msg) } else { r0 = ret.Error(0) diff --git a/network/mocknetwork/overlay.go b/network/mocknetwork/overlay.go index e36869114c1..913ecbe66de 100644 --- a/network/mocknetwork/overlay.go +++ b/network/mocknetwork/overlay.go @@ -4,9 +4,9 @@ package mocknetwork import ( flow "github.com/onflow/flow-go/model/flow" - mock "github.com/stretchr/testify/mock" + "github.com/onflow/flow-go/network/message" - network "github.com/onflow/flow-go/network" + mock "github.com/stretchr/testify/mock" peer "github.com/libp2p/go-libp2p/core/peer" ) @@ -59,11 +59,11 @@ func (_m *Overlay) Identity(_a0 peer.ID) (*flow.Identity, bool) { } // Receive provides a mock function with given fields: _a0 -func (_m *Overlay) Receive(_a0 *network.IncomingMessageScope) error { +func (_m *Overlay) Receive(_a0 *message.IncomingMessageScope) error { ret := _m.Called(_a0) var r0 error - if rf, ok := ret.Get(0).(func(*network.IncomingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(*message.IncomingMessageScope) error); ok { r0 = rf(_a0) } else { r0 = ret.Error(0) diff --git a/network/p2p/dht/dht_test.go b/network/p2p/dht/dht_test.go index a959a5a0496..3f5ccb64a18 100644 --- a/network/p2p/dht/dht_test.go +++ b/network/p2p/dht/dht_test.go @@ -15,7 +15,6 @@ import ( libp2pmsg "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/module/irrecoverable" mockmodule "github.com/onflow/flow-go/module/mock" - flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" @@ -155,7 +154,7 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { // hence expect count and not count - 1 messages to be received (one by each node, including the sender) ch := make(chan peer.ID, count) - messageScope, err := flownet.NewOutgoingScope( + messageScope, err := message.NewOutgoingScope( ids.NodeIDs(), channels.TestNetworkChannel, &libp2pmsg.TestMessage{}, diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 898a9c6916e..1b6764d9309 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -16,6 +16,7 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p/unicast/protocols" ) @@ -67,7 +68,7 @@ type LibP2PNode interface { // Unsubscribe cancels the subscriber and closes the topic corresponding to the given channel. Unsubscribe(channel channels.Channel) error // Publish publishes the given payload on the topic. - Publish(ctx context.Context, msgScope *network.OutgoingMessageScope) error + Publish(ctx context.Context, msgScope *message.OutgoingMessageScope) error // Host returns pointer to host object of node. Host() host.Host // WithDefaultUnicastProtocol overrides the default handler of the unicast manager and registers all preferred protocols. diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 4d3ae590247..9be8b271a41 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -358,7 +358,7 @@ func (m *Middleware) OnAllowListNotification(notification *network.AllowListingU // - failed to send message to peer. // // All errors returned from this function can be considered benign. -func (m *Middleware) SendDirect(msg *network.OutgoingMessageScope) error { +func (m *Middleware) SendDirect(msg *message.OutgoingMessageScope) error { // since it is a unicast, we only need to get the first peer ID. peerID, err := m.idTranslator.GetPeerID(msg.TargetIds()[0]) if err != nil { @@ -714,7 +714,7 @@ func (m *Middleware) processAuthenticatedMessage(msg *message.Message, peerID pe return } - scope, err := network.NewIncomingScope(originId, protocol, msg, decodedMsgPayload) + scope, err := message.NewIncomingScope(originId, protocol, msg, decodedMsgPayload) if err != nil { m.log.Error(). Err(err). @@ -728,7 +728,7 @@ func (m *Middleware) processAuthenticatedMessage(msg *message.Message, peerID pe } // processMessage processes a message and eventually passes it to the overlay -func (m *Middleware) processMessage(scope *network.IncomingMessageScope) { +func (m *Middleware) processMessage(scope *message.IncomingMessageScope) { logger := m.log.With(). Str("channel", scope.Channel().String()). Str("type", scope.Protocol().String()). @@ -763,7 +763,7 @@ func (m *Middleware) processMessage(scope *network.IncomingMessageScope) { // - the libP2P node fails to publish the message. // // All errors returned from this function can be considered benign. -func (m *Middleware) Publish(msg *network.OutgoingMessageScope) error { +func (m *Middleware) Publish(msg *message.OutgoingMessageScope) error { return m.libP2PNode.Publish(m.ctx, msg) } diff --git a/network/p2p/middleware/middleware_test.go b/network/p2p/middleware/middleware_test.go index 9b9cc1dbc0e..840d0c3bb07 100644 --- a/network/p2p/middleware/middleware_test.go +++ b/network/p2p/middleware/middleware_test.go @@ -8,7 +8,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/messages" - "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p/middleware" @@ -18,7 +17,7 @@ import ( // TestChunkDataPackMaxMessageSize tests that the max message size for a chunk data pack response is set to the large message size. func TestChunkDataPackMaxMessageSize(t *testing.T) { // creates an outgoing chunk data pack response message (imitating an EN is sending a chunk data pack response to VN). - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, channels.ProvideChunks, &messages.ChunkDataResponse{ diff --git a/network/p2p/mock/lib_p2_p_node.go b/network/p2p/mock/lib_p2_p_node.go index e67614d6212..b496c764bbd 100644 --- a/network/p2p/mock/lib_p2_p_node.go +++ b/network/p2p/mock/lib_p2_p_node.go @@ -5,6 +5,7 @@ package mockp2p import ( component "github.com/onflow/flow-go/module/component" channels "github.com/onflow/flow-go/network/channels" + "github.com/onflow/flow-go/network/message" context "context" @@ -285,11 +286,11 @@ func (_m *LibP2PNode) PeerScoreExposer() p2p.PeerScoreExposer { } // Publish provides a mock function with given fields: ctx, msgScope -func (_m *LibP2PNode) Publish(ctx context.Context, msgScope *flow_gonetwork.OutgoingMessageScope) error { +func (_m *LibP2PNode) Publish(ctx context.Context, msgScope *message.OutgoingMessageScope) error { ret := _m.Called(ctx, msgScope) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *flow_gonetwork.OutgoingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(context.Context, *message.OutgoingMessageScope) error); ok { r0 = rf(ctx, msgScope) } else { r0 = ret.Error(0) diff --git a/network/p2p/network.go b/network/p2p/network.go index f564aa0340d..8e5b0b102f1 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -378,7 +378,7 @@ func (n *Network) Identity(pid peer.ID) (*flow.Identity, bool) { return n.identityProvider.ByPeerID(pid) } -func (n *Network) Receive(msg *network.IncomingMessageScope) error { +func (n *Network) Receive(msg *message.IncomingMessageScope) error { n.metrics.InboundMessageReceived(msg.Size(), msg.Channel().String(), msg.Protocol().String(), msg.PayloadType()) err := n.processNetworkMessage(msg) @@ -388,7 +388,7 @@ func (n *Network) Receive(msg *network.IncomingMessageScope) error { return nil } -func (n *Network) processNetworkMessage(msg *network.IncomingMessageScope) error { +func (n *Network) processNetworkMessage(msg *message.IncomingMessageScope) error { // checks the cache for deduplication and adds the message if not already present if !n.receiveCache.Add(msg.EventID()) { // drops duplicate message @@ -429,7 +429,7 @@ func (n *Network) UnicastOnChannel(channel channels.Channel, payload interface{} return nil } - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{targetID}, channels.TopicFromChannel(channel, n.sporkId), payload, @@ -509,7 +509,7 @@ func (n *Network) sendOnChannel(channel channels.Channel, msg interface{}, targe Msg("sending new message on channel") // generate network message (encoding) based on list of recipients - scope, err := network.NewOutgoingScope( + scope, err := message.NewOutgoingScope( targetIDs, channels.TopicFromChannel(channel, n.sporkId), msg, diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index ed98a658b69..7e868b6ca90 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -24,6 +24,7 @@ import ( flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2putils" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/p2pnode/internal" "github.com/onflow/flow-go/network/p2p/unicast/protocols" @@ -341,7 +342,7 @@ func (n *Node) unsubscribeTopic(topic channels.Topic) error { // Publish publishes the given payload on the topic. // All errors returned from this function can be considered benign. -func (n *Node) Publish(ctx context.Context, msgScope *flownet.OutgoingMessageScope) error { +func (n *Node) Publish(ctx context.Context, msgScope *message.OutgoingMessageScope) error { lg := n.logger.With(). Str("topic", msgScope.Topic().String()). Interface("proto_message", msgScope.Proto()). diff --git a/network/p2p/scoring/app_score_test.go b/network/p2p/scoring/app_score_test.go index 37cfd52a6b5..d43eeeab800 100644 --- a/network/p2p/scoring/app_score_test.go +++ b/network/p2p/scoring/app_score_test.go @@ -13,7 +13,6 @@ import ( "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/mock" - flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2pfixtures" "github.com/onflow/flow-go/network/message" @@ -95,7 +94,7 @@ func TestFullGossipSubConnectivity(t *testing.T) { // checks end-to-end message delivery works // each node sends a distinct message to all and checks that all nodes receive it. for _, node := range nodes { - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( ids.NodeIDs(), channels.PushBlocks, unittest.ProposalFixture(), @@ -216,7 +215,7 @@ func testGossipSubMessageDeliveryUnderNetworkPartition(t *testing.T, honestPeerS // let nodes reside on a full topology, hence no partition is caused by the topology. p2ptest.LetNodesDiscoverEachOther(t, ctx, allNodes, allIds) - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( allIds.NodeIDs(), channels.PushBlocks, unittest.ProposalFixture(), diff --git a/network/p2p/scoring/subscription_validator_test.go b/network/p2p/scoring/subscription_validator_test.go index 795165ac9cc..b8243fbdadc 100644 --- a/network/p2p/scoring/subscription_validator_test.go +++ b/network/p2p/scoring/subscription_validator_test.go @@ -7,7 +7,6 @@ import ( "testing" "time" - flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" p2ptest "github.com/onflow/flow-go/network/p2p/test" @@ -132,8 +131,8 @@ func TestSubscriptionValidator_InvalidSubscriptions(t *testing.T) { for _, role := range flow.Roles() { peerId := p2pfixtures.PeerIdFixture(t) unauthorizedChannels := channels.Channels(). // all channels - ExcludeChannels(channels.ChannelsByRole(role)). // excluding the channels for the role - ExcludePattern(regexp.MustCompile("^(test).*")) // excluding the test channels. + ExcludeChannels(channels.ChannelsByRole(role)). // excluding the channels for the role + ExcludePattern(regexp.MustCompile("^(test).*")) // excluding the test channels. sporkID := unittest.IdentifierFixture() unauthorizedTopics := make([]string, 0, len(unauthorizedChannels)) for _, channel := range unauthorizedChannels { @@ -239,7 +238,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // let the subscriptions be established time.Sleep(2 * time.Second) - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( ids.NodeIDs(), channels.PushBlocks, unittest.ProposalFixture(), @@ -268,7 +267,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // consensus node publishes another proposal, but this time, it should not reach verification node. // since upon an unauthorized subscription, verification node should have slashed consensus node on // the GossipSub scoring protocol. - outgoingMessageScope, err = flownet.NewOutgoingScope( + outgoingMessageScope, err = message.NewOutgoingScope( ids.NodeIDs(), channels.PushBlocks, unittest.ProposalFixture(), @@ -283,7 +282,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // moreover, a verification node publishing a message to the request chunk topic should not reach consensus node. // however, both verification nodes should receive the message. - outgoingMessageScope, err = flownet.NewOutgoingScope( + outgoingMessageScope, err = message.NewOutgoingScope( ids.NodeIDs(), channels.RequestChunks, &messages.ChunkDataRequest{ diff --git a/network/p2p/subscription/subscription_filter_test.go b/network/p2p/subscription/subscription_filter_test.go index fdb098ea012..59cc739e9e3 100644 --- a/network/p2p/subscription/subscription_filter_test.go +++ b/network/p2p/subscription/subscription_filter_test.go @@ -14,7 +14,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/id" "github.com/onflow/flow-go/module/irrecoverable" - flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2pfixtures" "github.com/onflow/flow-go/network/message" @@ -81,7 +80,7 @@ func TestFilterSubscribe(t *testing.T) { testPublish := func(wg *sync.WaitGroup, from p2p.LibP2PNode, sub p2p.Subscription) { - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( ids.NodeIDs(), channels.SyncCommittee, []byte("hello"), diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index c5684a18480..39344980e87 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -587,7 +587,7 @@ func EnsurePubsubMessageExchange(t *testing.T, ctx context.Context, nodes []p2p. for i := 0; i < count; i++ { // creates a unique message to be published by the node payload := messageFactory() - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, topic, payload, @@ -629,7 +629,7 @@ func EnsurePubsubMessageExchangeFromNode(t *testing.T, ctx context.Context, send for i := 0; i < count; i++ { // creates a unique message to be published by the node payload := messageFactory() - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( flow.IdentifierList{}, topic, payload, @@ -713,7 +713,7 @@ func EnsureNoPubsubMessageExchange( // creates a unique message to be published by the node. payload := messageFactory() - outgoingMessageScope, err := flownet.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( toIdentifiers, topic, payload, diff --git a/network/p2p/test/sporking_test.go b/network/p2p/test/sporking_test.go index d637114366e..ba5778b17b6 100644 --- a/network/p2p/test/sporking_test.go +++ b/network/p2p/test/sporking_test.go @@ -12,7 +12,6 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" - "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" @@ -237,7 +236,7 @@ func TestOneToKCrosstalkPrevention(t *testing.T) { time.Sleep(time.Second) // assert that node 1 can successfully send a message to node 2 via PubSub - outgoingMessageScope, err := network.NewOutgoingScope( + outgoingMessageScope, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, topicBeforeSpork, &libp2pmessage.TestMessage{ @@ -278,7 +277,7 @@ func TestOneToKCrosstalkPrevention(t *testing.T) { require.NoError(t, err) // assert that node 1 can no longer send a message to node 2 via PubSub - outgoingMessageScope, err = network.NewOutgoingScope( + outgoingMessageScope, err = message.NewOutgoingScope( flow.IdentifierList{id2.NodeID}, topicAfterSpork, &libp2pmessage.TestMessage{ diff --git a/network/p2p/test/topic_validator_test.go b/network/p2p/test/topic_validator_test.go index e3d47c7c709..8b62e5465b3 100644 --- a/network/p2p/test/topic_validator_test.go +++ b/network/p2p/test/topic_validator_test.go @@ -94,7 +94,7 @@ func TestTopicValidator_Unstaked(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -153,7 +153,7 @@ func TestTopicValidator_PublicChannel(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, &messages.SyncRequest{Nonce: 0, Height: 0}, @@ -225,7 +225,7 @@ func TestTopicValidator_TopicMismatch(t *testing.T) { defer cancel5s() // create a dummy block proposal to publish from our SN node - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -285,7 +285,7 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() // create a dummy block proposal to publish from our SN node - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -383,7 +383,7 @@ func TestAuthorizedSenderValidator_Unauthorized(t *testing.T) { // sn2 publishes the block proposal, sn1 and an1 should receive the message because // SN nodes are authorized to send block proposals // create a dummy block proposal to publish from our SN node - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -410,7 +410,7 @@ func TestAuthorizedSenderValidator_Unauthorized(t *testing.T) { // the access node now publishes the block proposal message, AN are not authorized to publish block proposals // the message should be rejected by the topic validator on sn1 - outgoingMessageScope2, err := network.NewOutgoingScope( + outgoingMessageScope2, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -504,7 +504,7 @@ func TestAuthorizedSenderValidator_InvalidMsg(t *testing.T) { // create a dummy block proposal to publish from our SN node // sn2 publishes the block proposal on the sync committee channel - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -594,7 +594,7 @@ func TestAuthorizedSenderValidator_Ejected(t *testing.T) { // sn2 publishes the block proposal, sn1 and an1 should receive the message because // SN nodes are authorized to send block proposals // create a dummy block proposal to publish from our SN node - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -619,7 +619,7 @@ func TestAuthorizedSenderValidator_Ejected(t *testing.T) { // "eject" sn2 to ensure messages published by ejected nodes get rejected identity2.Ejected = true - outgoingMessageScope3, err := network.NewOutgoingScope( + outgoingMessageScope3, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, unittest.ProposalFixture(), @@ -704,7 +704,7 @@ func TestAuthorizedSenderValidator_ClusterChannel(t *testing.T) { defer cancel5s() // create a dummy sync request to publish from our LN node - outgoingMessageScope1, err := network.NewOutgoingScope( + outgoingMessageScope1, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, topic, &messages.RangeRequest{}, diff --git a/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go b/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go index 0016ae49f63..40befc19b48 100644 --- a/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go +++ b/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go @@ -6,7 +6,6 @@ import ( "github.com/onflow/flow-go/model/flow" libp2pmessage "github.com/onflow/flow-go/model/libp2p/message" - "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/message" "github.com/stretchr/testify/require" @@ -37,7 +36,7 @@ func TestBandWidthRateLimiter_Allow(t *testing.T) { b[i] = byte('X') } - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, channels.TestNetworkChannel, &libp2pmessage.TestMessage{ @@ -90,7 +89,7 @@ func TestBandWidthRateLimiter_IsRateLimited(t *testing.T) { require.False(t, bandwidthRateLimiter.IsRateLimited(peerID)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, channels.TestNetworkChannel, &libp2pmessage.TestMessage{ diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 65ce4f7cca3..a853659784b 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -226,7 +226,7 @@ func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { // needed to enable ID translation m.providers[0].SetIdentities(idList) - outMsg, err := network.NewOutgoingScope( + outMsg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -354,7 +354,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { // to be invoked at-least once. We send 10 messages due to the flakiness that is caused by async stream // handling of streams. for i := 0; i < 10; i++ { - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -378,7 +378,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { // eventually the rate limited node should be able to reconnect and send messages require.Eventually(m.T(), func() bool { - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -490,7 +490,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Bandwidth() { b[i] = byte('X') } - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -528,7 +528,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Bandwidth() { // eventually the rate limited node should be able to reconnect and send messages require.Eventually(m.T(), func() bool { - msg, err = network.NewOutgoingScope( + msg, err = message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -590,7 +590,7 @@ func (m *MiddlewareTestSuite) TestPing() { lastNodeIndex := m.size - 1 expectedPayload := "TestPingContentReception" - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{m.ids[lastNodeIndex].NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -604,7 +604,7 @@ func (m *MiddlewareTestSuite) TestPing() { Run(func(args mockery.Arguments) { receiveWG.Done() - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -648,7 +648,7 @@ func (m *MiddlewareTestSuite) MultiPing(count int) { sendWG.Add(1) expectedPayloadText := fmt.Sprintf("hello from: %d", i) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{m.ids[lastNodeIndex].NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -662,7 +662,7 @@ func (m *MiddlewareTestSuite) MultiPing(count int) { Run(func(args mockery.Arguments) { receiveWG.Done() - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -711,7 +711,7 @@ func (m *MiddlewareTestSuite) TestEcho() { // message sent from first node to the last node. expectedSendMsg := "TestEcho" - sendMsg, err := network.NewOutgoingScope( + sendMsg, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, testChannel, &libp2pmessage.TestMessage{ @@ -723,7 +723,7 @@ func (m *MiddlewareTestSuite) TestEcho() { // reply from last node to the first node. expectedReplyMsg := "TestEcho response" - replyMsg, err := network.NewOutgoingScope( + replyMsg, err := message.NewOutgoingScope( flow.IdentifierList{firstNode}, testChannel, &libp2pmessage.TestMessage{ @@ -739,7 +739,7 @@ func (m *MiddlewareTestSuite) TestEcho() { wg.Done() // sanity checks the message content. - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -762,7 +762,7 @@ func (m *MiddlewareTestSuite) TestEcho() { Run(func(args mockery.Arguments) { wg.Done() // sanity checks the message content. - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -805,7 +805,7 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_SendDirect() { Text: string(payload), } - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, testChannel, event, @@ -833,7 +833,7 @@ func (m *MiddlewareTestSuite) TestLargeMessageSize_SendDirect() { targetSize := uint64(middleware.DefaultMaxUnicastMsgSize) + 1000 event := unittest.ChunkDataResponseMsgFixture(unittest.IdentifierFixture(), unittest.WithApproximateSize(targetSize)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{targetNode}, channels.ProvideChunks, event, @@ -845,7 +845,7 @@ func (m *MiddlewareTestSuite) TestLargeMessageSize_SendDirect() { ch := make(chan struct{}) m.ov[targetIndex].On("Receive", mockery.Anything).Return(nil).Once(). Run(func(args mockery.Arguments) { - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), channels.ProvideChunks, msg.Channel()) @@ -886,7 +886,7 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { event := &libp2pmessage.TestMessage{ Text: string(payload), } - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, testChannel, event, @@ -921,7 +921,7 @@ func (m *MiddlewareTestSuite) TestUnsubscribe() { <-msgRcvd } - message1, err := network.NewOutgoingScope( + message1, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, testChannel, &libp2pmessage.TestMessage{ @@ -932,7 +932,7 @@ func (m *MiddlewareTestSuite) TestUnsubscribe() { require.NoError(m.T(), err) m.ov[last].On("Receive", mockery.Anything).Return(nil).Run(func(args mockery.Arguments) { - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), firstNode, msg.OriginId()) msgRcvd <- struct{}{} @@ -949,7 +949,7 @@ func (m *MiddlewareTestSuite) TestUnsubscribe() { assert.NoError(m.T(), err) // create and send a new message on the channel from the origin node - message2, err := network.NewOutgoingScope( + message2, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, testChannel, &libp2pmessage.TestMessage{ diff --git a/network/test/unicast_authorization_test.go b/network/test/unicast_authorization_test.go index 976f702c9de..1efafc75bd5 100644 --- a/network/test/unicast_authorization_test.go +++ b/network/test/unicast_authorization_test.go @@ -157,7 +157,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnstakedPeer() require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -219,7 +219,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_EjectedPeer() { require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -280,7 +280,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnauthorizedPee require.NoError(u.T(), u.receiverMW.Subscribe(channel)) require.NoError(u.T(), u.senderMW.Subscribe(channel)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, channel, &libp2pmessage.TestMessage{ @@ -343,7 +343,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnknownMsgCode( require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -413,7 +413,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_WrongMsgCode() require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -445,7 +445,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_PublicChannel() u.setupMiddlewaresAndProviders(slashingViolationsConsumer) expectedPayload := "hello" - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, testChannel, &libp2pmessage.TestMessage{ @@ -470,7 +470,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_PublicChannel() Run(func(args mockery.Arguments) { close(u.waitCh) - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(u.T(), ok) require.Equal(u.T(), testChannel, msg.Channel()) // channel @@ -543,7 +543,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnauthorizedUni // messages.BlockProposal is not authorized to be sent via unicast over the ConsensusCommittee channel payload := unittest.ProposalFixture() - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, channel, payload, @@ -600,7 +600,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_ReceiverHasNoSu channel := channels.TestNetworkChannel - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, channel, &libp2pmessage.TestMessage{ @@ -626,7 +626,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_ReceiverHasSubs u.setupMiddlewaresAndProviders(slashingViolationsConsumer) channel := channels.RequestReceiptsByBlockID - msg, err := network.NewOutgoingScope( + msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, channel, &messages.EntityRequest{}, @@ -652,7 +652,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_ReceiverHasSubs Run(func(args mockery.Arguments) { close(u.waitCh) - msg, ok := args[0].(*network.IncomingMessageScope) + msg, ok := args[0].(*message.IncomingMessageScope) require.True(u.T(), ok) require.Equal(u.T(), channel, msg.Channel()) // channel diff --git a/network/validator.go b/network/validator.go index 0d40b9290c5..3a9167ecde9 100644 --- a/network/validator.go +++ b/network/validator.go @@ -1,8 +1,10 @@ package network +import "github.com/onflow/flow-go/network/message" + // MessageValidator validates the incoming message. Message validation happens in the middleware right before it is // delivered to the network. type MessageValidator interface { // Validate validates the message and returns true if the message is to be retained and false if it needs to be dropped - Validate(msg IncomingMessageScope) bool + Validate(msg message.IncomingMessageScope) bool } diff --git a/network/validator/any_validator.go b/network/validator/any_validator.go index 92a09fd3ef0..c02fd3ab555 100644 --- a/network/validator/any_validator.go +++ b/network/validator/any_validator.go @@ -2,6 +2,7 @@ package validator import ( "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = (*AnyValidator)(nil) @@ -17,7 +18,7 @@ func NewAnyValidator(validators ...network.MessageValidator) network.MessageVali } } -func (v AnyValidator) Validate(msg network.IncomingMessageScope) bool { +func (v AnyValidator) Validate(msg message.IncomingMessageScope) bool { for _, validator := range v.validators { if validator.Validate(msg) { return true diff --git a/network/validator/not_validator.go b/network/validator/not_validator.go index b439e7cae54..14ab7460349 100644 --- a/network/validator/not_validator.go +++ b/network/validator/not_validator.go @@ -2,6 +2,7 @@ package validator import ( "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = (*NotValidator)(nil) @@ -17,6 +18,6 @@ func NewNotValidator(validator network.MessageValidator) network.MessageValidato } } -func (n NotValidator) Validate(msg network.IncomingMessageScope) bool { +func (n NotValidator) Validate(msg message.IncomingMessageScope) bool { return !n.validator.Validate(msg) } diff --git a/network/validator/origin_validator.go b/network/validator/origin_validator.go index 857f7b58271..0963fd9b68c 100644 --- a/network/validator/origin_validator.go +++ b/network/validator/origin_validator.go @@ -3,6 +3,7 @@ package validator import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = (*OriginValidator)(nil) @@ -17,6 +18,6 @@ func NewOriginValidator(provider module.IdentifierProvider) network.MessageValid return &OriginValidator{provider} } -func (v OriginValidator) Validate(msg network.IncomingMessageScope) bool { +func (v OriginValidator) Validate(msg message.IncomingMessageScope) bool { return v.idProvider.Identifiers().Contains(msg.OriginId()) } diff --git a/network/validator/sender_validator.go b/network/validator/sender_validator.go index 098e1dd9c49..3db517e1c8c 100644 --- a/network/validator/sender_validator.go +++ b/network/validator/sender_validator.go @@ -3,6 +3,7 @@ package validator import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = &SenderValidator{} @@ -22,7 +23,7 @@ func ValidateSender(sender flow.Identifier) network.MessageValidator { } // Validate returns true if the message origin id is the same as the sender ID. -func (sv *SenderValidator) Validate(msg network.IncomingMessageScope) bool { +func (sv *SenderValidator) Validate(msg message.IncomingMessageScope) bool { return sv.sender == msg.OriginId() } diff --git a/network/validator/target_validator.go b/network/validator/target_validator.go index 5a9b1ab73f9..4bfab62bb1b 100644 --- a/network/validator/target_validator.go +++ b/network/validator/target_validator.go @@ -5,6 +5,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/logging" ) @@ -28,7 +29,7 @@ func ValidateTarget(log zerolog.Logger, target flow.Identifier) network.MessageV } // Validate returns true if the message is intended for the given target ID else it returns false -func (tv *TargetValidator) Validate(msg network.IncomingMessageScope) bool { +func (tv *TargetValidator) Validate(msg message.IncomingMessageScope) bool { for _, t := range msg.TargetIDs() { if tv.target == t { return true From 49d464d268677c0b89eabe4a10764c5a3acf20ad Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 12:31:21 -0700 Subject: [PATCH 17/35] refactors message scopes to use the interface type --- network/message/message_scope.go | 4 ++++ network/message_scope.go | 12 ++++-------- network/middleware.go | 7 +++---- network/p2p/libp2pNode.go | 3 +-- network/p2p/middleware/middleware.go | 14 +++++++++----- network/p2p/network.go | 4 ++-- network/p2p/p2pnode/libp2pNode.go | 3 +-- network/test/middleware_test.go | 12 ++++++------ network/test/unicast_authorization_test.go | 4 ++-- network/validator.go | 4 +--- network/validator/any_validator.go | 3 +-- network/validator/not_validator.go | 3 +-- network/validator/sender_validator.go | 3 +-- network/validator/target_validator.go | 3 +-- 14 files changed, 37 insertions(+), 42 deletions(-) diff --git a/network/message/message_scope.go b/network/message/message_scope.go index aa15c0112a2..340a4184ea0 100644 --- a/network/message/message_scope.go +++ b/network/message/message_scope.go @@ -9,6 +9,10 @@ import ( "github.com/onflow/flow-go/network/channels" ) +// Ensure structs implement the interfaces +var _ network.IncomingMessageScope = &IncomingMessageScope{} +var _ network.OutgoingMessageScope = &OutgoingMessageScope{} + // IncomingMessageScope captures the context around an incoming message that is received by the network layer. type IncomingMessageScope struct { originId flow.Identifier // the origin node ID. diff --git a/network/message_scope.go b/network/message_scope.go index d39f7bc830f..e09595ff110 100644 --- a/network/message_scope.go +++ b/network/message_scope.go @@ -39,8 +39,8 @@ func MessageType(decodedPayload interface{}) string { return strings.TrimLeft(fmt.Sprintf("%T", decodedPayload), "*") } -// IncomingMessageScoper defines the interface for handling incoming message scope. -type IncomingMessageScoper interface { +// IncomingMessageScope defines the interface for handling incoming message scope. +type IncomingMessageScope interface { // OriginId returns the origin node ID. OriginId() flow.Identifier @@ -69,8 +69,8 @@ type IncomingMessageScoper interface { PayloadType() string } -// OutgoingMessageScoper defines the interface for handling outgoing message scope. -type OutgoingMessageScoper interface { +// OutgoingMessageScope defines the interface for handling outgoing message scope. +type OutgoingMessageScope interface { // TargetIds returns the target node IDs. TargetIds() flow.IdentifierList @@ -86,7 +86,3 @@ type OutgoingMessageScoper interface { // Proto returns the raw proto message sent on the wire. Proto() *message.Message } - -// Ensure structs implement the interfaces -var _ IncomingMessageScoper = &message.IncomingMessageScope{} -var _ OutgoingMessageScoper = &message.OutgoingMessageScope{} diff --git a/network/middleware.go b/network/middleware.go index f8bf3192d7b..a7ac5820b19 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -10,7 +10,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/network/channels" - "github.com/onflow/flow-go/network/message" ) // Middleware represents the middleware layer, which manages the connections to @@ -33,13 +32,13 @@ type Middleware interface { // Dispatch should be used whenever guaranteed delivery to a specific target is required. Otherwise, Publish is // a more efficient candidate. // All errors returned from this function can be considered benign. - SendDirect(msg *message.OutgoingMessageScope) error + SendDirect(msg OutgoingMessageScope) error // Publish publishes a message on the channel. It models a distributed broadcast where the message is meant for all or // a many nodes subscribing to the channel. It does not guarantee the delivery though, and operates on a best // effort. // All errors returned from this function can be considered benign. - Publish(msg *message.OutgoingMessageScope) error + Publish(msg OutgoingMessageScope) error // Subscribe subscribes the middleware to a channel. // No errors are expected during normal operation. @@ -72,7 +71,7 @@ type Overlay interface { // Identity returns the Identity associated with the given peer ID, if it exists Identity(peer.ID) (*flow.Identity, bool) - Receive(*message.IncomingMessageScope) error + Receive(IncomingMessageScope) error } // Connection represents an interface to read from & write to a connection. diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 1b6764d9309..0645be384e1 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -16,7 +16,6 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" - "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p/unicast/protocols" ) @@ -68,7 +67,7 @@ type LibP2PNode interface { // Unsubscribe cancels the subscriber and closes the topic corresponding to the given channel. Unsubscribe(channel channels.Channel) error // Publish publishes the given payload on the topic. - Publish(ctx context.Context, msgScope *message.OutgoingMessageScope) error + Publish(ctx context.Context, scope network.OutgoingMessageScope) error // Host returns pointer to host object of node. Host() host.Host // WithDefaultUnicastProtocol overrides the default handler of the unicast manager and registers all preferred protocols. diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 9be8b271a41..49e1a2b0835 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -358,7 +358,7 @@ func (m *Middleware) OnAllowListNotification(notification *network.AllowListingU // - failed to send message to peer. // // All errors returned from this function can be considered benign. -func (m *Middleware) SendDirect(msg *message.OutgoingMessageScope) error { +func (m *Middleware) SendDirect(msg network.OutgoingMessageScope) error { // since it is a unicast, we only need to get the first peer ID. peerID, err := m.idTranslator.GetPeerID(msg.TargetIds()[0]) if err != nil { @@ -381,7 +381,11 @@ func (m *Middleware) SendDirect(msg *message.OutgoingMessageScope) error { // protect the underlying connection from being inadvertently pruned by the peer manager while the stream and // connection creation is being attempted, and remove it from protected list once stream created. - tag := fmt.Sprintf("%v:%v", msg.Channel(), msg.PayloadType()) + channel, ok := channels.ChannelFromTopic(msg.Topic()) + if !ok { + return fmt.Errorf("could not find channel for topic %s", msg.Topic()) + } + tag := fmt.Sprintf("%v:%v", channel, msg.PayloadType()) m.libP2PNode.Host().ConnManager().Protect(peerID, tag) defer m.libP2PNode.Host().ConnManager().Unprotect(peerID, tag) @@ -728,7 +732,7 @@ func (m *Middleware) processAuthenticatedMessage(msg *message.Message, peerID pe } // processMessage processes a message and eventually passes it to the overlay -func (m *Middleware) processMessage(scope *message.IncomingMessageScope) { +func (m *Middleware) processMessage(scope network.IncomingMessageScope) { logger := m.log.With(). Str("channel", scope.Channel().String()). Str("type", scope.Protocol().String()). @@ -739,7 +743,7 @@ func (m *Middleware) processMessage(scope *message.IncomingMessageScope) { // run through all the message validators for _, v := range m.validators { // if any one fails, stop message propagation - if !v.Validate(*scope) { + if !v.Validate(scope) { logger.Debug().Msg("new message filtered by message validators") return } @@ -763,7 +767,7 @@ func (m *Middleware) processMessage(scope *message.IncomingMessageScope) { // - the libP2P node fails to publish the message. // // All errors returned from this function can be considered benign. -func (m *Middleware) Publish(msg *message.OutgoingMessageScope) error { +func (m *Middleware) Publish(msg network.OutgoingMessageScope) error { return m.libP2PNode.Publish(m.ctx, msg) } diff --git a/network/p2p/network.go b/network/p2p/network.go index 8e5b0b102f1..a3225f59925 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -378,7 +378,7 @@ func (n *Network) Identity(pid peer.ID) (*flow.Identity, bool) { return n.identityProvider.ByPeerID(pid) } -func (n *Network) Receive(msg *message.IncomingMessageScope) error { +func (n *Network) Receive(msg network.IncomingMessageScope) error { n.metrics.InboundMessageReceived(msg.Size(), msg.Channel().String(), msg.Protocol().String(), msg.PayloadType()) err := n.processNetworkMessage(msg) @@ -388,7 +388,7 @@ func (n *Network) Receive(msg *message.IncomingMessageScope) error { return nil } -func (n *Network) processNetworkMessage(msg *message.IncomingMessageScope) error { +func (n *Network) processNetworkMessage(msg network.IncomingMessageScope) error { // checks the cache for deduplication and adds the message if not already present if !n.receiveCache.Add(msg.EventID()) { // drops duplicate message diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 7e868b6ca90..6a772217c68 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -24,7 +24,6 @@ import ( flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2putils" - "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/p2pnode/internal" "github.com/onflow/flow-go/network/p2p/unicast/protocols" @@ -342,7 +341,7 @@ func (n *Node) unsubscribeTopic(topic channels.Topic) error { // Publish publishes the given payload on the topic. // All errors returned from this function can be considered benign. -func (n *Node) Publish(ctx context.Context, msgScope *message.OutgoingMessageScope) error { +func (n *Node) Publish(ctx context.Context, msgScope flownet.OutgoingMessageScope) error { lg := n.logger.With(). Str("topic", msgScope.Topic().String()). Interface("proto_message", msgScope.Proto()). diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index a853659784b..585b3418d9a 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -604,7 +604,7 @@ func (m *MiddlewareTestSuite) TestPing() { Run(func(args mockery.Arguments) { receiveWG.Done() - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -662,7 +662,7 @@ func (m *MiddlewareTestSuite) MultiPing(count int) { Run(func(args mockery.Arguments) { receiveWG.Done() - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -739,7 +739,7 @@ func (m *MiddlewareTestSuite) TestEcho() { wg.Done() // sanity checks the message content. - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -762,7 +762,7 @@ func (m *MiddlewareTestSuite) TestEcho() { Run(func(args mockery.Arguments) { wg.Done() // sanity checks the message content. - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), testChannel, msg.Channel()) // channel @@ -845,7 +845,7 @@ func (m *MiddlewareTestSuite) TestLargeMessageSize_SendDirect() { ch := make(chan struct{}) m.ov[targetIndex].On("Receive", mockery.Anything).Return(nil).Once(). Run(func(args mockery.Arguments) { - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), channels.ProvideChunks, msg.Channel()) @@ -932,7 +932,7 @@ func (m *MiddlewareTestSuite) TestUnsubscribe() { require.NoError(m.T(), err) m.ov[last].On("Receive", mockery.Anything).Return(nil).Run(func(args mockery.Arguments) { - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) require.Equal(m.T(), firstNode, msg.OriginId()) msgRcvd <- struct{}{} diff --git a/network/test/unicast_authorization_test.go b/network/test/unicast_authorization_test.go index 1efafc75bd5..0dfed501f4e 100644 --- a/network/test/unicast_authorization_test.go +++ b/network/test/unicast_authorization_test.go @@ -470,7 +470,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_PublicChannel() Run(func(args mockery.Arguments) { close(u.waitCh) - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(u.T(), ok) require.Equal(u.T(), testChannel, msg.Channel()) // channel @@ -652,7 +652,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_ReceiverHasSubs Run(func(args mockery.Arguments) { close(u.waitCh) - msg, ok := args[0].(*message.IncomingMessageScope) + msg, ok := args[0].(network.IncomingMessageScope) require.True(u.T(), ok) require.Equal(u.T(), channel, msg.Channel()) // channel diff --git a/network/validator.go b/network/validator.go index 3a9167ecde9..0d40b9290c5 100644 --- a/network/validator.go +++ b/network/validator.go @@ -1,10 +1,8 @@ package network -import "github.com/onflow/flow-go/network/message" - // MessageValidator validates the incoming message. Message validation happens in the middleware right before it is // delivered to the network. type MessageValidator interface { // Validate validates the message and returns true if the message is to be retained and false if it needs to be dropped - Validate(msg message.IncomingMessageScope) bool + Validate(msg IncomingMessageScope) bool } diff --git a/network/validator/any_validator.go b/network/validator/any_validator.go index c02fd3ab555..92a09fd3ef0 100644 --- a/network/validator/any_validator.go +++ b/network/validator/any_validator.go @@ -2,7 +2,6 @@ package validator import ( "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = (*AnyValidator)(nil) @@ -18,7 +17,7 @@ func NewAnyValidator(validators ...network.MessageValidator) network.MessageVali } } -func (v AnyValidator) Validate(msg message.IncomingMessageScope) bool { +func (v AnyValidator) Validate(msg network.IncomingMessageScope) bool { for _, validator := range v.validators { if validator.Validate(msg) { return true diff --git a/network/validator/not_validator.go b/network/validator/not_validator.go index 14ab7460349..b439e7cae54 100644 --- a/network/validator/not_validator.go +++ b/network/validator/not_validator.go @@ -2,7 +2,6 @@ package validator import ( "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = (*NotValidator)(nil) @@ -18,6 +17,6 @@ func NewNotValidator(validator network.MessageValidator) network.MessageValidato } } -func (n NotValidator) Validate(msg message.IncomingMessageScope) bool { +func (n NotValidator) Validate(msg network.IncomingMessageScope) bool { return !n.validator.Validate(msg) } diff --git a/network/validator/sender_validator.go b/network/validator/sender_validator.go index 3db517e1c8c..098e1dd9c49 100644 --- a/network/validator/sender_validator.go +++ b/network/validator/sender_validator.go @@ -3,7 +3,6 @@ package validator import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = &SenderValidator{} @@ -23,7 +22,7 @@ func ValidateSender(sender flow.Identifier) network.MessageValidator { } // Validate returns true if the message origin id is the same as the sender ID. -func (sv *SenderValidator) Validate(msg message.IncomingMessageScope) bool { +func (sv *SenderValidator) Validate(msg network.IncomingMessageScope) bool { return sv.sender == msg.OriginId() } diff --git a/network/validator/target_validator.go b/network/validator/target_validator.go index 4bfab62bb1b..5a9b1ab73f9 100644 --- a/network/validator/target_validator.go +++ b/network/validator/target_validator.go @@ -5,7 +5,6 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/logging" ) @@ -29,7 +28,7 @@ func ValidateTarget(log zerolog.Logger, target flow.Identifier) network.MessageV } // Validate returns true if the message is intended for the given target ID else it returns false -func (tv *TargetValidator) Validate(msg message.IncomingMessageScope) bool { +func (tv *TargetValidator) Validate(msg network.IncomingMessageScope) bool { for _, t := range msg.TargetIDs() { if tv.target == t { return true From 8071ab953f289e308e7cce45d5d229139c793bb8 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 12:35:03 -0700 Subject: [PATCH 18/35] resolves import cycles --- insecure/fixtures.go | 3 ++- insecure/orchestrator/network.go | 5 ++-- network/cache/rcvcache_test.go | 14 +++++------ network/message/message_scope.go | 36 +++++++++++++++++++++++++--- network/message_scope.go | 33 ------------------------- network/p2p/middleware/middleware.go | 2 +- network/test/middleware_test.go | 6 ++--- utils/unittest/fixtures.go | 4 ++-- 8 files changed, 51 insertions(+), 52 deletions(-) diff --git a/insecure/fixtures.go b/insecure/fixtures.go index 3abe3392f8c..9e85deb3bad 100644 --- a/insecure/fixtures.go +++ b/insecure/fixtures.go @@ -11,6 +11,7 @@ import ( "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" + message2 "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/unittest" ) @@ -40,7 +41,7 @@ func EgressMessageFixture(t *testing.T, codec network.Codec, protocol Protocol, // encodes event to create payload payload, err := codec.Encode(content) require.NoError(t, err) - eventIDHash, err := network.EventId(channel, payload) + eventIDHash, err := message2.EventId(channel, payload) require.NoError(t, err) eventID := flow.HashToID(eventIDHash) diff --git a/insecure/orchestrator/network.go b/insecure/orchestrator/network.go index a41a4781c2c..1c8ca95e417 100644 --- a/insecure/orchestrator/network.go +++ b/insecure/orchestrator/network.go @@ -13,6 +13,7 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" + message2 "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/logging" ) @@ -159,7 +160,7 @@ func (on *Network) processEgressMessage(message *insecure.EgressMessage) error { channel := channels.Channel(message.ChannelID) - egressEventIDHash, err := network.EventId(channel, message.Payload) + egressEventIDHash, err := message2.EventId(channel, message.Payload) if err != nil { return fmt.Errorf("could not create egress event ID: %w", err) } @@ -205,7 +206,7 @@ func (on *Network) processIngressMessage(message *insecure.IngressMessage) error defer on.orchestratorMutex.Unlock() channel := channels.Channel(message.ChannelID) - ingressEventIDHash, err := network.EventId(channel, message.Payload) + ingressEventIDHash, err := message2.EventId(channel, message.Payload) if err != nil { return fmt.Errorf("could not create ingress event ID: %w", err) } diff --git a/network/cache/rcvcache_test.go b/network/cache/rcvcache_test.go index 32551e1264f..28c65c52f72 100644 --- a/network/cache/rcvcache_test.go +++ b/network/cache/rcvcache_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/message" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -43,20 +43,20 @@ func (r *ReceiveCacheTestSuite) SetupTest() { // TestSingleElementAdd adds a single element to the cache and verifies its existence. func (r *ReceiveCacheTestSuite) TestSingleElementAdd() { - eventID, err := network.EventId(channels.Channel("0"), []byte("event-1")) + eventID, err := message.EventId(channels.Channel("0"), []byte("event-1")) require.NoError(r.T(), err) assert.True(r.Suite.T(), r.c.Add(eventID)) assert.False(r.Suite.T(), r.c.Add(eventID)) // same channel but different event should be treated as unseen - eventID2, err := network.EventId(channels.Channel("0"), []byte("event-2")) + eventID2, err := message.EventId(channels.Channel("0"), []byte("event-2")) require.NoError(r.T(), err) assert.True(r.Suite.T(), r.c.Add(eventID2)) assert.False(r.Suite.T(), r.c.Add(eventID2)) // same event but different channels should be treated as unseen - eventID3, err := network.EventId(channels.Channel("1"), []byte("event-2")) + eventID3, err := message.EventId(channels.Channel("1"), []byte("event-2")) require.NoError(r.T(), err) assert.True(r.Suite.T(), r.c.Add(eventID3)) assert.False(r.Suite.T(), r.c.Add(eventID3)) @@ -64,7 +64,7 @@ func (r *ReceiveCacheTestSuite) TestSingleElementAdd() { // TestNoneExistence evaluates the correctness of cache operation against non-existing element func (r *ReceiveCacheTestSuite) TestNoneExistence() { - eventID, err := network.EventId(channels.Channel("1"), []byte("non-existing event")) + eventID, err := message.EventId(channels.Channel("1"), []byte("non-existing event")) require.NoError(r.T(), err) // adding new event to cache should return true @@ -76,7 +76,7 @@ func (r *ReceiveCacheTestSuite) TestMultipleElementAdd() { // creates and populates slice of 10 events eventIDs := make([]hash.Hash, 0) for i := 0; i < r.size; i++ { - eventID, err := network.EventId(channels.Channel("1"), []byte(fmt.Sprintf("event-%d", i))) + eventID, err := message.EventId(channels.Channel("1"), []byte(fmt.Sprintf("event-%d", i))) require.NoError(r.T(), err) eventIDs = append(eventIDs, eventID) @@ -114,7 +114,7 @@ func (r *ReceiveCacheTestSuite) TestLRU() { eventIDs := make([]hash.Hash, 0) total := r.size + 1 for i := 0; i < total; i++ { - eventID, err := network.EventId(channels.Channel("1"), []byte(fmt.Sprintf("event-%d", i))) + eventID, err := message.EventId(channels.Channel("1"), []byte(fmt.Sprintf("event-%d", i))) require.NoError(r.T(), err) eventIDs = append(eventIDs, eventID) diff --git a/network/message/message_scope.go b/network/message/message_scope.go index 340a4184ea0..3c1076f0d79 100644 --- a/network/message/message_scope.go +++ b/network/message/message_scope.go @@ -2,6 +2,7 @@ package message import ( "fmt" + "strings" "github.com/onflow/flow-go/crypto/hash" "github.com/onflow/flow-go/model/flow" @@ -13,6 +14,35 @@ import ( var _ network.IncomingMessageScope = &IncomingMessageScope{} var _ network.OutgoingMessageScope = &OutgoingMessageScope{} +const ( + // eventIDPackingPrefix is used as a salt to generate payload hash for messages. + eventIDPackingPrefix = "libp2ppacking" +) + +// EventId computes the event ID for a given channel and payload (i.e., the hash of the payload and channel). +// All errors returned by this function are benign and should not cause the node to crash. +// It errors if the hash function fails to hash the payload and channel. +func EventId(channel channels.Channel, payload []byte) (hash.Hash, error) { + // use a hash with an engine-specific salt to get the payload hash + h := hash.NewSHA3_384() + _, err := h.Write([]byte(eventIDPackingPrefix + channel)) + if err != nil { + return nil, fmt.Errorf("could not hash channel as salt: %w", err) + } + + _, err = h.Write(payload) + if err != nil { + return nil, fmt.Errorf("could not hash event: %w", err) + } + + return h.SumHash(), nil +} + +// MessageType returns the type of the message payload. +func MessageType(decodedPayload interface{}) string { + return strings.TrimLeft(fmt.Sprintf("%T", decodedPayload), "*") +} + // IncomingMessageScope captures the context around an incoming message that is received by the network layer. type IncomingMessageScope struct { originId flow.Identifier // the origin node ID. @@ -29,7 +59,7 @@ type IncomingMessageScope struct { // It errors if event id (i.e., hash of the payload and channel) cannot be computed, or if it fails to // convert the target IDs from bytes slice to a flow.IdentifierList. func NewIncomingScope(originId flow.Identifier, protocol ProtocolType, msg *Message, decodedPayload interface{}) (*IncomingMessageScope, error) { - eventId, err := network.EventId(channels.Channel(msg.ChannelID), msg.Payload) + eventId, err := EventId(channels.Channel(msg.ChannelID), msg.Payload) if err != nil { return nil, fmt.Errorf("could not compute event id: %w", err) } @@ -81,7 +111,7 @@ func (m IncomingMessageScope) EventID() []byte { } func (m IncomingMessageScope) PayloadType() string { - return network.MessageType(m.decodedPayload) + return MessageType(m.decodedPayload) } // OutgoingMessageScope captures the context around an outgoing message that is about to be sent. @@ -143,7 +173,7 @@ func (o OutgoingMessageScope) Size() int { } func (o OutgoingMessageScope) PayloadType() string { - return network.MessageType(o.payload) + return MessageType(o.payload) } func (o OutgoingMessageScope) Topic() channels.Topic { diff --git a/network/message_scope.go b/network/message_scope.go index e09595ff110..fe37a795572 100644 --- a/network/message_scope.go +++ b/network/message_scope.go @@ -1,44 +1,11 @@ package network import ( - "fmt" - "strings" - - "github.com/onflow/flow-go/crypto/hash" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/message" ) -const ( - // eventIDPackingPrefix is used as a salt to generate payload hash for messages. - eventIDPackingPrefix = "libp2ppacking" -) - -// EventId computes the event ID for a given channel and payload (i.e., the hash of the payload and channel). -// All errors returned by this function are benign and should not cause the node to crash. -// It errors if the hash function fails to hash the payload and channel. -func EventId(channel channels.Channel, payload []byte) (hash.Hash, error) { - // use a hash with an engine-specific salt to get the payload hash - h := hash.NewSHA3_384() - _, err := h.Write([]byte(eventIDPackingPrefix + channel)) - if err != nil { - return nil, fmt.Errorf("could not hash channel as salt: %w", err) - } - - _, err = h.Write(payload) - if err != nil { - return nil, fmt.Errorf("could not hash event: %w", err) - } - - return h.SumHash(), nil -} - -// MessageType returns the type of the message payload. -func MessageType(decodedPayload interface{}) string { - return strings.TrimLeft(fmt.Sprintf("%T", decodedPayload), "*") -} - // IncomingMessageScope defines the interface for handling incoming message scope. type IncomingMessageScope interface { // OriginId returns the origin node ID. diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 49e1a2b0835..904f99aaa54 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -559,7 +559,7 @@ func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { remotePeer, role, msg.Size(), - network.MessageType(msg.Payload), + message.MessageType(msg.Payload), channels.Topic(msg.ChannelID)) { return } diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 585b3418d9a..38e5d331aef 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -748,7 +748,7 @@ func (m *MiddlewareTestSuite) TestEcho() { require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol require.Equal(m.T(), expectedSendMsg, msg.DecodedPayload().(*libp2pmessage.TestMessage).Text) // payload // event id - eventId, err := network.EventId(msg.Channel(), msg.Proto().Payload) + eventId, err := message.EventId(msg.Channel(), msg.Proto().Payload) require.NoError(m.T(), err) require.True(m.T(), bytes.Equal(eventId, msg.EventID())) @@ -771,7 +771,7 @@ func (m *MiddlewareTestSuite) TestEcho() { require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol require.Equal(m.T(), expectedReplyMsg, msg.DecodedPayload().(*libp2pmessage.TestMessage).Text) // payload // event id - eventId, err := network.EventId(msg.Channel(), msg.Proto().Payload) + eventId, err := message.EventId(msg.Channel(), msg.Proto().Payload) require.NoError(m.T(), err) require.True(m.T(), bytes.Equal(eventId, msg.EventID())) }) @@ -853,7 +853,7 @@ func (m *MiddlewareTestSuite) TestLargeMessageSize_SendDirect() { require.Equal(m.T(), targetNode, msg.TargetIDs()[0]) require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) - eventId, err := network.EventId(msg.Channel(), msg.Proto().Payload) + eventId, err := message.EventId(msg.Channel(), msg.Proto().Payload) require.NoError(m.T(), err) require.True(m.T(), bytes.Equal(eventId, msg.EventID())) close(ch) diff --git a/utils/unittest/fixtures.go b/utils/unittest/fixtures.go index e69a263ce6d..288494a2f99 100644 --- a/utils/unittest/fixtures.go +++ b/utils/unittest/fixtures.go @@ -15,6 +15,7 @@ import ( "github.com/onflow/cadence" sdk "github.com/onflow/flow-go-sdk" + "github.com/onflow/flow-go/network/message" hotstuff "github.com/onflow/flow-go/consensus/hotstuff/model" "github.com/onflow/flow-go/crypto" @@ -40,7 +41,6 @@ import ( "github.com/onflow/flow-go/module/mempool/entity" "github.com/onflow/flow-go/module/signature" "github.com/onflow/flow-go/module/updatable_configs" - "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/p2p/keyutils" "github.com/onflow/flow-go/state/protocol" @@ -2422,7 +2422,7 @@ func GetFlowProtocolEventID( ) flow.Identifier { payload, err := NetworkCodec().Encode(event) require.NoError(t, err) - eventIDHash, err := network.EventId(channel, payload) + eventIDHash, err := message.EventId(channel, payload) require.NoError(t, err) return flow.HashToID(eventIDHash) } From fb8820ddca21377ef356cf67272d21595ac04c21 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 12:36:46 -0700 Subject: [PATCH 19/35] fixes import ailiases --- insecure/fixtures.go | 4 ++-- insecure/orchestrator/network.go | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/insecure/fixtures.go b/insecure/fixtures.go index 9e85deb3bad..6292a7ac316 100644 --- a/insecure/fixtures.go +++ b/insecure/fixtures.go @@ -11,7 +11,7 @@ import ( "github.com/onflow/flow-go/model/libp2p/message" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" - message2 "github.com/onflow/flow-go/network/message" + flownetmsg "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/unittest" ) @@ -41,7 +41,7 @@ func EgressMessageFixture(t *testing.T, codec network.Codec, protocol Protocol, // encodes event to create payload payload, err := codec.Encode(content) require.NoError(t, err) - eventIDHash, err := message2.EventId(channel, payload) + eventIDHash, err := flownetmsg.EventId(channel, payload) require.NoError(t, err) eventID := flow.HashToID(eventIDHash) diff --git a/insecure/orchestrator/network.go b/insecure/orchestrator/network.go index 1c8ca95e417..11e24874e79 100644 --- a/insecure/orchestrator/network.go +++ b/insecure/orchestrator/network.go @@ -13,7 +13,7 @@ import ( "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" - message2 "github.com/onflow/flow-go/network/message" + flownetmsg "github.com/onflow/flow-go/network/message" "github.com/onflow/flow-go/utils/logging" ) @@ -160,7 +160,7 @@ func (on *Network) processEgressMessage(message *insecure.EgressMessage) error { channel := channels.Channel(message.ChannelID) - egressEventIDHash, err := message2.EventId(channel, message.Payload) + egressEventIDHash, err := flownetmsg.EventId(channel, message.Payload) if err != nil { return fmt.Errorf("could not create egress event ID: %w", err) } @@ -206,7 +206,7 @@ func (on *Network) processIngressMessage(message *insecure.IngressMessage) error defer on.orchestratorMutex.Unlock() channel := channels.Channel(message.ChannelID) - ingressEventIDHash, err := message2.EventId(channel, message.Payload) + ingressEventIDHash, err := flownetmsg.EventId(channel, message.Payload) if err != nil { return fmt.Errorf("could not create ingress event ID: %w", err) } From 7591e33a6235c7d9a91ae3c11e1bd3c914063ac9 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 12:38:35 -0700 Subject: [PATCH 20/35] fixes improt cycles --- network/message/message_scope.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/network/message/message_scope.go b/network/message/message_scope.go index 3c1076f0d79..fa4b0d667ef 100644 --- a/network/message/message_scope.go +++ b/network/message/message_scope.go @@ -6,14 +6,9 @@ import ( "github.com/onflow/flow-go/crypto/hash" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" ) -// Ensure structs implement the interfaces -var _ network.IncomingMessageScope = &IncomingMessageScope{} -var _ network.OutgoingMessageScope = &OutgoingMessageScope{} - const ( // eventIDPackingPrefix is used as a salt to generate payload hash for messages. eventIDPackingPrefix = "libp2ppacking" From c820722d5f3d68187720407b11c60edd1ef24875 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 12:41:00 -0700 Subject: [PATCH 21/35] re-generates mocks --- network/mocknetwork/incoming_message_scope.go | 168 ++++++++++++++++++ network/mocknetwork/message_validator.go | 7 +- network/mocknetwork/middleware.go | 10 +- network/mocknetwork/outgoing_message_scope.go | 106 +++++++++++ network/mocknetwork/overlay.go | 8 +- network/p2p/mock/lib_p2_p_node.go | 11 +- 6 files changed, 290 insertions(+), 20 deletions(-) create mode 100644 network/mocknetwork/incoming_message_scope.go create mode 100644 network/mocknetwork/outgoing_message_scope.go diff --git a/network/mocknetwork/incoming_message_scope.go b/network/mocknetwork/incoming_message_scope.go new file mode 100644 index 00000000000..543c9da9796 --- /dev/null +++ b/network/mocknetwork/incoming_message_scope.go @@ -0,0 +1,168 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mocknetwork + +import ( + flow "github.com/onflow/flow-go/model/flow" + channels "github.com/onflow/flow-go/network/channels" + + message "github.com/onflow/flow-go/network/message" + + mock "github.com/stretchr/testify/mock" +) + +// IncomingMessageScope is an autogenerated mock type for the IncomingMessageScope type +type IncomingMessageScope struct { + mock.Mock +} + +// Channel provides a mock function with given fields: +func (_m *IncomingMessageScope) Channel() channels.Channel { + ret := _m.Called() + + var r0 channels.Channel + if rf, ok := ret.Get(0).(func() channels.Channel); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(channels.Channel) + } + + return r0 +} + +// DecodedPayload provides a mock function with given fields: +func (_m *IncomingMessageScope) DecodedPayload() interface{} { + ret := _m.Called() + + var r0 interface{} + if rf, ok := ret.Get(0).(func() interface{}); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(interface{}) + } + } + + return r0 +} + +// EventID provides a mock function with given fields: +func (_m *IncomingMessageScope) EventID() []byte { + ret := _m.Called() + + var r0 []byte + if rf, ok := ret.Get(0).(func() []byte); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]byte) + } + } + + return r0 +} + +// OriginId provides a mock function with given fields: +func (_m *IncomingMessageScope) OriginId() flow.Identifier { + ret := _m.Called() + + var r0 flow.Identifier + if rf, ok := ret.Get(0).(func() flow.Identifier); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.Identifier) + } + } + + return r0 +} + +// PayloadType provides a mock function with given fields: +func (_m *IncomingMessageScope) PayloadType() string { + ret := _m.Called() + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// Proto provides a mock function with given fields: +func (_m *IncomingMessageScope) Proto() *message.Message { + ret := _m.Called() + + var r0 *message.Message + if rf, ok := ret.Get(0).(func() *message.Message); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*message.Message) + } + } + + return r0 +} + +// Protocol provides a mock function with given fields: +func (_m *IncomingMessageScope) Protocol() message.ProtocolType { + ret := _m.Called() + + var r0 message.ProtocolType + if rf, ok := ret.Get(0).(func() message.ProtocolType); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(message.ProtocolType) + } + + return r0 +} + +// Size provides a mock function with given fields: +func (_m *IncomingMessageScope) Size() int { + ret := _m.Called() + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// TargetIDs provides a mock function with given fields: +func (_m *IncomingMessageScope) TargetIDs() flow.IdentifierList { + ret := _m.Called() + + var r0 flow.IdentifierList + if rf, ok := ret.Get(0).(func() flow.IdentifierList); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.IdentifierList) + } + } + + return r0 +} + +type mockConstructorTestingTNewIncomingMessageScope interface { + mock.TestingT + Cleanup(func()) +} + +// NewIncomingMessageScope creates a new instance of IncomingMessageScope. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewIncomingMessageScope(t mockConstructorTestingTNewIncomingMessageScope) *IncomingMessageScope { + mock := &IncomingMessageScope{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/network/mocknetwork/message_validator.go b/network/mocknetwork/message_validator.go index c0cad7f1fef..f2c78f75d20 100644 --- a/network/mocknetwork/message_validator.go +++ b/network/mocknetwork/message_validator.go @@ -3,8 +3,7 @@ package mocknetwork import ( - "github.com/onflow/flow-go/network/message" - + network "github.com/onflow/flow-go/network" mock "github.com/stretchr/testify/mock" ) @@ -14,11 +13,11 @@ type MessageValidator struct { } // Validate provides a mock function with given fields: msg -func (_m *MessageValidator) Validate(msg message.IncomingMessageScope) bool { +func (_m *MessageValidator) Validate(msg network.IncomingMessageScope) bool { ret := _m.Called(msg) var r0 bool - if rf, ok := ret.Get(0).(func(message.IncomingMessageScope) bool); ok { + if rf, ok := ret.Get(0).(func(network.IncomingMessageScope) bool); ok { r0 = rf(msg) } else { r0 = ret.Get(0).(bool) diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index 6f5a0fab528..2b0f7409df4 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -4,9 +4,7 @@ package mocknetwork import ( datastore "github.com/ipfs/go-datastore" - channels "github.com/onflow/flow-go/network/channels" - "github.com/onflow/flow-go/network/message" irrecoverable "github.com/onflow/flow-go/module/irrecoverable" @@ -88,11 +86,11 @@ func (_m *Middleware) OnDisallowListNotification(_a0 *network.DisallowListingUpd } // Publish provides a mock function with given fields: msg -func (_m *Middleware) Publish(msg *message.OutgoingMessageScope) error { +func (_m *Middleware) Publish(msg network.OutgoingMessageScope) error { ret := _m.Called(msg) var r0 error - if rf, ok := ret.Get(0).(func(*message.OutgoingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(network.OutgoingMessageScope) error); ok { r0 = rf(msg) } else { r0 = ret.Error(0) @@ -118,11 +116,11 @@ func (_m *Middleware) Ready() <-chan struct{} { } // SendDirect provides a mock function with given fields: msg -func (_m *Middleware) SendDirect(msg *message.OutgoingMessageScope) error { +func (_m *Middleware) SendDirect(msg network.OutgoingMessageScope) error { ret := _m.Called(msg) var r0 error - if rf, ok := ret.Get(0).(func(*message.OutgoingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(network.OutgoingMessageScope) error); ok { r0 = rf(msg) } else { r0 = ret.Error(0) diff --git a/network/mocknetwork/outgoing_message_scope.go b/network/mocknetwork/outgoing_message_scope.go new file mode 100644 index 00000000000..f630c94e01a --- /dev/null +++ b/network/mocknetwork/outgoing_message_scope.go @@ -0,0 +1,106 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mocknetwork + +import ( + flow "github.com/onflow/flow-go/model/flow" + channels "github.com/onflow/flow-go/network/channels" + + message "github.com/onflow/flow-go/network/message" + + mock "github.com/stretchr/testify/mock" +) + +// OutgoingMessageScope is an autogenerated mock type for the OutgoingMessageScope type +type OutgoingMessageScope struct { + mock.Mock +} + +// PayloadType provides a mock function with given fields: +func (_m *OutgoingMessageScope) PayloadType() string { + ret := _m.Called() + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// Proto provides a mock function with given fields: +func (_m *OutgoingMessageScope) Proto() *message.Message { + ret := _m.Called() + + var r0 *message.Message + if rf, ok := ret.Get(0).(func() *message.Message); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*message.Message) + } + } + + return r0 +} + +// Size provides a mock function with given fields: +func (_m *OutgoingMessageScope) Size() int { + ret := _m.Called() + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// TargetIds provides a mock function with given fields: +func (_m *OutgoingMessageScope) TargetIds() flow.IdentifierList { + ret := _m.Called() + + var r0 flow.IdentifierList + if rf, ok := ret.Get(0).(func() flow.IdentifierList); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(flow.IdentifierList) + } + } + + return r0 +} + +// Topic provides a mock function with given fields: +func (_m *OutgoingMessageScope) Topic() channels.Topic { + ret := _m.Called() + + var r0 channels.Topic + if rf, ok := ret.Get(0).(func() channels.Topic); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(channels.Topic) + } + + return r0 +} + +type mockConstructorTestingTNewOutgoingMessageScope interface { + mock.TestingT + Cleanup(func()) +} + +// NewOutgoingMessageScope creates a new instance of OutgoingMessageScope. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewOutgoingMessageScope(t mockConstructorTestingTNewOutgoingMessageScope) *OutgoingMessageScope { + mock := &OutgoingMessageScope{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/network/mocknetwork/overlay.go b/network/mocknetwork/overlay.go index 913ecbe66de..7eab360e012 100644 --- a/network/mocknetwork/overlay.go +++ b/network/mocknetwork/overlay.go @@ -4,10 +4,10 @@ package mocknetwork import ( flow "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/network/message" - mock "github.com/stretchr/testify/mock" + network "github.com/onflow/flow-go/network" + peer "github.com/libp2p/go-libp2p/core/peer" ) @@ -59,11 +59,11 @@ func (_m *Overlay) Identity(_a0 peer.ID) (*flow.Identity, bool) { } // Receive provides a mock function with given fields: _a0 -func (_m *Overlay) Receive(_a0 *message.IncomingMessageScope) error { +func (_m *Overlay) Receive(_a0 network.IncomingMessageScope) error { ret := _m.Called(_a0) var r0 error - if rf, ok := ret.Get(0).(func(*message.IncomingMessageScope) error); ok { + if rf, ok := ret.Get(0).(func(network.IncomingMessageScope) error); ok { r0 = rf(_a0) } else { r0 = ret.Error(0) diff --git a/network/p2p/mock/lib_p2_p_node.go b/network/p2p/mock/lib_p2_p_node.go index b496c764bbd..008ad125a96 100644 --- a/network/p2p/mock/lib_p2_p_node.go +++ b/network/p2p/mock/lib_p2_p_node.go @@ -5,7 +5,6 @@ package mockp2p import ( component "github.com/onflow/flow-go/module/component" channels "github.com/onflow/flow-go/network/channels" - "github.com/onflow/flow-go/network/message" context "context" @@ -285,13 +284,13 @@ func (_m *LibP2PNode) PeerScoreExposer() p2p.PeerScoreExposer { return r0 } -// Publish provides a mock function with given fields: ctx, msgScope -func (_m *LibP2PNode) Publish(ctx context.Context, msgScope *message.OutgoingMessageScope) error { - ret := _m.Called(ctx, msgScope) +// Publish provides a mock function with given fields: ctx, scope +func (_m *LibP2PNode) Publish(ctx context.Context, scope flow_gonetwork.OutgoingMessageScope) error { + ret := _m.Called(ctx, scope) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *message.OutgoingMessageScope) error); ok { - r0 = rf(ctx, msgScope) + if rf, ok := ret.Get(0).(func(context.Context, flow_gonetwork.OutgoingMessageScope) error); ok { + r0 = rf(ctx, scope) } else { r0 = ret.Error(0) } From ff4122bb21e97962d68fcffeaa7e62c703f8fad5 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 12:41:55 -0700 Subject: [PATCH 22/35] fixes build --- network/validator/origin_validator.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/network/validator/origin_validator.go b/network/validator/origin_validator.go index 0963fd9b68c..857f7b58271 100644 --- a/network/validator/origin_validator.go +++ b/network/validator/origin_validator.go @@ -3,7 +3,6 @@ package validator import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/network" - "github.com/onflow/flow-go/network/message" ) var _ network.MessageValidator = (*OriginValidator)(nil) @@ -18,6 +17,6 @@ func NewOriginValidator(provider module.IdentifierProvider) network.MessageValid return &OriginValidator{provider} } -func (v OriginValidator) Validate(msg message.IncomingMessageScope) bool { +func (v OriginValidator) Validate(msg network.IncomingMessageScope) bool { return v.idProvider.Identifiers().Contains(msg.OriginId()) } From 996ac1f2578cfd673d6c53540aa0b986b652191d Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 13:22:36 -0700 Subject: [PATCH 23/35] fixes tests --- network/p2p/test/topic_validator_test.go | 26 ++++++++++++++++-------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/network/p2p/test/topic_validator_test.go b/network/p2p/test/topic_validator_test.go index 8b62e5465b3..0245985dc27 100644 --- a/network/p2p/test/topic_validator_test.go +++ b/network/p2p/test/topic_validator_test.go @@ -262,7 +262,7 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { p2ptest.StartNodes(t, signalerCtx, nodes, 100*time.Millisecond) defer p2ptest.StopNodes(t, nodes, cancel, 100*time.Millisecond) - topic := channels.TopicFromChannel(channels.ConsensusCommittee, sporkId) + topic := channels.Topic("invalid-topic") pInfo2, err := utils.PeerAddressInfo(identity2) require.NoError(t, err) @@ -284,24 +284,32 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { timedCtx, cancel5s := context.WithTimeout(ctx, 5*time.Second) defer cancel5s() - // create a dummy block proposal to publish from our SN node - outgoingMessageScope1, err := message.NewOutgoingScope( + + // invalid topic is malformed, hence it cannot be used to create a message scope, as it faces an error. + // Hence, we create a dummy block proposal message scope to publish on a legit topic, and then override + // the topic in the next step to a malformed topic. + dummyMessageScope, err := message.NewOutgoingScope( flow.IdentifierList{identity1.NodeID, identity2.NodeID}, - topic, + channels.TopicFromChannel(channels.PushBlocks, sporkId), unittest.ProposalFixture(), unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) require.NoError(t, err) - // intentionally overriding the channel id to be a valid channel id (though the topic is invalid) - // hence imitating a message that was published to the wrong topic - outgoingMessageScope1.Proto().ChannelID = channels.PushBlocks.String() + // overrides the topic to be an invalid topic + corruptOutgoingMessageScope := mocknetwork.NewOutgoingMessageScope(t) + corruptOutgoingMessageScope.On("TargetIds").Return(dummyMessageScope.TargetIds()).Maybe() + corruptOutgoingMessageScope.On("Topic").Return(topic).Maybe() + corruptOutgoingMessageScope.On("Proto").Return(dummyMessageScope.Proto()).Maybe() + corruptOutgoingMessageScope.On("PayloadType").Return(dummyMessageScope.PayloadType()).Maybe() + corruptOutgoingMessageScope.On("Size").Return(dummyMessageScope.Size()).Maybe() - err = sn2.Publish(timedCtx, outgoingMessageScope1) + // create a dummy block proposal to publish from our SN node + err = sn2.Publish(timedCtx, corruptOutgoingMessageScope) // publish fails because the topic conversion fails require.Error(t, err) - + fmt.Println(err) // ensure the correct error is contained in the logged error require.Contains(t, hook.Logs(), "could not convert topic to channel") } From dc8a3b594b6f23572fe804c593e07d110e0635e9 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 13:50:34 -0700 Subject: [PATCH 24/35] fixes tests --- network/test/middleware_test.go | 48 ++++++++++----------- network/test/unicast_authorization_test.go | 49 +++++++++++----------- 2 files changed, 48 insertions(+), 49 deletions(-) diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 38e5d331aef..90ef9709627 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -42,8 +42,6 @@ import ( "github.com/onflow/flow-go/utils/unittest" ) -const testChannel = channels.TestNetworkChannel - // libp2p emits a call to `Protect` with a topic-specific tag upon establishing each peering connection in a GossipSUb mesh, see: // https://github.com/libp2p/go-libp2p-pubsub/blob/master/tag_tracer.go // One way to make sure such a mesh has formed, asynchronously, in unit tests, is to wait for libp2p.GossipSubD such calls, @@ -172,7 +170,7 @@ func (m *MiddlewareTestSuite) SetupTest() { mw.SetOverlay(m.ov[i]) mw.Start(m.mwCtx) unittest.RequireComponentsReadyBefore(m.T(), 100*time.Millisecond, mw) - require.NoError(m.T(), mw.Subscribe(testChannel)) + require.NoError(m.T(), mw.Subscribe(channels.TestNetworkChannel)) } } @@ -228,7 +226,7 @@ func (m *MiddlewareTestSuite) TestUpdateNodeAddresses() { outMsg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: "TestUpdateNodeAddresses", }, @@ -317,7 +315,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { calls := atomic.NewUint64(0) ch := make(chan struct{}) - overlay.On("Receive", mockery.AnythingOfType("*network.IncomingMessageScope")).Return(nil).Run(func(args mockery.Arguments) { + overlay.On("Receive", mockery.AnythingOfType("*message.IncomingMessageScope")).Return(nil).Run(func(args mockery.Arguments) { calls.Inc() if calls.Load() >= 5 { close(ch) @@ -335,7 +333,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { newMw.Start(irrecoverableCtx) unittest.RequireComponentsReadyBefore(m.T(), 100*time.Millisecond, newMw) - require.NoError(m.T(), newMw.Subscribe(testChannel)) + require.NoError(m.T(), newMw.Subscribe(channels.TestNetworkChannel)) // needed to enable ID translation m.providers[0].SetIdentities(idList) @@ -356,7 +354,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { for i := 0; i < 10; i++ { msg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: fmt.Sprintf("hello-%d", i), }, @@ -380,7 +378,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { require.Eventually(m.T(), func() bool { msg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: "hello", }, @@ -477,7 +475,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Bandwidth() { newMw.Start(irrecoverableCtx) unittest.RequireComponentsReadyBefore(m.T(), 100*time.Millisecond, newMw) - require.NoError(m.T(), newMw.Subscribe(testChannel)) + require.NoError(m.T(), newMw.Subscribe(channels.TestNetworkChannel)) idList := flow.IdentityList(append(m.ids, newId)) @@ -492,7 +490,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Bandwidth() { msg, err := message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: string(b), }, @@ -530,7 +528,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Bandwidth() { require.Eventually(m.T(), func() bool { msg, err = message.NewOutgoingScope( flow.IdentifierList{newId.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: "", }, @@ -592,7 +590,7 @@ func (m *MiddlewareTestSuite) TestPing() { expectedPayload := "TestPingContentReception" msg, err := message.NewOutgoingScope( flow.IdentifierList{m.ids[lastNodeIndex].NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: expectedPayload, }, @@ -607,7 +605,7 @@ func (m *MiddlewareTestSuite) TestPing() { msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) - require.Equal(m.T(), testChannel, msg.Channel()) // channel + require.Equal(m.T(), channels.TestNetworkChannel, msg.Channel()) // channel require.Equal(m.T(), m.ids[firstNodeIndex].NodeID, msg.OriginId()) // sender id require.Equal(m.T(), m.ids[lastNodeIndex].NodeID, msg.TargetIDs()[0]) // target id require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol @@ -650,7 +648,7 @@ func (m *MiddlewareTestSuite) MultiPing(count int) { expectedPayloadText := fmt.Sprintf("hello from: %d", i) msg, err := message.NewOutgoingScope( flow.IdentifierList{m.ids[lastNodeIndex].NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: expectedPayloadText, }, @@ -665,7 +663,7 @@ func (m *MiddlewareTestSuite) MultiPing(count int) { msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) - require.Equal(m.T(), testChannel, msg.Channel()) // channel + require.Equal(m.T(), channels.TestNetworkChannel, msg.Channel()) // channel require.Equal(m.T(), m.ids[firstNodeIndex].NodeID, msg.OriginId()) // sender id require.Equal(m.T(), m.ids[lastNodeIndex].NodeID, msg.TargetIDs()[0]) // target id require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol @@ -713,7 +711,7 @@ func (m *MiddlewareTestSuite) TestEcho() { expectedSendMsg := "TestEcho" sendMsg, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: expectedSendMsg, }, @@ -725,7 +723,7 @@ func (m *MiddlewareTestSuite) TestEcho() { expectedReplyMsg := "TestEcho response" replyMsg, err := message.NewOutgoingScope( flow.IdentifierList{firstNode}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: expectedReplyMsg, }, @@ -742,7 +740,7 @@ func (m *MiddlewareTestSuite) TestEcho() { msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) - require.Equal(m.T(), testChannel, msg.Channel()) // channel + require.Equal(m.T(), channels.TestNetworkChannel, msg.Channel()) // channel require.Equal(m.T(), m.ids[first].NodeID, msg.OriginId()) // sender id require.Equal(m.T(), lastNode, msg.TargetIDs()[0]) // target id require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol @@ -765,7 +763,7 @@ func (m *MiddlewareTestSuite) TestEcho() { msg, ok := args[0].(network.IncomingMessageScope) require.True(m.T(), ok) - require.Equal(m.T(), testChannel, msg.Channel()) // channel + require.Equal(m.T(), channels.TestNetworkChannel, msg.Channel()) // channel require.Equal(m.T(), m.ids[last].NodeID, msg.OriginId()) // sender id require.Equal(m.T(), firstNode, msg.TargetIDs()[0]) // target id require.Equal(m.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol @@ -807,7 +805,7 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_SendDirect() { msg, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), event, unittest.NetworkCodec().Encode, message.ProtocolTypeUnicast) @@ -835,7 +833,7 @@ func (m *MiddlewareTestSuite) TestLargeMessageSize_SendDirect() { msg, err := message.NewOutgoingScope( flow.IdentifierList{targetNode}, - channels.ProvideChunks, + channels.TopicFromChannel(channels.ProvideChunks, m.sporkId), event, unittest.NetworkCodec().Encode, message.ProtocolTypeUnicast) @@ -888,7 +886,7 @@ func (m *MiddlewareTestSuite) TestMaxMessageSize_Publish() { } msg, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), event, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) @@ -923,7 +921,7 @@ func (m *MiddlewareTestSuite) TestUnsubscribe() { message1, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: string("hello1"), }, @@ -945,13 +943,13 @@ func (m *MiddlewareTestSuite) TestUnsubscribe() { unittest.RequireReturnsBefore(m.T(), msgRcvdFun, 2*time.Second, "message not received") // now unsubscribe the target node from the channel - err = m.mws[last].Unsubscribe(testChannel) + err = m.mws[last].Unsubscribe(channels.TestNetworkChannel) assert.NoError(m.T(), err) // create and send a new message on the channel from the origin node message2, err := message.NewOutgoingScope( flow.IdentifierList{lastNode}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, m.sporkId), &libp2pmessage.TestMessage{ Text: string("hello2"), }, diff --git a/network/test/unicast_authorization_test.go b/network/test/unicast_authorization_test.go index 0dfed501f4e..b2915fc1bd8 100644 --- a/network/test/unicast_authorization_test.go +++ b/network/test/unicast_authorization_test.go @@ -48,7 +48,8 @@ type UnicastAuthorizationTestSuite struct { // providers id providers generated at beginning of a test run providers []*unittest.UpdatableIDProvider // cancel is the cancel func from the context that was used to start the middlewares in a test run - cancel context.CancelFunc + cancel context.CancelFunc + sporkId flow.Identifier // waitCh is the channel used to wait for the middleware to perform authorization and invoke the slashing //violation's consumer before making mock assertions and cleaning up resources waitCh chan struct{} @@ -73,9 +74,9 @@ func (u *UnicastAuthorizationTestSuite) TearDownTest() { // setupMiddlewaresAndProviders will setup 2 middlewares that will be used as a sender and receiver in each suite test. func (u *UnicastAuthorizationTestSuite) setupMiddlewaresAndProviders(slashingViolationsConsumer network.ViolationsConsumer) { - sporkId := unittest.IdentifierFixture() - ids, libP2PNodes := testutils.LibP2PNodeForMiddlewareFixture(u.T(), sporkId, 2) - cfg := testutils.MiddlewareConfigFixture(u.T(), sporkId) + u.sporkId = unittest.IdentifierFixture() + ids, libP2PNodes := testutils.LibP2PNodeForMiddlewareFixture(u.T(), u.sporkId, 2) + cfg := testutils.MiddlewareConfigFixture(u.T(), u.sporkId) mws, providers := testutils.MiddlewareFixtures(u.T(), ids, libP2PNodes, cfg, slashingViolationsConsumer) require.Len(u.T(), ids, 2) require.Len(u.T(), providers, 2) @@ -154,12 +155,12 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnstakedPeer() u.startMiddlewares(overlay) - require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) - require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) + require.NoError(u.T(), u.receiverMW.Subscribe(channels.TestNetworkChannel)) + require.NoError(u.T(), u.senderMW.Subscribe(channels.TestNetworkChannel)) msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, u.sporkId), &libp2pmessage.TestMessage{ Text: string("hello"), }, @@ -216,12 +217,12 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_EjectedPeer() { u.startMiddlewares(overlay) - require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) - require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) + require.NoError(u.T(), u.receiverMW.Subscribe(channels.TestNetworkChannel)) + require.NoError(u.T(), u.senderMW.Subscribe(channels.TestNetworkChannel)) msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, u.sporkId), &libp2pmessage.TestMessage{ Text: string("hello"), }, @@ -282,7 +283,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnauthorizedPee msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - channel, + channels.TopicFromChannel(channels.ConsensusCommittee, u.sporkId), &libp2pmessage.TestMessage{ Text: string("hello"), }, @@ -340,12 +341,12 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnknownMsgCode( u.startMiddlewares(overlay) - require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) - require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) + require.NoError(u.T(), u.receiverMW.Subscribe(channels.TestNetworkChannel)) + require.NoError(u.T(), u.senderMW.Subscribe(channels.TestNetworkChannel)) msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, u.sporkId), &libp2pmessage.TestMessage{ Text: "hello", }, @@ -410,12 +411,12 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_WrongMsgCode() u.startMiddlewares(overlay) - require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) - require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) + require.NoError(u.T(), u.receiverMW.Subscribe(channels.TestNetworkChannel)) + require.NoError(u.T(), u.senderMW.Subscribe(channels.TestNetworkChannel)) msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, u.sporkId), &libp2pmessage.TestMessage{ Text: "hello", }, @@ -447,7 +448,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_PublicChannel() expectedPayload := "hello" msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - testChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, u.sporkId), &libp2pmessage.TestMessage{ Text: expectedPayload, }, @@ -473,7 +474,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_PublicChannel() msg, ok := args[0].(network.IncomingMessageScope) require.True(u.T(), ok) - require.Equal(u.T(), testChannel, msg.Channel()) // channel + require.Equal(u.T(), channels.TestNetworkChannel, msg.Channel()) // channel require.Equal(u.T(), u.senderID.NodeID, msg.OriginId()) // sender id require.Equal(u.T(), u.receiverID.NodeID, msg.TargetIDs()[0]) // target id require.Equal(u.T(), message.ProtocolTypeUnicast, msg.Protocol()) // protocol @@ -482,8 +483,8 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_PublicChannel() u.startMiddlewares(overlay) - require.NoError(u.T(), u.receiverMW.Subscribe(testChannel)) - require.NoError(u.T(), u.senderMW.Subscribe(testChannel)) + require.NoError(u.T(), u.receiverMW.Subscribe(channels.TestNetworkChannel)) + require.NoError(u.T(), u.senderMW.Subscribe(channels.TestNetworkChannel)) // send message via unicast err = u.senderMW.SendDirect(msg) @@ -545,7 +546,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_UnauthorizedUni msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - channel, + channels.TopicFromChannel(channel, u.sporkId), payload, unittest.NetworkCodec().Encode, message.ProtocolTypeUnicast) @@ -602,7 +603,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_ReceiverHasNoSu msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - channel, + channels.TopicFromChannel(channel, u.sporkId), &libp2pmessage.TestMessage{ Text: "TestUnicastAuthorization_ReceiverHasNoSubscription", }, @@ -628,7 +629,7 @@ func (u *UnicastAuthorizationTestSuite) TestUnicastAuthorization_ReceiverHasSubs msg, err := message.NewOutgoingScope( flow.IdentifierList{u.receiverID.NodeID}, - channel, + channels.TopicFromChannel(channel, u.sporkId), &messages.EntityRequest{}, unittest.NetworkCodec().Encode, message.ProtocolTypeUnicast) From 67f60536ab5622306cf994c940b0cf1bf0c5ecfb Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 14:41:41 -0700 Subject: [PATCH 25/35] fixes tests --- network/p2p/dht/dht_test.go | 2 +- network/p2p/middleware/middleware_test.go | 2 +- network/p2p/scoring/app_score_test.go | 4 ++-- network/p2p/scoring/subscription_validator_test.go | 6 +++--- network/p2p/subscription/subscription_filter_test.go | 2 +- .../p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go | 6 ++++-- 6 files changed, 12 insertions(+), 10 deletions(-) diff --git a/network/p2p/dht/dht_test.go b/network/p2p/dht/dht_test.go index 3f5ccb64a18..3bbbf25079c 100644 --- a/network/p2p/dht/dht_test.go +++ b/network/p2p/dht/dht_test.go @@ -156,7 +156,7 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { messageScope, err := message.NewOutgoingScope( ids.NodeIDs(), - channels.TestNetworkChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, sporkId), &libp2pmsg.TestMessage{}, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) diff --git a/network/p2p/middleware/middleware_test.go b/network/p2p/middleware/middleware_test.go index 840d0c3bb07..b1708701418 100644 --- a/network/p2p/middleware/middleware_test.go +++ b/network/p2p/middleware/middleware_test.go @@ -19,7 +19,7 @@ func TestChunkDataPackMaxMessageSize(t *testing.T) { // creates an outgoing chunk data pack response message (imitating an EN is sending a chunk data pack response to VN). msg, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, - channels.ProvideChunks, + channels.TopicFromChannel(channels.ProvideChunks, unittest.IdentifierFixture()), &messages.ChunkDataResponse{ ChunkDataPack: *unittest.ChunkDataPackFixture(unittest.IdentifierFixture()), Nonce: rand.Uint64(), diff --git a/network/p2p/scoring/app_score_test.go b/network/p2p/scoring/app_score_test.go index d43eeeab800..43cbba7f966 100644 --- a/network/p2p/scoring/app_score_test.go +++ b/network/p2p/scoring/app_score_test.go @@ -96,7 +96,7 @@ func TestFullGossipSubConnectivity(t *testing.T) { for _, node := range nodes { outgoingMessageScope, err := message.NewOutgoingScope( ids.NodeIDs(), - channels.PushBlocks, + channels.TopicFromChannel(channels.PushBlocks, sporkId), unittest.ProposalFixture(), unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) @@ -217,7 +217,7 @@ func testGossipSubMessageDeliveryUnderNetworkPartition(t *testing.T, honestPeerS outgoingMessageScope, err := message.NewOutgoingScope( allIds.NodeIDs(), - channels.PushBlocks, + channels.TopicFromChannel(channels.PushBlocks, sporkId), unittest.ProposalFixture(), unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) diff --git a/network/p2p/scoring/subscription_validator_test.go b/network/p2p/scoring/subscription_validator_test.go index b8243fbdadc..2bda9d78a1b 100644 --- a/network/p2p/scoring/subscription_validator_test.go +++ b/network/p2p/scoring/subscription_validator_test.go @@ -240,7 +240,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { outgoingMessageScope, err := message.NewOutgoingScope( ids.NodeIDs(), - channels.PushBlocks, + channels.TopicFromChannel(channels.PushBlocks, sporkId), unittest.ProposalFixture(), unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) @@ -269,7 +269,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // the GossipSub scoring protocol. outgoingMessageScope, err = message.NewOutgoingScope( ids.NodeIDs(), - channels.PushBlocks, + channels.TopicFromChannel(channels.PushBlocks, sporkId), unittest.ProposalFixture(), unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) @@ -284,7 +284,7 @@ func TestSubscriptionValidator_Integration(t *testing.T) { // however, both verification nodes should receive the message. outgoingMessageScope, err = message.NewOutgoingScope( ids.NodeIDs(), - channels.RequestChunks, + channels.TopicFromChannel(channels.RequestChunks, sporkId), &messages.ChunkDataRequest{ ChunkID: unittest.IdentifierFixture(), Nonce: rand.Uint64(), diff --git a/network/p2p/subscription/subscription_filter_test.go b/network/p2p/subscription/subscription_filter_test.go index 59cc739e9e3..ec43a039ab1 100644 --- a/network/p2p/subscription/subscription_filter_test.go +++ b/network/p2p/subscription/subscription_filter_test.go @@ -82,7 +82,7 @@ func TestFilterSubscribe(t *testing.T) { outgoingMessageScope, err := message.NewOutgoingScope( ids.NodeIDs(), - channels.SyncCommittee, + channels.TopicFromChannel(channels.SyncCommittee, sporkId), []byte("hello"), unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) diff --git a/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go b/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go index 40befc19b48..1bcb28a46cc 100644 --- a/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go +++ b/network/p2p/unicast/ratelimit/bandwidth_rate_limiter_test.go @@ -36,9 +36,10 @@ func TestBandWidthRateLimiter_Allow(t *testing.T) { b[i] = byte('X') } + sporkId := unittest.IdentifierFixture() msg, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, - channels.TestNetworkChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, sporkId), &libp2pmessage.TestMessage{ Text: string(b), }, @@ -89,9 +90,10 @@ func TestBandWidthRateLimiter_IsRateLimited(t *testing.T) { require.False(t, bandwidthRateLimiter.IsRateLimited(peerID)) + sporkId := unittest.IdentifierFixture() msg, err := message.NewOutgoingScope( flow.IdentifierList{unittest.IdentifierFixture()}, - channels.TestNetworkChannel, + channels.TopicFromChannel(channels.TestNetworkChannel, sporkId), &libp2pmessage.TestMessage{ Text: string(b), }, From a5eabe199d85bd3caa3a248dd514cd30d460f59b Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 17 Aug 2023 14:47:04 -0700 Subject: [PATCH 26/35] fixes tests --- .../test/gossipsub/scoring/scoring_test.go | 2 +- network/p2p/test/fixtures.go | 17 +++++++++++++---- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go index 2366678576e..11248d109da 100644 --- a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go +++ b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go @@ -483,7 +483,7 @@ func TestGossipSubMeshDeliveryScoring_Replay_Will_Not_Counted(t *testing.T) { proposalList[i] = unittest.ProposalFixture() } i := -1 - p2ptest.EnsurePubsubMessageExchangeFromNode(t, ctx, replayingNode, thisNode, blockTopic, len(proposalList), func() interface{} { + p2ptest.EnsurePubsubMessageExchangeFromNode(t, ctx, replayingNode, thisNode, thisId.NodeID, blockTopic, len(proposalList), func() interface{} { i += 1 return proposalList[i] }) diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 39344980e87..f124ac4d17b 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -612,15 +612,24 @@ func EnsurePubsubMessageExchange(t *testing.T, ctx context.Context, nodes []p2p. // // - ctx: the context- the test will fail if the context expires. // - sender: the node that sends the message to the other node. -// - receiver: the node that receives the message from the other node. +// - receiverNode: the node that receives the message from the other node. +// - receiverIdentifier: the identifier of the receiver node. // - topic: the topic to exchange messages on. // - count: the number of messages to exchange from `sender` to `receiver`. // - messageFactory: a function that creates a unique message to be published by the node. -func EnsurePubsubMessageExchangeFromNode(t *testing.T, ctx context.Context, sender p2p.LibP2PNode, receiver p2p.LibP2PNode, topic channels.Topic, count int, messageFactory func() interface{}) { +func EnsurePubsubMessageExchangeFromNode( + t *testing.T, + ctx context.Context, + sender p2p.LibP2PNode, + receiverNode p2p.LibP2PNode, + receiverIdentifier flow.Identifier, + topic channels.Topic, + count int, + messageFactory func() interface{}) { _, err := sender.Subscribe(topic, validator.TopicValidator(unittest.Logger(), unittest.AllowAllPeerFilter())) require.NoError(t, err) - toSub, err := receiver.Subscribe(topic, validator.TopicValidator(unittest.Logger(), unittest.AllowAllPeerFilter())) + toSub, err := receiverNode.Subscribe(topic, validator.TopicValidator(unittest.Logger(), unittest.AllowAllPeerFilter())) require.NoError(t, err) // let subscriptions propagate @@ -630,7 +639,7 @@ func EnsurePubsubMessageExchangeFromNode(t *testing.T, ctx context.Context, send // creates a unique message to be published by the node payload := messageFactory() outgoingMessageScope, err := message.NewOutgoingScope( - flow.IdentifierList{}, + flow.IdentifierList{receiverIdentifier}, topic, payload, unittest.NetworkCodec().Encode, From 6db0d8d5640d7db34add1ae9756bf9f8241b00d7 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 18 Aug 2023 10:06:06 -0700 Subject: [PATCH 27/35] skims off the spork id from libp2p node --- insecure/corruptlibp2p/p2p_node.go | 4 +--- network/p2p/builder.go | 2 +- network/p2p/libp2pNode.go | 2 +- network/p2p/p2pbuilder/libp2pNodeBuilder.go | 5 ++--- network/p2p/p2pnode/libp2pNode.go | 10 +++++----- 5 files changed, 10 insertions(+), 13 deletions(-) diff --git a/insecure/corruptlibp2p/p2p_node.go b/insecure/corruptlibp2p/p2p_node.go index 02ce4e3a3f3..b3fbd0cb36d 100644 --- a/insecure/corruptlibp2p/p2p_node.go +++ b/insecure/corruptlibp2p/p2p_node.go @@ -8,7 +8,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/rs/zerolog" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/codec/cbor" @@ -54,11 +53,10 @@ func (n *CorruptP2PNode) Subscribe(topic channels.Topic, _ p2p.TopicValidatorFun // NewCorruptLibP2PNode returns corrupted libP2PNode that will subscribe to topics using the AcceptAllTopicValidator. func NewCorruptLibP2PNode( logger zerolog.Logger, - sporkId flow.Identifier, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager, disallowListCacheCfg *p2p.DisallowListCacheConfig) p2p.LibP2PNode { - node := p2pnode.NewNode(logger, sporkId, host, pCache, peerManager, disallowListCacheCfg) + node := p2pnode.NewNode(logger, host, pCache, peerManager, disallowListCacheCfg) return &CorruptP2PNode{Node: node, logger: logger, codec: cbor.NewCodec()} } diff --git a/network/p2p/builder.go b/network/p2p/builder.go index b530f4cc49c..b856f931a29 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -23,7 +23,7 @@ import ( ) type GossipSubFactoryFunc func(context.Context, zerolog.Logger, host.Host, PubSubAdapterConfig, CollectionClusterChangesConsumer) (PubSubAdapter, error) -type CreateNodeFunc func(zerolog.Logger, flow.Identifier, host.Host, ProtocolPeerCache, PeerManager, *DisallowListCacheConfig) LibP2PNode +type CreateNodeFunc func(zerolog.Logger, host.Host, ProtocolPeerCache, PeerManager, *DisallowListCacheConfig) LibP2PNode type GossipSubAdapterConfigFunc func(*BasePubSubAdapterConfig) PubSubAdapterConfig // GossipSubBuilder provides a builder pattern for creating a GossipSub pubsub system. diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 0645be384e1..68f907a80b4 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -65,7 +65,7 @@ type LibP2PNode interface { // Subscribe subscribes the node to the given topic and returns the subscription Subscribe(topic channels.Topic, topicValidator TopicValidatorFunc) (Subscription, error) // Unsubscribe cancels the subscriber and closes the topic corresponding to the given channel. - Unsubscribe(channel channels.Channel) error + Unsubscribe(topic channels.Topic) error // Publish publishes the given payload on the topic. Publish(ctx context.Context, scope network.OutgoingMessageScope) error // Host returns pointer to host object of node. diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index 42e22a2fb02..7ded51080bc 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -301,7 +301,7 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { } } - node := builder.createNode(builder.logger, builder.sporkId, h, pCache, peerManager, builder.disallowListCacheCfg) + node := builder.createNode(builder.logger, h, pCache, peerManager, builder.disallowListCacheCfg) if builder.connGater != nil { builder.connGater.SetDisallowListOracle(node) @@ -414,12 +414,11 @@ func defaultLibP2POptions(address string, key fcrypto.PrivateKey) ([]config.Opti // DefaultCreateNodeFunc returns new libP2P node. func DefaultCreateNodeFunc(logger zerolog.Logger, - sporkId flow.Identifier, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager, disallowListCacheCfg *p2p.DisallowListCacheConfig) p2p.LibP2PNode { - return p2pnode.NewNode(logger, sporkId, host, pCache, peerManager, disallowListCacheCfg) + return p2pnode.NewNode(logger, host, pCache, peerManager, disallowListCacheCfg) } // DefaultNodeBuilder returns a node builder. diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 6a772217c68..8a19c715c56 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -66,13 +66,11 @@ type Node struct { // Cache of temporary disallow-listed peers, when a peer is disallow-listed, the connections to that peer // are closed and further connections are not allowed till the peer is removed from the disallow-list. disallowListedCache p2p.DisallowListCache - sporkId flow.Identifier } // NewNode creates a new libp2p node and sets its parameters. func NewNode( logger zerolog.Logger, - sporkId flow.Identifier, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager, @@ -81,7 +79,6 @@ func NewNode( return &Node{ host: host, logger: logger.With().Str("component", "libp2p-node").Logger(), - sporkId: sporkId, topics: make(map[channels.Topic]p2p.Topic), subs: make(map[channels.Topic]p2p.Subscription), pCache: pCache, @@ -281,9 +278,12 @@ func (n *Node) Subscribe(topic channels.Topic, topicValidator p2p.TopicValidator } // Unsubscribe cancels the subscriber and closes the topic. +// Args: +// topic: topic to unsubscribe from. +// Returns: +// error: error if any, which means unsubscribe failed. // All errors returned from this function can be considered benign. -func (n *Node) Unsubscribe(channel channels.Channel) error { - topic := channels.TopicFromChannel(channel, n.sporkId) +func (n *Node) Unsubscribe(topic channels.Topic) error { err := n.unsubscribeTopic(topic) if err != nil { return fmt.Errorf("failed to unsubscribe from topic: %w", err) From 14c893987a4d604c8740b5ae1a5247b9b12b7d58 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 18 Aug 2023 10:34:17 -0700 Subject: [PATCH 28/35] lint and test fix --- cmd/access/node_builder/access_node_builder.go | 2 +- cmd/observer/node_builder/observer_builder.go | 2 +- cmd/scaffold.go | 2 +- follower/follower_builder.go | 2 +- network/internal/testutils/testUtil.go | 2 +- network/message_scope.go | 6 ++++-- network/p2p/dht/dht_test.go | 2 +- network/p2p/libp2pNode.go | 2 +- network/p2p/middleware/middleware.go | 15 ++++++++------- network/p2p/p2pnode/libp2pNode.go | 18 +++++++++--------- 10 files changed, 28 insertions(+), 25 deletions(-) diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 5d33051cf96..e87f1e6327d 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -1405,7 +1405,7 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, Libp2pNode: libp2pNode, FlowId: nodeID, BitSwapMetrics: builder.Metrics.Bitswap, - RootBlockID: builder.SporkID, + SporkId: builder.SporkID, UnicastMessageTimeout: middleware.DefaultUnicastTimeout, IdTranslator: builder.IDTranslator, Codec: builder.CodecFactory(), diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index 207e6c167be..1e294e70d01 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -1032,7 +1032,7 @@ func (builder *ObserverServiceBuilder) initMiddleware(nodeID flow.Identifier, Libp2pNode: libp2pNode, FlowId: nodeID, BitSwapMetrics: builder.Metrics.Bitswap, - RootBlockID: builder.SporkID, + SporkId: builder.SporkID, UnicastMessageTimeout: middleware.DefaultUnicastTimeout, IdTranslator: builder.IDTranslator, Codec: builder.CodecFactory(), diff --git a/cmd/scaffold.go b/cmd/scaffold.go index e7ba460a53d..61d9680f524 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -440,7 +440,7 @@ func (fnb *FlowNodeBuilder) InitFlowNetworkWithConduitFactory( Libp2pNode: fnb.LibP2PNode, FlowId: fnb.Me.NodeID(), BitSwapMetrics: fnb.Metrics.Bitswap, - RootBlockID: fnb.SporkID, + SporkId: fnb.SporkID, UnicastMessageTimeout: fnb.FlowConfig.NetworkConfig.UnicastMessageTimeout, IdTranslator: fnb.IDTranslator, Codec: fnb.CodecFactory(), diff --git a/follower/follower_builder.go b/follower/follower_builder.go index d03ccde45c6..4b561916fe4 100644 --- a/follower/follower_builder.go +++ b/follower/follower_builder.go @@ -759,7 +759,7 @@ func (builder *FollowerServiceBuilder) initMiddleware(nodeID flow.Identifier, Libp2pNode: libp2pNode, FlowId: nodeID, BitSwapMetrics: builder.Metrics.Bitswap, - RootBlockID: builder.SporkID, + SporkId: builder.SporkID, UnicastMessageTimeout: middleware.DefaultUnicastTimeout, IdTranslator: builder.IDTranslator, Codec: builder.CodecFactory(), diff --git a/network/internal/testutils/testUtil.go b/network/internal/testutils/testUtil.go index ddfc19f186c..e08482e6acd 100644 --- a/network/internal/testutils/testUtil.go +++ b/network/internal/testutils/testUtil.go @@ -154,7 +154,7 @@ func MiddlewareConfigFixture(t *testing.T, sporkId flow.Identifier) *middleware. return &middleware.Config{ Logger: unittest.Logger(), BitSwapMetrics: metrics.NewNoopCollector(), - RootBlockID: sporkId, + SporkId: sporkId, UnicastMessageTimeout: middleware.DefaultUnicastTimeout, Codec: unittest.NetworkCodec(), } diff --git a/network/message_scope.go b/network/message_scope.go index fe37a795572..4e4ded4b9cc 100644 --- a/network/message_scope.go +++ b/network/message_scope.go @@ -6,7 +6,8 @@ import ( "github.com/onflow/flow-go/network/message" ) -// IncomingMessageScope defines the interface for handling incoming message scope. +// IncomingMessageScope defines the interface for incoming message scopes, i.e., self-contained messages that have been +// received on the wire and are ready to be processed. type IncomingMessageScope interface { // OriginId returns the origin node ID. OriginId() flow.Identifier @@ -36,7 +37,8 @@ type IncomingMessageScope interface { PayloadType() string } -// OutgoingMessageScope defines the interface for handling outgoing message scope. +// OutgoingMessageScope defines the interface for building outgoing message scopes, i.e., self-contained messages +// that are ready to be sent on the wire. type OutgoingMessageScope interface { // TargetIds returns the target node IDs. TargetIds() flow.IdentifierList diff --git a/network/p2p/dht/dht_test.go b/network/p2p/dht/dht_test.go index 3bbbf25079c..ae306f2effd 100644 --- a/network/p2p/dht/dht_test.go +++ b/network/p2p/dht/dht_test.go @@ -163,7 +163,7 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { require.NoError(t, err) logger := unittest.Logger() - topic := channels.TopicFromChannel(channels.TestNetworkChannel, sporkId) + topic := channels.Topic("/flow/" + unittest.IdentifierFixture().String()) topicValidator := flowpubsub.TopicValidator(logger, unittest.AllowAllPeerFilter()) for _, n := range nodes { s, err := n.Subscribe(topic, topicValidator) diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 68f907a80b4..8426bd18ba6 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -67,7 +67,7 @@ type LibP2PNode interface { // Unsubscribe cancels the subscriber and closes the topic corresponding to the given channel. Unsubscribe(topic channels.Topic) error // Publish publishes the given payload on the topic. - Publish(ctx context.Context, scope network.OutgoingMessageScope) error + Publish(ctx context.Context, messageScope network.OutgoingMessageScope) error // Host returns pointer to host object of node. Host() host.Host // WithDefaultUnicastProtocol overrides the default handler of the unicast manager and registers all preferred protocols. diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 904f99aaa54..90a82fc9cb6 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -88,7 +88,7 @@ type Middleware struct { libP2PNode p2p.LibP2PNode preferredUnicasts []protocols.ProtocolName bitswapMetrics module.BitswapMetrics - rootBlockID flow.Identifier + sporkId flow.Identifier validators []network.MessageValidator peerManagerFilters []p2p.PeerFilter unicastMessageTimeout time.Duration @@ -135,7 +135,7 @@ type Config struct { Libp2pNode p2p.LibP2PNode FlowId flow.Identifier // This node's Flow ID BitSwapMetrics module.BitswapMetrics - RootBlockID flow.Identifier + SporkId flow.Identifier UnicastMessageTimeout time.Duration IdTranslator p2p.IDTranslator Codec network.Codec @@ -165,7 +165,7 @@ func NewMiddleware(cfg *Config, opts ...OptionFn) *Middleware { log: cfg.Logger, libP2PNode: cfg.Libp2pNode, bitswapMetrics: cfg.BitSwapMetrics, - rootBlockID: cfg.RootBlockID, + sporkId: cfg.SporkId, validators: DefaultValidators(cfg.Logger, cfg.FlowId), unicastMessageTimeout: cfg.UnicastMessageTimeout, idTranslator: cfg.IdTranslator, @@ -512,7 +512,7 @@ func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { } channel := channels.Channel(msg.ChannelID) - topic := channels.TopicFromChannel(channel, m.rootBlockID) + topic := channels.TopicFromChannel(channel, m.sporkId) // ignore messages if node does not have subscription to topic if !m.libP2PNode.HasSubscription(topic) { @@ -577,8 +577,7 @@ func (m *Middleware) handleIncomingStream(s libp2pnetwork.Stream) { // Subscribe subscribes the middleware to a channel. // No errors are expected during normal operation. func (m *Middleware) Subscribe(channel channels.Channel) error { - - topic := channels.TopicFromChannel(channel, m.rootBlockID) + topic := channels.TopicFromChannel(channel, m.sporkId) var peerFilter p2p.PeerFilter var validators []validator.PubSubMessageValidator @@ -628,7 +627,8 @@ func (m *Middleware) processPubSubMessages(msg *message.Message, peerID peer.ID) // // All errors returned from this function can be considered benign. func (m *Middleware) Unsubscribe(channel channels.Channel) error { - return m.libP2PNode.Unsubscribe(channel) + topic := channels.TopicFromChannel(channel, m.sporkId) + return m.libP2PNode.Unsubscribe(topic) } // processUnicastStreamMessage will decode, perform authorized sender validation and process a message @@ -767,6 +767,7 @@ func (m *Middleware) processMessage(scope network.IncomingMessageScope) { // - the libP2P node fails to publish the message. // // All errors returned from this function can be considered benign. +// TODO: publish has made ready to be removed from middleware, and instead the libp2pNode.Publish should be used directly. func (m *Middleware) Publish(msg network.OutgoingMessageScope) error { return m.libP2PNode.Publish(m.ctx, msg) } diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 8a19c715c56..9fcbcc9a466 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -341,16 +341,16 @@ func (n *Node) unsubscribeTopic(topic channels.Topic) error { // Publish publishes the given payload on the topic. // All errors returned from this function can be considered benign. -func (n *Node) Publish(ctx context.Context, msgScope flownet.OutgoingMessageScope) error { +func (n *Node) Publish(ctx context.Context, messageScope flownet.OutgoingMessageScope) error { lg := n.logger.With(). - Str("topic", msgScope.Topic().String()). - Interface("proto_message", msgScope.Proto()). - Str("payload_type", msgScope.PayloadType()). - Int("message_size", msgScope.Size()).Logger() + Str("topic", messageScope.Topic().String()). + Interface("proto_message", messageScope.Proto()). + Str("payload_type", messageScope.PayloadType()). + Int("message_size", messageScope.Size()).Logger() lg.Debug().Msg("received message to publish") // convert the message to bytes to be put on the wire. - data, err := msgScope.Proto().Marshal() + data, err := messageScope.Proto().Marshal() if err != nil { return fmt.Errorf("failed to marshal the message: %w", err) } @@ -362,13 +362,13 @@ func (n *Node) Publish(ctx context.Context, msgScope flownet.OutgoingMessageScop return fmt.Errorf("message size %d exceeds configured max message size %d", msgSize, DefaultMaxPubSubMsgSize) } - ps, found := n.topics[msgScope.Topic()] + ps, found := n.topics[messageScope.Topic()] if !found { - return fmt.Errorf("could not find topic (%s)", msgScope.Topic()) + return fmt.Errorf("could not find topic (%s)", messageScope.Topic()) } err = ps.Publish(ctx, data) if err != nil { - return fmt.Errorf("could not publish to topic (%s): %w", msgScope.Topic(), err) + return fmt.Errorf("could not publish to topic (%s): %w", messageScope.Topic(), err) } lg.Debug().Msg("published message to topic") From 4d296ee7c4f8dad495a352dbc41e7f4561464337 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 18 Aug 2023 10:43:46 -0700 Subject: [PATCH 29/35] fixes build errors --- network/p2p/dht/dht_test.go | 6 ++--- network/p2p/test/sporking_test.go | 24 ++++++++----------- .../p2p/tracer/gossipSubMeshTracer_test.go | 6 ++--- 3 files changed, 16 insertions(+), 20 deletions(-) diff --git a/network/p2p/dht/dht_test.go b/network/p2p/dht/dht_test.go index ae306f2effd..534cca7eaa6 100644 --- a/network/p2p/dht/dht_test.go +++ b/network/p2p/dht/dht_test.go @@ -121,6 +121,7 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { // N4 N5 N4-----N5 sporkId := unittest.IdentifierFixture() + topic := channels.TopicFromChannel(channels.TestNetworkChannel, sporkId) idProvider := mockmodule.NewIdentityProvider(t) // create one node running the DHT Server (mimicking the staked AN) dhtServerNodes, serverIDs := p2ptest.NodesFixture(t, sporkId, "dht_test", 1, idProvider, p2ptest.WithDHTOptions(dht.AsServer())) @@ -156,14 +157,13 @@ func TestPubSubWithDHTDiscovery(t *testing.T) { messageScope, err := message.NewOutgoingScope( ids.NodeIDs(), - channels.TopicFromChannel(channels.TestNetworkChannel, sporkId), + topic, &libp2pmsg.TestMessage{}, unittest.NetworkCodec().Encode, message.ProtocolTypePubSub) require.NoError(t, err) logger := unittest.Logger() - topic := channels.Topic("/flow/" + unittest.IdentifierFixture().String()) topicValidator := flowpubsub.TopicValidator(logger, unittest.AllowAllPeerFilter()) for _, n := range nodes { s, err := n.Subscribe(topic, topicValidator) @@ -218,6 +218,6 @@ loop: // Step 6: unsubscribes all nodes from the topic for _, n := range nodes { - assert.NoError(t, n.Unsubscribe(channels.TestNetworkChannel)) + assert.NoError(t, n.Unsubscribe(topic)) } } diff --git a/network/p2p/test/sporking_test.go b/network/p2p/test/sporking_test.go index ba5778b17b6..41a0379b2ac 100644 --- a/network/p2p/test/sporking_test.go +++ b/network/p2p/test/sporking_test.go @@ -86,10 +86,12 @@ func TestCrosstalkPreventionOnNetworkKeyChange(t *testing.T) { require.NoError(t, err) // create stream from node 1 to node 2 - testOneToOneMessagingSucceeds(t, node1, peerInfo2) + node1.Host().Peerstore().AddAddrs(peerInfo2.ID, peerInfo2.Addrs, peerstore.AddressTTL) + s, err := node1.CreateStream(context.Background(), peerInfo2.ID) + require.NoError(t, err) + assert.NotNil(t, s) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain - // stop node 2 and start it again with a different networking key but on the same IP and port p2ptest.StopNode(t, node2, cancel2, 100*time.Millisecond) @@ -150,8 +152,11 @@ func TestOneToOneCrosstalkPrevention(t *testing.T) { idProvider.SetIdentities(flow.IdentityList{&id1, &id2}) p2ptest.StartNode(t, signalerCtx2, node2, 100*time.Millisecond) - // create stream from node 2 to node 1 - testOneToOneMessagingSucceeds(t, node2, peerInfo1) + // create stream from node 1 to node 2 + node2.Host().Peerstore().AddAddrs(peerInfo1.ID, peerInfo1.Addrs, peerstore.AddressTTL) + s, err := node2.CreateStream(context.Background(), peerInfo1.ID) + require.NoError(t, err) + assert.NotNil(t, s) // Simulate a hard-spoon: node1 is on the old chain, but node2 is moved from the old chain to the new chain // stop node 2 and start it again with a different libp2p protocol id to listen for @@ -271,7 +276,7 @@ func TestOneToKCrosstalkPrevention(t *testing.T) { // mimic that node1 is now part of the new spork while node2 remains on the old spork // by unsubscribing node1 from 'topicBeforeSpork' and subscribing it to 'topicAfterSpork' // and keeping node2 subscribed to topic 'topicBeforeSpork' - err = node1.Unsubscribe(channels.TestNetworkChannel) + err = node1.Unsubscribe(topicBeforeSpork) require.NoError(t, err) _, err = node1.Subscribe(topicAfterSpork, topicValidator) require.NoError(t, err) @@ -300,15 +305,6 @@ func TestOneToKCrosstalkPrevention(t *testing.T) { "nodes on different sporks were able to communicate") } -func testOneToOneMessagingSucceeds(t *testing.T, sourceNode p2p.LibP2PNode, peerInfo peer.AddrInfo) { - // create stream from node 1 to node 2 - sourceNode.Host().Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) - s, err := sourceNode.CreateStream(context.Background(), peerInfo.ID) - // assert that stream creation succeeded - require.NoError(t, err) - assert.NotNil(t, s) -} - func testOneToOneMessagingFails(t *testing.T, sourceNode p2p.LibP2PNode, peerInfo peer.AddrInfo) { // create stream from source node to destination address sourceNode.Host().Peerstore().AddAddrs(peerInfo.ID, peerInfo.Addrs, peerstore.AddressTTL) diff --git a/network/p2p/tracer/gossipSubMeshTracer_test.go b/network/p2p/tracer/gossipSubMeshTracer_test.go index 4384ffcf9f1..6e7bc77e0f6 100644 --- a/network/p2p/tracer/gossipSubMeshTracer_test.go +++ b/network/p2p/tracer/gossipSubMeshTracer_test.go @@ -178,9 +178,9 @@ func TestGossipSubMeshTracer(t *testing.T) { // all nodes except the tracerNode unsubscribe from the topic1, which triggers sending a PRUNE to the tracerNode for each unsubscription. // We expect the tracerNode to remove the otherNode1, otherNode2, and unknownNode from its mesh. - require.NoError(t, otherNode1.Unsubscribe(channel1)) - require.NoError(t, otherNode2.Unsubscribe(channel1)) - require.NoError(t, unknownNode.Unsubscribe(channel1)) + require.NoError(t, otherNode1.Unsubscribe(topic1)) + require.NoError(t, otherNode2.Unsubscribe(topic1)) + require.NoError(t, unknownNode.Unsubscribe(topic1)) assert.Eventually(t, func() bool { // eventually, the tracerNode should not have the other node in its mesh for topic1. From d306a89fe6b9cf95eb88c1d704d9e751f55ea714 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 18 Aug 2023 10:50:40 -0700 Subject: [PATCH 30/35] updates mocks --- network/p2p/mock/create_node_func.go | 12 +++++------- network/p2p/mock/lib_p2_p_node.go | 18 +++++++++--------- 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/network/p2p/mock/create_node_func.go b/network/p2p/mock/create_node_func.go index b2db19f13ee..1a57772cbeb 100644 --- a/network/p2p/mock/create_node_func.go +++ b/network/p2p/mock/create_node_func.go @@ -4,8 +4,6 @@ package mockp2p import ( host "github.com/libp2p/go-libp2p/core/host" - flow "github.com/onflow/flow-go/model/flow" - mock "github.com/stretchr/testify/mock" p2p "github.com/onflow/flow-go/network/p2p" @@ -18,13 +16,13 @@ type CreateNodeFunc struct { mock.Mock } -// Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4, _a5 -func (_m *CreateNodeFunc) Execute(_a0 zerolog.Logger, _a1 flow.Identifier, _a2 host.Host, _a3 p2p.ProtocolPeerCache, _a4 p2p.PeerManager, _a5 *p2p.DisallowListCacheConfig) p2p.LibP2PNode { - ret := _m.Called(_a0, _a1, _a2, _a3, _a4, _a5) +// Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4 +func (_m *CreateNodeFunc) Execute(_a0 zerolog.Logger, _a1 host.Host, _a2 p2p.ProtocolPeerCache, _a3 p2p.PeerManager, _a4 *p2p.DisallowListCacheConfig) p2p.LibP2PNode { + ret := _m.Called(_a0, _a1, _a2, _a3, _a4) var r0 p2p.LibP2PNode - if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, host.Host, p2p.ProtocolPeerCache, p2p.PeerManager, *p2p.DisallowListCacheConfig) p2p.LibP2PNode); ok { - r0 = rf(_a0, _a1, _a2, _a3, _a4, _a5) + if rf, ok := ret.Get(0).(func(zerolog.Logger, host.Host, p2p.ProtocolPeerCache, p2p.PeerManager, *p2p.DisallowListCacheConfig) p2p.LibP2PNode); ok { + r0 = rf(_a0, _a1, _a2, _a3, _a4) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(p2p.LibP2PNode) diff --git a/network/p2p/mock/lib_p2_p_node.go b/network/p2p/mock/lib_p2_p_node.go index 008ad125a96..2fcdc336ad7 100644 --- a/network/p2p/mock/lib_p2_p_node.go +++ b/network/p2p/mock/lib_p2_p_node.go @@ -284,13 +284,13 @@ func (_m *LibP2PNode) PeerScoreExposer() p2p.PeerScoreExposer { return r0 } -// Publish provides a mock function with given fields: ctx, scope -func (_m *LibP2PNode) Publish(ctx context.Context, scope flow_gonetwork.OutgoingMessageScope) error { - ret := _m.Called(ctx, scope) +// Publish provides a mock function with given fields: ctx, messageScope +func (_m *LibP2PNode) Publish(ctx context.Context, messageScope flow_gonetwork.OutgoingMessageScope) error { + ret := _m.Called(ctx, messageScope) var r0 error if rf, ok := ret.Get(0).(func(context.Context, flow_gonetwork.OutgoingMessageScope) error); ok { - r0 = rf(ctx, scope) + r0 = rf(ctx, messageScope) } else { r0 = ret.Error(0) } @@ -430,13 +430,13 @@ func (_m *LibP2PNode) Subscribe(topic channels.Topic, topicValidator p2p.TopicVa return r0, r1 } -// Unsubscribe provides a mock function with given fields: channel -func (_m *LibP2PNode) Unsubscribe(channel channels.Channel) error { - ret := _m.Called(channel) +// Unsubscribe provides a mock function with given fields: topic +func (_m *LibP2PNode) Unsubscribe(topic channels.Topic) error { + ret := _m.Called(topic) var r0 error - if rf, ok := ret.Get(0).(func(channels.Channel) error); ok { - r0 = rf(channel) + if rf, ok := ret.Get(0).(func(channels.Topic) error); ok { + r0 = rf(topic) } else { r0 = ret.Error(0) } From d899924114f47da1b514acf437b7f5bcb2e956fc Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Wed, 23 Aug 2023 10:30:26 -0700 Subject: [PATCH 31/35] Update network/p2p/test/topic_validator_test.go Co-authored-by: Khalil Claybon --- network/p2p/test/topic_validator_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/network/p2p/test/topic_validator_test.go b/network/p2p/test/topic_validator_test.go index 0245985dc27..c7959ee1a02 100644 --- a/network/p2p/test/topic_validator_test.go +++ b/network/p2p/test/topic_validator_test.go @@ -309,7 +309,6 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { // publish fails because the topic conversion fails require.Error(t, err) - fmt.Println(err) // ensure the correct error is contained in the logged error require.Contains(t, hook.Logs(), "could not convert topic to channel") } From 7fefe6be5f00735ed650594dba933a81010c6160 Mon Sep 17 00:00:00 2001 From: Yahya Date: Wed, 23 Aug 2023 12:11:00 -0700 Subject: [PATCH 32/35] fixes merge conflicts --- network/p2p/scoring/app_score_test.go | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/network/p2p/scoring/app_score_test.go b/network/p2p/scoring/app_score_test.go index e15aafd9e01..771a442a213 100644 --- a/network/p2p/scoring/app_score_test.go +++ b/network/p2p/scoring/app_score_test.go @@ -201,14 +201,6 @@ func TestFullGossipSubConnectivityAmongHonestNodesWithMaliciousMajority(t *testi // let nodes reside on a full topology, hence no partition is caused by the topology. p2ptest.LetNodesDiscoverEachOther(t, ctx, allNodes, allIds) - outgoingMessageScope, err := message.NewOutgoingScope( - allIds.NodeIDs(), - channels.TopicFromChannel(channels.PushBlocks, sporkId), - unittest.ProposalFixture(), - unittest.NetworkCodec().Encode, - message.ProtocolTypePubSub) - require.NoError(t, err) - require.NoError(t, con1Node.Publish(ctx, outgoingMessageScope)) // checks whether con1 and con2 are in the same mesh tick := time.Second // Set the tick duration as needed timeout := 5 * time.Second // Set the timeout duration as needed @@ -246,20 +238,6 @@ func TestFullGossipSubConnectivityAmongHonestNodesWithMaliciousMajority(t *testi require.Fail(t, "timed out waiting for con1 to have con2 in its mesh; honest nodes are not on each others' topic mesh on GossipSub") } } - // we check that whether within a one-second window the message is received by the other honest consensus node. - // the one-second window is important because it triggers the heartbeat of the con1Node to perform a lazy pull (iHave). - // And con1Node may randomly choose con2Node as the peer to perform the lazy pull. - // However, under a network partition con2Node is not in the mesh of con1Node, and hence is deprived of the eager push from con1Node. - // - // If no honest peer scoring is enabled, then con1Node and con2Node are less-likely to be in the same mesh, and hence the message is not delivered. - // If honest peer scoring is enabled, then con1Node and con2Node are certainly in the same mesh, and hence the message is delivered. - ctx1s, cancel1s := context.WithTimeout(ctx, 1*time.Second) - defer cancel1s() - - expectedReceivedData, err := outgoingMessageScope.Proto().Marshal() - require.NoError(t, err) - - return p2pfixtures.HasSubReceivedMessage(t, ctx1s, expectedReceivedData, con2Sub) } // maliciousAppSpecificScore returns a malicious app specific penalty function that rewards the malicious node and From daddb3157ea72c27858ac00a4a63fd696129e992 Mon Sep 17 00:00:00 2001 From: Yahya Date: Wed, 23 Aug 2023 13:32:27 -0700 Subject: [PATCH 33/35] fixes lint --- network/alsp/manager/manager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/alsp/manager/manager_test.go b/network/alsp/manager/manager_test.go index 3f6e5a1629d..a02bc5b3002 100644 --- a/network/alsp/manager/manager_test.go +++ b/network/alsp/manager/manager_test.go @@ -323,7 +323,7 @@ func TestHandleReportedMisbehavior_And_DisallowListing_RepeatOffender_Integratio ids, nodes := testutils.LibP2PNodeForMiddlewareFixture(t, sporkId, 3, p2ptest.WithPeerManagerEnabled(p2ptest.PeerManagerConfigFixture(p2ptest.WithZeroJitterAndZeroBackoff(t)), nil)) mws, _ := testutils.MiddlewareFixtures(t, ids, nodes, testutils.MiddlewareConfigFixture(t, sporkId), mocknetwork.NewViolationsConsumer(t)) - networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, mws[0], p2p.WithAlspConfig(cfg)) + networkCfg := testutils.NetworkConfigFixture(t, *ids[0], ids, sporkId, mws[0], p2p.WithAlspConfig(cfg)) victimNetwork, err := p2p.NewNetwork(networkCfg) require.NoError(t, err) From b59818fc585db56877b229546d9e597e82ca00d3 Mon Sep 17 00:00:00 2001 From: "Yahya Hassanzadeh, Ph.D" Date: Thu, 24 Aug 2023 08:44:13 -0700 Subject: [PATCH 34/35] Update network/p2p/middleware/middleware.go Co-authored-by: Misha --- network/p2p/middleware/middleware.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 6cabb7103b0..37cbce33c6e 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -767,7 +767,7 @@ func (m *Middleware) processMessage(scope network.IncomingMessageScope) { // - the libP2P node fails to publish the message. // // All errors returned from this function can be considered benign. -// TODO: publish has made ready to be removed from middleware, and instead the libp2pNode.Publish should be used directly. +// TODO: DO NOT USE. Publish is ready to be removed from middleware. Use libp2pNode.Publish directly. func (m *Middleware) Publish(msg network.OutgoingMessageScope) error { return m.libP2PNode.Publish(m.ctx, msg) } From 701e14d8dd940b6da2749925d6c01b858bf3cdd1 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 24 Aug 2023 09:04:33 -0700 Subject: [PATCH 35/35] removes maybe from the test --- network/p2p/test/topic_validator_test.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/network/p2p/test/topic_validator_test.go b/network/p2p/test/topic_validator_test.go index af72371d0df..f5ea66d44a2 100644 --- a/network/p2p/test/topic_validator_test.go +++ b/network/p2p/test/topic_validator_test.go @@ -298,11 +298,10 @@ func TestTopicValidator_InvalidTopic(t *testing.T) { // overrides the topic to be an invalid topic corruptOutgoingMessageScope := mocknetwork.NewOutgoingMessageScope(t) - corruptOutgoingMessageScope.On("TargetIds").Return(dummyMessageScope.TargetIds()).Maybe() - corruptOutgoingMessageScope.On("Topic").Return(topic).Maybe() - corruptOutgoingMessageScope.On("Proto").Return(dummyMessageScope.Proto()).Maybe() - corruptOutgoingMessageScope.On("PayloadType").Return(dummyMessageScope.PayloadType()).Maybe() - corruptOutgoingMessageScope.On("Size").Return(dummyMessageScope.Size()).Maybe() + corruptOutgoingMessageScope.On("Topic").Return(topic) + corruptOutgoingMessageScope.On("Proto").Return(dummyMessageScope.Proto()) + corruptOutgoingMessageScope.On("PayloadType").Return(dummyMessageScope.PayloadType()) + corruptOutgoingMessageScope.On("Size").Return(dummyMessageScope.Size()) // create a dummy block proposal to publish from our SN node err = sn2.Publish(timedCtx, corruptOutgoingMessageScope)