diff --git a/cmd/access/node_builder/access_node_builder.go b/cmd/access/node_builder/access_node_builder.go index 2568a0577e0..e50dadc9948 100644 --- a/cmd/access/node_builder/access_node_builder.go +++ b/cmd/access/node_builder/access_node_builder.go @@ -812,11 +812,11 @@ func (builder *FlowAccessNodeBuilder) InitIDProviders() { } builder.IDTranslator = translator.NewHierarchicalIDTranslator(idCache, translator.NewPublicNetworkIDTranslator()) - builder.NodeDisallowListDistributor = cmd.BuildDisallowListNotificationDisseminator(builder.DisallowListNotificationCacheSize, builder.MetricsRegisterer, builder.Logger, builder.MetricsEnabled) - // The following wrapper allows to disallow-list byzantine nodes via an admin command: // the wrapper overrides the 'Ejected' flag of disallow-listed nodes to true - disallowListWrapper, err := cache.NewNodeBlocklistWrapper(idCache, node.DB, builder.NodeDisallowListDistributor) + disallowListWrapper, err := cache.NewNodeDisallowListWrapper(idCache, node.DB, func() network.DisallowListNotificationConsumer { + return builder.Middleware + }) if err != nil { return fmt.Errorf("could not initialize NodeBlockListWrapper: %w", err) } @@ -824,9 +824,9 @@ func (builder *FlowAccessNodeBuilder) InitIDProviders() { // register the wrapper for dynamic configuration via admin command err = node.ConfigManager.RegisterIdentifierListConfig("network-id-provider-blocklist", - disallowListWrapper.GetBlocklist, disallowListWrapper.Update) + disallowListWrapper.GetDisallowList, disallowListWrapper.Update) if err != nil { - return fmt.Errorf("failed to register blocklist with config manager: %w", err) + return fmt.Errorf("failed to register disallow-list wrapper with config manager: %w", err) } builder.SyncEngineParticipantsProviderFactory = func() module.IdentifierProvider { @@ -841,11 +841,6 @@ func (builder *FlowAccessNodeBuilder) InitIDProviders() { } return nil }) - - builder.Component("disallow list notification distributor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // distributor is returned as a component to be started and stopped. - return builder.NodeDisallowListDistributor, nil - }) } func (builder *FlowAccessNodeBuilder) Initialize() error { @@ -1221,7 +1216,11 @@ func (builder *FlowAccessNodeBuilder) initPublicLibp2pNode(networkKey crypto.Pri bindAddress, networkKey, builder.SporkID, - builder.LibP2PResourceManagerConfig). + builder.LibP2PResourceManagerConfig, + &p2p.DisallowListCacheConfig{ + MaxSize: builder.BaseConfig.NetworkConfig.DisallowListCacheSize, + Metrics: metrics.DisallowListCacheMetricsFactory(builder.HeroCacheMetricsFactory(), network.PublicNetwork), + }). SetBasicResolver(builder.Resolver). SetSubscriptionFilter( subscription.NewRoleBasedFilter( @@ -1262,20 +1261,19 @@ func (builder *FlowAccessNodeBuilder) initMiddleware(nodeID flow.Identifier, validators ...network.MessageValidator, ) network.Middleware { logger := builder.Logger.With().Bool("staked", false).Logger() - slashingViolationsConsumer := slashing.NewSlashingViolationsConsumer(logger, networkMetrics) - mw := middleware.NewMiddleware( - logger, - libp2pNode, - nodeID, - builder.Metrics.Bitswap, - builder.SporkID, - middleware.DefaultUnicastTimeout, - builder.IDTranslator, - builder.CodecFactory(), - slashingViolationsConsumer, + mw := middleware.NewMiddleware(&middleware.Config{ + Logger: logger, + Libp2pNode: libp2pNode, + FlowId: nodeID, + BitSwapMetrics: builder.Metrics.Bitswap, + RootBlockID: builder.SporkID, + UnicastMessageTimeout: middleware.DefaultUnicastTimeout, + IdTranslator: builder.IDTranslator, + Codec: builder.CodecFactory(), + SlashingViolationsConsumer: slashing.NewSlashingViolationsConsumer(logger, networkMetrics), + }, middleware.WithMessageValidators(validators...), // use default identifier provider ) - builder.NodeDisallowListDistributor.AddConsumer(mw) builder.Middleware = mw return builder.Middleware } diff --git a/cmd/node_builder.go b/cmd/node_builder.go index 06d824bc90b..e637ab3715c 100644 --- a/cmd/node_builder.go +++ b/cmd/node_builder.go @@ -27,7 +27,6 @@ import ( "github.com/onflow/flow-go/network/codec/cbor" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/connection" - "github.com/onflow/flow-go/network/p2p/distributor" "github.com/onflow/flow-go/network/p2p/dns" "github.com/onflow/flow-go/network/p2p/middleware" "github.com/onflow/flow-go/network/p2p/p2pbuilder" @@ -197,8 +196,11 @@ type NetworkConfig struct { ConnectionManagerConfig *connection.ManagerConfig // UnicastCreateStreamRetryDelay initial delay used in the exponential backoff for create stream retries UnicastCreateStreamRetryDelay time.Duration - // size of the queue for notifications about new peers in the disallow list. - DisallowListNotificationCacheSize uint32 + // size of the cache keeping the status of disallow-listed peers. Recommended to be 100 * number of authorized nodes. + // Note: this cache is meant to only keep the authorized (i.e., staked) but disallow-listed peers. There is no sybil + // vulnerability for this cache. However, the cache must be large enough to ensure it never runs out of space even if + // the node has not been re-configured for a long time. + DisallowListCacheSize uint32 // UnicastRateLimitersConfig configuration for all unicast rate limiters. UnicastRateLimitersConfig *UnicastRateLimitersConfig AlspConfig *AlspConfig @@ -291,8 +293,6 @@ type NodeConfig struct { // UnicastRateLimiterDistributor notifies consumers when a peer's unicast message is rate limited. UnicastRateLimiterDistributor p2p.UnicastRateLimiterDistributor - // NodeDisallowListDistributor notifies consumers of updates to disallow listing of nodes. - NodeDisallowListDistributor p2p.DisallowListNotificationDistributor } // StateExcerptAtBoot stores information about the root snapshot and latest finalized block for use in bootstrapping. @@ -333,12 +333,12 @@ func DefaultBaseConfig() *BaseConfig { BandwidthRateLimit: 0, BandwidthBurstLimit: middleware.LargeMsgMaxUnicastMsgSize, }, - GossipSubConfig: p2pbuilder.DefaultGossipSubConfig(), - DNSCacheTTL: dns.DefaultTimeToLive, - LibP2PResourceManagerConfig: p2pbuilder.DefaultResourceManagerConfig(), - ConnectionManagerConfig: connection.DefaultConnManagerConfig(), - NetworkConnectionPruning: connection.PruningEnabled, - DisallowListNotificationCacheSize: distributor.DefaultDisallowListNotificationQueueCacheSize, + GossipSubConfig: p2pbuilder.DefaultGossipSubConfig(), + DNSCacheTTL: dns.DefaultTimeToLive, + LibP2PResourceManagerConfig: p2pbuilder.DefaultResourceManagerConfig(), + ConnectionManagerConfig: connection.DefaultConnManagerConfig(), + NetworkConnectionPruning: connection.PruningEnabled, + DisallowListCacheSize: middleware.DisallowListCacheSize, AlspConfig: &AlspConfig{ SpamRecordCacheSize: alsp.DefaultSpamRecordCacheSize, SpamReportQueueSize: alsp.DefaultSpamReportQueueSize, diff --git a/cmd/observer/node_builder/observer_builder.go b/cmd/observer/node_builder/observer_builder.go index f9866ae1413..be2a8b35260 100644 --- a/cmd/observer/node_builder/observer_builder.go +++ b/cmd/observer/node_builder/observer_builder.go @@ -581,11 +581,11 @@ func (builder *ObserverServiceBuilder) InitIDProviders() { } builder.IDTranslator = translator.NewHierarchicalIDTranslator(idCache, translator.NewPublicNetworkIDTranslator()) - builder.NodeDisallowListDistributor = cmd.BuildDisallowListNotificationDisseminator(builder.DisallowListNotificationCacheSize, builder.MetricsRegisterer, builder.Logger, builder.MetricsEnabled) - // The following wrapper allows to black-list byzantine nodes via an admin command: // the wrapper overrides the 'Ejected' flag of disallow-listed nodes to true - builder.IdentityProvider, err = cache.NewNodeBlocklistWrapper(idCache, node.DB, builder.NodeDisallowListDistributor) + builder.IdentityProvider, err = cache.NewNodeDisallowListWrapper(idCache, node.DB, func() network.DisallowListNotificationConsumer { + return builder.Middleware + }) if err != nil { return fmt.Errorf("could not initialize NodeBlockListWrapper: %w", err) } @@ -616,11 +616,6 @@ func (builder *ObserverServiceBuilder) InitIDProviders() { return nil }) - - builder.Component("disallow list notification distributor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // distributor is returned as a component to be started and stopped. - return builder.NodeDisallowListDistributor, nil - }) } func (builder *ObserverServiceBuilder) Initialize() error { @@ -731,7 +726,11 @@ func (builder *ObserverServiceBuilder) initPublicLibp2pNode(networkKey crypto.Pr builder.BaseConfig.BindAddr, networkKey, builder.SporkID, - builder.LibP2PResourceManagerConfig). + builder.LibP2PResourceManagerConfig, + &p2p.DisallowListCacheConfig{ + MaxSize: builder.BaseConfig.NetworkConfig.DisallowListCacheSize, + Metrics: metrics.DisallowListCacheMetricsFactory(builder.HeroCacheMetricsFactory(), network.PublicNetwork), + }). SetSubscriptionFilter( subscription.NewRoleBasedFilter( subscription.UnstakedRole, builder.IdentityProvider, @@ -912,18 +911,19 @@ func (builder *ObserverServiceBuilder) initMiddleware(nodeID flow.Identifier, validators ...network.MessageValidator, ) network.Middleware { slashingViolationsConsumer := slashing.NewSlashingViolationsConsumer(builder.Logger, builder.Metrics.Network) - mw := middleware.NewMiddleware( - builder.Logger, - libp2pNode, nodeID, - builder.Metrics.Bitswap, - builder.SporkID, - middleware.DefaultUnicastTimeout, - builder.IDTranslator, - builder.CodecFactory(), - slashingViolationsConsumer, + mw := middleware.NewMiddleware(&middleware.Config{ + Logger: builder.Logger, + Libp2pNode: libp2pNode, + FlowId: nodeID, + BitSwapMetrics: builder.Metrics.Bitswap, + RootBlockID: builder.SporkID, + UnicastMessageTimeout: middleware.DefaultUnicastTimeout, + IdTranslator: builder.IDTranslator, + Codec: builder.CodecFactory(), + SlashingViolationsConsumer: slashingViolationsConsumer, + }, middleware.WithMessageValidators(validators...), // use default identifier provider ) - builder.NodeDisallowListDistributor.AddConsumer(mw) builder.Middleware = mw return builder.Middleware } diff --git a/cmd/scaffold.go b/cmd/scaffold.go index 59bd3ca1b2f..cc9771734a3 100644 --- a/cmd/scaffold.go +++ b/cmd/scaffold.go @@ -225,7 +225,6 @@ func (fnb *FlowNodeBuilder) BaseFlags() { // networking event notifications fnb.flags.Uint32Var(&fnb.BaseConfig.GossipSubConfig.RpcInspector.GossipSubRPCInspectorNotificationCacheSize, "gossipsub-rpc-inspector-notification-cache-size", defaultConfig.GossipSubConfig.RpcInspector.GossipSubRPCInspectorNotificationCacheSize, "cache size for notification events from gossipsub rpc inspector") - fnb.flags.Uint32Var(&fnb.BaseConfig.DisallowListNotificationCacheSize, "disallow-list-notification-cache-size", defaultConfig.DisallowListNotificationCacheSize, "cache size for notification events from disallow list") // unicast manager options fnb.flags.DurationVar(&fnb.BaseConfig.UnicastCreateStreamRetryDelay, "unicast-manager-create-stream-retry-delay", defaultConfig.NetworkConfig.UnicastCreateStreamRetryDelay, "Initial delay between failing to establish a connection with another node and retrying. This delay increases exponentially (exponential backoff) with the number of subsequent failures to establish a connection.") @@ -413,7 +412,12 @@ func (fnb *FlowNodeBuilder) EnqueueNetworkInit() { fnb.GossipSubConfig, fnb.GossipSubRpcInspectorSuite, fnb.LibP2PResourceManagerConfig, - uniCfg) + uniCfg, + &p2p.DisallowListCacheConfig{ + MaxSize: fnb.BaseConfig.NetworkConfig.DisallowListCacheSize, + Metrics: metrics.DisallowListCacheMetricsFactory(fnb.HeroCacheMetricsFactory(), network.PrivateNetwork), + }, + ) if err != nil { return nil, fmt.Errorf("could not create libp2p node builder: %w", err) @@ -458,8 +462,13 @@ func (fnb *FlowNodeBuilder) HeroCacheMetricsFactory() metrics.HeroCacheMetricsFa return metrics.NewNoopHeroCacheMetricsFactory() } -func (fnb *FlowNodeBuilder) InitFlowNetworkWithConduitFactory(node *NodeConfig, cf network.ConduitFactory, unicastRateLimiters *ratelimit.RateLimiters, peerManagerFilters []p2p.PeerFilter) (network.Network, error) { - var mwOpts []middleware.MiddlewareOption +func (fnb *FlowNodeBuilder) InitFlowNetworkWithConduitFactory( + node *NodeConfig, + cf network.ConduitFactory, + unicastRateLimiters *ratelimit.RateLimiters, + peerManagerFilters []p2p.PeerFilter) (network.Network, error) { + + var mwOpts []middleware.OptionFn if len(fnb.MsgValidators) > 0 { mwOpts = append(mwOpts, middleware.WithMessageValidators(fnb.MsgValidators...)) } @@ -476,20 +485,19 @@ func (fnb *FlowNodeBuilder) InitFlowNetworkWithConduitFactory(node *NodeConfig, if len(peerManagerFilters) > 0 { mwOpts = append(mwOpts, middleware.WithPeerManagerFilters(peerManagerFilters)) } - - slashingViolationsConsumer := slashing.NewSlashingViolationsConsumer(fnb.Logger, fnb.Metrics.Network) - mw := middleware.NewMiddleware( - fnb.Logger, - fnb.LibP2PNode, - fnb.Me.NodeID(), - fnb.Metrics.Bitswap, - fnb.SporkID, - fnb.BaseConfig.UnicastMessageTimeout, - fnb.IDTranslator, - fnb.CodecFactory(), - slashingViolationsConsumer, + mw := middleware.NewMiddleware(&middleware.Config{ + Logger: fnb.Logger, + Libp2pNode: fnb.LibP2PNode, + FlowId: fnb.Me.NodeID(), + BitSwapMetrics: fnb.Metrics.Bitswap, + RootBlockID: fnb.SporkID, + UnicastMessageTimeout: fnb.BaseConfig.UnicastMessageTimeout, + IdTranslator: fnb.IDTranslator, + Codec: fnb.CodecFactory(), + SlashingViolationsConsumer: slashing.NewSlashingViolationsConsumer(fnb.Logger, fnb.Metrics.Network), + }, mwOpts...) - fnb.NodeDisallowListDistributor.AddConsumer(mw) + fnb.Middleware = mw subscriptionManager := subscription.NewChannelSubscriptionManager(fnb.Middleware) @@ -1037,13 +1045,6 @@ func (fnb *FlowNodeBuilder) initStorage() error { } func (fnb *FlowNodeBuilder) InitIDProviders() { - fnb.Component("disallow list notification distributor", func(node *NodeConfig) (module.ReadyDoneAware, error) { - // distributor is returned as a component to be started and stopped. - if fnb.NodeDisallowListDistributor == nil { - return nil, fmt.Errorf("disallow list notification distributor has not been set") - } - return fnb.NodeDisallowListDistributor, nil - }) fnb.Module("id providers", func(node *NodeConfig) error { idCache, err := cache.NewProtocolStateIDCache(node.Logger, node.State, node.ProtocolEvents) if err != nil { @@ -1051,11 +1052,11 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { } node.IDTranslator = idCache - fnb.NodeDisallowListDistributor = BuildDisallowListNotificationDisseminator(fnb.DisallowListNotificationCacheSize, fnb.MetricsRegisterer, fnb.Logger, fnb.MetricsEnabled) - // The following wrapper allows to disallow-list byzantine nodes via an admin command: // the wrapper overrides the 'Ejected' flag of disallow-listed nodes to true - disallowListWrapper, err := cache.NewNodeBlocklistWrapper(idCache, node.DB, fnb.NodeDisallowListDistributor) + disallowListWrapper, err := cache.NewNodeDisallowListWrapper(idCache, node.DB, func() network.DisallowListNotificationConsumer { + return fnb.Middleware + }) if err != nil { return fmt.Errorf("could not initialize NodeBlockListWrapper: %w", err) } @@ -1063,9 +1064,9 @@ func (fnb *FlowNodeBuilder) InitIDProviders() { // register the disallow list wrapper for dynamic configuration via admin command err = node.ConfigManager.RegisterIdentifierListConfig("network-id-provider-blocklist", - disallowListWrapper.GetBlocklist, disallowListWrapper.Update) + disallowListWrapper.GetDisallowList, disallowListWrapper.Update) if err != nil { - return fmt.Errorf("failed to register blocklist with config manager: %w", err) + return fmt.Errorf("failed to register disallow-list wrapper with config manager: %w", err) } node.SyncEngineIdentifierProvider = id.NewIdentityFilterIdentifierProvider( diff --git a/cmd/utils.go b/cmd/utils.go index bfc77542c8d..05763933ebc 100644 --- a/cmd/utils.go +++ b/cmd/utils.go @@ -6,15 +6,10 @@ import ( "path/filepath" "github.com/libp2p/go-libp2p/core/peer" - "github.com/prometheus/client_golang/prometheus" - "github.com/rs/zerolog" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/mempool/queue" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/distributor" "github.com/onflow/flow-go/state/protocol/inmem" "github.com/onflow/flow-go/utils/io" ) @@ -68,13 +63,3 @@ func rateLimiterPeerFilter(rateLimiter p2p.RateLimiter) p2p.PeerFilter { return nil } } - -// BuildDisallowListNotificationDisseminator builds the disallow list notification distributor. -func BuildDisallowListNotificationDisseminator(size uint32, metricsRegistry prometheus.Registerer, logger zerolog.Logger, metricsEnabled bool) p2p.DisallowListNotificationDistributor { - heroStoreOpts := []queue.HeroStoreConfigOption{queue.WithHeroStoreSizeLimit(size)} - if metricsEnabled { - collector := metrics.DisallowListNotificationQueueMetricFactory(metricsRegistry) - heroStoreOpts = append(heroStoreOpts, queue.WithHeroStoreCollector(collector)) - } - return distributor.DefaultDisallowListNotificationDistributor(logger, heroStoreOpts...) -} diff --git a/follower/follower_builder.go b/follower/follower_builder.go index 1bd4c14b6bf..5d2d2d15582 100644 --- a/follower/follower_builder.go +++ b/follower/follower_builder.go @@ -484,11 +484,11 @@ func (builder *FollowerServiceBuilder) InitIDProviders() { } builder.IDTranslator = translator.NewHierarchicalIDTranslator(idCache, translator.NewPublicNetworkIDTranslator()) - builder.NodeDisallowListDistributor = cmd.BuildDisallowListNotificationDisseminator(builder.DisallowListNotificationCacheSize, builder.MetricsRegisterer, builder.Logger, builder.MetricsEnabled) - // The following wrapper allows to disallow-list byzantine nodes via an admin command: // the wrapper overrides the 'Ejected' flag of the disallow-listed nodes to true - builder.IdentityProvider, err = cache.NewNodeBlocklistWrapper(idCache, node.DB, builder.NodeDisallowListDistributor) + builder.IdentityProvider, err = cache.NewNodeDisallowListWrapper(idCache, node.DB, func() network.DisallowListNotificationConsumer { + return builder.Middleware + }) if err != nil { return fmt.Errorf("could not initialize NodeBlockListWrapper: %w", err) } @@ -519,11 +519,6 @@ func (builder *FollowerServiceBuilder) InitIDProviders() { return nil }) - - builder.Component("disallow list notification distributor", func(node *cmd.NodeConfig) (module.ReadyDoneAware, error) { - // distributor is returned as a component to be started and stopped. - return builder.NodeDisallowListDistributor, nil - }) } func (builder *FollowerServiceBuilder) Initialize() error { @@ -627,7 +622,11 @@ func (builder *FollowerServiceBuilder) initPublicLibp2pNode(networkKey crypto.Pr builder.BaseConfig.BindAddr, networkKey, builder.SporkID, - builder.LibP2PResourceManagerConfig). + builder.LibP2PResourceManagerConfig, + &p2p.DisallowListCacheConfig{ + MaxSize: builder.BaseConfig.NetworkConfig.DisallowListCacheSize, + Metrics: metrics.DisallowListCacheMetricsFactory(builder.HeroCacheMetricsFactory(), network.PublicNetwork), + }). SetSubscriptionFilter( subscription.NewRoleBasedFilter( subscription.UnstakedRole, builder.IdentityProvider, @@ -748,21 +747,19 @@ func (builder *FollowerServiceBuilder) initMiddleware(nodeID flow.Identifier, libp2pNode p2p.LibP2PNode, validators ...network.MessageValidator, ) network.Middleware { - slashingViolationsConsumer := slashing.NewSlashingViolationsConsumer(builder.Logger, builder.Metrics.Network) - mw := middleware.NewMiddleware( - builder.Logger, - libp2pNode, - nodeID, - builder.Metrics.Bitswap, - builder.SporkID, - middleware.DefaultUnicastTimeout, - builder.IDTranslator, - builder.CodecFactory(), - slashingViolationsConsumer, + mw := middleware.NewMiddleware(&middleware.Config{ + Logger: builder.Logger, + Libp2pNode: libp2pNode, + FlowId: nodeID, + BitSwapMetrics: builder.Metrics.Bitswap, + RootBlockID: builder.SporkID, + UnicastMessageTimeout: middleware.DefaultUnicastTimeout, + IdTranslator: builder.IDTranslator, + Codec: builder.CodecFactory(), + SlashingViolationsConsumer: slashing.NewSlashingViolationsConsumer(builder.Logger, builder.Metrics.Network), + }, middleware.WithMessageValidators(validators...), - // use default identifier provider ) - builder.NodeDisallowListDistributor.AddConsumer(mw) builder.Middleware = mw return builder.Middleware } diff --git a/insecure/cmd/corrupted_builder.go b/insecure/cmd/corrupted_builder.go index 7de352609a8..5ce4c3411ad 100644 --- a/insecure/cmd/corrupted_builder.go +++ b/insecure/cmd/corrupted_builder.go @@ -11,6 +11,8 @@ import ( "github.com/onflow/flow-go/insecure/corruptnet" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" p2pconfig "github.com/onflow/flow-go/network/p2p/p2pbuilder/config" "github.com/onflow/flow-go/network/p2p/unicast/ratelimit" @@ -101,6 +103,10 @@ func (cnb *CorruptedNodeBuilder) enqueueNetworkingLayer() { peerManagerCfg, uniCfg, cnb.GossipSubConfig, + &p2p.DisallowListCacheConfig{ + MaxSize: cnb.BaseConfig.NetworkConfig.DisallowListCacheSize, + Metrics: metrics.DisallowListCacheMetricsFactory(cnb.HeroCacheMetricsFactory(), network.PrivateNetwork), + }, cnb.TopicValidatorDisabled, cnb.WithPubSubMessageSigning, cnb.WithPubSubStrictSignatureVerification, diff --git a/insecure/corruptlibp2p/libp2p_node_factory.go b/insecure/corruptlibp2p/libp2p_node_factory.go index f65c06ce731..1f3e539079c 100644 --- a/insecure/corruptlibp2p/libp2p_node_factory.go +++ b/insecure/corruptlibp2p/libp2p_node_factory.go @@ -59,6 +59,7 @@ func InitCorruptLibp2pNode( peerManagerCfg *p2pconfig.PeerManagerConfig, uniCfg *p2pconfig.UnicastConfig, gossipSubCfg *p2pbuilder.GossipSubConfig, + disallowListCacheCfg *p2p.DisallowListCacheConfig, topicValidatorDisabled, withMessageSigning, withStrictSignatureVerification bool, @@ -94,7 +95,8 @@ func InitCorruptLibp2pNode( gossipSubCfg, rpcInspectorSuite, p2pbuilder.DefaultResourceManagerConfig(), - uniCfg) + uniCfg, + disallowListCacheCfg) if err != nil { return nil, fmt.Errorf("could not create corrupt libp2p node builder: %w", err) diff --git a/insecure/corruptlibp2p/p2p_node.go b/insecure/corruptlibp2p/p2p_node.go index 143e1a9e938..b3fbd0cb36d 100644 --- a/insecure/corruptlibp2p/p2p_node.go +++ b/insecure/corruptlibp2p/p2p_node.go @@ -51,7 +51,12 @@ 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, host host.Host, pCache p2p.ProtocolPeerCache, peerManager p2p.PeerManager) p2p.LibP2PNode { - node := p2pnode.NewNode(logger, host, pCache, peerManager) +func NewCorruptLibP2PNode( + logger zerolog.Logger, + host host.Host, + pCache p2p.ProtocolPeerCache, + peerManager p2p.PeerManager, + disallowListCacheCfg *p2p.DisallowListCacheConfig) p2p.LibP2PNode { + node := p2pnode.NewNode(logger, host, pCache, peerManager, disallowListCacheCfg) return &CorruptP2PNode{Node: node, logger: logger, codec: cbor.NewCodec()} } diff --git a/insecure/corruptlibp2p/spam_test.go b/insecure/corruptlibp2p/spam_test.go index f446e9f4c65..cd759fd52e9 100644 --- a/insecure/corruptlibp2p/spam_test.go +++ b/insecure/corruptlibp2p/spam_test.go @@ -71,7 +71,7 @@ func TestSpam_IHave(t *testing.T) { // prior to the test we should ensure that spammer and victim connect. // this is vital as the spammer will circumvent the normal pubsub subscription mechanism and send iHAVE messages directly to the victim. // without a prior connection established, directly spamming pubsub messages may cause a race condition in the pubsub implementation. - p2ptest.EnsureConnected(t, ctx, nodes) + p2ptest.TryConnectionAndEnsureConnected(t, ctx, nodes) p2ptest.EnsurePubsubMessageExchange(t, ctx, nodes, func() (interface{}, channels.Topic) { blockTopic := channels.TopicFromChannel(channels.PushBlocks, sporkId) return unittest.ProposalFixture(), blockTopic diff --git a/insecure/rpc_inspector/utils.go b/insecure/rpc_inspector/utils.go index 02cf9492f7c..164634236bc 100644 --- a/insecure/rpc_inspector/utils.go +++ b/insecure/rpc_inspector/utils.go @@ -19,7 +19,7 @@ func startNodesAndEnsureConnected(t *testing.T, ctx irrecoverable.SignalerContex // prior to the test we should ensure that spammer and victim connect. // this is vital as the spammer will circumvent the normal pubsub subscription mechanism and send iHAVE messages directly to the victim. // without a prior connection established, directly spamming pubsub messages may cause a race condition in the pubsub implementation. - p2ptest.EnsureConnected(t, ctx, nodes) + p2ptest.TryConnectionAndEnsureConnected(t, ctx, nodes) p2ptest.EnsurePubsubMessageExchange(t, ctx, nodes, func() (interface{}, channels.Topic) { blockTopic := channels.TopicFromChannel(channels.PushBlocks, sporkID) return unittest.ProposalFixture(), blockTopic diff --git a/module/metrics/herocache.go b/module/metrics/herocache.go index 1c932d90bf1..54e287bdb1b 100644 --- a/module/metrics/herocache.go +++ b/module/metrics/herocache.go @@ -72,6 +72,21 @@ func NetworkReceiveCacheMetricsFactory(f HeroCacheMetricsFactory, networkType ne return f(namespaceNetwork, r) } +// DisallowListCacheMetricsFactory is the factory method for creating a new HeroCacheCollector for the disallow list cache. +// The disallow-list cache is used to keep track of peers that are disallow-listed and the reasons for it. +// Args: +// - f: the HeroCacheMetricsFactory to create the collector +// - networkingType: the networking type of the cache, i.e., whether it is used for the public or the private network +// Returns: +// - a HeroCacheMetrics for the disallow list cache +func DisallowListCacheMetricsFactory(f HeroCacheMetricsFactory, networkingType network.NetworkingType) module.HeroCacheMetrics { + r := ResourceNetworkingDisallowListCache + if networkingType == network.PublicNetwork { + r = PrependPublicPrefix(r) + } + return f(namespaceNetwork, r) +} + func NetworkDnsTxtCacheMetricsFactory(registrar prometheus.Registerer) *HeroCacheCollector { return NewHeroCacheCollector(namespaceNetwork, ResourceNetworkingDnsTxtCache, registrar) } diff --git a/module/metrics/labels.go b/module/metrics/labels.go index a4c8a848720..353e1b3ca25 100644 --- a/module/metrics/labels.go +++ b/module/metrics/labels.go @@ -91,6 +91,7 @@ const ( ResourceNetworkingApplicationLayerSpamRecordCache = "application_layer_spam_record_cache" ResourceNetworkingApplicationLayerSpamReportQueue = "application_layer_spam_report_queue" ResourceNetworkingRpcClusterPrefixReceivedCache = "rpc_cluster_prefixed_received_cache" + ResourceNetworkingDisallowListCache = "disallow_list_cache" ResourceFollowerPendingBlocksCache = "follower_pending_block_cache" // follower engine ResourceFollowerLoopCertifiedBlocksChannel = "follower_loop_certified_blocks_channel" // follower loop, certified blocks buffered channel diff --git a/network/alsp/internal/cache.go b/network/alsp/internal/cache.go index 2b7dc8236cd..c29ae4bd988 100644 --- a/network/alsp/internal/cache.go +++ b/network/alsp/internal/cache.go @@ -44,7 +44,7 @@ func NewSpamRecordCache(sizeLimit uint32, logger zerolog.Logger, collector modul // the spam records of the authorized nodes. Also, this cache is keeping at most one record per origin id, so the // size of the cache must be at least the number of authorized nodes. heropool.NoEjection, - logger.With().Str("mempool", "aslp=spam-records").Logger(), + logger.With().Str("mempool", "aslp-spam-records").Logger(), collector) return &SpamRecordCache{ @@ -166,10 +166,11 @@ func (s *SpamRecordCache) Get(originId flow.Identifier) (*model.ProtocolSpamReco // return a copy of the record (we do not want the caller to modify the record). return &model.ProtocolSpamRecord{ - OriginId: record.OriginId, - Decay: record.Decay, - CutoffCounter: record.CutoffCounter, - Penalty: record.Penalty, + OriginId: record.OriginId, + Decay: record.Decay, + CutoffCounter: record.CutoffCounter, + Penalty: record.Penalty, + DisallowListed: record.DisallowListed, }, true } diff --git a/network/alsp/internal/cache_test.go b/network/alsp/internal/cache_test.go index d41b3cec331..22efd456a8e 100644 --- a/network/alsp/internal/cache_test.go +++ b/network/alsp/internal/cache_test.go @@ -75,12 +75,13 @@ func TestSpamRecordCache_Adjust_Init(t *testing.T) { // adjusting a spam record for an origin ID that does not exist in the cache should initialize the record. initializedPenalty, err := cache.Adjust(originID1, adjustFuncIncrement) require.NoError(t, err, "expected no error") - require.Equal(t, float64(1), initializedPenalty, "expected initialized penalty to be 0") + require.Equal(t, float64(1), initializedPenalty, "expected initialized penalty to be 1") record1, ok := cache.Get(originID1) require.True(t, ok, "expected record to exist") require.NotNil(t, record1, "expected non-nil record") require.Equal(t, originID1, record1.OriginId, "expected record to have correct origin ID") + require.False(t, record1.DisallowListed, "expected record to not be disallow listed") require.Equal(t, cache.Size(), uint(1), "expected cache to have one record") // adjusting a spam record for an origin ID that already exists in the cache should not initialize the record, @@ -92,6 +93,7 @@ func TestSpamRecordCache_Adjust_Init(t *testing.T) { require.True(t, ok, "expected record to still exist") require.NotNil(t, record1Again, "expected non-nil record") require.Equal(t, originID1, record1Again.OriginId, "expected record to have correct origin ID") + require.False(t, record1Again.DisallowListed, "expected record not to be disallow listed") require.Equal(t, cache.Size(), uint(1), "expected cache to still have one record") // adjusting a spam record for a different origin ID should initialize the record. @@ -103,6 +105,7 @@ func TestSpamRecordCache_Adjust_Init(t *testing.T) { require.True(t, ok, "expected record to exist") require.NotNil(t, record2, "expected non-nil record") require.Equal(t, originID2, record2.OriginId, "expected record to have correct origin ID") + require.False(t, record2.DisallowListed, "expected record not to be disallow listed") require.Equal(t, cache.Size(), uint(2), "expected cache to have two records") } @@ -356,6 +359,8 @@ func TestSpamRecordCache_ConcurrentSameRecordAdjust(t *testing.T) { } adjustFn := func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { record.Penalty -= 1.0 + record.DisallowListed = true + record.Decay += 1.0 return record, nil // no-op } @@ -380,10 +385,13 @@ func TestSpamRecordCache_ConcurrentSameRecordAdjust(t *testing.T) { unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") // ensure that the record is correctly initialized and adjusted in the cache + initDecay := model.SpamRecordFactory()(originID).Decay record, found := cache.Get(originID) require.True(t, found) require.NotNil(t, record) require.Equal(t, concurrentAttempts*-1.0, record.Penalty) + require.Equal(t, initDecay+concurrentAttempts*1.0, record.Decay) + require.True(t, record.DisallowListed) require.Equal(t, originID, record.OriginId) } diff --git a/network/alsp/manager/README.md b/network/alsp/manager/README.md new file mode 100644 index 00000000000..36bf3deda9d --- /dev/null +++ b/network/alsp/manager/README.md @@ -0,0 +1,84 @@ +# Application Layer Spam Prevention (ASLP) Manager +Implementation of ALSP manager is available here: [manager.go](manager.go) +Note that this readme is primarily focusing on the ALSP manager. For more details regarding the ALSP system please refer to [readme.md](..%2Freadme.md). +--- +## Architectural Overview +### Reporting Misbehavior and Managing Node Penalties +Figure below illustrates the ALSP manager’s role in the reporting of misbehavior and the management of node penalties as +well as the interactions between the ALSP manager and the `LibP2PNode`, `ConnectionGater`, and `PeerManager` components for +the disallow listing and allow listing processes. + +#### Reporting Misbehavior +In the event that an engine detects misbehavior within a channel, +it is imperative to report this finding to the ALSP manager. +This is achieved by invoking the `ReportMisbehavior` method on the conduit corresponding to the engine. + +#### Managing Penalties +The ALSP manager is responsible for maintaining records of misbehavior reports associated with +remote nodes and for calculating their accumulated misbehavior penalties. +Should a node’s misbehavior penalty surpass a certain threshold +(referred to as `DisallowListingThreshold`), the ALSP manager initiates the disallow listing process. When a remote node is disallow-listed, +it is effectively isolated from the network by the `ConnectionGater` and `PeerManager` components, i.e., the existing +connections to that remote node are closed and new connections attempts are rejected. + +##### Disallow Listing Process +1. The ALSP manager communicates with the `LibP2PNode` by calling its `OnDisallowListNotification` method to indicate that a particular remote node has been disallow-listed. +2. In response, the `LibP2PNode` takes two important actions: + + a. It alerts the `PeerManager`, instructing it to sever the connection with the disallow-listed node. + b. It notifies the `ConnectionGater` to block any incoming or outgoing connections to and from the disallow-listed node. +This ensures that the disallow-listed node is effectively isolated from the local node's network. + +##### Penalty Decay and Allow Listing Process +The ALSP manager also includes a penalty decay mechanism, which gradually reduces the penalties of nodes over time upon regular heartbeat intervals (default is every one second). +Once a disallow-listed node's penalty decays back to zero, the node can be reintegrated into the network through the allow listing process. The allow-listing process involves allowing +the `ConnectionGater` to lift the block on the disallow-listed node and instructing the `PeerManager` to initiate an outbound connection with the allow-listed node. + +1. The ALSP manager calls the `OnAllowListNotification` method on the `LibP2PNode` to signify that a previously disallow-listed node is now allow-listed. +2. The `LibP2PNode` responds by: + + a. Instructing the `ConnectionGater` to lift the block, thereby permitting connections with the now allow-listed node. + b. Requesting the `PeerManager` to initiate an outbound connection with the allow-listed node. + +This series of actions allows the rehabilitated node to be reintegrated and actively participate in the network once again. +![alsp-manager.png](alsp-manager.png) +--- + + + +## Developer Guidelines +The ALSP (Application Layer Spam Prevention) Manager handles application layer spamming misbehavior reports and penalizes misbehaving nodes. It also disallow-lists nodes whose penalties drop below a threshold. + + +- **Misbehavior Reports**: When a local engine detects a spamming misbehavior of a remote node, it sends a report to the ALSP manager, by invoking the `HandleMisbehaviorReport` method of the corresponding +conduit on which the misbehavior was detected. The manager handles the report in a thread-safe and non-blocking manner, using worker pools. + +```go +func (m *MisbehaviorReportManager) HandleMisbehaviorReport(channel channels.Channel, report network.MisbehaviorReport) { + // Handle the report +} +``` + +- **Penalties**: Misbehaving nodes are penalized by the manager. +The manager keeps a cache of records with penalties for each node. +The penalties are decayed over time through periodic heartbeats. + +- **Disallow-listing**: Nodes whose penalties drop below a threshold are disallow-listed. + +- **Heartbeats**: Periodic heartbeats allow the manager to perform recurring tasks, such as decaying the penalties of misbehaving nodes. +```go +func (m *MisbehaviorReportManager) heartbeatLoop(ctx irrecoverable.SignalerContext, interval time.Duration) { + // Handle heartbeats +} +``` + +- **Disallow-list Notification Consumer**: is the interface of the consumer of disallow-list notifications, which is +responsible for taking actions when a node is disallow-listed, i.e., closing exisitng connections with the remote disallow-listed +node and blocking any incoming or outgoing connections to that node. The consumer is passed to the manager when it is created. +In the current implementation the consumer is the instance of the `LibP2PNode` component of the node. +```go +disallowListingConsumer network.DisallowListNotificationConsumer +``` + +### Configuration +The configuration includes settings like cache size, heartbeat intervals, and network type. \ No newline at end of file diff --git a/network/alsp/manager/alsp-manager.png b/network/alsp/manager/alsp-manager.png new file mode 100644 index 00000000000..97e111e532b Binary files /dev/null and b/network/alsp/manager/alsp-manager.png differ diff --git a/network/alsp/manager/manager.go b/network/alsp/manager/manager.go index 6f35ba8c311..941407e2637 100644 --- a/network/alsp/manager/manager.go +++ b/network/alsp/manager/manager.go @@ -10,6 +10,7 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/engine/common/worker" + "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "github.com/onflow/flow-go/module/irrecoverable" @@ -53,11 +54,8 @@ func defaultSpamRecordCacheFactory() SpamRecordCacheFactory { } } -// MisbehaviorReportManager is responsible for handling misbehavior reports. -// The current version is at the minimum viable product stage and only logs the reports. -// TODO: the mature version should be able to handle the reports and take actions accordingly, i.e., penalize the misbehaving node -// -// and report the node to be disallow-listed if the overall penalty of the misbehaving node drops below the disallow-listing threshold. +// MisbehaviorReportManager is responsible for handling misbehavior reports, i.e., penalizing the misbehaving node +// and report the node to be disallow-listed if the overall penalty of the misbehaving node drops below the disallow-listing threshold. type MisbehaviorReportManager struct { component.Component logger zerolog.Logger @@ -75,6 +73,10 @@ type MisbehaviorReportManager struct { // Note: under normal circumstances, the ALSP module should not be disabled. disablePenalty bool + // disallowListingConsumer is the consumer for the disallow-listing notifications. + // It is notified when a node is disallow-listed by this manager. + disallowListingConsumer network.DisallowListNotificationConsumer + // workerPool is the worker pool for handling the misbehavior reports in a thread-safe and non-blocking manner. workerPool *worker.Pool[internal.ReportedMisbehaviorWork] } @@ -152,26 +154,28 @@ func WithSpamRecordsCacheFactory(f SpamRecordCacheFactory) MisbehaviorReportMana // NewMisbehaviorReportManager creates a new instance of the MisbehaviorReportManager. // Args: -// -// logger: the logger instance. -// metrics: the metrics instance. -// cache: the spam record cache instance. +// cfg: the configuration for the MisbehaviorReportManager. +// consumer: the consumer for the disallow-listing notifications. When the manager decides to disallow-list a node, it notifies the consumer to +// perform the lower-level disallow-listing action at the networking layer. +// All connections to the disallow-listed node are closed and the node is removed from the overlay, and +// no further connections are established to the disallow-listed node, either inbound or outbound. // // Returns: // // A new instance of the MisbehaviorReportManager. // An error if the config is invalid. The error is considered irrecoverable. -func NewMisbehaviorReportManager(cfg *MisbehaviorReportManagerConfig) (*MisbehaviorReportManager, error) { +func NewMisbehaviorReportManager(cfg *MisbehaviorReportManagerConfig, consumer network.DisallowListNotificationConsumer) (*MisbehaviorReportManager, error) { if err := cfg.validate(); err != nil { return nil, fmt.Errorf("invalid configuration for MisbehaviorReportManager: %w", err) } lg := cfg.Logger.With().Str("module", "misbehavior_report_manager").Logger() m := &MisbehaviorReportManager{ - logger: lg, - metrics: cfg.AlspMetrics, - disablePenalty: cfg.DisablePenalty, - cacheFactory: defaultSpamRecordCacheFactory(), + logger: lg, + metrics: cfg.AlspMetrics, + disablePenalty: cfg.DisablePenalty, + disallowListingConsumer: consumer, + cacheFactory: defaultSpamRecordCacheFactory(), } store := queue.NewHeroStore( @@ -308,10 +312,56 @@ func (m *MisbehaviorReportManager) onHeartbeat() error { return record, fmt.Errorf("illegal state: spam record %x has non-positive decay %f", id, record.Decay) } + // TODO: this can be done in batch but at this stage let's send individual notifications. + // (it requires enabling the batch mode end-to-end including the cache in middleware). + // as long as record.Penalty is NOT below model.DisallowListingThreshold, + // the node is considered allow-listed and can conduct inbound and outbound connections. + // Once it falls below model.DisallowListingThreshold, it needs to be disallow listed. + if record.Penalty < model.DisallowListingThreshold && !record.DisallowListed { + // cutoff counter keeps track of how many times the penalty has been below the threshold. + record.CutoffCounter++ + record.DisallowListed = true + m.logger.Warn(). + Str("key", logging.KeySuspicious). + Hex("identifier", logging.ID(id)). + Uint64("cutoff_counter", record.CutoffCounter). + Float64("decay_speed", record.Decay). + Bool("disallow_listed", record.DisallowListed). + Msg("node penalty is below threshold, disallow listing") + m.disallowListingConsumer.OnDisallowListNotification(&network.DisallowListingUpdate{ + FlowIds: flow.IdentifierList{id}, + Cause: network.DisallowListedCauseAlsp, // sets the ALSP disallow listing cause on node + }) + } + // each time we decay the penalty by the decay speed, the penalty is a negative number, and the decay speed // is a positive number. So the penalty is getting closer to zero. // We use math.Min() to make sure the penalty is never positive. record.Penalty = math.Min(record.Penalty+record.Decay, 0) + + // TODO: this can be done in batch but at this stage let's send individual notifications. + // (it requires enabling the batch mode end-to-end including the cache in middleware). + if record.Penalty == float64(0) && record.DisallowListed { + record.DisallowListed = false + m.logger.Info(). + Hex("identifier", logging.ID(id)). + Uint64("cutoff_counter", record.CutoffCounter). + Float64("decay_speed", record.Decay). + Bool("disallow_listed", record.DisallowListed). + Msg("allow-listing a node that was disallow listed") + // Penalty has fully decayed to zero and the node can be back in the allow list. + m.disallowListingConsumer.OnAllowListNotification(&network.AllowListingUpdate{ + FlowIds: flow.IdentifierList{id}, + Cause: network.DisallowListedCauseAlsp, // clears the ALSP disallow listing cause from node + }) + } + + m.logger.Trace(). + Hex("identifier", logging.ID(id)). + Uint64("cutoff_counter", record.CutoffCounter). + Float64("decay_speed", record.Decay). + Bool("disallow_listed", record.DisallowListed). + Msg("spam record decayed successfully") return record, nil }) diff --git a/network/alsp/manager/manager_test.go b/network/alsp/manager/manager_test.go index fb63a2609c2..3fd57430e21 100644 --- a/network/alsp/manager/manager_test.go +++ b/network/alsp/manager/manager_test.go @@ -2,7 +2,6 @@ package alspmgr_test import ( "context" - "fmt" "math" "math/rand" "sync" @@ -29,6 +28,7 @@ import ( "github.com/onflow/flow-go/network/internal/testutils" "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/network/p2p" + p2ptest "github.com/onflow/flow-go/network/p2p/test" "github.com/onflow/flow-go/utils/unittest" ) @@ -102,10 +102,12 @@ func TestNetworkPassesReportedMisbehavior(t *testing.T) { // It prepares a set of misbehavior reports and reports them to the conduit on the test channel. // The test ensures that the MisbehaviorReportManager receives and handles all reported misbehavior // without any duplicate reports and within a specified time. -func TestHandleReportedMisbehavior_Integration(t *testing.T) { +func TestHandleReportedMisbehavior_Cache_Integration(t *testing.T) { cfg := managerCfgFixture() - // create a new MisbehaviorReportManager + // this test is assessing the integration of the ALSP manager with the network. As the ALSP manager is an attribute + // of the network, we need to configure the ALSP manager via the network configuration, and let the network create + // the ALSP manager. var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ alspmgr.WithSpamRecordsCacheFactory(func(logger zerolog.Logger, size uint32, metrics module.HeroCacheMetrics) alsp.SpamRecordCache { @@ -126,7 +128,6 @@ func TestHandleReportedMisbehavior_Integration(t *testing.T) { require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) - signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) testutils.StartNodesAndNetworks(signalerCtx, t, nodes, []network.Network{net}, 100*time.Millisecond) defer testutils.StopComponents[p2p.LibP2PNode](t, nodes, 100*time.Millisecond) @@ -152,11 +153,11 @@ func TestHandleReportedMisbehavior_Integration(t *testing.T) { wg.Add(len(reports)) // reports the misbehavior for _, report := range reports { - report := report // capture range variable + r := report // capture range variable go func() { defer wg.Done() - con.ReportMisbehavior(report) + con.ReportMisbehavior(r) }() } } @@ -180,6 +181,8 @@ func TestHandleReportedMisbehavior_Integration(t *testing.T) { require.Equal(t, totalPenalty, record.Penalty) // with just reporting a single misbehavior report, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) + // with just reporting a single misbehavior report, the node should not be disallowed. + require.False(t, record.DisallowListed) // the decay should be the default decay value. require.Equal(t, model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay, record.Decay) } @@ -188,11 +191,105 @@ func TestHandleReportedMisbehavior_Integration(t *testing.T) { }, 1*time.Second, 10*time.Millisecond, "ALSP manager did not handle the misbehavior report") } +// TestHandleReportedMisbehavior_And_DisallowListing_Integration implements an end-to-end integration test for the +// handling of reported misbehavior and disallow listing. +// +// The test sets up 3 nodes, one victim, one honest, and one (alledged) spammer. +// Initially, the test ensures that all nodes are connected to each other. +// Then, test imitates that victim node reports the spammer node for spamming. +// The test generates enough spam reports to trigger the disallow-listing of the victim node. +// The test ensures that the victim node is disconnected from the spammer node. +// The test ensures that despite attempting on connections, no inbound or outbound connections between the victim and +// the disallow-listed spammer node are established. +func TestHandleReportedMisbehavior_And_DisallowListing_Integration(t *testing.T) { + cfg := managerCfgFixture() + + // this test is assessing the integration of the ALSP manager with the network. As the ALSP manager is an attribute + // of the network, we need to configure the ALSP manager via the network configuration, and let the network create + // the ALSP manager. + var victimSpamRecordCacheCache alsp.SpamRecordCache + cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ + alspmgr.WithSpamRecordsCacheFactory(func(logger zerolog.Logger, size uint32, metrics module.HeroCacheMetrics) alsp.SpamRecordCache { + victimSpamRecordCacheCache = internal.NewSpamRecordCache(size, logger, metrics, model.SpamRecordFactory()) + return victimSpamRecordCacheCache + }), + } + + ids, nodes, mws, _, _ := testutils.GenerateIDsAndMiddlewares( + t, + 3, + unittest.Logger(), + unittest.NetworkCodec(), + unittest.NetworkSlashingViolationsConsumer(unittest.Logger(), metrics.NewNoopCollector())) + sms := testutils.GenerateSubscriptionManagers(t, mws) + networkCfg := testutils.NetworkConfigFixture(t, unittest.Logger(), *ids[0], ids, mws[0], sms[0], p2p.WithAlspConfig(cfg)) + victimNetwork, err := p2p.NewNetwork(networkCfg) + require.NoError(t, err) + + // index of the victim node in the nodes slice. + victimIndex := 0 + // index of the spammer node in the nodes slice (the node that will be reported for misbehavior and disallow-listed by victim). + spammerIndex := 1 + // other node (not victim and not spammer) that we have to ensure is not affected by the disallow-listing of the spammer. + honestIndex := 2 + + ctx, cancel := context.WithCancel(context.Background()) + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + testutils.StartNodesAndNetworks(signalerCtx, t, nodes, []network.Network{victimNetwork}, 100*time.Millisecond) + defer testutils.StopComponents[p2p.LibP2PNode](t, nodes, 100*time.Millisecond) + defer cancel() + + p2ptest.LetNodesDiscoverEachOther(t, ctx, nodes, ids) + // initially victim and spammer should be able to connect to each other. + p2ptest.TryConnectionAndEnsureConnected(t, ctx, nodes) + + e := mocknetwork.NewEngine(t) + con, err := victimNetwork.Register(channels.TestNetworkChannel, e) + require.NoError(t, err) + + // creates a misbehavior report for the spammer + report := misbehaviorReportFixtureWithPenalty(t, ids[spammerIndex].NodeID, model.DefaultPenaltyValue) + + // simulates the victim node reporting the spammer node misbehavior 120 times + // to the network. As each report has the default penalty, ideally the spammer should be disallow-listed after + // 100 reports (each having 0.01 * disallow-listing penalty). But we take 120 as a safe number to ensure that + // the spammer is definitely disallow-listed. + reportCount := 120 + wg := sync.WaitGroup{} + for i := 0; i < reportCount; i++ { + wg.Add(1) + // reports the misbehavior + r := report // capture range variable + go func() { + defer wg.Done() + + con.ReportMisbehavior(r) + }() + } + + 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) + + // 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) + p2ptest.RequireConnectedEventually(t, []p2p.LibP2PNode{nodes[honestIndex], nodes[victimIndex]}, 1*time.Millisecond, 100*time.Millisecond) + + // while the spammer node is disallow-listed, it cannot connect to the victim node. Also, the victim node cannot directly dial and connect to the spammer node, unless + // it is allow-listed again. + p2ptest.EnsureNotConnectedBetweenGroups(t, ctx, []p2p.LibP2PNode{nodes[victimIndex]}, []p2p.LibP2PNode{nodes[spammerIndex]}) +} + // TestMisbehaviorReportMetrics tests the recording of misbehavior report metrics. // It checks that when a misbehavior report is received by the ALSP manager, the metrics are recorded. // It fails the test if the metrics are not recorded or if they are recorded incorrectly. func TestMisbehaviorReportMetrics(t *testing.T) { cfg := managerCfgFixture() + + // this test is assessing the integration of the ALSP manager with the network. As the ALSP manager is an attribute + // of the network, we need to configure the ALSP manager via the network configuration, and let the network create + // the ALSP manager. alspMetrics := mockmodule.NewAlspMetrics(t) cfg.AlspMetrics = alspMetrics @@ -282,6 +379,7 @@ func TestReportCreation(t *testing.T) { // In other words, variation of input values do not cause a nil ALSP manager to be created or a panic. func TestNewMisbehaviorReportManager(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ alspmgr.WithSpamRecordsCacheFactory(func(logger zerolog.Logger, size uint32, metrics module.HeroCacheMetrics) alsp.SpamRecordCache { @@ -291,25 +389,25 @@ func TestNewMisbehaviorReportManager(t *testing.T) { } t.Run("with default values", func(t *testing.T) { - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) assert.NotNil(t, m) }) t.Run("with a custom spam record cache", func(t *testing.T) { - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) assert.NotNil(t, m) }) t.Run("with ALSP module enabled", func(t *testing.T) { - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) assert.NotNil(t, m) }) t.Run("with ALSP module disabled", func(t *testing.T) { - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) assert.NotNil(t, m) }) @@ -319,10 +417,11 @@ func TestNewMisbehaviorReportManager(t *testing.T) { // It is a minimum viable test that ensures that a nil ALSP manager is created with invalid set of inputs. func TestMisbehaviorReportManager_InitializationError(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) t.Run("missing spam report queue size", func(t *testing.T) { cfg.SpamReportQueueSize = 0 - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.Error(t, err) require.ErrorIs(t, err, alspmgr.ErrSpamReportQueueSizeNotSet) assert.Nil(t, m) @@ -330,7 +429,7 @@ func TestMisbehaviorReportManager_InitializationError(t *testing.T) { t.Run("missing spam record cache size", func(t *testing.T) { cfg.SpamRecordCacheSize = 0 - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.Error(t, err) require.ErrorIs(t, err, alspmgr.ErrSpamRecordCacheSizeNotSet) assert.Nil(t, m) @@ -338,7 +437,7 @@ func TestMisbehaviorReportManager_InitializationError(t *testing.T) { t.Run("missing heartbeat intervals", func(t *testing.T) { cfg.HeartBeatInterval = 0 - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.Error(t, err) require.ErrorIs(t, err, alspmgr.ErrSpamRecordCacheSizeNotSet) assert.Nil(t, m) @@ -349,6 +448,8 @@ func TestMisbehaviorReportManager_InitializationError(t *testing.T) { // The test ensures that the misbehavior report is handled correctly and the penalty is applied to the peer in the cache. func TestHandleMisbehaviorReport_SinglePenaltyReport(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) + // create a new MisbehaviorReportManager var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -358,7 +459,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReport(t *testing.T) { }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -391,6 +492,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReport(t *testing.T) { } require.NotNil(t, record) require.Equal(t, penalty, record.Penalty) + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet require.Equal(t, uint64(0), record.CutoffCounter) // with just reporting a misbehavior, the cutoff counter should not be incremented. require.Equal(t, model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay, record.Decay) // the decay should be the default decay value. @@ -402,6 +504,8 @@ func TestHandleMisbehaviorReport_SinglePenaltyReport(t *testing.T) { // The test ensures that the misbehavior is reported on metrics but the penalty is not applied to the peer in the cache. func TestHandleMisbehaviorReport_SinglePenaltyReport_PenaltyDisable(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) + cfg.DisablePenalty = true // disable penalty for misbehavior reports alspMetrics := mockmodule.NewAlspMetrics(t) cfg.AlspMetrics = alspMetrics @@ -414,7 +518,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReport_PenaltyDisable(t *testing.T return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -459,6 +563,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReport_PenaltyDisable(t *testing.T // The test ensures that each misbehavior report is handled correctly and the penalties are cumulatively applied to the peer in the cache. func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Sequentially(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) // create a new MisbehaviorReportManager var cache alsp.SpamRecordCache @@ -468,7 +573,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Sequentiall return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -506,6 +611,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Sequentiall // all the misbehavior reports should be processed by now, so the penalty should be equal to the total penalty return false } + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. // with just reporting a few misbehavior reports, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) // the decay should be the default decay value. @@ -520,6 +626,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Sequentiall // The test ensures that each misbehavior report is handled correctly and the penalties are cumulatively applied to the peer in the cache. func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Concurrently(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -528,7 +635,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Concurrentl return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -552,12 +659,12 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Concurrentl // handle the misbehavior reports totalPenalty := float64(0) for _, report := range reports { - report := report // capture range variable + r := report // capture range variable totalPenalty += report.Penalty() go func() { defer wg.Done() - m.HandleMisbehaviorReport(channel, report) + m.HandleMisbehaviorReport(channel, r) }() } @@ -575,6 +682,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Concurrentl // all the misbehavior reports should be processed by now, so the penalty should be equal to the total penalty return false } + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. // with just reporting a few misbehavior reports, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) // the decay should be the default decay value. @@ -589,6 +697,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForSinglePeer_Concurrentl // The test ensures that each misbehavior report is handled correctly and the penalties are applied to the corresponding peers in the cache. func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Sequentially(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -597,7 +706,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Sequential return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -630,7 +739,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Sequential return false } require.NotNil(t, record) - + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. require.Equal(t, report.Penalty(), record.Penalty) // with just reporting a single misbehavior report, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) @@ -648,6 +757,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Sequential // The test ensures that each misbehavior report is handled correctly and the penalties are applied to the corresponding peers in the cache. func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Concurrently(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -656,7 +766,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Concurrent return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -680,12 +790,12 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Concurrent // handle the misbehavior reports totalPenalty := float64(0) for _, report := range reports { - report := report // capture range variable + r := report // capture range variable totalPenalty += report.Penalty() go func() { defer wg.Done() - m.HandleMisbehaviorReport(channel, report) + m.HandleMisbehaviorReport(channel, r) }() } @@ -700,7 +810,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Concurrent return false } require.NotNil(t, record) - + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. require.Equal(t, report.Penalty(), record.Penalty) // with just reporting a single misbehavior report, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) @@ -717,6 +827,7 @@ func TestHandleMisbehaviorReport_SinglePenaltyReportsForMultiplePeers_Concurrent // The test ensures that each misbehavior report is handled correctly and the penalties are cumulatively applied to the corresponding peers in the cache. func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Sequentially(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -725,7 +836,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Sequenti return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -756,11 +867,11 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Sequenti for _, reports := range peersReports { wg.Add(len(reports)) for _, report := range reports { - report := report // capture range variable + r := report // capture range variable go func() { defer wg.Done() - m.HandleMisbehaviorReport(channel, report) + m.HandleMisbehaviorReport(channel, r) }() } } @@ -777,11 +888,10 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Sequenti record, ok := cache.Get(originID) if !ok { - fmt.Println("not ok") return false } require.NotNil(t, record) - + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. require.Equal(t, totalPenalty, record.Penalty) // with just reporting a single misbehavior report, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) @@ -798,6 +908,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Sequenti // The test ensures that each misbehavior report is handled correctly and the penalties are cumulatively applied to the corresponding peers in the cache. func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Concurrently(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -806,7 +917,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Concurre return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -852,7 +963,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Concurre return false } require.NotNil(t, record) - + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. require.Equal(t, totalPenalty, record.Penalty) // with just reporting a single misbehavior report, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) @@ -872,6 +983,7 @@ func TestHandleMisbehaviorReport_MultiplePenaltyReportsForMultiplePeers_Concurre // is uniquely identifying a traffic violation, even though the description of the violation is the same. func TestHandleMisbehaviorReport_DuplicateReportsForSinglePeer_Concurrently(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -880,7 +992,7 @@ func TestHandleMisbehaviorReport_DuplicateReportsForSinglePeer_Concurrently(t *t return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -925,6 +1037,7 @@ func TestHandleMisbehaviorReport_DuplicateReportsForSinglePeer_Concurrently(t *t if record.Penalty != report.Penalty()*float64(times) { return false } + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. // with just reporting a few misbehavior reports, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) // the decay should be the default decay value. @@ -938,6 +1051,7 @@ func TestHandleMisbehaviorReport_DuplicateReportsForSinglePeer_Concurrently(t *t // is decayed after a single heartbeat. The test guarantees waiting for at least one heartbeat by waiting for the first decay to happen. func TestDecayMisbehaviorPenalty_SingleHeartbeat(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -946,7 +1060,7 @@ func TestDecayMisbehaviorPenalty_SingleHeartbeat(t *testing.T) { return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -995,6 +1109,7 @@ func TestDecayMisbehaviorPenalty_SingleHeartbeat(t *testing.T) { if record.Penalty != report.Penalty()*float64(times) { return false } + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. // with just reporting a few misbehavior reports, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) // the decay should be the default decay value. @@ -1026,6 +1141,7 @@ func TestDecayMisbehaviorPenalty_SingleHeartbeat(t *testing.T) { // The test ensures that the misbehavior penalty is decayed with a linear progression within multiple heartbeats. func TestDecayMisbehaviorPenalty_MultipleHeartbeats(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -1034,7 +1150,7 @@ func TestDecayMisbehaviorPenalty_MultipleHeartbeats(t *testing.T) { return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -1104,6 +1220,7 @@ func TestDecayMisbehaviorPenalty_MultipleHeartbeats(t *testing.T) { require.Greater(t, record.Penalty, penaltyBeforeDecay) // with 3 heartbeats processed, the decayed penalty should be less than the value after 4 heartbeats. require.Less(t, record.Penalty, penaltyBeforeDecay+4*record.Decay) + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. // with just reporting a few misbehavior reports, the cutoff counter should not be incremented. require.Equal(t, uint64(0), record.CutoffCounter) // the decay should be the default decay value. @@ -1114,6 +1231,7 @@ func TestDecayMisbehaviorPenalty_MultipleHeartbeats(t *testing.T) { // The test ensures that the misbehavior penalty is decayed with a linear progression within multiple heartbeats. func TestDecayMisbehaviorPenalty_DecayToZero(t *testing.T) { cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) var cache alsp.SpamRecordCache cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ @@ -1122,7 +1240,7 @@ func TestDecayMisbehaviorPenalty_DecayToZero(t *testing.T) { return cache }), } - m, err := alspmgr.NewMisbehaviorReportManager(cfg) + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) require.NoError(t, err) // start the ALSP manager @@ -1187,12 +1305,175 @@ func TestDecayMisbehaviorPenalty_DecayToZero(t *testing.T) { require.True(t, ok) // the record should be in the cache require.NotNil(t, record) + require.False(t, record.DisallowListed) // the peer should not be disallow listed yet. // with a single heartbeat and decay speed of 1000, the penalty should be decayed to zero. require.Equal(t, float64(0), record.Penalty) // the decay should be the default decay value. require.Equal(t, model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay, record.Decay) } +// TestDecayMisbehaviorPenalty_DecayToZero_AllowListing tests that when the misbehavior penalty of an already disallow-listed +// peer is decayed to zero, the peer is allow-listed back in the network, and its spam record cache is updated accordingly. +func TestDecayMisbehaviorPenalty_DecayToZero_AllowListing(t *testing.T) { + cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) + + var cache alsp.SpamRecordCache + cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ + alspmgr.WithSpamRecordsCacheFactory(func(logger zerolog.Logger, size uint32, metrics module.HeroCacheMetrics) alsp.SpamRecordCache { + cache = internal.NewSpamRecordCache(size, logger, metrics, model.SpamRecordFactory()) + return cache + }), + } + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) + require.NoError(t, err) + + // start the ALSP manager + ctx, cancel := context.WithCancel(context.Background()) + defer func() { + cancel() + unittest.RequireCloseBefore(t, m.Done(), 100*time.Millisecond, "ALSP manager did not stop") + }() + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + m.Start(signalerCtx) + unittest.RequireCloseBefore(t, m.Ready(), 100*time.Millisecond, "ALSP manager did not start") + + // simulates a disallow-listed peer in cache. + originId := unittest.IdentifierFixture() + penalty, err := cache.Adjust(originId, func(record model.ProtocolSpamRecord) (model.ProtocolSpamRecord, error) { + record.Penalty = -10 // set the penalty to -10 to simulate that the penalty has already been decayed for a while. + record.CutoffCounter = 1 + record.DisallowListed = true + record.OriginId = originId + record.Decay = model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay + return record, nil + }) + require.NoError(t, err) + require.Equal(t, float64(-10), penalty) + + // sanity check + record, ok := cache.Get(originId) + require.True(t, ok) // the record should be in the cache + require.NotNil(t, record) + require.Equal(t, float64(-10), record.Penalty) + require.True(t, record.DisallowListed) + require.Equal(t, uint64(1), record.CutoffCounter) + require.Equal(t, model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay, record.Decay) + + // eventually, we expect the ALSP manager to emit an allow list notification to the network layer when the penalty is decayed to zero. + consumer.On("OnAllowListNotification", &network.AllowListingUpdate{ + FlowIds: flow.IdentifierList{originId}, + Cause: network.DisallowListedCauseAlsp, + }).Return(nil).Once() + + // wait for at most two heartbeats; default decay speed is 1000 and with a penalty of -10, the penalty should be decayed to zero in a single heartbeat. + require.Eventually(t, func() bool { + record, ok = cache.Get(originId) + if !ok { + return false + } + if record.DisallowListed { + return false // the peer should not be allow-listed yet. + } + if record.Penalty != float64(0) { + return false // the penalty should be decayed to zero. + } + if record.CutoffCounter != 1 { + return false // the cutoff counter should be incremented. + } + if record.Decay != model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay { + return false // the decay should be the default decay value. + } + + return true + + }, 2*time.Second, 10*time.Millisecond, "penalty was not decayed to zero") + +} + +// TestDisallowListNotification tests the emission of the allow list notification to the network layer when the misbehavior +// penalty of a node is dropped below the disallow-listing threshold. The test ensures that the disallow list notification is +// emitted to the network layer when the misbehavior penalty is dropped below the disallow-listing threshold and that the +// cutoff counter of the spam record for the misbehaving node is incremented indicating that the node is disallow-listed once. +func TestDisallowListNotification(t *testing.T) { + cfg := managerCfgFixture() + consumer := mocknetwork.NewDisallowListNotificationConsumer(t) + + var cache alsp.SpamRecordCache + cfg.Opts = []alspmgr.MisbehaviorReportManagerOption{ + alspmgr.WithSpamRecordsCacheFactory(func(logger zerolog.Logger, size uint32, metrics module.HeroCacheMetrics) alsp.SpamRecordCache { + cache = internal.NewSpamRecordCache(size, logger, metrics, model.SpamRecordFactory()) + return cache + }), + } + m, err := alspmgr.NewMisbehaviorReportManager(cfg, consumer) + require.NoError(t, err) + + // start the ALSP manager + ctx, cancel := context.WithCancel(context.Background()) + defer func() { + cancel() + unittest.RequireCloseBefore(t, m.Done(), 100*time.Millisecond, "ALSP manager did not stop") + }() + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + m.Start(signalerCtx) + unittest.RequireCloseBefore(t, m.Ready(), 100*time.Millisecond, "ALSP manager did not start") + + // creates a single misbehavior report + originId := unittest.IdentifierFixture() + report := misbehaviorReportFixtureWithDefaultPenalty(t, originId) + require.Less(t, report.Penalty(), float64(0)) // ensure the penalty is negative + + channel := channels.Channel("test-channel") + + // reporting the same misbehavior 120 times, should result in a single disallow list notification, since each + // misbehavior report is reported with the same penalty 0.01 * diallowlisting-threshold. We go over the threshold + // to ensure that the disallow list notification is emitted only once. + times := 120 + wg := sync.WaitGroup{} + wg.Add(times) + + // concurrently reports the same misbehavior report twice + for i := 0; i < times; i++ { + go func() { + defer wg.Done() + + m.HandleMisbehaviorReport(channel, report) + }() + } + + // at this point, we expect a single disallow list notification to be emitted to the network layer when all the misbehavior + // reports are processed by the ALSP manager (the notification is emitted when at the next heartbeat). + consumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: flow.IdentifierList{report.OriginId()}, + Cause: network.DisallowListedCauseAlsp, + }).Return().Once() + + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "not all misbehavior reports have been processed") + + require.Eventually(t, func() bool { + // check if the misbehavior reports have been processed by verifying that the Adjust method was called on the cache + record, ok := cache.Get(originId) + if !ok { + return false + } + require.NotNil(t, record) + + // eventually, the penalty should be the accumulated penalty of all the duplicate misbehavior reports (with the default decay). + // the decay is added to the penalty as we allow for a single heartbeat before the disallow list notification is emitted. + if record.Penalty != report.Penalty()*float64(times)+record.Decay { + return false + } + require.True(t, record.DisallowListed) // the peer should be disallow-listed. + // cuttoff counter should be incremented since the penalty is above the disallowlisting threshold. + require.Equal(t, uint64(1), record.CutoffCounter) + // the decay should be the default decay value. + require.Equal(t, model.SpamRecordFactory()(unittest.IdentifierFixture()).Decay, record.Decay) + + return true + }, 1*time.Second, 10*time.Millisecond, "ALSP manager did not handle the misbehavior report") +} + // misbehaviorReportFixture creates a mock misbehavior report for a single origin id. // Args: // - t: the testing.T instance diff --git a/network/alsp/model/params.go b/network/alsp/model/params.go index e675ee6b990..04a53a8f0c8 100644 --- a/network/alsp/model/params.go +++ b/network/alsp/model/params.go @@ -1,7 +1,7 @@ package model // To give a summary with the default value: -// 1. The penalty of each misbehavior is 0.01 * misbehaviorDisallowListingThreshold = -864 +// 1. The penalty of each misbehavior is 0.01 * DisallowListingThreshold = -864 // 2. The penalty of each misbehavior is decayed by a decay value at each decay interval. The default decay value is 1000. // This means that by default if a node misbehaves 100 times in a second, it gets disallow-listed, and takes 86.4 seconds to recover. // We emphasize on the default penalty value can be amplified by the engine that reports the misbehavior. @@ -13,12 +13,12 @@ package model // around a day to recover. From this point on, the decay speed is 1, and it takes around a day to recover from each // disallow-listing. const ( - // misbehaviorDisallowListingThreshold is the threshold for concluding a node behavior is malicious and disallow-listing the node. + // DisallowListingThreshold is the threshold for concluding a node behavior is malicious and disallow-listing the node. // If the overall penalty of this node drops below this threshold, the node is reported to be disallow-listed by // the networking layer, i.e., existing connections to the node are closed and the node is no longer allowed to connect till // its penalty is decayed back to zero. // maximum block-list period is 1 day - misbehaviorDisallowListingThreshold = -24 * 60 * 60 // (Don't change this value) + DisallowListingThreshold = -24 * 60 * 60 // (Don't change this value) // DefaultPenaltyValue is the default penalty value for misbehaving nodes. // By default, each reported infringement will be penalized by this value. However, the penalty can be amplified @@ -27,7 +27,7 @@ const ( // decrease the number of misbehavior/sec that will result in disallow-listing the node. For example, if the engine // amplifies the penalty by 10, the number of misbehavior/sec that will result in disallow-listing the node will be // 10 times less than the default penalty value and the node will be disallow-listed after 10 times more misbehavior/sec. - DefaultPenaltyValue = 0.01 * misbehaviorDisallowListingThreshold // (Don't change this value) + DefaultPenaltyValue = 0.01 * DisallowListingThreshold // (Don't change this value) // InitialDecaySpeed is the initial decay speed of the penalty of a misbehaving node. // The decay speed is applied on an arithmetic progression. The penalty value of the node is the first term of the diff --git a/network/alsp/model/record.go b/network/alsp/model/record.go index cde105c1d11..088e7d81916 100644 --- a/network/alsp/model/record.go +++ b/network/alsp/model/record.go @@ -20,6 +20,11 @@ type ProtocolSpamRecord struct { // Note that the cutoff connections are recovered after a certain amount of time. CutoffCounter uint64 + // DisallowListed indicates whether the node is currently disallow-listed or not. When a node is in the disallow-list, + // the existing connections to the node are cut and no new connections are allowed to be established, neither incoming + // nor outgoing. + DisallowListed bool + // total Penalty value of the misbehaving node. Should be a negative value. Penalty float64 } @@ -44,10 +49,11 @@ type SpamRecordFactoryFunc func(flow.Identifier) ProtocolSpamRecord func SpamRecordFactory() SpamRecordFactoryFunc { return func(originId flow.Identifier) ProtocolSpamRecord { return ProtocolSpamRecord{ - OriginId: originId, - Decay: InitialDecaySpeed, - CutoffCounter: uint64(0), - Penalty: float64(0), + OriginId: originId, + Decay: InitialDecaySpeed, + DisallowListed: false, + CutoffCounter: uint64(0), + Penalty: float64(0), } } } diff --git a/network/disallow.go b/network/disallow.go new file mode 100644 index 00000000000..feaa6d2b27b --- /dev/null +++ b/network/disallow.go @@ -0,0 +1,49 @@ +package network + +import ( + "github.com/onflow/flow-go/model/flow" +) + +// DisallowListedCause is a type representing the cause of disallow listing. A remote node may be disallow-listed by the +// current node for a variety of reasons. This type is used to represent the reason for disallow-listing, so that if +// a node is disallow-listed for reasons X and Y, allow-listing it back for reason X does not automatically allow-list +// it for reason Y. +type DisallowListedCause string + +func (c DisallowListedCause) String() string { + return string(c) +} + +const ( + // DisallowListedCauseAdmin is the cause of disallow-listing a node by an admin command. + DisallowListedCauseAdmin DisallowListedCause = "disallow-listed-admin" + // DisallowListedCauseAlsp is the cause of disallow-listing a node by the ALSP (Application Layer Spam Prevention). + DisallowListedCauseAlsp DisallowListedCause = "disallow-listed-alsp" +) + +// DisallowListingUpdate is a notification of a new disallow list update, it contains a list of Flow identities that +// are now disallow listed for a specific reason. +type DisallowListingUpdate struct { + FlowIds flow.IdentifierList + Cause DisallowListedCause +} + +// AllowListingUpdate is a notification of a new allow list update, it contains a list of Flow identities that +// are now allow listed for a specific reason, i.e., their disallow list entry for that reason is removed. +type AllowListingUpdate struct { + FlowIds flow.IdentifierList + Cause DisallowListedCause +} + +// DisallowListNotificationConsumer is an interface for consuming disallow/allow list update notifications. +type DisallowListNotificationConsumer interface { + // OnDisallowListNotification is called when a new disallow list update notification is distributed. + // Any error on consuming an event must be handled internally. + // The implementation must be concurrency safe. + OnDisallowListNotification(*DisallowListingUpdate) + + // OnAllowListNotification is called when a new allow list update notification is distributed. + // Any error on consuming an event must be handled internally. + // The implementation must be concurrency safe. + OnAllowListNotification(*AllowListingUpdate) +} diff --git a/network/internal/p2pfixtures/fixtures.go b/network/internal/p2pfixtures/fixtures.go index 0d4b0b549f5..477b529302c 100644 --- a/network/internal/p2pfixtures/fixtures.go +++ b/network/internal/p2pfixtures/fixtures.go @@ -115,7 +115,11 @@ func CreateNode(t *testing.T, networkKey crypto.PrivateKey, sporkID flow.Identif unittest.DefaultAddress, networkKey, sporkID, - p2pbuilder.DefaultResourceManagerConfig()). + p2pbuilder.DefaultResourceManagerConfig(), + &p2p.DisallowListCacheConfig{ + MaxSize: uint32(1000), + Metrics: metrics.NewNoopCollector(), + }). SetRoutingSystem(func(c context.Context, h host.Host) (routing.Routing, error) { return p2pdht.NewDHT(c, h, protocols.FlowDHTProtocolID(sporkID), zerolog.Nop(), metrics.NewNoopCollector()) }). diff --git a/network/internal/testutils/testUtil.go b/network/internal/testutils/testUtil.go index ca55f6b638c..ea25564c37a 100644 --- a/network/internal/testutils/testUtil.go +++ b/network/internal/testutils/testUtil.go @@ -11,7 +11,6 @@ import ( "time" dht "github.com/libp2p/go-libp2p-kad-dht" - "github.com/libp2p/go-libp2p/core/connmgr" "github.com/libp2p/go-libp2p/core/host" p2pNetwork "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" @@ -138,9 +137,11 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, opts ...func(*optsC o := &optsConfig{ peerUpdateInterval: connection.DefaultPeerUpdateInterval, unicastRateLimiterDistributor: ratelimit.NewUnicastRateLimiterDistributor(), - connectionGater: NewConnectionGater(idProvider, func(p peer.ID) error { - return nil - }), + connectionGaterFactory: func() p2p.ConnectionGater { + return NewConnectionGater(idProvider, func(p peer.ID) error { + return nil + }) + }, createStreamRetryInterval: unicast.DefaultRetryDelay, } for _, opt := range opts { @@ -164,7 +165,7 @@ func GenerateIDs(t *testing.T, logger zerolog.Logger, n int, opts ...func(*optsC opts = append(opts, withDHT(o.dhtPrefix, o.dhtOpts...)) opts = append(opts, withPeerManagerOptions(connection.PruningEnabled, o.peerUpdateInterval)) opts = append(opts, withRateLimiterDistributor(o.unicastRateLimiterDistributor)) - opts = append(opts, withConnectionGater(o.connectionGater)) + opts = append(opts, withConnectionGater(o.connectionGaterFactory())) opts = append(opts, withUnicastManagerOpts(o.createStreamRetryInterval)) libP2PNodes[i], tagObservables[i] = generateLibP2PNode(t, logger, key, idProvider, opts...) @@ -210,16 +211,17 @@ func GenerateMiddlewares(t *testing.T, idProviders[i] = NewUpdatableIDProvider(identities) // creating middleware of nodes - mws[i] = middleware.NewMiddleware( - logger, - node, - nodeId, - bitswapmet, - sporkID, - middleware.DefaultUnicastTimeout, - translator.NewIdentityProviderIDTranslator(idProviders[i]), - codec, - consumer, + mws[i] = middleware.NewMiddleware(&middleware.Config{ + Logger: logger, + Libp2pNode: node, + FlowId: nodeId, + BitSwapMetrics: bitswapmet, + RootBlockID: sporkID, + UnicastMessageTimeout: middleware.DefaultUnicastTimeout, + IdTranslator: translator.NewIdentityProviderIDTranslator(idProviders[i]), + Codec: codec, + SlashingViolationsConsumer: consumer, + }, middleware.WithUnicastRateLimiters(o.unicastRateLimiters), middleware.WithPeerManagerFilters(o.peerManagerFilters)) } @@ -311,7 +313,7 @@ type optsConfig struct { networkMetrics module.NetworkMetrics peerManagerFilters []p2p.PeerFilter unicastRateLimiterDistributor p2p.UnicastRateLimiterDistributor - connectionGater connmgr.ConnectionGater + connectionGaterFactory func() p2p.ConnectionGater createStreamRetryInterval time.Duration } @@ -358,9 +360,9 @@ func WithUnicastRateLimiters(limiters *ratelimit.RateLimiters) func(*optsConfig) } } -func WithConnectionGater(connectionGater connmgr.ConnectionGater) func(*optsConfig) { +func WithConnectionGaterFactory(connectionGaterFactory func() p2p.ConnectionGater) func(*optsConfig) { return func(o *optsConfig) { - o.connectionGater = connectionGater + o.connectionGaterFactory = connectionGaterFactory } } @@ -471,7 +473,7 @@ func withRateLimiterDistributor(distributor p2p.UnicastRateLimiterDistributor) n } } -func withConnectionGater(connectionGater connmgr.ConnectionGater) nodeBuilderOption { +func withConnectionGater(connectionGater p2p.ConnectionGater) nodeBuilderOption { return func(nb p2p.NodeBuilder) { nb.SetConnectionGater(connectionGater) } @@ -501,7 +503,11 @@ func generateLibP2PNode(t *testing.T, logger zerolog.Logger, key crypto.PrivateK unittest.DefaultAddress, key, sporkID, - p2pbuilder.DefaultResourceManagerConfig()). + p2pbuilder.DefaultResourceManagerConfig(), + &p2p.DisallowListCacheConfig{ + MaxSize: uint32(1000), + Metrics: metrics.NewNoopCollector(), + }). SetConnectionManager(connManager). SetResourceManager(NewResourceManager(t)). SetStreamCreationRetryInterval(unicast.DefaultRetryDelay). @@ -586,7 +592,7 @@ func NewResourceManager(t *testing.T) p2pNetwork.ResourceManager { } // NewConnectionGater creates a new connection gater for testing with given allow listing filter. -func NewConnectionGater(idProvider module.IdentityProvider, allowListFilter p2p.PeerFilter) connmgr.ConnectionGater { +func NewConnectionGater(idProvider module.IdentityProvider, allowListFilter p2p.PeerFilter) p2p.ConnectionGater { filters := []p2p.PeerFilter{allowListFilter} return connection.NewConnGater(unittest.Logger(), idProvider, diff --git a/network/middleware.go b/network/middleware.go index 7bc600fbc8f..c2eeef98905 100644 --- a/network/middleware.go +++ b/network/middleware.go @@ -17,6 +17,7 @@ import ( // connections, as well as reading & writing to/from the connections. type Middleware interface { component.Component + DisallowListNotificationConsumer // SetOverlay sets the overlay used by the middleware. This must be called before the middleware can be Started. SetOverlay(Overlay) diff --git a/network/mocknetwork/connector_host.go b/network/mocknetwork/connector_host.go index 51c7ac7b539..e656391a11f 100644 --- a/network/mocknetwork/connector_host.go +++ b/network/mocknetwork/connector_host.go @@ -14,13 +14,13 @@ type ConnectorHost struct { mock.Mock } -// ClosePeer provides a mock function with given fields: id -func (_m *ConnectorHost) ClosePeer(id peer.ID) error { - ret := _m.Called(id) +// ClosePeer provides a mock function with given fields: peerId +func (_m *ConnectorHost) ClosePeer(peerId peer.ID) error { + ret := _m.Called(peerId) var r0 error if rf, ok := ret.Get(0).(func(peer.ID) error); ok { - r0 = rf(id) + r0 = rf(peerId) } else { r0 = ret.Error(0) } @@ -58,13 +58,13 @@ func (_m *ConnectorHost) ID() peer.ID { return r0 } -// IsProtected provides a mock function with given fields: id -func (_m *ConnectorHost) IsProtected(id peer.ID) bool { - ret := _m.Called(id) +// IsConnectedTo provides a mock function with given fields: peerId +func (_m *ConnectorHost) IsConnectedTo(peerId peer.ID) bool { + ret := _m.Called(peerId) var r0 bool if rf, ok := ret.Get(0).(func(peer.ID) bool); ok { - r0 = rf(id) + r0 = rf(peerId) } else { r0 = ret.Get(0).(bool) } @@ -72,13 +72,27 @@ func (_m *ConnectorHost) IsProtected(id peer.ID) bool { return r0 } -// PeerInfo provides a mock function with given fields: id -func (_m *ConnectorHost) PeerInfo(id peer.ID) peer.AddrInfo { - ret := _m.Called(id) +// IsProtected provides a mock function with given fields: peerId +func (_m *ConnectorHost) IsProtected(peerId peer.ID) bool { + ret := _m.Called(peerId) + + var r0 bool + if rf, ok := ret.Get(0).(func(peer.ID) bool); ok { + r0 = rf(peerId) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// PeerInfo provides a mock function with given fields: peerId +func (_m *ConnectorHost) PeerInfo(peerId peer.ID) peer.AddrInfo { + ret := _m.Called(peerId) var r0 peer.AddrInfo if rf, ok := ret.Get(0).(func(peer.ID) peer.AddrInfo); ok { - r0 = rf(id) + r0 = rf(peerId) } else { r0 = ret.Get(0).(peer.AddrInfo) } diff --git a/network/mocknetwork/disallow_list_notification_consumer.go b/network/mocknetwork/disallow_list_notification_consumer.go new file mode 100644 index 00000000000..802caddf023 --- /dev/null +++ b/network/mocknetwork/disallow_list_notification_consumer.go @@ -0,0 +1,38 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mocknetwork + +import ( + network "github.com/onflow/flow-go/network" + mock "github.com/stretchr/testify/mock" +) + +// DisallowListNotificationConsumer is an autogenerated mock type for the DisallowListNotificationConsumer type +type DisallowListNotificationConsumer struct { + mock.Mock +} + +// OnAllowListNotification provides a mock function with given fields: _a0 +func (_m *DisallowListNotificationConsumer) OnAllowListNotification(_a0 *network.AllowListingUpdate) { + _m.Called(_a0) +} + +// OnDisallowListNotification provides a mock function with given fields: _a0 +func (_m *DisallowListNotificationConsumer) OnDisallowListNotification(_a0 *network.DisallowListingUpdate) { + _m.Called(_a0) +} + +type mockConstructorTestingTNewDisallowListNotificationConsumer interface { + mock.TestingT + Cleanup(func()) +} + +// NewDisallowListNotificationConsumer creates a new instance of DisallowListNotificationConsumer. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewDisallowListNotificationConsumer(t mockConstructorTestingTNewDisallowListNotificationConsumer) *DisallowListNotificationConsumer { + mock := &DisallowListNotificationConsumer{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/network/mocknetwork/disallow_list_oracle.go b/network/mocknetwork/disallow_list_oracle.go new file mode 100644 index 00000000000..3bae6e851f3 --- /dev/null +++ b/network/mocknetwork/disallow_list_oracle.go @@ -0,0 +1,46 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mocknetwork + +import ( + network "github.com/onflow/flow-go/network" + mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p/core/peer" +) + +// DisallowListOracle is an autogenerated mock type for the DisallowListOracle type +type DisallowListOracle struct { + mock.Mock +} + +// GetAllDisallowedListCausesFor provides a mock function with given fields: _a0 +func (_m *DisallowListOracle) GetAllDisallowedListCausesFor(_a0 peer.ID) []network.DisallowListedCause { + ret := _m.Called(_a0) + + var r0 []network.DisallowListedCause + if rf, ok := ret.Get(0).(func(peer.ID) []network.DisallowListedCause); ok { + r0 = rf(_a0) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]network.DisallowListedCause) + } + } + + return r0 +} + +type mockConstructorTestingTNewDisallowListOracle interface { + mock.TestingT + Cleanup(func()) +} + +// NewDisallowListOracle creates a new instance of DisallowListOracle. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewDisallowListOracle(t mockConstructorTestingTNewDisallowListOracle) *DisallowListOracle { + mock := &DisallowListOracle{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/network/mocknetwork/middleware.go b/network/mocknetwork/middleware.go index 457d8fd7360..64167ce9ed8 100644 --- a/network/mocknetwork/middleware.go +++ b/network/mocknetwork/middleware.go @@ -101,6 +101,16 @@ func (_m *Middleware) NewPingService(pingProtocol protocol.ID, provider network. return r0 } +// OnAllowListNotification provides a mock function with given fields: _a0 +func (_m *Middleware) OnAllowListNotification(_a0 *network.AllowListingUpdate) { + _m.Called(_a0) +} + +// OnDisallowListNotification provides a mock function with given fields: _a0 +func (_m *Middleware) OnDisallowListNotification(_a0 *network.DisallowListingUpdate) { + _m.Called(_a0) +} + // Publish provides a mock function with given fields: msg func (_m *Middleware) Publish(msg *network.OutgoingMessageScope) error { ret := _m.Called(msg) diff --git a/network/p2p/builder.go b/network/p2p/builder.go index 43037f5a90e..a8a89367013 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -19,7 +19,7 @@ import ( ) type GossipSubFactoryFunc func(context.Context, zerolog.Logger, host.Host, PubSubAdapterConfig) (PubSubAdapter, error) -type CreateNodeFunc func(zerolog.Logger, host.Host, ProtocolPeerCache, PeerManager) 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. @@ -96,7 +96,7 @@ type NodeBuilder interface { SetSubscriptionFilter(pubsub.SubscriptionFilter) NodeBuilder SetResourceManager(network.ResourceManager) NodeBuilder SetConnectionManager(connmgr.ConnManager) NodeBuilder - SetConnectionGater(connmgr.ConnectionGater) NodeBuilder + SetConnectionGater(ConnectionGater) NodeBuilder SetRoutingSystem(func(context.Context, host.Host) (routing.Routing, error)) NodeBuilder SetPeerManagerOptions(bool, time.Duration) NodeBuilder diff --git a/network/p2p/cache/node_blocklist_wrapper.go b/network/p2p/cache/node_blocklist_wrapper.go index ae045ecff62..f655215178a 100644 --- a/network/p2p/cache/node_blocklist_wrapper.go +++ b/network/p2p/cache/node_blocklist_wrapper.go @@ -10,7 +10,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" - "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/storage" "github.com/onflow/flow-go/storage/badger/operation" ) @@ -24,84 +24,101 @@ func (s IdentifierSet) Contains(id flow.Identifier) bool { return found } -// NodeBlocklistWrapper is a wrapper for an `module.IdentityProvider` instance, where the -// wrapper overrides the `Ejected` flag to true for all NodeIDs in a `blocklist`. +// NodeDisallowListingWrapper is a wrapper for an `module.IdentityProvider` instance, where the +// wrapper overrides the `Ejected` flag to true for all NodeIDs in a `disallowList`. // To avoid modifying the source of the identities, the wrapper creates shallow copies // of the identities (whenever necessary) and modifies the `Ejected` flag only in // the copy. -// The `NodeBlocklistWrapper` internally represents the `blocklist` as a map, to enable +// The `NodeDisallowListingWrapper` internally represents the `disallowList` as a map, to enable // performant lookup. However, the exported API works with `flow.IdentifierList` for -// blocklist, as this is a broadly supported data structure which lends itself better +// disallowList, as this is a broadly supported data structure which lends itself better // to config or command-line inputs. -type NodeBlocklistWrapper struct { +// When a node is disallow-listed, the networking layer connection to that node is closed and no +// incoming or outgoing connections are established with that node. +// TODO: terminology change - rename `blocklist` to `disallowList` everywhere to be consistent with the code. +type NodeDisallowListingWrapper struct { m sync.RWMutex db *badger.DB identityProvider module.IdentityProvider - blocklist IdentifierSet // `IdentifierSet` is a map, hence efficient O(1) lookup - distributor p2p.DisallowListNotificationDistributor // distributor for the blocklist update notifications + disallowList IdentifierSet // `IdentifierSet` is a map, hence efficient O(1) lookup + + // updateConsumerOracle is called whenever the disallow-list is updated. + // Note that we do not use the `updateConsumer` directly due to the circular dependency between the + // middleware (i.e., updateConsumer), and the wrapper (i.e., NodeDisallowListingWrapper). + // Middleware needs identity provider to be initialized, and identity provider needs this wrapper to be initialized. + // Hence, if we pass the updateConsumer by the interface value, it will be nil at the time of initialization. + // Instead, we use the oracle function to get the updateConsumer whenever we need it. + updateConsumerOracle func() network.DisallowListNotificationConsumer } -var _ module.IdentityProvider = (*NodeBlocklistWrapper)(nil) +var _ module.IdentityProvider = (*NodeDisallowListingWrapper)(nil) -// NewNodeBlocklistWrapper wraps the given `IdentityProvider`. The blocklist is +// NewNodeDisallowListWrapper wraps the given `IdentityProvider`. The disallow-list is // loaded from the database (or assumed to be empty if no database entry is present). -func NewNodeBlocklistWrapper( +func NewNodeDisallowListWrapper( identityProvider module.IdentityProvider, db *badger.DB, - distributor p2p.DisallowListNotificationDistributor) (*NodeBlocklistWrapper, error) { + updateConsumerOracle func() network.DisallowListNotificationConsumer) (*NodeDisallowListingWrapper, error) { - blocklist, err := retrieveBlocklist(db) + disallowList, err := retrieveDisallowList(db) if err != nil { - return nil, fmt.Errorf("failed to read set of blocked node IDs from data base: %w", err) + return nil, fmt.Errorf("failed to read set of disallowed node IDs from data base: %w", err) } - return &NodeBlocklistWrapper{ - db: db, - identityProvider: identityProvider, - blocklist: blocklist, - distributor: distributor, + return &NodeDisallowListingWrapper{ + db: db, + identityProvider: identityProvider, + disallowList: disallowList, + updateConsumerOracle: updateConsumerOracle, }, nil } -// Update sets the wrapper's internal set of blocked nodes to `blocklist`. Empty list and `nil` -// (equivalent to empty list) are accepted inputs. To avoid legacy entries in the data base, this -// function purges the entire data base entry if `blocklist` is empty. -// This implementation is _eventually consistent_, where changes are written to the data base first +// Update sets the wrapper's internal set of blocked nodes to `disallowList`. Empty list and `nil` +// (equivalent to empty list) are accepted inputs. To avoid legacy entries in the database, this +// function purges the entire data base entry if `disallowList` is empty. +// This implementation is _eventually consistent_, where changes are written to the database first // and then (non-atomically!) the in-memory set of blocked nodes is updated. This strongly // benefits performance and modularity. No errors are expected during normal operations. -func (w *NodeBlocklistWrapper) Update(blocklist flow.IdentifierList) error { - b := blocklist.Lookup() // converts slice to map +// +// Args: +// - disallowList: list of node IDs to be disallow-listed from the networking layer, i.e., the existing connections +// to these nodes will be closed and no new connections will be established (neither incoming nor outgoing). +// +// Returns: +// - error: if the update fails, e.g., due to a database error. Any returned error is irrecoverable and the caller +// should abort the process. +func (w *NodeDisallowListingWrapper) Update(disallowList flow.IdentifierList) error { + b := disallowList.Lookup() // converts slice to map w.m.Lock() defer w.m.Unlock() - err := persistBlocklist(b, w.db) + err := persistDisallowList(b, w.db) if err != nil { return fmt.Errorf("failed to persist set of blocked nodes to the data base: %w", err) } - w.blocklist = b - err = w.distributor.DistributeBlockListNotification(blocklist) - - if err != nil { - return fmt.Errorf("failed to distribute blocklist update notification: %w", err) - } + w.disallowList = b + w.updateConsumerOracle().OnDisallowListNotification(&network.DisallowListingUpdate{ + FlowIds: disallowList, + Cause: network.DisallowListedCauseAdmin, + }) return nil } -// ClearBlocklist purges the set of blocked node IDs. Convenience function +// ClearDisallowList purges the set of blocked node IDs. Convenience function // equivalent to w.Update(nil). No errors are expected during normal operations. -func (w *NodeBlocklistWrapper) ClearBlocklist() error { +func (w *NodeDisallowListingWrapper) ClearDisallowList() error { return w.Update(nil) } -// GetBlocklist returns the set of blocked node IDs. -func (w *NodeBlocklistWrapper) GetBlocklist() flow.IdentifierList { +// GetDisallowList returns the set of blocked node IDs. +func (w *NodeDisallowListingWrapper) GetDisallowList() flow.IdentifierList { w.m.RLock() defer w.m.RUnlock() - identifiers := make(flow.IdentifierList, 0, len(w.blocklist)) - for i := range w.blocklist { + identifiers := make(flow.IdentifierList, 0, len(w.disallowList)) + for i := range w.disallowList { identifiers = append(identifiers, i) } return identifiers @@ -111,7 +128,7 @@ func (w *NodeBlocklistWrapper) GetBlocklist() flow.IdentifierList { // protocol that pass the provided filter. Caution, this includes ejected nodes. // Please check the `Ejected` flag in the returned identities (or provide a // filter for removing ejected nodes). -func (w *NodeBlocklistWrapper) Identities(filter flow.IdentityFilter) flow.IdentityList { +func (w *NodeDisallowListingWrapper) Identities(filter flow.IdentityFilter) flow.IdentityList { identities := w.identityProvider.Identities(filter) if len(identities) == 0 { return identities @@ -123,7 +140,7 @@ func (w *NodeBlocklistWrapper) Identities(filter flow.IdentityFilter) flow.Ident idtx := make(flow.IdentityList, 0, len(identities)) w.m.RLock() for _, identity := range identities { - if w.blocklist.Contains(identity.NodeID) { + if w.disallowList.Contains(identity.NodeID) { var i = *identity // shallow copy is sufficient, because `Ejected` flag is in top-level struct i.Ejected = true if filter(&i) { // we need to check the filter here again, because the filter might drop ejected nodes and we are modifying the ejected status here @@ -143,22 +160,22 @@ func (w *NodeBlocklistWrapper) Identities(filter flow.IdentityFilter) flow.Ident // true if and only if Identity has been found, i.e. `Identity` is not nil. // Caution: function returns include ejected nodes. Please check the `Ejected` // flag in the identity. -func (w *NodeBlocklistWrapper) ByNodeID(identifier flow.Identifier) (*flow.Identity, bool) { +func (w *NodeDisallowListingWrapper) ByNodeID(identifier flow.Identifier) (*flow.Identity, bool) { identity, b := w.identityProvider.ByNodeID(identifier) return w.setEjectedIfBlocked(identity), b } -// setEjectedIfBlocked checks whether the node with the given identity is on the `blocklist`. +// setEjectedIfBlocked checks whether the node with the given identity is on the `disallowList`. // Shortcuts: // - If the node's identity is nil, there is nothing to do because we don't generate identities here. -// - If the node is already ejected, we don't have to check the blocklist. -func (w *NodeBlocklistWrapper) setEjectedIfBlocked(identity *flow.Identity) *flow.Identity { +// - If the node is already ejected, we don't have to check the disallowList. +func (w *NodeDisallowListingWrapper) setEjectedIfBlocked(identity *flow.Identity) *flow.Identity { if identity == nil || identity.Ejected { return identity } w.m.RLock() - isBlocked := w.blocklist.Contains(identity.NodeID) + isBlocked := w.disallowList.Contains(identity.NodeID) w.m.RUnlock() if !isBlocked { return identity @@ -178,25 +195,25 @@ func (w *NodeBlocklistWrapper) setEjectedIfBlocked(identity *flow.Identity) *flo // true if and only if Identity has been found, i.e. `Identity` is not nil. // Caution: function returns include ejected nodes. Please check the `Ejected` // flag in the identity. -func (w *NodeBlocklistWrapper) ByPeerID(p peer.ID) (*flow.Identity, bool) { +func (w *NodeDisallowListingWrapper) ByPeerID(p peer.ID) (*flow.Identity, bool) { identity, b := w.identityProvider.ByPeerID(p) return w.setEjectedIfBlocked(identity), b } -// persistBlocklist writes the given blocklist to the database. To avoid legacy -// entries in the database, we prune the entire data base entry if `blocklist` is +// persistDisallowList writes the given disallowList to the database. To avoid legacy +// entries in the database, we prune the entire data base entry if `disallowList` is // empty. No errors are expected during normal operations. -func persistBlocklist(blocklist IdentifierSet, db *badger.DB) error { - if len(blocklist) == 0 { +func persistDisallowList(disallowList IdentifierSet, db *badger.DB) error { + if len(disallowList) == 0 { return db.Update(operation.PurgeBlocklist()) } - return db.Update(operation.PersistBlocklist(blocklist)) + return db.Update(operation.PersistBlocklist(disallowList)) } -// retrieveBlocklist reads the set of blocked nodes from the data base. +// retrieveDisallowList reads the set of blocked nodes from the data base. // In case no database entry exists, an empty set (nil map) is returned. // No errors are expected during normal operations. -func retrieveBlocklist(db *badger.DB) (IdentifierSet, error) { +func retrieveDisallowList(db *badger.DB) (IdentifierSet, error) { var blocklist map[flow.Identifier]struct{} err := db.View(operation.RetrieveBlocklist(&blocklist)) if err != nil && !errors.Is(err, storage.ErrNotFound) { diff --git a/network/p2p/cache/node_blocklist_wrapper_test.go b/network/p2p/cache/node_blocklist_wrapper_test.go index cdc32b546f5..01e23e5a276 100644 --- a/network/p2p/cache/node_blocklist_wrapper_test.go +++ b/network/p2p/cache/node_blocklist_wrapper_test.go @@ -14,39 +14,42 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/model/flow/filter" mocks "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/mocknetwork" "github.com/onflow/flow-go/network/p2p" "github.com/onflow/flow-go/network/p2p/cache" - mockp2p "github.com/onflow/flow-go/network/p2p/mock" "github.com/onflow/flow-go/utils/unittest" ) -type NodeBlocklistWrapperTestSuite struct { +type NodeDisallowListWrapperTestSuite struct { suite.Suite DB *badger.DB provider *mocks.IdentityProvider - wrapper *cache.NodeBlocklistWrapper - distributor *mockp2p.DisallowListNotificationDistributor + wrapper *cache.NodeDisallowListingWrapper + updateConsumer *mocknetwork.DisallowListNotificationConsumer } -func (s *NodeBlocklistWrapperTestSuite) SetupTest() { +func (s *NodeDisallowListWrapperTestSuite) SetupTest() { s.DB, _ = unittest.TempBadgerDB(s.T()) s.provider = new(mocks.IdentityProvider) var err error - s.distributor = mockp2p.NewDisallowListNotificationDistributor(s.T()) - s.wrapper, err = cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + s.updateConsumer = mocknetwork.NewDisallowListNotificationConsumer(s.T()) + s.wrapper, err = cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) } -func TestNodeBlocklistWrapperTestSuite(t *testing.T) { - suite.Run(t, new(NodeBlocklistWrapperTestSuite)) +func TestNodeDisallowListWrapperTestSuite(t *testing.T) { + suite.Run(t, new(NodeDisallowListWrapperTestSuite)) } // TestHonestNode verifies: -// For nodes _not_ on the blocklist, the `cache.NodeBlocklistWrapper` should forward +// For nodes _not_ on the disallowList, the `cache.NodeDisallowListingWrapper` should forward // the identities from the wrapped `IdentityProvider` without modification. -func (s *NodeBlocklistWrapperTestSuite) TestHonestNode() { +func (s *NodeDisallowListWrapperTestSuite) TestHonestNode() { s.Run("ByNodeID", func() { identity := unittest.IdentityFixture() s.provider.On("ByNodeID", identity.NodeID).Return(identity, true) @@ -78,8 +81,8 @@ func (s *NodeBlocklistWrapperTestSuite) TestHonestNode() { }) } -// TestDenylistedNode tests proper handling of identities _on_ the blocklist: -// - For any identity `i` with `i.NodeID ∈ blocklist`, the returned identity +// TestDisallowListNode tests proper handling of identities _on_ the disallowList: +// - For any identity `i` with `i.NodeID ∈ disallowList`, the returned identity // should have `i.Ejected` set to `true` (irrespective of the `Ejected` // flag's initial returned by the wrapped `IdentityProvider`). // - The wrapper should _copy_ the identity and _not_ write into the wrapped @@ -91,9 +94,12 @@ func (s *NodeBlocklistWrapperTestSuite) TestHonestNode() { // While returning (non-nil identity, false) is not a defined return value, // we expect the wrapper to nevertheless handle this case to increase its // generality. -func (s *NodeBlocklistWrapperTestSuite) TestDenylistedNode() { +func (s *NodeDisallowListWrapperTestSuite) TestDisallowListNode() { blocklist := unittest.IdentityListFixture(11) - s.distributor.On("DistributeBlockListNotification", blocklist.NodeIDs()).Return(nil).Once() + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: blocklist.NodeIDs(), + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() err := s.wrapper.Update(blocklist.NodeIDs()) require.NoError(s.T(), err) @@ -188,7 +194,7 @@ func (s *NodeBlocklistWrapperTestSuite) TestDenylistedNode() { // TestUnknownNode verifies that the wrapper forwards nil identities // irrespective of the boolean return values. -func (s *NodeBlocklistWrapperTestSuite) TestUnknownNode() { +func (s *NodeDisallowListWrapperTestSuite) TestUnknownNode() { for _, b := range []bool{true, false} { s.Run(fmt.Sprintf("IdentityProvider.ByNodeID returning (nil, %v)", b), func() { id := unittest.IdentifierFixture() @@ -210,13 +216,13 @@ func (s *NodeBlocklistWrapperTestSuite) TestUnknownNode() { } } -// TestBlocklistAddRemove checks that adding and subsequently removing a node from the blocklist +// TestDisallowListAddRemove checks that adding and subsequently removing a node from the disallowList // it in combination a no-op. We test two scenarious // - Node whose original `Identity` has `Ejected = false`: -// After adding the node to the blocklist and then removing it again, the `Ejected` should be false. +// After adding the node to the disallowList and then removing it again, the `Ejected` should be false. // - Node whose original `Identity` has `Ejected = true`: -// After adding the node to the blocklist and then removing it again, the `Ejected` should be still be true. -func (s *NodeBlocklistWrapperTestSuite) TestBlocklistAddRemove() { +// After adding the node to the disallowList and then removing it again, the `Ejected` should be still be true. +func (s *NodeDisallowListWrapperTestSuite) TestDisallowListAddRemove() { for _, originalEjected := range []bool{true, false} { s.Run(fmt.Sprintf("Add & remove node with Ejected = %v", originalEjected), func() { originalIdentity := unittest.IdentityFixture() @@ -225,7 +231,7 @@ func (s *NodeBlocklistWrapperTestSuite) TestBlocklistAddRemove() { s.provider.On("ByNodeID", originalIdentity.NodeID).Return(originalIdentity, true) s.provider.On("ByPeerID", peerID).Return(originalIdentity, true) - // step 1: before putting node on blocklist, + // step 1: before putting node on disallowList, // an Identity with `Ejected` equal to the original value should be returned i, found := s.wrapper.ByNodeID(originalIdentity.NodeID) require.True(s.T(), found) @@ -235,9 +241,12 @@ func (s *NodeBlocklistWrapperTestSuite) TestBlocklistAddRemove() { require.True(s.T(), found) require.Equal(s.T(), originalEjected, i.Ejected) - // step 2: _after_ putting node on blocklist, + // step 2: _after_ putting node on disallowList, // an Identity with `Ejected` equal to `true` should be returned - s.distributor.On("DistributeBlockListNotification", flow.IdentifierList{originalIdentity.NodeID}).Return(nil).Once() + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: flow.IdentifierList{originalIdentity.NodeID}, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() err := s.wrapper.Update(flow.IdentifierList{originalIdentity.NodeID}) require.NoError(s.T(), err) @@ -249,9 +258,12 @@ func (s *NodeBlocklistWrapperTestSuite) TestBlocklistAddRemove() { require.True(s.T(), found) require.True(s.T(), i.Ejected) - // step 3: after removing the node from the blocklist, + // step 3: after removing the node from the disallowList, // an Identity with `Ejected` equal to the original value should be returned - s.distributor.On("DistributeBlockListNotification", flow.IdentifierList{}).Return(nil).Once() + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: flow.IdentifierList{}, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() err = s.wrapper.Update(flow.IdentifierList{}) require.NoError(s.T(), err) @@ -266,125 +278,170 @@ func (s *NodeBlocklistWrapperTestSuite) TestBlocklistAddRemove() { } } -// TestUpdate tests updating, clearing and retrieving the blocklist. +// TestUpdate tests updating, clearing and retrieving the disallowList. // This test verifies that the wrapper updates _its own internal state_ correctly. // Note: -// conceptually, the blocklist is a set, i.e. not order dependent. +// conceptually, the disallowList is a set, i.e. not order dependent. // The wrapper internally converts the list to a set and vice versa. Therefore -// the order is not preserved by `GetBlocklist`. Consequently, we compare +// the order is not preserved by `GetDisallowList`. Consequently, we compare // map-based representations here. -func (s *NodeBlocklistWrapperTestSuite) TestUpdate() { - blocklist1 := unittest.IdentifierListFixture(8) - blocklist2 := unittest.IdentifierListFixture(11) - blocklist3 := unittest.IdentifierListFixture(5) - - s.distributor.On("DistributeBlockListNotification", blocklist1).Return(nil).Once() - err := s.wrapper.Update(blocklist1) +func (s *NodeDisallowListWrapperTestSuite) TestUpdate() { + disallowList1 := unittest.IdentifierListFixture(8) + disallowList2 := unittest.IdentifierListFixture(11) + disallowList3 := unittest.IdentifierListFixture(5) + + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList1, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err := s.wrapper.Update(disallowList1) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist1.Lookup(), s.wrapper.GetBlocklist().Lookup()) + require.Equal(s.T(), disallowList1.Lookup(), s.wrapper.GetDisallowList().Lookup()) - s.distributor.On("DistributeBlockListNotification", blocklist2).Return(nil).Once() - err = s.wrapper.Update(blocklist2) + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList2, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err = s.wrapper.Update(disallowList2) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist2.Lookup(), s.wrapper.GetBlocklist().Lookup()) + require.Equal(s.T(), disallowList2.Lookup(), s.wrapper.GetDisallowList().Lookup()) - s.distributor.On("DistributeBlockListNotification", (flow.IdentifierList)(nil)).Return(nil).Once() - err = s.wrapper.ClearBlocklist() + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: nil, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err = s.wrapper.ClearDisallowList() require.NoError(s.T(), err) - require.Empty(s.T(), s.wrapper.GetBlocklist()) + require.Empty(s.T(), s.wrapper.GetDisallowList()) - s.distributor.On("DistributeBlockListNotification", blocklist3).Return(nil).Once() - err = s.wrapper.Update(blocklist3) + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList3, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err = s.wrapper.Update(disallowList3) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist3.Lookup(), s.wrapper.GetBlocklist().Lookup()) + require.Equal(s.T(), disallowList3.Lookup(), s.wrapper.GetDisallowList().Lookup()) } // TestDataBasePersist verifies database interactions of the wrapper with the data base. -// This test verifies that the blocklist updates are persisted across restarts. +// This test verifies that the disallowList updates are persisted across restarts. // To decouple this test from the lower-level data base design, we proceed as follows: -// - We do data-base operation through the exported methods from `NodeBlocklistWrapper` -// - Then, we create a new `NodeBlocklistWrapper` backed by the same data base. Since it is a +// - We do data-base operation through the exported methods from `NodeDisallowListingWrapper` +// - Then, we create a new `NodeDisallowListingWrapper` backed by the same data base. Since it is a // new wrapper, it must read its state from the data base. Hence, if the new wrapper returns // the correct data, we have strong evidence that data-base interactions are correct. // // Note: The wrapper internally converts the list to a set and vice versa. Therefore -// the order is not preserved by `GetBlocklist`. Consequently, we compare +// the order is not preserved by `GetDisallowList`. Consequently, we compare // map-based representations here. -func (s *NodeBlocklistWrapperTestSuite) TestDataBasePersist() { - blocklist := unittest.IdentifierListFixture(8) - blocklist2 := unittest.IdentifierListFixture(8) +func (s *NodeDisallowListWrapperTestSuite) TestDataBasePersist() { + disallowList1 := unittest.IdentifierListFixture(8) + disallowList2 := unittest.IdentifierListFixture(8) - s.Run("Get blocklist from empty database", func() { - require.Empty(s.T(), s.wrapper.GetBlocklist()) + s.Run("Get disallowList from empty database", func() { + require.Empty(s.T(), s.wrapper.GetDisallowList()) }) - s.Run("Clear blocklist on empty database", func() { - s.distributor.On("DistributeBlockListNotification", (flow.IdentifierList)(nil)).Return(nil).Once() - err := s.wrapper.ClearBlocklist() // No-op as data base does not contain any block list + s.Run("Clear disallow-list on empty database", func() { + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: nil, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err := s.wrapper.ClearDisallowList() // No-op as data base does not contain any block list require.NoError(s.T(), err) - require.Empty(s.T(), s.wrapper.GetBlocklist()) + require.Empty(s.T(), s.wrapper.GetDisallowList()) - // newly created wrapper should read `blocklist` from data base during initialization - w, err := cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + // newly created wrapper should read `disallowList` from data base during initialization + w, err := cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) - require.Empty(s.T(), w.GetBlocklist()) + require.Empty(s.T(), w.GetDisallowList()) }) - s.Run("Update blocklist and init new wrapper from database", func() { - s.distributor.On("DistributeBlockListNotification", blocklist).Return(nil).Once() - err := s.wrapper.Update(blocklist) + s.Run("Update disallowList and init new wrapper from database", func() { + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList1, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err := s.wrapper.Update(disallowList1) require.NoError(s.T(), err) - // newly created wrapper should read `blocklist` from data base during initialization - w, err := cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + // newly created wrapper should read `disallowList` from data base during initialization + w, err := cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist.Lookup(), w.GetBlocklist().Lookup()) + require.Equal(s.T(), disallowList1.Lookup(), w.GetDisallowList().Lookup()) }) - s.Run("Update and overwrite blocklist and then init new wrapper from database", func() { - s.distributor.On("DistributeBlockListNotification", blocklist).Return(nil).Once() - err := s.wrapper.Update(blocklist) + s.Run("Update and overwrite disallowList and then init new wrapper from database", func() { + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList1, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err := s.wrapper.Update(disallowList1) require.NoError(s.T(), err) - s.distributor.On("DistributeBlockListNotification", blocklist2).Return(nil).Once() - err = s.wrapper.Update(blocklist2) + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList2, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err = s.wrapper.Update(disallowList2) require.NoError(s.T(), err) // newly created wrapper should read initial state from data base - w, err := cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + w, err := cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist2.Lookup(), w.GetBlocklist().Lookup()) + require.Equal(s.T(), disallowList2.Lookup(), w.GetDisallowList().Lookup()) }) s.Run("Update & clear & update and then init new wrapper from database", func() { - // set blocklist -> + // set disallowList -> // newly created wrapper should now read this list from data base during initialization - s.distributor.On("DistributeBlockListNotification", blocklist).Return(nil).Once() - err := s.wrapper.Update(blocklist) + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList1, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err := s.wrapper.Update(disallowList1) require.NoError(s.T(), err) - w0, err := cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + w0, err := cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist.Lookup(), w0.GetBlocklist().Lookup()) - - // clear blocklist -> - // newly created wrapper should now read empty blocklist from data base during initialization - s.distributor.On("DistributeBlockListNotification", (flow.IdentifierList)(nil)).Return(nil).Once() - err = s.wrapper.ClearBlocklist() + require.Equal(s.T(), disallowList1.Lookup(), w0.GetDisallowList().Lookup()) + + // clear disallowList -> + // newly created wrapper should now read empty disallowList from data base during initialization + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: nil, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err = s.wrapper.ClearDisallowList() require.NoError(s.T(), err) - w1, err := cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + w1, err := cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) - require.Empty(s.T(), w1.GetBlocklist()) + require.Empty(s.T(), w1.GetDisallowList()) - // set blocklist2 -> + // set disallowList2 -> // newly created wrapper should now read this list from data base during initialization - s.distributor.On("DistributeBlockListNotification", blocklist2).Return(nil).Once() - err = s.wrapper.Update(blocklist2) + s.updateConsumer.On("OnDisallowListNotification", &network.DisallowListingUpdate{ + FlowIds: disallowList2, + Cause: network.DisallowListedCauseAdmin, + }).Return().Once() + err = s.wrapper.Update(disallowList2) require.NoError(s.T(), err) - w2, err := cache.NewNodeBlocklistWrapper(s.provider, s.DB, s.distributor) + w2, err := cache.NewNodeDisallowListWrapper(s.provider, s.DB, func() network.DisallowListNotificationConsumer { + return s.updateConsumer + }) require.NoError(s.T(), err) - require.Equal(s.T(), blocklist2.Lookup(), w2.GetBlocklist().Lookup()) + require.Equal(s.T(), disallowList2.Lookup(), w2.GetDisallowList().Lookup()) }) } diff --git a/network/p2p/connection/connection_gater.go b/network/p2p/connection/connection_gater.go index 2ee0df16331..3603d15d227 100644 --- a/network/p2p/connection/connection_gater.go +++ b/network/p2p/connection/connection_gater.go @@ -1,9 +1,9 @@ package connection import ( + "fmt" "sync" - "github.com/libp2p/go-libp2p/core/connmgr" "github.com/libp2p/go-libp2p/core/control" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" @@ -15,7 +15,7 @@ import ( "github.com/onflow/flow-go/utils/logging" ) -var _ connmgr.ConnectionGater = (*ConnGater)(nil) +var _ p2p.ConnectionGater = (*ConnGater)(nil) // ConnGaterOption allow the connection gater to be configured with a list of PeerFilter funcs for a specific conn gater callback. // In the current implementation of the ConnGater the following callbacks can be configured with peer filters. @@ -44,6 +44,11 @@ type ConnGater struct { onInterceptPeerDialFilters []p2p.PeerFilter onInterceptSecuredFilters []p2p.PeerFilter + // disallowListOracle is consulted upon every incoming or outgoing connection attempt, and the connection is only + // allowed if the remote peer is not on the disallow list. + // A ConnGater must have a disallowListOracle set, and if one is not set the ConnGater will panic. + disallowListOracle p2p.DisallowListOracle + // identityProvider provides the identity of a node given its peer ID for logging purposes only. // It is not used for allowlisting or filtering. We use the onInterceptPeerDialFilters and onInterceptSecuredFilters // to determine if a node should be allowed to connect. @@ -68,6 +73,14 @@ func NewConnGater(log zerolog.Logger, identityProvider module.IdentityProvider, func (c *ConnGater) InterceptPeerDial(p peer.ID) bool { lg := c.log.With().Str("peer_id", p.String()).Logger() + disallowListCauses, disallowListed := c.disallowListOracle.IsDisallowListed(p) + if disallowListed { + lg.Warn(). + Str("disallow_list_causes", fmt.Sprintf("%v", disallowListCauses)). + Msg("outbound connection attempt to disallow listed peer is rejected") + return false + } + if len(c.onInterceptPeerDialFilters) == 0 { lg.Warn(). Msg("outbound connection established with no intercept peer dial filters") @@ -119,6 +132,14 @@ func (c *ConnGater) InterceptSecured(dir network.Direction, p peer.ID, addr netw Str("remote_address", addr.RemoteMultiaddr().String()). Logger() + disallowListCauses, disallowListed := c.disallowListOracle.IsDisallowListed(p) + if disallowListed { + lg.Warn(). + Str("disallow_list_causes", fmt.Sprintf("%v", disallowListCauses)). + Msg("inbound connection attempt to disallow listed peer is rejected") + return false + } + if len(c.onInterceptSecuredFilters) == 0 { lg.Warn().Msg("inbound connection established with no intercept secured filters") return true @@ -169,3 +190,28 @@ func (c *ConnGater) peerIDPassesAllFilters(p peer.ID, filters []p2p.PeerFilter) return nil } + +// SetDisallowListOracle sets the disallow list oracle for the connection gater. +// If one is set, the oracle is consulted upon every incoming or outgoing connection attempt, and +// the connection is only allowed if the remote peer is not on the disallow list. +// In Flow blockchain, it is not optional to dismiss the disallow list oracle, and if one is not set +// the connection gater will panic. +// Also, it follows a dependency injection pattern and does not allow to set the disallow list oracle more than once, +// any subsequent calls to this method will result in a panic. +// Args: +// +// oracle: the disallow list oracle to set. +// +// Returns: +// +// none +// +// Panics: +// +// if the disallow list oracle is already set. +func (c *ConnGater) SetDisallowListOracle(oracle p2p.DisallowListOracle) { + if c.disallowListOracle != nil { + panic("disallow list oracle already set") + } + c.disallowListOracle = oracle +} diff --git a/network/p2p/connection/connection_gater_test.go b/network/p2p/connection/connection_gater_test.go index e3f723ef71e..07c3f0e2115 100644 --- a/network/p2p/connection/connection_gater_test.go +++ b/network/p2p/connection/connection_gater_test.go @@ -396,7 +396,7 @@ func ensureCommunicationSilenceAmongGroups(t *testing.T, ctx context.Context, sp // ensureCommunicationOverAllProtocols ensures that all nodes are connected to each other, and they can exchange messages over the pubsub and unicast. func ensureCommunicationOverAllProtocols(t *testing.T, ctx context.Context, sporkId flow.Identifier, nodes []p2p.LibP2PNode, inbounds []chan string) { - p2ptest.EnsureConnected(t, ctx, nodes) + p2ptest.TryConnectionAndEnsureConnected(t, ctx, nodes) p2ptest.EnsurePubsubMessageExchange(t, ctx, nodes, func() (interface{}, channels.Topic) { blockTopic := channels.TopicFromChannel(channels.PushBlocks, sporkId) return unittest.ProposalFixture(), blockTopic diff --git a/network/p2p/connection/connector.go b/network/p2p/connection/connector.go index bfbba1e15d1..0065c395d46 100644 --- a/network/p2p/connection/connector.go +++ b/network/p2p/connection/connector.go @@ -109,6 +109,10 @@ func (l *Libp2pConnector) connectToPeers(ctx context.Context, peerIDs peer.IDSli } for _, peerID := range peerIDs { + if l.host.IsConnectedTo(peerID) { + l.log.Trace().Str("peer_id", peerID.String()).Msg("already connected to peer, skipping connection") + continue + } peerCh <- peer.AddrInfo{ID: peerID} } @@ -152,6 +156,11 @@ func (l *Libp2pConnector) pruneAllConnectionsExcept(peerIDs peer.IDSlice) { if flowStream != nil { lg = lg.With().Str("flow_stream", string(flowStream.Protocol())).Logger() } + for _, stream := range conn.GetStreams() { + if err := stream.Close(); err != nil { + lg.Warn().Err(err).Msg("failed to close stream, when pruning connections") + } + } // close the connection with the peer if it is not part of the current fanout err := l.host.ClosePeer(peerID) diff --git a/network/p2p/connection/connector_factory.go b/network/p2p/connection/connector_factory.go index a5c8be29704..c3ecfaeee9c 100644 --- a/network/p2p/connection/connector_factory.go +++ b/network/p2p/connection/connector_factory.go @@ -11,7 +11,10 @@ import ( const ( // minBackoff is the minimum backoff duration for the backoff connector. - minBackoff = time.Second * 10 + // We set it to 1 second as we want to let the LibP2PNode be in charge of connection establishment and can disconnect + // and reconnect to peers as soon as it needs. This is essential to ensure that the allow-listing and disallow-listing + // time intervals are working as expected. + minBackoff = 1 * time.Second // maxBackoff is the maximum backoff duration for the backoff connector. When the backoff duration reaches this value, // it will not increase any further. maxBackoff = time.Hour @@ -21,7 +24,7 @@ const ( timeUnit = time.Second // exponentialBackOffBase is the base for the exponential backoff. The backoff duration will be a multiple of the time unit // multiplied by the exponential base raised to the exponential offset, i.e., exponentialBase^(timeUnit*attempt). - exponentialBackOffBase = 5.0 + exponentialBackOffBase = 2.0 // exponentialBackOffOffset is the offset for the exponential backoff. It acts as a constant that is added result // of the exponential base raised to the exponential offset, i.e., exponentialBase^(timeUnit*attempt) + exponentialBackOffOffset. // This is used to ensure that the backoff duration is always greater than the time unit. We set this to 0 as we want the diff --git a/network/p2p/connection/connector_host.go b/network/p2p/connection/connector_host.go index 6af6ecc4777..04cfd56b28a 100644 --- a/network/p2p/connection/connector_host.go +++ b/network/p2p/connection/connector_host.go @@ -27,6 +27,18 @@ func (c *ConnectorHost) Connections() []network.Conn { return c.h.Network().Conns() } +// IsConnectedTo returns true if the given peer.ID is connected to the underlying host. +// Args: +// +// peerID: peer.ID for which the connection status is requested +// +// Returns: +// +// true if the given peer.ID is connected to the underlying host. +func (c *ConnectorHost) IsConnectedTo(peerID peer.ID) bool { + return c.h.Network().Connectedness(peerID) == network.Connected && len(c.h.Network().ConnsToPeer(peerID)) > 0 +} + // PeerInfo returns the peer.AddrInfo for the given peer.ID. // Args: // diff --git a/network/p2p/connection/peerManager.go b/network/p2p/connection/peerManager.go index d82c5b779b6..05cc7c47129 100644 --- a/network/p2p/connection/peerManager.go +++ b/network/p2p/connection/peerManager.go @@ -16,8 +16,9 @@ import ( "github.com/onflow/flow-go/utils/logging" ) -// DefaultPeerUpdateInterval is default duration for which the peer manager waits in between attempts to update peer connections -var DefaultPeerUpdateInterval = 10 * time.Minute +// DefaultPeerUpdateInterval is default duration for which the peer manager waits in between attempts to update peer connections. +// We set it to 1 second to be aligned with the heartbeat intervals of libp2p, alsp, and gossipsub. +var DefaultPeerUpdateInterval = time.Second var _ p2p.PeerManager = (*PeerManager)(nil) var _ component.Component = (*PeerManager)(nil) diff --git a/network/p2p/connectionGater.go b/network/p2p/connectionGater.go index d2732fbd713..212dea51102 100644 --- a/network/p2p/connectionGater.go +++ b/network/p2p/connectionGater.go @@ -1,23 +1,24 @@ package p2p -import ( - "github.com/libp2p/go-libp2p/core/control" - "github.com/libp2p/go-libp2p/core/network" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/multiformats/go-multiaddr" -) +import "github.com/libp2p/go-libp2p/core/connmgr" -// ConnectionGater is a copy of the libp2p ConnectionGater interface: -// https://github.com/libp2p/go-libp2p/blob/master/core/connmgr/gater.go#L54 -// We use it here to generate a mock for testing through testify mock. +// ConnectionGater the customized interface for the connection gater in the p2p package. +// It acts as a wrapper around the libp2p connmgr.ConnectionGater interface and adds some custom methods. type ConnectionGater interface { - InterceptPeerDial(p peer.ID) (allow bool) + connmgr.ConnectionGater - InterceptAddrDial(peer.ID, multiaddr.Multiaddr) (allow bool) - - InterceptAccept(network.ConnMultiaddrs) (allow bool) - - InterceptSecured(network.Direction, peer.ID, network.ConnMultiaddrs) (allow bool) - - InterceptUpgraded(network.Conn) (allow bool, reason control.DisconnectReason) + // SetDisallowListOracle sets the disallow list oracle for the connection gater. + // If one is set, the oracle is consulted upon every incoming or outgoing connection attempt, and + // the connection is only allowed if the remote peer is not on the disallow list. + // In Flow blockchain, it is not optional to dismiss the disallow list oracle, and if one is not set + // the connection gater will panic. + // Also, it follows a dependency injection pattern and does not allow to set the disallow list oracle more than once, + // any subsequent calls to this method will result in a panic. + // Args: + // oracle: the disallow list oracle to set. + // Returns: + // none + // Panics: + // if the disallow list oracle is already set. + SetDisallowListOracle(oracle DisallowListOracle) } diff --git a/network/p2p/connector.go b/network/p2p/connector.go index 2bbf9f24dea..5ba291d7063 100644 --- a/network/p2p/connector.go +++ b/network/p2p/connector.go @@ -31,26 +31,33 @@ type ConnectorHost interface { // Connections returns all the connections of the underlying host. Connections() []network.Conn + // IsConnectedTo returns true if the given peer.ID is connected to the underlying host. + // Args: + // peerID: peer.ID for which the connection status is requested + // Returns: + // true if the given peer.ID is connected to the underlying host. + IsConnectedTo(peerId peer.ID) bool + // PeerInfo returns the peer.AddrInfo for the given peer.ID. // Args: // id: peer.ID for which the peer.AddrInfo is requested // Returns: // peer.AddrInfo for the given peer.ID - PeerInfo(id peer.ID) peer.AddrInfo + PeerInfo(peerId peer.ID) peer.AddrInfo // IsProtected returns true if the given peer.ID is protected from pruning. // Args: // id: peer.ID for which the protection status is requested // Returns: // true if the given peer.ID is protected from pruning - IsProtected(id peer.ID) bool + IsProtected(peerId peer.ID) bool // ClosePeer closes the connection to the given peer.ID. // Args: // id: peer.ID for which the connection is to be closed // Returns: // error if there is any error while closing the connection to the given peer.ID. All errors are benign. - ClosePeer(id peer.ID) error + ClosePeer(peerId peer.ID) error // ID returns the peer.ID of the underlying host. // Returns: diff --git a/network/p2p/consumers.go b/network/p2p/consumers.go index afe7b9e6efd..ee12521aadb 100644 --- a/network/p2p/consumers.go +++ b/network/p2p/consumers.go @@ -4,22 +4,9 @@ import ( pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/peer" - "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module/component" ) -// DisallowListConsumer consumes notifications from the cache.NodeBlocklistWrapper whenever the block list is updated. -// Implementations must: -// - be concurrency safe -// - be non-blocking -type DisallowListConsumer interface { - // OnNodeDisallowListUpdate notifications whenever the node block list is updated. - // Prerequisites: - // Implementation must be concurrency safe; Non-blocking; - // and must handle repetition of the same events (with some processing overhead). - OnNodeDisallowListUpdate(list flow.IdentifierList) -} - // ControlMessageType is the type of control message, as defined in the libp2p pubsub spec. type ControlMessageType string @@ -39,31 +26,6 @@ func ControlMessageTypes() []ControlMessageType { return []ControlMessageType{CtrlMsgIHave, CtrlMsgIWant, CtrlMsgGraft, CtrlMsgPrune} } -// DisallowListUpdateNotification is the event that is submitted to the distributor when the disallow list is updated. -type DisallowListUpdateNotification struct { - DisallowList flow.IdentifierList -} - -type DisallowListNotificationConsumer interface { - // OnDisallowListNotification is called when a new disallow list update notification is distributed. - // Any error on consuming event must handle internally. - // The implementation must be concurrency safe, but can be blocking. - OnDisallowListNotification(*DisallowListUpdateNotification) -} - -type DisallowListNotificationDistributor interface { - component.Component - // DistributeBlockListNotification distributes the event to all the consumers. - // Any error returned by the distributor is non-recoverable and will cause the node to crash. - // Implementation must be concurrency safe, and non-blocking. - DistributeBlockListNotification(list flow.IdentifierList) error - - // AddConsumer adds a consumer to the distributor. The consumer will be called the distributor distributes a new event. - // AddConsumer must be concurrency safe. Once a consumer is added, it must be called for all future events. - // There is no guarantee that the consumer will be called for events that were already received by the distributor. - AddConsumer(DisallowListNotificationConsumer) -} - // GossipSubInspectorNotifDistributor is the interface for the distributor that distributes gossip sub inspector notifications. // It is used to distribute notifications to the consumers in an asynchronous manner and non-blocking manner. // The implementation should guarantee that all registered consumers are called upon distribution of a new event. diff --git a/network/p2p/disallowListCache.go b/network/p2p/disallowListCache.go new file mode 100644 index 00000000000..b153084b6cf --- /dev/null +++ b/network/p2p/disallowListCache.go @@ -0,0 +1,51 @@ +package p2p + +import ( + "github.com/libp2p/go-libp2p/core/peer" + + "github.com/onflow/flow-go/module" + "github.com/onflow/flow-go/network" +) + +// DisallowListCache is an interface for a cache that keeps the list of disallow-listed peers. +// It is designed to present a centralized interface for keeping track of disallow-listed peers for different reasons. +type DisallowListCache interface { + // IsDisallowListed determines whether the given peer is disallow-listed for any reason. + // Args: + // - peerID: the peer to check. + // Returns: + // - []network.DisallowListedCause: the list of causes for which the given peer is disallow-listed. If the peer is not disallow-listed for any reason, + // a nil slice is returned. + // - bool: true if the peer is disallow-listed for any reason, false otherwise. + IsDisallowListed(peerID peer.ID) ([]network.DisallowListedCause, bool) + + // DisallowFor disallow-lists a peer for a cause. + // Args: + // - peerID: the peerID of the peer to be disallow-listed. + // - cause: the cause for disallow-listing the peer. + // Returns: + // - the list of causes for which the peer is disallow-listed. + // - error if the operation fails, error is irrecoverable. + DisallowFor(peerID peer.ID, cause network.DisallowListedCause) ([]network.DisallowListedCause, error) + + // AllowFor removes a cause from the disallow list cache entity for the peerID. + // Args: + // - peerID: the peerID of the peer to be allow-listed. + // - cause: the cause for allow-listing the peer. + // Returns: + // - the list of causes for which the peer is disallow-listed. If the peer is not disallow-listed for any reason, + // an empty list is returned. + AllowFor(peerID peer.ID, cause network.DisallowListedCause) []network.DisallowListedCause +} + +// DisallowListCacheConfig is the configuration for the disallow-list cache. +// The disallow-list cache is used to temporarily disallow-list peers. +type DisallowListCacheConfig struct { + // MaxSize is the maximum number of peers that can be disallow-listed at any given time. + // When the cache is full, no further new peers can be disallow-listed. + // Recommended size is 100 * number of staked nodes. + MaxSize uint32 + + // Metrics is the HeroCache metrics collector to be used for the disallow-list cache. + Metrics module.HeroCacheMetrics +} diff --git a/network/p2p/distributor/disallow_list.go b/network/p2p/distributor/disallow_list.go deleted file mode 100644 index 848baa925bb..00000000000 --- a/network/p2p/distributor/disallow_list.go +++ /dev/null @@ -1,114 +0,0 @@ -package distributor - -import ( - "sync" - - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/common/worker" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/component" - "github.com/onflow/flow-go/module/mempool/queue" - "github.com/onflow/flow-go/module/metrics" - "github.com/onflow/flow-go/network/p2p" -) - -const ( - // DefaultDisallowListNotificationQueueCacheSize is the default size of the disallow list notification queue. - DefaultDisallowListNotificationQueueCacheSize = 100 -) - -// DisallowListNotificationDistributor is a component that distributes disallow list updates to registered consumers in an -// asynchronous, fan-out manner. It is thread-safe and can be used concurrently from multiple goroutines. -type DisallowListNotificationDistributor struct { - component.Component - cm *component.ComponentManager - logger zerolog.Logger - - consumerLock sync.RWMutex // protects the consumer field from concurrent updates - consumers []p2p.DisallowListNotificationConsumer - workerPool *worker.Pool[*p2p.DisallowListUpdateNotification] -} - -var _ p2p.DisallowListNotificationDistributor = (*DisallowListNotificationDistributor)(nil) - -// DefaultDisallowListNotificationDistributor creates a new disallow list notification distributor with default configuration. -func DefaultDisallowListNotificationDistributor(logger zerolog.Logger, opts ...queue.HeroStoreConfigOption) *DisallowListNotificationDistributor { - cfg := &queue.HeroStoreConfig{ - SizeLimit: DefaultDisallowListNotificationQueueCacheSize, - Collector: metrics.NewNoopCollector(), - } - - for _, opt := range opts { - opt(cfg) - } - - store := queue.NewHeroStore(cfg.SizeLimit, logger, cfg.Collector) - return NewDisallowListConsumer(logger, store) -} - -// NewDisallowListConsumer creates a new disallow list notification distributor. -// It takes a message store as a parameter, which is used to store the events that are distributed to the consumers. -// The message store is used to ensure that DistributeBlockListNotification is non-blocking. -func NewDisallowListConsumer(logger zerolog.Logger, store engine.MessageStore) *DisallowListNotificationDistributor { - lg := logger.With().Str("component", "node_disallow_distributor").Logger() - - d := &DisallowListNotificationDistributor{ - logger: lg, - } - - pool := worker.NewWorkerPoolBuilder[*p2p.DisallowListUpdateNotification]( - lg, - store, - d.distribute).Build() - - d.workerPool = pool - - cm := component.NewComponentManagerBuilder() - cm.AddWorker(d.workerPool.WorkerLogic()) - - d.cm = cm.Build() - d.Component = d.cm - - return d -} - -// distribute is called by the workers to process the event. It calls the OnDisallowListNotification method on all registered -// consumers. -// It does not return an error because the event is already in the store, so it will be retried. -func (d *DisallowListNotificationDistributor) distribute(notification *p2p.DisallowListUpdateNotification) error { - d.consumerLock.RLock() - defer d.consumerLock.RUnlock() - - for _, consumer := range d.consumers { - consumer.OnDisallowListNotification(notification) - } - - return nil -} - -// AddConsumer adds a consumer to the distributor. The consumer will be called the distributor distributes a new event. -// AddConsumer must be concurrency safe. Once a consumer is added, it must be called for all future events. -// There is no guarantee that the consumer will be called for events that were already received by the distributor. -func (d *DisallowListNotificationDistributor) AddConsumer(consumer p2p.DisallowListNotificationConsumer) { - d.consumerLock.Lock() - defer d.consumerLock.Unlock() - - d.consumers = append(d.consumers, consumer) -} - -// DistributeBlockListNotification distributes the event to all the consumers. -// Implementation is non-blocking, it submits the event to the worker pool and returns immediately. -// The event will be distributed to the consumers in the order it was submitted but asynchronously. -// If the worker pool is full, the event will be dropped and a warning will be logged. -// This implementation returns no error. -func (d *DisallowListNotificationDistributor) DistributeBlockListNotification(disallowList flow.IdentifierList) error { - ok := d.workerPool.Submit(&p2p.DisallowListUpdateNotification{DisallowList: disallowList}) - if !ok { - // we use a queue to buffer the events, so this may happen if the queue is full or the event is duplicate. In this case, we log a warning. - d.logger.Warn().Msg("node disallow list update notification queue is full or the event is duplicate, dropping event") - } - - return nil -} diff --git a/network/p2p/distributor/disallow_list_test.go b/network/p2p/distributor/disallow_list_test.go deleted file mode 100644 index 39cf9532f46..00000000000 --- a/network/p2p/distributor/disallow_list_test.go +++ /dev/null @@ -1,100 +0,0 @@ -package distributor_test - -import ( - "context" - "math/rand" - "sync" - "testing" - "time" - - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/distributor" - mockp2p "github.com/onflow/flow-go/network/p2p/mock" - "github.com/onflow/flow-go/utils/unittest" -) - -// TestDisallowListNotificationDistributor tests the disallow list notification distributor by adding two consumers to the -// notification distributor component and sending a random set of notifications to the notification component. The test -// verifies that the consumers receive the notifications and that each consumer sees each notification only once. -func TestDisallowListNotificationDistributor(t *testing.T) { - d := distributor.DefaultDisallowListNotificationDistributor(unittest.Logger()) - - c1 := mockp2p.NewDisallowListNotificationConsumer(t) - c2 := mockp2p.NewDisallowListNotificationConsumer(t) - - d.AddConsumer(c1) - d.AddConsumer(c2) - - tt := disallowListUpdateNotificationsFixture(50) - - c1Done := sync.WaitGroup{} - c1Done.Add(len(tt)) - c1Seen := unittest.NewProtectedMap[flow.Identifier, struct{}]() - c1.On("OnDisallowListNotification", mock.Anything).Run(func(args mock.Arguments) { - n, ok := args.Get(0).(*p2p.DisallowListUpdateNotification) - require.True(t, ok) - - require.Contains(t, tt, n) - - // ensure consumer see each peer once - hash := flow.MerkleRoot(n.DisallowList...) - require.False(t, c1Seen.Has(hash)) - c1Seen.Add(hash, struct{}{}) - - c1Done.Done() - }).Return() - - c2Done := sync.WaitGroup{} - c2Done.Add(len(tt)) - c2Seen := unittest.NewProtectedMap[flow.Identifier, struct{}]() - c2.On("OnDisallowListNotification", mock.Anything, mock.Anything, mock.Anything).Run(func(args mock.Arguments) { - n, ok := args.Get(0).(*p2p.DisallowListUpdateNotification) - require.True(t, ok) - - require.Contains(t, tt, n) - - // ensure consumer see each peer once - hash := flow.MerkleRoot(n.DisallowList...) - require.False(t, c2Seen.Has(hash)) - c2Seen.Add(hash, struct{}{}) - - c2Done.Done() - }).Return() - - cancelCtx, cancel := context.WithCancel(context.Background()) - defer cancel() - ctx, _ := irrecoverable.WithSignaler(cancelCtx) - d.Start(ctx) - - unittest.RequireCloseBefore(t, d.Ready(), 100*time.Millisecond, "could not start distributor") - - for i := 0; i < len(tt); i++ { - go func(i int) { - require.NoError(t, d.DistributeBlockListNotification(tt[i].DisallowList)) - }(i) - } - - unittest.RequireReturnsBefore(t, c1Done.Wait, 1*time.Second, "events are not received by consumer 1") - unittest.RequireReturnsBefore(t, c2Done.Wait, 1*time.Second, "events are not received by consumer 2") - cancel() - unittest.RequireCloseBefore(t, d.Done(), 100*time.Millisecond, "could not stop distributor") -} - -func disallowListUpdateNotificationsFixture(n int) []*p2p.DisallowListUpdateNotification { - tt := make([]*p2p.DisallowListUpdateNotification, n) - for i := 0; i < n; i++ { - tt[i] = disallowListUpdateNotificationFixture() - } - return tt -} - -func disallowListUpdateNotificationFixture() *p2p.DisallowListUpdateNotification { - return &p2p.DisallowListUpdateNotification{ - DisallowList: unittest.IdentifierListFixture(rand.Int()%100 + 1), - } -} diff --git a/network/p2p/libp2pNode.go b/network/p2p/libp2pNode.go index 1a7a87bd03d..a5a92d5cc70 100644 --- a/network/p2p/libp2pNode.go +++ b/network/p2p/libp2pNode.go @@ -13,6 +13,7 @@ import ( "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/component" "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/p2p/unicast/protocols" ) @@ -31,6 +32,11 @@ type LibP2PNode interface { PeerConnections // PeerScore exposes the peer score API. PeerScore + // DisallowListNotificationConsumer exposes the disallow list notification consumer API for the node so that + // it will be notified when a new disallow list update is distributed. + DisallowListNotificationConsumer + // DisallowListOracle exposes the disallow list oracle API for external consumers to query about the disallow list. + DisallowListOracle // Start the libp2p node. Start(ctx irrecoverable.SignalerContext) // Stop terminates the libp2p node. @@ -109,3 +115,38 @@ type PeerConnections interface { // to the peer is not empty. This indicates a bug within libp2p. IsConnected(peerID peer.ID) (bool, error) } + +// DisallowListNotificationConsumer is an interface for consuming disallow/allow list update notifications. +type DisallowListNotificationConsumer interface { + // OnDisallowListNotification is called when a new disallow list update notification is distributed. + // Any error on consuming event must handle internally. + // The implementation must be concurrency safe. + // Args: + // id: peer ID of the peer being disallow-listed. + // cause: cause of the peer being disallow-listed (only this cause is added to the peer's disallow-listed causes). + // Returns: + // none + OnDisallowListNotification(id peer.ID, cause network.DisallowListedCause) + + // OnAllowListNotification is called when a new allow list update notification is distributed. + // Any error on consuming event must handle internally. + // The implementation must be concurrency safe. + // Args: + // id: peer ID of the peer being allow-listed. + // cause: cause of the peer being allow-listed (only this cause is removed from the peer's disallow-listed causes). + // Returns: + // none + OnAllowListNotification(id peer.ID, cause network.DisallowListedCause) +} + +// DisallowListOracle is an interface for querying disallow-listed peers. +type DisallowListOracle interface { + // IsDisallowListed determines whether the given peer is disallow-listed for any reason. + // Args: + // - peerID: the peer to check. + // Returns: + // - []network.DisallowListedCause: the list of causes for which the given peer is disallow-listed. If the peer is not disallow-listed for any reason, + // a nil slice is returned. + // - bool: true if the peer is disallow-listed for any reason, false otherwise. + IsDisallowListed(peerId peer.ID) ([]network.DisallowListedCause, bool) +} diff --git a/network/p2p/middleware/middleware.go b/network/p2p/middleware/middleware.go index 58e15638943..ac5f349264a 100644 --- a/network/p2p/middleware/middleware.go +++ b/network/p2p/middleware/middleware.go @@ -62,11 +62,17 @@ const ( // LargeMsgUnicastTimeout is the maximum time to wait for a unicast request to complete for large message size LargeMsgUnicastTimeout = 1000 * time.Second + + // DisallowListCacheSize is the maximum number of peers that can be disallow-listed at a time. The recommended + // size is 100 * number of staked nodes. Note that when the cache is full, there is no eviction policy and + // disallow-listing a new peer will fail. Hence, the cache size should be set to a value that is large enough + // to accommodate all the peers that can be disallow-listed at a time. Also, note that this cache is only taking + // the staked (authorized) peers. Hence, Sybil attacks are not possible. + DisallowListCacheSize = 100 * 1000 ) var ( - _ network.Middleware = (*Middleware)(nil) - _ p2p.DisallowListNotificationConsumer = (*Middleware)(nil) + _ network.Middleware = (*Middleware)(nil) // ErrUnicastMsgWithoutSub error is provided to the slashing violations consumer in the case where // the middleware receives a message via unicast but does not have a corresponding subscription for @@ -78,9 +84,11 @@ var ( // our neighbours on the peer-to-peer network. type Middleware struct { sync.Mutex + component.Component ctx context.Context log zerolog.Logger ov network.Overlay + // TODO: using a waitgroup here doesn't actually guarantee that we'll wait for all // goroutines to exit, because new goroutines could be started after we've already // returned from wg.Wait(). We need to solve this the right way using ComponentManager @@ -88,7 +96,6 @@ type Middleware struct { wg sync.WaitGroup libP2PNode p2p.LibP2PNode preferredUnicasts []protocols.ProtocolName - me flow.Identifier bitswapMetrics module.BitswapMetrics rootBlockID flow.Identifier validators []network.MessageValidator @@ -100,18 +107,17 @@ type Middleware struct { slashingViolationsConsumer slashing.ViolationsConsumer unicastRateLimiters *ratelimit.RateLimiters authorizedSenderValidator *validator.AuthorizedSenderValidator - component.Component } -type MiddlewareOption func(*Middleware) +type OptionFn func(*Middleware) -func WithMessageValidators(validators ...network.MessageValidator) MiddlewareOption { +func WithMessageValidators(validators ...network.MessageValidator) OptionFn { return func(mw *Middleware) { mw.validators = validators } } -func WithPreferredUnicastProtocols(unicasts []protocols.ProtocolName) MiddlewareOption { +func WithPreferredUnicastProtocols(unicasts []protocols.ProtocolName) OptionFn { return func(mw *Middleware) { mw.preferredUnicasts = unicasts } @@ -119,19 +125,39 @@ func WithPreferredUnicastProtocols(unicasts []protocols.ProtocolName) Middleware // WithPeerManagerFilters sets a list of p2p.PeerFilter funcs that are used to // filter out peers provided by the peer manager PeersProvider. -func WithPeerManagerFilters(peerManagerFilters []p2p.PeerFilter) MiddlewareOption { +func WithPeerManagerFilters(peerManagerFilters []p2p.PeerFilter) OptionFn { return func(mw *Middleware) { mw.peerManagerFilters = peerManagerFilters } } // WithUnicastRateLimiters sets the unicast rate limiters. -func WithUnicastRateLimiters(rateLimiters *ratelimit.RateLimiters) MiddlewareOption { +func WithUnicastRateLimiters(rateLimiters *ratelimit.RateLimiters) OptionFn { return func(mw *Middleware) { mw.unicastRateLimiters = rateLimiters } } +// Config is the configuration for the middleware. +type Config struct { + Logger zerolog.Logger + Libp2pNode p2p.LibP2PNode + FlowId flow.Identifier // This node's Flow ID + BitSwapMetrics module.BitswapMetrics + RootBlockID flow.Identifier + UnicastMessageTimeout time.Duration + IdTranslator p2p.IDTranslator + Codec network.Codec + SlashingViolationsConsumer slashing.ViolationsConsumer +} + +// Validate validates the configuration, and sets default values for any missing fields. +func (cfg *Config) Validate() { + if cfg.UnicastMessageTimeout <= 0 { + cfg.UnicastMessageTimeout = DefaultUnicastTimeout + } +} + // NewMiddleware creates a new middleware instance // libP2PNodeFactory is the factory used to create a LibP2PNode // flowID is this node's Flow ID @@ -141,34 +167,20 @@ func WithUnicastRateLimiters(rateLimiters *ratelimit.RateLimiters) MiddlewareOpt // validators are the set of the different message validators that each inbound messages is passed through // During normal operations any error returned by Middleware.start is considered to be catastrophic // and will be thrown by the irrecoverable.SignalerContext causing the node to crash. -func NewMiddleware( - log zerolog.Logger, - libP2PNode p2p.LibP2PNode, - flowID flow.Identifier, - bitswapMet module.BitswapMetrics, - rootBlockID flow.Identifier, - unicastMessageTimeout time.Duration, - idTranslator p2p.IDTranslator, - codec network.Codec, - slashingViolationsConsumer slashing.ViolationsConsumer, - opts ...MiddlewareOption) *Middleware { - - if unicastMessageTimeout <= 0 { - unicastMessageTimeout = DefaultUnicastTimeout - } +func NewMiddleware(cfg *Config, opts ...OptionFn) *Middleware { + cfg.Validate() // create the node entity and inject dependencies & config mw := &Middleware{ - log: log, - me: flowID, - libP2PNode: libP2PNode, - bitswapMetrics: bitswapMet, - rootBlockID: rootBlockID, - validators: DefaultValidators(log, flowID), - unicastMessageTimeout: unicastMessageTimeout, - idTranslator: idTranslator, - codec: codec, - slashingViolationsConsumer: slashingViolationsConsumer, + log: cfg.Logger, + libP2PNode: cfg.Libp2pNode, + bitswapMetrics: cfg.BitSwapMetrics, + rootBlockID: cfg.RootBlockID, + validators: DefaultValidators(cfg.Logger, cfg.FlowId), + unicastMessageTimeout: cfg.UnicastMessageTimeout, + idTranslator: cfg.IdTranslator, + codec: cfg.Codec, + slashingViolationsConsumer: cfg.SlashingViolationsConsumer, unicastRateLimiters: ratelimit.NoopRateLimiters(), } @@ -188,13 +200,24 @@ func NewMiddleware( }) } builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - // TODO: refactor to avoid storing ctx altogether mw.ctx = ctx + if mw.ov == nil { + ctx.Throw(fmt.Errorf("overlay has not been set")) + } - if err := mw.start(ctx); err != nil { - ctx.Throw(err) + mw.authorizedSenderValidator = validator.NewAuthorizedSenderValidator( + mw.log, + mw.slashingViolationsConsumer, + mw.ov.Identity) + + err := mw.libP2PNode.WithDefaultUnicastProtocol(mw.handleIncomingStream, mw.preferredUnicasts) + if err != nil { + ctx.Throw(fmt.Errorf("could not register preferred unicast protocols on libp2p node: %w", err)) } + mw.UpdateNodeAddresses() + mw.libP2PNode.WithPeersProvider(mw.authorizedPeers) + ready() <-ctx.Done() @@ -202,7 +225,6 @@ func NewMiddleware( // wait for the readConnection and readSubscription routines to stop mw.wg.Wait() - mw.log.Info().Str("component", "middleware").Msg("stopped subroutines") }) @@ -253,22 +275,6 @@ func (m *Middleware) peerIDs(flowIDs flow.IdentifierList) peer.IDSlice { return result } -// Me returns the flow identifier of this middleware -func (m *Middleware) Me() flow.Identifier { - return m.me -} - -// GetIPPort returns the ip address and port number associated with the middleware -// All errors returned from this function can be considered benign. -func (m *Middleware) GetIPPort() (string, string, error) { - ipOrHostname, port, err := m.libP2PNode.GetIPPort() - if err != nil { - return "", "", fmt.Errorf("failed to get ip and port from libP2P node: %w", err) - } - - return ipOrHostname, port, nil -} - func (m *Middleware) UpdateNodeAddresses() { m.log.Info().Msg("Updating protocol state node addresses") @@ -298,33 +304,18 @@ func (m *Middleware) SetOverlay(ov network.Overlay) { m.ov = ov } -// start will start the middleware. -// No errors are expected during normal operation. -func (m *Middleware) start(ctx context.Context) error { - if m.ov == nil { - return fmt.Errorf("could not start middleware: overlay must be configured by calling SetOverlay before middleware can be started") - } - - m.authorizedSenderValidator = validator.NewAuthorizedSenderValidator(m.log, m.slashingViolationsConsumer, m.ov.Identity) - - err := m.libP2PNode.WithDefaultUnicastProtocol(m.handleIncomingStream, m.preferredUnicasts) - if err != nil { - return fmt.Errorf("could not register preferred unicast protocols on libp2p node: %w", err) - } - - m.UpdateNodeAddresses() - - m.libP2PNode.WithPeersProvider(m.topologyPeers) - - return nil -} - -// topologyPeers callback used by the peer manager to get the list of peer ID's -// which this node should be directly connected to as peers. The peer ID list -// returned will be filtered through any configured m.peerManagerFilters. If the -// underlying libp2p node has a peer manager configured this func will be used as the -// peers provider. -func (m *Middleware) topologyPeers() peer.IDSlice { +// authorizedPeers is a peer manager callback used by the underlying libp2p node that updates who can connect to this node (as +// well as who this node can connect to). +// and who is not allowed to connect to this node. This function is called by the peer manager and connection gater components +// of libp2p. +// +// Args: +// none +// Returns: +// - peer.IDSlice: a list of peer IDs that are allowed to connect to this node (and that this node can connect to). Any peer +// not in this list is assumed to be disconnected from this node (if connected) and not allowed to connect to this node. +// This is the guarantee that the underlying libp2p node implementation makes. +func (m *Middleware) authorizedPeers() peer.IDSlice { peerIDs := make([]peer.ID, 0) for _, id := range m.peerIDs(m.ov.Topology().NodeIDs()) { peerAllowed := true @@ -348,14 +339,15 @@ func (m *Middleware) topologyPeers() peer.IDSlice { return peerIDs } -// OnDisallowListNotification is called when a new disallow list update notification is distributed. -// It disconnects from all peers in the disallow list. -func (m *Middleware) OnDisallowListNotification(notification *p2p.DisallowListUpdateNotification) { - for _, pid := range m.peerIDs(notification.DisallowList) { - err := m.libP2PNode.RemovePeer(pid) - if err != nil { - m.log.Error().Err(err).Str("peer_id", pid.String()).Msg("failed to disconnect from blocklisted peer") - } +func (m *Middleware) OnDisallowListNotification(notification *network.DisallowListingUpdate) { + for _, pid := range m.peerIDs(notification.FlowIds) { + m.libP2PNode.OnDisallowListNotification(pid, notification.Cause) + } +} + +func (m *Middleware) OnAllowListNotification(notification *network.AllowListingUpdate) { + for _, pid := range m.peerIDs(notification.FlowIds) { + m.libP2PNode.OnAllowListNotification(pid, notification.Cause) } } diff --git a/network/p2p/mock/connection_gater.go b/network/p2p/mock/connection_gater.go index d5943e8efa9..c316e3f31d0 100644 --- a/network/p2p/mock/connection_gater.go +++ b/network/p2p/mock/connection_gater.go @@ -10,6 +10,8 @@ import ( network "github.com/libp2p/go-libp2p/core/network" + p2p "github.com/onflow/flow-go/network/p2p" + peer "github.com/libp2p/go-libp2p/core/peer" ) @@ -98,6 +100,11 @@ func (_m *ConnectionGater) InterceptUpgraded(_a0 network.Conn) (bool, control.Di return r0, r1 } +// SetDisallowListOracle provides a mock function with given fields: oracle +func (_m *ConnectionGater) SetDisallowListOracle(oracle p2p.DisallowListOracle) { + _m.Called(oracle) +} + type mockConstructorTestingTNewConnectionGater interface { mock.TestingT Cleanup(func()) diff --git a/network/p2p/mock/connector_host.go b/network/p2p/mock/connector_host.go index 549c013db28..5cb468884e8 100644 --- a/network/p2p/mock/connector_host.go +++ b/network/p2p/mock/connector_host.go @@ -14,13 +14,13 @@ type ConnectorHost struct { mock.Mock } -// ClosePeer provides a mock function with given fields: id -func (_m *ConnectorHost) ClosePeer(id peer.ID) error { - ret := _m.Called(id) +// ClosePeer provides a mock function with given fields: peerId +func (_m *ConnectorHost) ClosePeer(peerId peer.ID) error { + ret := _m.Called(peerId) var r0 error if rf, ok := ret.Get(0).(func(peer.ID) error); ok { - r0 = rf(id) + r0 = rf(peerId) } else { r0 = ret.Error(0) } @@ -58,13 +58,13 @@ func (_m *ConnectorHost) ID() peer.ID { return r0 } -// IsProtected provides a mock function with given fields: id -func (_m *ConnectorHost) IsProtected(id peer.ID) bool { - ret := _m.Called(id) +// IsConnectedTo provides a mock function with given fields: peerId +func (_m *ConnectorHost) IsConnectedTo(peerId peer.ID) bool { + ret := _m.Called(peerId) var r0 bool if rf, ok := ret.Get(0).(func(peer.ID) bool); ok { - r0 = rf(id) + r0 = rf(peerId) } else { r0 = ret.Get(0).(bool) } @@ -72,13 +72,27 @@ func (_m *ConnectorHost) IsProtected(id peer.ID) bool { return r0 } -// PeerInfo provides a mock function with given fields: id -func (_m *ConnectorHost) PeerInfo(id peer.ID) peer.AddrInfo { - ret := _m.Called(id) +// IsProtected provides a mock function with given fields: peerId +func (_m *ConnectorHost) IsProtected(peerId peer.ID) bool { + ret := _m.Called(peerId) + + var r0 bool + if rf, ok := ret.Get(0).(func(peer.ID) bool); ok { + r0 = rf(peerId) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// PeerInfo provides a mock function with given fields: peerId +func (_m *ConnectorHost) PeerInfo(peerId peer.ID) peer.AddrInfo { + ret := _m.Called(peerId) var r0 peer.AddrInfo if rf, ok := ret.Get(0).(func(peer.ID) peer.AddrInfo); ok { - r0 = rf(id) + r0 = rf(peerId) } else { r0 = ret.Get(0).(peer.AddrInfo) } diff --git a/network/p2p/mock/create_node_func.go b/network/p2p/mock/create_node_func.go index 3169c71cb1e..1a57772cbeb 100644 --- a/network/p2p/mock/create_node_func.go +++ b/network/p2p/mock/create_node_func.go @@ -16,13 +16,13 @@ type CreateNodeFunc struct { mock.Mock } -// Execute provides a mock function with given fields: _a0, _a1, _a2, _a3 -func (_m *CreateNodeFunc) Execute(_a0 zerolog.Logger, _a1 host.Host, _a2 p2p.ProtocolPeerCache, _a3 p2p.PeerManager) p2p.LibP2PNode { - ret := _m.Called(_a0, _a1, _a2, _a3) +// 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, host.Host, p2p.ProtocolPeerCache, p2p.PeerManager) p2p.LibP2PNode); ok { - r0 = rf(_a0, _a1, _a2, _a3) + 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/disallow_list_cache.go b/network/p2p/mock/disallow_list_cache.go new file mode 100644 index 00000000000..54d7fcf0d3c --- /dev/null +++ b/network/p2p/mock/disallow_list_cache.go @@ -0,0 +1,98 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mockp2p + +import ( + network "github.com/onflow/flow-go/network" + mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p/core/peer" +) + +// DisallowListCache is an autogenerated mock type for the DisallowListCache type +type DisallowListCache struct { + mock.Mock +} + +// AllowFor provides a mock function with given fields: peerID, cause +func (_m *DisallowListCache) AllowFor(peerID peer.ID, cause network.DisallowListedCause) []network.DisallowListedCause { + ret := _m.Called(peerID, cause) + + var r0 []network.DisallowListedCause + if rf, ok := ret.Get(0).(func(peer.ID, network.DisallowListedCause) []network.DisallowListedCause); ok { + r0 = rf(peerID, cause) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]network.DisallowListedCause) + } + } + + return r0 +} + +// DisallowFor provides a mock function with given fields: peerID, cause +func (_m *DisallowListCache) DisallowFor(peerID peer.ID, cause network.DisallowListedCause) ([]network.DisallowListedCause, error) { + ret := _m.Called(peerID, cause) + + var r0 []network.DisallowListedCause + var r1 error + if rf, ok := ret.Get(0).(func(peer.ID, network.DisallowListedCause) ([]network.DisallowListedCause, error)); ok { + return rf(peerID, cause) + } + if rf, ok := ret.Get(0).(func(peer.ID, network.DisallowListedCause) []network.DisallowListedCause); ok { + r0 = rf(peerID, cause) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]network.DisallowListedCause) + } + } + + if rf, ok := ret.Get(1).(func(peer.ID, network.DisallowListedCause) error); ok { + r1 = rf(peerID, cause) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// IsDisallowListed provides a mock function with given fields: peerID +func (_m *DisallowListCache) IsDisallowListed(peerID peer.ID) ([]network.DisallowListedCause, bool) { + ret := _m.Called(peerID) + + var r0 []network.DisallowListedCause + var r1 bool + if rf, ok := ret.Get(0).(func(peer.ID) ([]network.DisallowListedCause, bool)); ok { + return rf(peerID) + } + if rf, ok := ret.Get(0).(func(peer.ID) []network.DisallowListedCause); ok { + r0 = rf(peerID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]network.DisallowListedCause) + } + } + + if rf, ok := ret.Get(1).(func(peer.ID) bool); ok { + r1 = rf(peerID) + } else { + r1 = ret.Get(1).(bool) + } + + return r0, r1 +} + +type mockConstructorTestingTNewDisallowListCache interface { + mock.TestingT + Cleanup(func()) +} + +// NewDisallowListCache creates a new instance of DisallowListCache. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewDisallowListCache(t mockConstructorTestingTNewDisallowListCache) *DisallowListCache { + mock := &DisallowListCache{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/network/p2p/mock/disallow_list_consumer.go b/network/p2p/mock/disallow_list_consumer.go deleted file mode 100644 index 2800a5aa909..00000000000 --- a/network/p2p/mock/disallow_list_consumer.go +++ /dev/null @@ -1,33 +0,0 @@ -// Code generated by mockery v2.21.4. DO NOT EDIT. - -package mockp2p - -import ( - flow "github.com/onflow/flow-go/model/flow" - mock "github.com/stretchr/testify/mock" -) - -// DisallowListConsumer is an autogenerated mock type for the DisallowListConsumer type -type DisallowListConsumer struct { - mock.Mock -} - -// OnNodeDisallowListUpdate provides a mock function with given fields: list -func (_m *DisallowListConsumer) OnNodeDisallowListUpdate(list flow.IdentifierList) { - _m.Called(list) -} - -type mockConstructorTestingTNewDisallowListConsumer interface { - mock.TestingT - Cleanup(func()) -} - -// NewDisallowListConsumer creates a new instance of DisallowListConsumer. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewDisallowListConsumer(t mockConstructorTestingTNewDisallowListConsumer) *DisallowListConsumer { - mock := &DisallowListConsumer{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/network/p2p/mock/disallow_list_notification_consumer.go b/network/p2p/mock/disallow_list_notification_consumer.go index 7df8437ddcf..0d30cddea03 100644 --- a/network/p2p/mock/disallow_list_notification_consumer.go +++ b/network/p2p/mock/disallow_list_notification_consumer.go @@ -3,8 +3,10 @@ package mockp2p import ( - p2p "github.com/onflow/flow-go/network/p2p" + network "github.com/onflow/flow-go/network" mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p/core/peer" ) // DisallowListNotificationConsumer is an autogenerated mock type for the DisallowListNotificationConsumer type @@ -12,9 +14,14 @@ type DisallowListNotificationConsumer struct { mock.Mock } -// OnDisallowListNotification provides a mock function with given fields: _a0 -func (_m *DisallowListNotificationConsumer) OnDisallowListNotification(_a0 *p2p.DisallowListUpdateNotification) { - _m.Called(_a0) +// OnAllowListNotification provides a mock function with given fields: id, cause +func (_m *DisallowListNotificationConsumer) OnAllowListNotification(id peer.ID, cause network.DisallowListedCause) { + _m.Called(id, cause) +} + +// OnDisallowListNotification provides a mock function with given fields: id, cause +func (_m *DisallowListNotificationConsumer) OnDisallowListNotification(id peer.ID, cause network.DisallowListedCause) { + _m.Called(id, cause) } type mockConstructorTestingTNewDisallowListNotificationConsumer interface { diff --git a/network/p2p/mock/disallow_list_notification_distributor.go b/network/p2p/mock/disallow_list_notification_distributor.go deleted file mode 100644 index 82419cb87e1..00000000000 --- a/network/p2p/mock/disallow_list_notification_distributor.go +++ /dev/null @@ -1,88 +0,0 @@ -// Code generated by mockery v2.21.4. DO NOT EDIT. - -package mockp2p - -import ( - flow "github.com/onflow/flow-go/model/flow" - irrecoverable "github.com/onflow/flow-go/module/irrecoverable" - - mock "github.com/stretchr/testify/mock" - - p2p "github.com/onflow/flow-go/network/p2p" -) - -// DisallowListNotificationDistributor is an autogenerated mock type for the DisallowListNotificationDistributor type -type DisallowListNotificationDistributor struct { - mock.Mock -} - -// AddConsumer provides a mock function with given fields: _a0 -func (_m *DisallowListNotificationDistributor) AddConsumer(_a0 p2p.DisallowListNotificationConsumer) { - _m.Called(_a0) -} - -// DistributeBlockListNotification provides a mock function with given fields: list -func (_m *DisallowListNotificationDistributor) DistributeBlockListNotification(list flow.IdentifierList) error { - ret := _m.Called(list) - - var r0 error - if rf, ok := ret.Get(0).(func(flow.IdentifierList) error); ok { - r0 = rf(list) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Done provides a mock function with given fields: -func (_m *DisallowListNotificationDistributor) Done() <-chan struct{} { - ret := _m.Called() - - var r0 <-chan struct{} - if rf, ok := ret.Get(0).(func() <-chan struct{}); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(<-chan struct{}) - } - } - - return r0 -} - -// Ready provides a mock function with given fields: -func (_m *DisallowListNotificationDistributor) Ready() <-chan struct{} { - ret := _m.Called() - - var r0 <-chan struct{} - if rf, ok := ret.Get(0).(func() <-chan struct{}); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(<-chan struct{}) - } - } - - return r0 -} - -// Start provides a mock function with given fields: _a0 -func (_m *DisallowListNotificationDistributor) Start(_a0 irrecoverable.SignalerContext) { - _m.Called(_a0) -} - -type mockConstructorTestingTNewDisallowListNotificationDistributor interface { - mock.TestingT - Cleanup(func()) -} - -// NewDisallowListNotificationDistributor creates a new instance of DisallowListNotificationDistributor. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewDisallowListNotificationDistributor(t mockConstructorTestingTNewDisallowListNotificationDistributor) *DisallowListNotificationDistributor { - mock := &DisallowListNotificationDistributor{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/network/p2p/mock/disallow_list_oracle.go b/network/p2p/mock/disallow_list_oracle.go new file mode 100644 index 00000000000..8779bce7186 --- /dev/null +++ b/network/p2p/mock/disallow_list_oracle.go @@ -0,0 +1,56 @@ +// Code generated by mockery v2.21.4. DO NOT EDIT. + +package mockp2p + +import ( + network "github.com/onflow/flow-go/network" + mock "github.com/stretchr/testify/mock" + + peer "github.com/libp2p/go-libp2p/core/peer" +) + +// DisallowListOracle is an autogenerated mock type for the DisallowListOracle type +type DisallowListOracle struct { + mock.Mock +} + +// IsDisallowListed provides a mock function with given fields: peerId +func (_m *DisallowListOracle) IsDisallowListed(peerId peer.ID) ([]network.DisallowListedCause, bool) { + ret := _m.Called(peerId) + + var r0 []network.DisallowListedCause + var r1 bool + if rf, ok := ret.Get(0).(func(peer.ID) ([]network.DisallowListedCause, bool)); ok { + return rf(peerId) + } + if rf, ok := ret.Get(0).(func(peer.ID) []network.DisallowListedCause); ok { + r0 = rf(peerId) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]network.DisallowListedCause) + } + } + + if rf, ok := ret.Get(1).(func(peer.ID) bool); ok { + r1 = rf(peerId) + } else { + r1 = ret.Get(1).(bool) + } + + return r0, r1 +} + +type mockConstructorTestingTNewDisallowListOracle interface { + mock.TestingT + Cleanup(func()) +} + +// NewDisallowListOracle creates a new instance of DisallowListOracle. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewDisallowListOracle(t mockConstructorTestingTNewDisallowListOracle) *DisallowListOracle { + mock := &DisallowListOracle{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/network/p2p/mock/lib_p2_p_node.go b/network/p2p/mock/lib_p2_p_node.go index 326b2280eca..6665b1a9cb2 100644 --- a/network/p2p/mock/lib_p2_p_node.go +++ b/network/p2p/mock/lib_p2_p_node.go @@ -8,6 +8,8 @@ import ( context "context" + flow_gonetwork "github.com/onflow/flow-go/network" + host "github.com/libp2p/go-libp2p/core/host" irrecoverable "github.com/onflow/flow-go/module/irrecoverable" @@ -191,6 +193,32 @@ func (_m *LibP2PNode) IsConnected(peerID peer.ID) (bool, error) { return r0, r1 } +// IsDisallowListed provides a mock function with given fields: peerId +func (_m *LibP2PNode) IsDisallowListed(peerId peer.ID) ([]flow_gonetwork.DisallowListedCause, bool) { + ret := _m.Called(peerId) + + var r0 []flow_gonetwork.DisallowListedCause + var r1 bool + if rf, ok := ret.Get(0).(func(peer.ID) ([]flow_gonetwork.DisallowListedCause, bool)); ok { + return rf(peerId) + } + if rf, ok := ret.Get(0).(func(peer.ID) []flow_gonetwork.DisallowListedCause); ok { + r0 = rf(peerId) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]flow_gonetwork.DisallowListedCause) + } + } + + if rf, ok := ret.Get(1).(func(peer.ID) bool); ok { + r1 = rf(peerId) + } else { + r1 = ret.Get(1).(bool) + } + + return r0, r1 +} + // ListPeers provides a mock function with given fields: topic func (_m *LibP2PNode) ListPeers(topic string) []peer.ID { ret := _m.Called(topic) @@ -207,6 +235,16 @@ func (_m *LibP2PNode) ListPeers(topic string) []peer.ID { return r0 } +// OnAllowListNotification provides a mock function with given fields: id, cause +func (_m *LibP2PNode) OnAllowListNotification(id peer.ID, cause flow_gonetwork.DisallowListedCause) { + _m.Called(id, cause) +} + +// OnDisallowListNotification provides a mock function with given fields: id, cause +func (_m *LibP2PNode) OnDisallowListNotification(id peer.ID, cause flow_gonetwork.DisallowListedCause) { + _m.Called(id, cause) +} + // PeerManagerComponent provides a mock function with given fields: func (_m *LibP2PNode) PeerManagerComponent() component.Component { ret := _m.Called() diff --git a/network/p2p/mock/node_builder.go b/network/p2p/mock/node_builder.go index 70184e2ecaf..a14e07363ae 100644 --- a/network/p2p/mock/node_builder.go +++ b/network/p2p/mock/node_builder.go @@ -90,11 +90,11 @@ func (_m *NodeBuilder) SetBasicResolver(_a0 madns.BasicResolver) p2p.NodeBuilder } // SetConnectionGater provides a mock function with given fields: _a0 -func (_m *NodeBuilder) SetConnectionGater(_a0 connmgr.ConnectionGater) p2p.NodeBuilder { +func (_m *NodeBuilder) SetConnectionGater(_a0 p2p.ConnectionGater) p2p.NodeBuilder { ret := _m.Called(_a0) var r0 p2p.NodeBuilder - if rf, ok := ret.Get(0).(func(connmgr.ConnectionGater) p2p.NodeBuilder); ok { + if rf, ok := ret.Get(0).(func(p2p.ConnectionGater) p2p.NodeBuilder); ok { r0 = rf(_a0) } else { if ret.Get(0) != nil { diff --git a/network/p2p/network.go b/network/p2p/network.go index dea89b92eda..54d1a0fc285 100644 --- a/network/p2p/network.go +++ b/network/p2p/network.go @@ -152,7 +152,7 @@ func NewNetwork(param *NetworkConfig, opts ...NetworkOption) (*Network, error) { if err != nil { return nil, fmt.Errorf("could not create middleware: %w", err) } - misbehaviorMngr, err := alspmgr.NewMisbehaviorReportManager(param.AlspCfg) + misbehaviorMngr, err := alspmgr.NewMisbehaviorReportManager(param.AlspCfg, mw) if err != nil { return nil, fmt.Errorf("could not create misbehavior report manager: %w", err) } diff --git a/network/p2p/p2pbuilder/libp2pNodeBuilder.go b/network/p2p/p2pbuilder/libp2pNodeBuilder.go index e388f35397e..3e3ef3364d4 100644 --- a/network/p2p/p2pbuilder/libp2pNodeBuilder.go +++ b/network/p2p/p2pbuilder/libp2pNodeBuilder.go @@ -128,7 +128,7 @@ type LibP2PNodeBuilder struct { resourceManager network.ResourceManager resourceManagerCfg *ResourceManagerConfig connManager connmgr.ConnManager - connGater connmgr.ConnectionGater + connGater p2p.ConnectionGater routingFactory func(context.Context, host.Host) (routing.Routing, error) peerManagerEnablePruning bool peerManagerUpdateInterval time.Duration @@ -136,6 +136,7 @@ type LibP2PNodeBuilder struct { createStreamRetryInterval time.Duration rateLimiterDistributor p2p.UnicastRateLimiterDistributor gossipSubTracer p2p.PubSubTracer + disallowListCacheCfg *p2p.DisallowListCacheConfig } func NewNodeBuilder(logger zerolog.Logger, @@ -143,16 +144,18 @@ func NewNodeBuilder(logger zerolog.Logger, addr string, networkKey fcrypto.PrivateKey, sporkID flow.Identifier, - rCfg *ResourceManagerConfig) *LibP2PNodeBuilder { + rCfg *ResourceManagerConfig, + disallowListCacheCfg *p2p.DisallowListCacheConfig) *LibP2PNodeBuilder { return &LibP2PNodeBuilder{ - logger: logger, - sporkID: sporkID, - addr: addr, - networkKey: networkKey, - createNode: DefaultCreateNodeFunc, - metrics: metrics, - resourceManagerCfg: rCfg, - gossipSubBuilder: gossipsubbuilder.NewGossipSubBuilder(logger, metrics), + logger: logger, + sporkID: sporkID, + addr: addr, + networkKey: networkKey, + createNode: DefaultCreateNodeFunc, + metrics: metrics, + resourceManagerCfg: rCfg, + gossipSubBuilder: gossipsubbuilder.NewGossipSubBuilder(logger, metrics), + disallowListCacheCfg: disallowListCacheCfg, } } @@ -181,7 +184,7 @@ func (builder *LibP2PNodeBuilder) SetConnectionManager(manager connmgr.ConnManag } // SetConnectionGater sets the connection gater for the node. -func (builder *LibP2PNodeBuilder) SetConnectionGater(gater connmgr.ConnectionGater) p2p.NodeBuilder { +func (builder *LibP2PNodeBuilder) SetConnectionGater(gater p2p.ConnectionGater) p2p.NodeBuilder { builder.connGater = gater return builder } @@ -368,7 +371,11 @@ func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { } } - node := builder.createNode(builder.logger, h, pCache, peerManager) + node := builder.createNode(builder.logger, h, pCache, peerManager, builder.disallowListCacheCfg) + + if builder.connGater != nil { + builder.connGater.SetDisallowListOracle(node) + } unicastManager := unicast.NewUnicastManager(builder.logger, stream.NewLibP2PStreamFactory(h), @@ -482,8 +489,9 @@ func defaultLibP2POptions(address string, key fcrypto.PrivateKey) ([]config.Opti func DefaultCreateNodeFunc(logger zerolog.Logger, host host.Host, pCache p2p.ProtocolPeerCache, - peerManager p2p.PeerManager) p2p.LibP2PNode { - return p2pnode.NewNode(logger, host, pCache, peerManager) + peerManager p2p.PeerManager, + disallowListCacheCfg *p2p.DisallowListCacheConfig) p2p.LibP2PNode { + return p2pnode.NewNode(logger, host, pCache, peerManager, disallowListCacheCfg) } // DefaultNodeBuilder returns a node builder. @@ -500,7 +508,8 @@ func DefaultNodeBuilder(log zerolog.Logger, gossipCfg *GossipSubConfig, rpcInspectorSuite p2p.GossipSubInspectorSuite, rCfg *ResourceManagerConfig, - uniCfg *p2pconfig.UnicastConfig) (p2p.NodeBuilder, error) { + uniCfg *p2pconfig.UnicastConfig, + disallowListCacheCfg *p2p.DisallowListCacheConfig) (p2p.NodeBuilder, error) { connManager, err := connection.NewConnManager(log, metricsCfg.Metrics, connection.DefaultConnManagerConfig()) if err != nil { @@ -516,7 +525,7 @@ func DefaultNodeBuilder(log zerolog.Logger, connection.WithOnInterceptPeerDialFilters(append(peerFilters, connGaterCfg.InterceptPeerDialFilters...)), connection.WithOnInterceptSecuredFilters(append(peerFilters, connGaterCfg.InterceptSecuredFilters...))) - builder := NewNodeBuilder(log, metricsCfg.Metrics, address, flowKey, sporkId, rCfg). + builder := NewNodeBuilder(log, metricsCfg.Metrics, address, flowKey, sporkId, rCfg, disallowListCacheCfg). SetBasicResolver(resolver). SetConnectionManager(connManager). SetConnectionGater(connGater). diff --git a/network/p2p/p2pnode/disallow_listing_test.go b/network/p2p/p2pnode/disallow_listing_test.go new file mode 100644 index 00000000000..0249c3ee91f --- /dev/null +++ b/network/p2p/p2pnode/disallow_listing_test.go @@ -0,0 +1,96 @@ +package p2pnode_test + +import ( + "context" + "testing" + "time" + + "github.com/libp2p/go-libp2p/core/peer" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module/irrecoverable" + mockmodule "github.com/onflow/flow-go/module/mock" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/internal/testutils" + "github.com/onflow/flow-go/network/p2p" + "github.com/onflow/flow-go/network/p2p/connection" + p2ptest "github.com/onflow/flow-go/network/p2p/test" + "github.com/onflow/flow-go/utils/unittest" +) + +// TestDisconnectingFromDisallowListedNode ensures that: +// (1) the node disconnects from a disallow listed node while the node is connected to other (allow listed) nodes. +// (2) new inbound or outbound connections to and from disallow-listed nodes are rejected. +// (3) When a disallow-listed node is allow-listed again, the node reconnects to it. +func TestDisconnectingFromDisallowListedNode(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + + sporkID := unittest.IdentifierFixture() + idProvider := mockmodule.NewIdentityProvider(t) + + peerIDSlice := peer.IDSlice{} + // node 1 is the node that will be disallow-listing another node (node 2). + node1, identity1 := p2ptest.NodeFixture(t, + sporkID, + t.Name(), + idProvider, + p2ptest.WithPeerManagerEnabled(true, connection.DefaultPeerUpdateInterval, + func() peer.IDSlice { + return peerIDSlice + }), + p2ptest.WithConnectionGater(testutils.NewConnectionGater(idProvider, func(p peer.ID) error { + // allow all the connections, except for the ones that are disallow-listed, which are determined when + // this connection gater object queries the disallow listing oracle that will be provided to it by + // the libp2p node. So, here, we don't need to do anything except just enabling the connection gater. + return nil + }))) + idProvider.On("ByPeerID", node1.Host().ID()).Return(&identity1, true).Maybe() + peerIDSlice = append(peerIDSlice, node1.Host().ID()) + + // node 2 is the node that will be disallow-listed by node 1. + node2, identity2 := p2ptest.NodeFixture(t, sporkID, t.Name(), idProvider) + idProvider.On("ByPeerID", node2.Host().ID()).Return(&identity2, true).Maybe() + peerIDSlice = append(peerIDSlice, node2.Host().ID()) + + // node 3 is the node that will be connected to node 1 (to ensure that node 1 is still able to connect to other nodes + // after disallow-listing node 2). + node3, identity3 := p2ptest.NodeFixture(t, sporkID, t.Name(), idProvider) + idProvider.On("ByPeerID", node3.Host().ID()).Return(&identity3, true).Maybe() + peerIDSlice = append(peerIDSlice, node3.Host().ID()) + + nodes := []p2p.LibP2PNode{node1, node2, node3} + ids := flow.IdentityList{&identity1, &identity2, &identity3} + + p2ptest.StartNodes(t, signalerCtx, nodes, 100*time.Millisecond) + defer p2ptest.StopNodes(t, nodes, cancel, 100*time.Millisecond) + + p2ptest.LetNodesDiscoverEachOther(t, ctx, nodes, ids) + + // initially all nodes should be connected to each other. + p2ptest.RequireConnectedEventually(t, nodes, 100*time.Millisecond, 2*time.Second) + + // phase-1: node 1 disallow-lists node 2. + node1.OnDisallowListNotification(node2.Host().ID(), network.DisallowListedCauseAlsp) + + // eventually node 1 should be disconnected from node 2 while other nodes should remain connected. + // we choose a timeout of 2 seconds because peer manager updates peers every 1 second. + p2ptest.RequireEventuallyNotConnected(t, []p2p.LibP2PNode{node1}, []p2p.LibP2PNode{node2}, 100*time.Millisecond, 2*time.Second) + + // but nodes 1 and 3 should remain connected as well as nodes 2 and 3. + // we choose a short timeout because we expect the nodes to remain connected. + p2ptest.RequireConnectedEventually(t, []p2p.LibP2PNode{node1, node3}, 1*time.Millisecond, 100*time.Millisecond) + p2ptest.RequireConnectedEventually(t, []p2p.LibP2PNode{node2, node3}, 1*time.Millisecond, 100*time.Millisecond) + + // while node 2 is disallow-listed, it cannot connect to node 1. Also, node 1 cannot directly dial and connect to node 2, unless + // it is allow-listed again. + p2ptest.EnsureNotConnectedBetweenGroups(t, ctx, []p2p.LibP2PNode{node1}, []p2p.LibP2PNode{node2}) + + // phase-2: now we allow-list node 1 back + node1.OnAllowListNotification(node2.Host().ID(), network.DisallowListedCauseAlsp) + + // eventually node 1 should be connected to node 2 again, hence all nodes should be connected to each other. + // we choose a timeout of 5 seconds because peer manager updates peers every 1 second and we need to wait for + // any potential random backoffs to expire (min 1 second). + p2ptest.RequireConnectedEventually(t, nodes, 100*time.Millisecond, 5*time.Second) +} diff --git a/network/p2p/p2pnode/internal/cache.go b/network/p2p/p2pnode/internal/cache.go new file mode 100644 index 00000000000..6d8952e6628 --- /dev/null +++ b/network/p2p/p2pnode/internal/cache.go @@ -0,0 +1,192 @@ +package internal + +import ( + "errors" + "fmt" + + "github.com/libp2p/go-libp2p/core/peer" + "github.com/rs/zerolog" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" + herocache "github.com/onflow/flow-go/module/mempool/herocache/backdata" + "github.com/onflow/flow-go/module/mempool/herocache/backdata/heropool" + "github.com/onflow/flow-go/module/mempool/stdmap" + "github.com/onflow/flow-go/network" +) + +var ( + ErrDisallowCacheEntityNotFound = errors.New("disallow list cache entity not found") +) + +// DisallowListCache is the disallow-list cache. It is used to keep track of the disallow-listed peers and the reasons for it. +type DisallowListCache struct { + c *stdmap.Backend +} + +// NewDisallowListCache creates a new disallow-list cache. The cache is backed by a stdmap.Backend. +// Args: +// - sizeLimit: the size limit of the cache, i.e., the maximum number of records that the cache can hold, recommended size is 100 * number of authorized nodes. +// - logger: the logger used by the cache. +// - collector: the metrics collector used by the cache. +// Returns: +// - *DisallowListCache: the created cache. +func NewDisallowListCache(sizeLimit uint32, logger zerolog.Logger, collector module.HeroCacheMetrics) *DisallowListCache { + backData := herocache.NewCache(sizeLimit, + herocache.DefaultOversizeFactor, + // this cache is supposed to keep the disallow-list causes for the authorized (staked) nodes. Since the number of such nodes is + // expected to be small, we do not eject any records from the cache. The cache size must be large enough to hold all + // the spam records of the authorized nodes. Also, this cache is keeping at most one record per peer id, so the + // size of the cache must be at least the number of authorized nodes. + heropool.NoEjection, + logger.With().Str("mempool", "disallow-list-records").Logger(), + collector) + + return &DisallowListCache{ + c: stdmap.NewBackend(stdmap.WithBackData(backData)), + } +} + +// IsDisallowListed determines whether the given peer is disallow-listed for any reason. +// Args: +// - peerID: the peer to check. +// Returns: +// - []network.DisallowListedCause: the list of causes for which the given peer is disallow-listed. If the peer is not disallow-listed for any reason, +// a nil slice is returned. +// - bool: true if the peer is disallow-listed for any reason, false otherwise. +func (d *DisallowListCache) IsDisallowListed(peerID peer.ID) ([]network.DisallowListedCause, bool) { + entity, exists := d.c.ByID(makeId(peerID)) + if !exists { + return nil, false + } + + dEntity := mustBeDisallowListEntity(entity) + if len(dEntity.causes) == 0 { + return nil, false + } + + causes := make([]network.DisallowListedCause, 0, len(dEntity.causes)) + for c := range dEntity.causes { + causes = append(causes, c) + } + return causes, true +} + +// init initializes the disallow-list cache entity for the peerID. +// Args: +// - peerID: the peerID of the peer to be disallow-listed. +// Returns: +// - bool: true if the entity is successfully added to the cache. +// false if the entity already exists in the cache. +func (d *DisallowListCache) init(peerID peer.ID) bool { + return d.c.Add(&disallowListCacheEntity{ + peerID: peerID, + causes: make(map[network.DisallowListedCause]struct{}), + id: makeId(peerID), + }) +} + +// DisallowFor disallow-lists a peer for a cause. +// Args: +// - peerID: the peerID of the peer to be disallow-listed. +// - cause: the cause for disallow-listing the peer. +// Returns: +// - []network.DisallowListedCause: the list of causes for which the peer is disallow-listed. +// - error: if the operation fails, error is irrecoverable. +func (d *DisallowListCache) DisallowFor(peerID peer.ID, cause network.DisallowListedCause) ([]network.DisallowListedCause, error) { + // first, we try to optimistically add the peer to the disallow list. + causes, err := d.disallowListFor(peerID, cause) + switch { + case err == nil: + return causes, nil + case err == ErrDisallowCacheEntityNotFound: + // if the entity not exist, we initialize it and try again. + // Note: there is an edge case where the entity is initialized by another goroutine between the two calls. + // In this case, the init function is invoked twice, but it is not a problem because the underlying + // cache is thread-safe. Hence, we do not need to synchronize the two calls. In such cases, one of the + // two calls returns false, and the other call returns true. We do not care which call returns false, hence, + // we ignore the return value of the init function. + _ = d.init(peerID) + causes, err = d.disallowListFor(peerID, cause) + if err != nil { + // any error after the init is irrecoverable. + return nil, fmt.Errorf("failed to disallow list peer %s for cause %s: %w", peerID, cause, err) + } + return causes, nil + default: + return nil, fmt.Errorf("failed to disallow list peer %s for cause %s: %w", peerID, cause, err) + } +} + +// disallowListFor is a helper function for disallowing a peer for a cause. +// It adds the cause to the disallow list cache entity for the peerID and returns the updated list of causes for the peer. +// Args: +// - peerID: the peerID of the peer to be disallow-listed. +// - cause: the cause for disallow-listing the peer. +// Returns: +// - the updated list of causes for the peer. +// - error if the entity for the peerID is not found in the cache it returns ErrDisallowCacheEntityNotFound, which is a benign error. +func (d *DisallowListCache) disallowListFor(peerID peer.ID, cause network.DisallowListedCause) ([]network.DisallowListedCause, error) { + adjustedEntity, adjusted := d.c.Adjust(makeId(peerID), func(entity flow.Entity) flow.Entity { + dEntity := mustBeDisallowListEntity(entity) + dEntity.causes[cause] = struct{}{} + return dEntity + }) + + if !adjusted { + // if the entity is not found in the cache, we return a benign error. + return nil, ErrDisallowCacheEntityNotFound + } + + dEntity := mustBeDisallowListEntity(adjustedEntity) + updatedCauses := make([]network.DisallowListedCause, 0, len(dEntity.causes)) + for c := range dEntity.causes { + updatedCauses = append(updatedCauses, c) + } + + return updatedCauses, nil +} + +// AllowFor removes a cause from the disallow list cache entity for the peerID. +// Args: +// - peerID: the peerID of the peer to be allow-listed. +// - cause: the cause for allow-listing the peer. +// Returns: +// - the list of causes for which the peer is disallow-listed. +// - error if the entity for the peerID is not found in the cache it returns ErrDisallowCacheEntityNotFound, which is a benign error. +func (d *DisallowListCache) AllowFor(peerID peer.ID, cause network.DisallowListedCause) []network.DisallowListedCause { + adjustedEntity, adjusted := d.c.Adjust(makeId(peerID), func(entity flow.Entity) flow.Entity { + dEntity := mustBeDisallowListEntity(entity) + delete(dEntity.causes, cause) + return dEntity + }) + + if !adjusted { + // if the entity is not found in the cache, we return an empty list. + // we don't return a nil to be consistent with the case that entity is found but the list of causes is empty. + return make([]network.DisallowListedCause, 0) + } + + dEntity := mustBeDisallowListEntity(adjustedEntity) + // returning a deep copy of causes (to avoid being mutated externally). + causes := make([]network.DisallowListedCause, 0, len(dEntity.causes)) + for c := range dEntity.causes { + causes = append(causes, c) + } + return causes +} + +// mustBeDisallowListEntity is a helper function for type assertion of the flow.Entity to disallowListCacheEntity. +// It panics if the type assertion fails. +// Args: +// - entity: the flow.Entity to be type asserted. +// Returns: +// - the disallowListCacheEntity. +func mustBeDisallowListEntity(entity flow.Entity) *disallowListCacheEntity { + dEntity, ok := entity.(*disallowListCacheEntity) + if !ok { + // this should never happen, unless there is a bug. We should crash the node and do not proceed. + panic(fmt.Errorf("disallow list cache entity is not of type disallowListCacheEntity, got: %T", entity)) + } + return dEntity +} diff --git a/network/p2p/p2pnode/internal/cacheEntity.go b/network/p2p/p2pnode/internal/cacheEntity.go new file mode 100644 index 00000000000..e55b0d127b5 --- /dev/null +++ b/network/p2p/p2pnode/internal/cacheEntity.go @@ -0,0 +1,44 @@ +package internal + +import ( + "github.com/libp2p/go-libp2p/core/peer" + + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/network" +) + +// disallowListCacheEntity is the model data type for the disallow list cache. +// It represents a single peer that is disallow-listed and the reasons for it. +// The key for storing this entity is the id field which is the hash of the peerID. +// This means that the entities are deduplicated by their peerID. +type disallowListCacheEntity struct { + peerID peer.ID + causes map[network.DisallowListedCause]struct{} + // id is the hash of the peerID which is used as the key for storing the entity in the cache. + // we cache it internally to avoid hashing the peerID multiple times. + id flow.Identifier +} + +var _ flow.Entity = (*disallowListCacheEntity)(nil) + +// ID returns the hash of the peerID which is used as the key for storing the entity in the cache. +// Returns: +// - the hash of the peerID as a flow.Identifier. +func (d *disallowListCacheEntity) ID() flow.Identifier { + return d.id +} + +// Checksum returns the hash of the peerID, there is no use for this method in the cache. It is implemented to satisfy +// the flow.Entity interface. +// Returns: +// - the hash of the peerID as a flow.Identifier. +func (d *disallowListCacheEntity) Checksum() flow.Identifier { + return d.id +} + +// makeId is a helper function for creating the id field of the disallowListCacheEntity by hashing the peerID. +// Returns: +// - the hash of the peerID as a flow.Identifier. +func makeId(peerID peer.ID) flow.Identifier { + return flow.MakeID([]byte(peerID)) +} diff --git a/network/p2p/p2pnode/internal/cache_test.go b/network/p2p/p2pnode/internal/cache_test.go new file mode 100644 index 00000000000..d4ab02d5c9b --- /dev/null +++ b/network/p2p/p2pnode/internal/cache_test.go @@ -0,0 +1,355 @@ +package internal_test + +import ( + "fmt" + "sync" + "testing" + "time" + + "github.com/libp2p/go-libp2p/core/peer" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/onflow/flow-go/module/metrics" + "github.com/onflow/flow-go/network" + "github.com/onflow/flow-go/network/p2p/p2pnode/internal" + "github.com/onflow/flow-go/utils/unittest" +) + +// TestNewDisallowListCache tests the NewDisallowListCache function. It verifies that the returned disallowListCache +// is not nil. +func TestNewDisallowListCache(t *testing.T) { + disallowListCache := internal.NewDisallowListCache(uint32(100), unittest.Logger(), metrics.NewNoopCollector()) + + // Verify that the new disallowListCache is not nil + assert.NotNil(t, disallowListCache) +} + +// TestDisallowFor_SinglePeer tests the DisallowFor function for a single peer. It verifies that the peerID is +// disallow-listed for the given cause and that the cause is returned when the peerID is disallow-listed again. +func TestDisallowFor_SinglePeer(t *testing.T) { + disallowListCache := internal.NewDisallowListCache(uint32(100), unittest.Logger(), metrics.NewNoopCollector()) + require.NotNil(t, disallowListCache) + + // disallowing a peerID for a cause when the peerID doesn't exist in the cache + causes, err := disallowListCache.DisallowFor(peer.ID("peer1"), network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAdmin) + + // disallowing a peerID for a cause when the peerID already exists in the cache + causes, err = disallowListCache.DisallowFor(peer.ID("peer1"), network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) + + // disallowing a peerID for a duplicate cause + causes, err = disallowListCache.DisallowFor(peer.ID("peer1"), network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) +} + +// TestDisallowFor_MultiplePeers tests the DisallowFor function for multiple peers. It verifies that the peerIDs are +// disallow-listed for the given cause and that the cause is returned when the peerIDs are disallow-listed again. +func TestDisallowFor_MultiplePeers(t *testing.T) { + disallowListCache := internal.NewDisallowListCache(uint32(100), unittest.Logger(), metrics.NewNoopCollector()) + require.NotNil(t, disallowListCache) + + for i := 0; i <= 10; i++ { + // disallowing a peerID for a cause when the peerID doesn't exist in the cache + causes, err := disallowListCache.DisallowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAdmin) + } + + for i := 0; i <= 10; i++ { + // disallowing a peerID for a cause when the peerID already exists in the cache + causes, err := disallowListCache.DisallowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) + } + + for i := 0; i <= 10; i++ { + // getting the disallow-listed causes for a peerID + causes, disallowListed := disallowListCache.IsDisallowListed(peer.ID(fmt.Sprintf("peer-%d", i))) + require.True(t, disallowListed) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) + } +} + +// TestAllowFor_SinglePeer is a unit test function to verify the behavior of DisallowListCache for a single peer. +// The test checks the following functionalities in sequence: +// 1. Allowing a peerID for a cause when the peerID already exists in the cache. +// 2. Disallowing the peerID for a cause when the peerID doesn't exist in the cache. +// 3. Getting the disallow-listed causes for the peerID. +// 4. Allowing a peerID for a cause when the peerID already exists in the cache. +// 5. Getting the disallow-listed causes for the peerID. +// 6. Disallowing the peerID for a cause. +// 7. Allowing the peerID for a different cause than it is disallowed when the peerID already exists in the cache. +// 8. Disallowing the peerID for another cause. +// 9. Allowing the peerID for the first cause. +// 10. Allowing the peerID for the second cause. +func TestAllowFor_SinglePeer(t *testing.T) { + disallowListCache := internal.NewDisallowListCache(uint32(100), unittest.Logger(), metrics.NewNoopCollector()) + require.NotNil(t, disallowListCache) + peerID := peer.ID("peer1") + + // allowing the peerID for a cause when the peerID already exists in the cache + causes := disallowListCache.AllowFor(peerID, network.DisallowListedCauseAdmin) + require.Len(t, causes, 0) + + // disallowing the peerID for a cause when the peerID doesn't exist in the cache + causes, err := disallowListCache.DisallowFor(peerID, network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAdmin) + + // getting the disallow-listed causes for the peerID + causes, disallowListed := disallowListCache.IsDisallowListed(peerID) + require.True(t, disallowListed) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAdmin) + + // allowing a peerID for a cause when the peerID already exists in the cache + causes = disallowListCache.AllowFor(peerID, network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 0) + + // getting the disallow-listed causes for the peerID + causes, disallowListed = disallowListCache.IsDisallowListed(peerID) + require.False(t, disallowListed) + require.Len(t, causes, 0) + + // disallowing the peerID for a cause + causes, err = disallowListCache.DisallowFor(peerID, network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 1) + + // allowing the peerID for a different cause than it is disallowed when the peerID already exists in the cache + causes = disallowListCache.AllowFor(peerID, network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAdmin) // the peerID is still disallow-listed for the previous cause + + // disallowing the peerID for another cause + causes, err = disallowListCache.DisallowFor(peerID, network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) + + // allowing the peerID for the first cause + causes = disallowListCache.AllowFor(peerID, network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) // the peerID is still disallow-listed for the previous cause + + // allowing the peerID for the second cause + causes = disallowListCache.AllowFor(peerID, network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 0) +} + +// TestAllowFor_MultiplePeers_Sequentially is a unit test function to test the behavior of DisallowListCache with multiple peers. +// The test checks the following functionalities in sequence: +// 1. Allowing a peerID for a cause when the peerID doesn't exist in the cache. +// 2. Disallowing peers for a cause. +// 3. Getting the disallow-listed causes for a peerID. +// 4. Allowing the peer ids for a cause different than the one they are disallow-listed for. +// 5. Disallowing the peer ids for a different cause. +// 6. Allowing the peer ids for the first cause. +// 7. Allowing the peer ids for the second cause. +func TestAllowFor_MultiplePeers_Sequentially(t *testing.T) { + disallowListCache := internal.NewDisallowListCache(uint32(100), unittest.Logger(), metrics.NewNoopCollector()) + require.NotNil(t, disallowListCache) + + for i := 0; i <= 10; i++ { + // allowing a peerID for a cause when the peerID doesn't exist in the cache + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 0) + } + + for i := 0; i <= 10; i++ { + // disallowing peers for a cause + causes, err := disallowListCache.DisallowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + } + + for i := 0; i <= 10; i++ { + // getting the disallow-listed causes for a peerID + causes, disallowListed := disallowListCache.IsDisallowListed(peer.ID(fmt.Sprintf("peer-%d", i))) + require.True(t, disallowListed) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + } + + for i := 0; i <= 10; i++ { + // allowing the peer ids for a cause different than the one they are disallow-listed for + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + } + + for i := 0; i <= 10; i++ { + // disallowing the peer ids for a different cause + causes, err := disallowListCache.DisallowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) + } + + for i := 0; i <= 10; i++ { + // allowing the peer ids for the first cause + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + } + + for i := 0; i <= 10; i++ { + // allowing the peer ids for the second cause + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAlsp) + require.Len(t, causes, 0) + } +} + +// TestAllowFor_MultiplePeers_Concurrently is a unit test function that verifies the behavior of DisallowListCache +// when multiple peerIDs are added and managed concurrently. This test is designed to confirm that DisallowListCache +// works as expected under concurrent access, an important aspect for a system dealing with multiple connections. +// +// The test runs multiple goroutines simultaneously, each handling a different peerID and performs the following +// operations in the sequence: +// 1. Allowing a peerID for a cause when the peerID doesn't exist in the cache. +// 2. Disallowing peers for a cause. +// 3. Getting the disallow-listed causes for a peerID. +// 4. Allowing the peer ids for a cause different than the one they are disallow-listed for. +// 5. Disallowing the peer ids for a different cause. +// 6. Allowing the peer ids for the first cause. +// 7. Allowing the peer ids for the second cause. +// 8. Getting the disallow-listed causes for a peerID. +// 9. Allowing a peerID for a cause when the peerID doesn't exist in the cache for a new set of peers. +func TestAllowFor_MultiplePeers_Concurrently(t *testing.T) { + disallowListCache := internal.NewDisallowListCache(uint32(100), unittest.Logger(), metrics.NewNoopCollector()) + require.NotNil(t, disallowListCache) + + var wg sync.WaitGroup + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // allowing a peerID for a cause when the peerID doesn't exist in the cache + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 0) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // disallowing peers for a cause + causes, err := disallowListCache.DisallowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAlsp) + require.NoError(t, err) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // getting the disallow-listed causes for a peerID + causes, disallowListed := disallowListCache.IsDisallowListed(peer.ID(fmt.Sprintf("peer-%d", i))) + require.Len(t, causes, 1) + require.True(t, disallowListed) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // allowing the peer ids for a cause different than the one they are disallow-listed for + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // disallowing the peer ids for a different cause + causes, err := disallowListCache.DisallowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.NoError(t, err) + require.Len(t, causes, 2) + require.ElementsMatch(t, causes, []network.DisallowListedCause{network.DisallowListedCauseAdmin, network.DisallowListedCauseAlsp}) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // allowing the peer ids for the first cause + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 1) + require.Contains(t, causes, network.DisallowListedCauseAlsp) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // allowing the peer ids for the second cause + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAlsp) + require.Len(t, causes, 0) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 0; i <= 10; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // getting the disallow-listed causes for a peerID + causes, disallowListed := disallowListCache.IsDisallowListed(peer.ID(fmt.Sprintf("peer-%d", i))) + require.False(t, disallowListed) + require.Len(t, causes, 0) + }(i) + } + unittest.RequireReturnsBefore(t, wg.Wait, 100*time.Millisecond, "timed out waiting for goroutines to finish") + + for i := 11; i <= 20; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + + // allowing a peerID for a cause when the peerID doesn't exist in the cache + causes := disallowListCache.AllowFor(peer.ID(fmt.Sprintf("peer-%d", i)), network.DisallowListedCauseAdmin) + require.Len(t, causes, 0) + }(i) + } +} diff --git a/network/p2p/p2pnode/libp2pNode.go b/network/p2p/p2pnode/libp2pNode.go index 977a5b393d3..60a548c694b 100644 --- a/network/p2p/p2pnode/libp2pNode.go +++ b/network/p2p/p2pnode/libp2pNode.go @@ -24,6 +24,7 @@ import ( "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/internal/p2putils" "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" "github.com/onflow/flow-go/utils/logging" ) @@ -46,6 +47,8 @@ const ( findPeerQueryTimeout = 10 * time.Second ) +var _ p2p.LibP2PNode = (*Node)(nil) + // Node is a wrapper around the LibP2P host. type Node struct { component.Component @@ -60,6 +63,9 @@ type Node struct { pCache p2p.ProtocolPeerCache peerManager p2p.PeerManager peerScoreExposer p2p.PeerScoreExposer + // 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 } // NewNode creates a new libp2p node and sets its parameters. @@ -68,14 +74,20 @@ func NewNode( 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: logger.With().Str("component", "libp2p-node").Logger(), + logger: lg, topics: make(map[channels.Topic]p2p.Topic), subs: make(map[channels.Topic]p2p.Subscription), pCache: pCache, peerManager: peerManager, + disallowListedCache: internal.NewDisallowListCache( + disallowLstCacheCfg.MaxSize, + logger.With().Str("module", "disallow-list-cache").Logger(), + disallowLstCacheCfg.Metrics), } } @@ -346,8 +358,29 @@ func (n *Node) WithDefaultUnicastProtocol(defaultHandler libp2pnet.StreamHandler // WithPeersProvider sets the PeersProvider for the peer manager. // If a peer manager factory is set, this method will set the peer manager's PeersProvider. func (n *Node) WithPeersProvider(peersProvider p2p.PeersProvider) { + // TODO: chore: we should not allow overriding the peers provider if one is already set. if n.peerManager != nil { - n.peerManager.SetPeersProvider(peersProvider) + n.peerManager.SetPeersProvider( + func() peer.IDSlice { + authorizedPeersIds := peersProvider() + allowListedPeerIds := peer.IDSlice{} // subset of authorizedPeersIds that are not disallowed + for _, peerId := range authorizedPeersIds { + // exclude the disallowed peers from the authorized peers list + causes, disallowListed := n.disallowListedCache.IsDisallowListed(peerId) + if disallowListed { + n.logger.Warn(). + Str("peer_id", peerId.String()). + Str("causes", fmt.Sprintf("%v", causes)). + Msg("peer is disallowed for a cause, removing from authorized peers of peer manager") + + // exclude the peer from the authorized peers list + continue + } + allowListedPeerIds = append(allowListedPeerIds, peerId) + } + + return allowListedPeerIds + }) } } @@ -444,3 +477,61 @@ func (n *Node) SetUnicastManager(uniMgr p2p.UnicastManager) { } n.uniMgr = uniMgr } + +// OnDisallowListNotification is called when a new disallow list update notification is distributed. +// Any error on consuming event must handle internally. +// The implementation must be concurrency safe. +// Args: +// +// id: peer ID of the peer being disallow-listed. +// cause: cause of the peer being disallow-listed (only this cause is added to the peer's disallow-listed causes). +// +// Returns: +// +// none +func (n *Node) OnDisallowListNotification(peerId peer.ID, cause flownet.DisallowListedCause) { + causes, err := n.disallowListedCache.DisallowFor(peerId, cause) + if err != nil { + // returned error is fatal. + n.logger.Fatal().Err(err).Str("peer_id", peerId.String()).Msg("failed to add peer to disallow list") + } + + // TODO: this code should further be refactored to also log the Flow id. + n.logger.Warn(). + Str("peer_id", peerId.String()). + Str("notification_cause", cause.String()). + Str("causes", fmt.Sprintf("%v", causes)). + Msg("peer added to disallow list cache") +} + +// OnAllowListNotification is called when a new allow list update notification is distributed. +// Any error on consuming event must handle internally. +// The implementation must be concurrency safe. +// Args: +// +// id: peer ID of the peer being allow-listed. +// cause: cause of the peer being allow-listed (only this cause is removed from the peer's disallow-listed causes). +// +// Returns: +// +// none +func (n *Node) OnAllowListNotification(peerId peer.ID, cause flownet.DisallowListedCause) { + remainingCauses := n.disallowListedCache.AllowFor(peerId, cause) + + n.logger.Info(). + Str("peer_id", peerId.String()). + Str("causes", fmt.Sprintf("%v", cause)). + Str("remaining_causes", fmt.Sprintf("%v", remainingCauses)). + Msg("peer is allow-listed for cause") +} + +// IsDisallowListed determines whether the given peer is disallow-listed for any reason. +// Args: +// - peerID: the peer to check. +// Returns: +// - []network.DisallowListedCause: the list of causes for which the given peer is disallow-listed. If the peer is not disallow-listed for any reason, +// a nil slice is returned. +// - bool: true if the peer is disallow-listed for any reason, false otherwise. +func (n *Node) IsDisallowListed(peerId peer.ID) ([]flownet.DisallowListedCause, bool) { + return n.disallowListedCache.IsDisallowListed(peerId) +} diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 505a90a3f47..556da6c3fd7 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -57,7 +57,7 @@ func NodeFixture( opts ...NodeFixtureParameterOption, ) (p2p.LibP2PNode, flow.Identity) { - logger := unittest.Logger().Level(zerolog.ErrorLevel) + logger := unittest.Logger().Level(zerolog.WarnLevel) rpcInspectorSuite, err := inspectorbuilder.NewGossipSubInspectorBuilder(logger, sporkID, inspectorbuilder.DefaultGossipSubRPCInspectorsConfig(), idProvider, metrics.NewNoopCollector()). Build() @@ -97,7 +97,11 @@ func NodeFixture( parameters.Address, parameters.Key, sporkID, - p2pbuilder.DefaultResourceManagerConfig()). + p2pbuilder.DefaultResourceManagerConfig(), + &p2p.DisallowListCacheConfig{ + MaxSize: uint32(1000), + Metrics: metrics.NewNoopCollector(), + }). SetConnectionManager(connManager). SetRoutingSystem(func(c context.Context, h host.Host) (routing.Routing, error) { return p2pdht.NewDHT(c, h, @@ -177,7 +181,7 @@ type NodeFixtureParameters struct { ConnectionPruning bool // peer manager parameter UpdateInterval time.Duration // peer manager parameter PeerProvider p2p.PeersProvider // peer manager parameter - ConnGater connmgr.ConnectionGater + ConnGater p2p.ConnectionGater ConnManager connmgr.ConnManager GossipSubFactory p2p.GossipSubFactoryFunc GossipSubConfig p2p.GossipSubAdapterConfigFunc @@ -252,7 +256,7 @@ func WithDHTOptions(opts ...dht.Option) NodeFixtureParameterOption { } } -func WithConnectionGater(connGater connmgr.ConnectionGater) NodeFixtureParameterOption { +func WithConnectionGater(connGater p2p.ConnectionGater) NodeFixtureParameterOption { return func(p *NodeFixtureParameters) { p.ConnGater = connGater } @@ -386,20 +390,71 @@ func LetNodesDiscoverEachOther(t *testing.T, ctx context.Context, nodes []p2p.Li } } -// EnsureConnected ensures that the given nodes are connected to each other. +// TryConnectionAndEnsureConnected tries connecting nodes to each other and ensures that the given nodes are connected to each other. // It fails the test if any of the nodes is not connected to any other node. -func EnsureConnected(t *testing.T, ctx context.Context, nodes []p2p.LibP2PNode) { +func TryConnectionAndEnsureConnected(t *testing.T, ctx context.Context, nodes []p2p.LibP2PNode) { for _, node := range nodes { for _, other := range nodes { if node == other { continue } require.NoError(t, node.Host().Connect(ctx, other.Host().Peerstore().PeerInfo(other.Host().ID()))) + // the other node should be connected to this node require.Equal(t, node.Host().Network().Connectedness(other.Host().ID()), network.Connected) + // at least one connection should be established + require.True(t, len(node.Host().Network().ConnsToPeer(other.Host().ID())) > 0) } } } +// RequireConnectedEventually ensures eventually that the given nodes are already connected to each other. +// It fails the test if any of the nodes is not connected to any other node. +// Args: +// - nodes: the nodes to check +// - tick: the tick duration +// - timeout: the timeout duration +func RequireConnectedEventually(t *testing.T, nodes []p2p.LibP2PNode, tick time.Duration, timeout time.Duration) { + require.Eventually(t, func() bool { + for _, node := range nodes { + for _, other := range nodes { + if node == other { + continue + } + if node.Host().Network().Connectedness(other.Host().ID()) != network.Connected { + return false + } + if len(node.Host().Network().ConnsToPeer(other.Host().ID())) == 0 { + return false + } + } + } + return true + }, timeout, tick) +} + +// RequireEventuallyNotConnected ensures eventually that the given groups of nodes are not connected to each other. +// It fails the test if any of the nodes from groupA is connected to any of the nodes from groupB. +// Args: +// - groupA: the first group of nodes +// - groupB: the second group of nodes +// - tick: the tick duration +// - timeout: the timeout duration +func RequireEventuallyNotConnected(t *testing.T, groupA []p2p.LibP2PNode, groupB []p2p.LibP2PNode, tick time.Duration, timeout time.Duration) { + require.Eventually(t, func() bool { + for _, node := range groupA { + for _, other := range groupB { + if node.Host().Network().Connectedness(other.Host().ID()) == network.Connected { + return false + } + if len(node.Host().Network().ConnsToPeer(other.Host().ID())) > 0 { + return false + } + } + } + return true + }, timeout, tick) +} + // EnsureStreamCreationInBothDirections ensure that between each pair of nodes in the given list, a stream is created in both directions. func EnsureStreamCreationInBothDirections(t *testing.T, ctx context.Context, nodes []p2p.LibP2PNode) { for _, this := range nodes { @@ -416,7 +471,7 @@ func EnsureStreamCreationInBothDirections(t *testing.T, ctx context.Context, nod } // EnsurePubsubMessageExchange ensures that the given connected nodes exchange the given message on the given channel through pubsub. -// Note: EnsureConnected() must be called to connect all nodes before calling this function. +// Note: TryConnectionAndEnsureConnected() must be called to connect all nodes before calling this function. func EnsurePubsubMessageExchange(t *testing.T, ctx context.Context, nodes []p2p.LibP2PNode, messageFactory func() (interface{}, channels.Topic)) { _, topic := messageFactory() diff --git a/network/test/middleware_test.go b/network/test/middleware_test.go index 3fe9ecc042f..631808230cf 100644 --- a/network/test/middleware_test.go +++ b/network/test/middleware_test.go @@ -250,17 +250,20 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { idProvider := testutils.NewUpdatableIDProvider(m.ids) // create a new staked identity - connGater := testutils.NewConnectionGater(idProvider, func(pid peer.ID) error { - if messageRateLimiter.IsRateLimited(pid) { - return fmt.Errorf("rate-limited peer") - } - return nil - }) + connGaterFactory := func() p2p.ConnectionGater { + return testutils.NewConnectionGater(idProvider, func(pid peer.ID) error { + if messageRateLimiter.IsRateLimited(pid) { + return fmt.Errorf("rate-limited peer") + } + return nil + }) + } + ids, libP2PNodes, _ := testutils.GenerateIDs(m.T(), m.logger, 1, testutils.WithUnicastRateLimiterDistributor(distributor), - testutils.WithConnectionGater(connGater)) + testutils.WithConnectionGaterFactory(connGaterFactory)) idProvider.SetIdentities(append(m.ids, ids...)) // create middleware @@ -317,7 +320,7 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Messages() { // return true only if the node is a direct peer of the other, after rate limiting this direct // peer should be removed by the peer manager. p2ptest.LetNodesDiscoverEachOther(m.T(), ctx, []p2p.LibP2PNode{libP2PNodes[0], m.nodes[0]}, flow.IdentityList{ids[0], m.ids[0]}) - p2ptest.EnsureConnected(m.T(), ctx, []p2p.LibP2PNode{libP2PNodes[0], m.nodes[0]}) + p2ptest.TryConnectionAndEnsureConnected(m.T(), ctx, []p2p.LibP2PNode{libP2PNodes[0], m.nodes[0]}) // with the rate limit configured to 5 msg/sec we send 10 messages at once and expect the rate limiter // to be invoked at-least once. We send 10 messages due to the flakiness that is caused by async stream @@ -405,19 +408,21 @@ func (m *MiddlewareTestSuite) TestUnicastRateLimit_Bandwidth() { idProvider := testutils.NewUpdatableIDProvider(m.ids) // create connection gater, connection gater will refuse connections from rate limited nodes - connGater := testutils.NewConnectionGater(idProvider, func(pid peer.ID) error { - if bandwidthRateLimiter.IsRateLimited(pid) { - return fmt.Errorf("rate-limited peer") - } + connGaterFactory := func() p2p.ConnectionGater { + return testutils.NewConnectionGater(idProvider, func(pid peer.ID) error { + if bandwidthRateLimiter.IsRateLimited(pid) { + return fmt.Errorf("rate-limited peer") + } - return nil - }) + return nil + }) + } // create a new staked identity ids, libP2PNodes, _ := testutils.GenerateIDs(m.T(), m.logger, 1, testutils.WithUnicastRateLimiterDistributor(distributor), - testutils.WithConnectionGater(connGater)) + testutils.WithConnectionGaterFactory(connGaterFactory)) idProvider.SetIdentities(append(m.ids, ids...)) // create middleware