From e7320d078b74d350407e1926e9499640f7b15471 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 10:03:56 -0800 Subject: [PATCH 01/52] moves distributor logic into score registry --- .../p2p/distributor/gossipsub_inspector.go | 20 +----------------- network/p2p/scoring/registry.go | 21 ++++++++++++++++--- 2 files changed, 19 insertions(+), 22 deletions(-) diff --git a/network/p2p/distributor/gossipsub_inspector.go b/network/p2p/distributor/gossipsub_inspector.go index d466bf5a134..f3152ad5d79 100644 --- a/network/p2p/distributor/gossipsub_inspector.go +++ b/network/p2p/distributor/gossipsub_inspector.go @@ -6,12 +6,10 @@ import ( "github.com/rs/zerolog" "github.com/onflow/flow-go/engine" - "github.com/onflow/flow-go/engine/common/worker" "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" - p2plogging "github.com/onflow/flow-go/network/p2p/logging" ) const ( @@ -33,7 +31,6 @@ type GossipSubInspectorNotifDistributor struct { cm *component.ComponentManager logger zerolog.Logger - workerPool *worker.Pool[*p2p.InvCtrlMsgNotif] consumerLock sync.RWMutex // protects the consumer field from concurrent updates consumers []p2p.GossipSubInvCtrlMsgNotifConsumer } @@ -62,15 +59,6 @@ func NewGossipSubInspectorNotificationDistributor(log zerolog.Logger, store engi logger: lg, } - pool := worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, store, d.distribute).Build() - d.workerPool = pool - - cm := component.NewComponentManagerBuilder() - - for i := 0; i < defaultGossipSubInspectorNotificationQueueWorkerCount; i++ { - cm.AddWorker(pool.WorkerLogic()) - } - d.cm = cm.Build() d.Component = d.cm @@ -81,13 +69,7 @@ func NewGossipSubInspectorNotificationDistributor(log zerolog.Logger, store engi // The distribution is done asynchronously and non-blocking. The notification is added to a queue and processed by a worker pool. // DistributeEvent in this implementation does not return an error, but it logs a warning if the queue is full. func (g *GossipSubInspectorNotifDistributor) Distribute(notification *p2p.InvCtrlMsgNotif) error { - lg := g.logger.With().Str("peer_id", p2plogging.PeerId(notification.PeerID)).Logger() - if ok := g.workerPool.Submit(notification); !ok { - // we use a queue with a fixed size, so this can happen when queue is full or when the notification is duplicate. - lg.Warn().Msg("gossipsub rpc inspector notification queue is full or notification is duplicate, discarding notification") - } - lg.Trace().Msg("gossipsub rpc inspector notification submitted to the queue") - return nil + } // AddConsumer adds a consumer to the distributor. The consumer will be called when distributor distributes a new event. diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index 16f3471f2c8..9b46a2edd9f 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -66,6 +66,7 @@ type GossipSubAppSpecificScoreRegistry struct { // appScoreUpdateWorkerPool is the worker pool for handling the application specific score update of peers in a non-blocking way. appScoreUpdateWorkerPool *worker.Pool[peer.ID] + workerPool *worker.Pool[*p2p.InvCtrlMsgNotif] appSpecificScoreParams p2pconfig.ApplicationSpecificScoreParameters duplicateMessageThreshold float64 @@ -163,6 +164,9 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis store, reg.processAppSpecificScoreUpdateWork).Build() + pool := worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, store, reg.handleMisbehaviourReport).Build() + reg.workerPool = pool + builder := component.NewComponentManagerBuilder() builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { reg.logger.Info().Msg("starting subscription validator") @@ -185,7 +189,7 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis } reg.silencePeriodStartTime = time.Now() ready() - }) + }).AddWorker(pool.WorkerLogic()) // we must NOT have more than one worker for processing notifications; handling notifications are NOT idempotent. for i := 0; i < config.Parameters.ScoreUpdateWorkerNum; i++ { builder.AddWorker(reg.appScoreUpdateWorkerPool.WorkerLogic()) @@ -404,16 +408,25 @@ func (r *GossipSubAppSpecificScoreRegistry) duplicateMessagesPenalty(pid peer.ID // Any error on consuming event must handle internally. // The implementation must be concurrency safe, but can be blocking. func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification(notification *p2p.InvCtrlMsgNotif) { + lg := r.logger.With().Str("peer_id", p2plogging.PeerId(notification.PeerID)).Logger() + if ok := r.workerPool.Submit(notification); !ok { + // we use a queue with a fixed size, so this can happen when queue is full or when the notification is duplicate. + // TODO: we have to add a metric for this case. + lg.Warn().Msg("gossipsub rpc inspector notification queue is full or notification is duplicate, discarding notification") + } + lg.Trace().Msg("gossipsub rpc inspector notification submitted to the queue") +} + +func (r *GossipSubAppSpecificScoreRegistry) handleMisbehaviourReport(notification *p2p.InvCtrlMsgNotif) error { // we use mutex to ensure the method is concurrency safe. lg := r.logger.With(). Err(notification.Error). - Str("peer_id", p2plogging.PeerId(notification.PeerID)). Str("misbehavior_type", notification.MsgType.String()).Logger() // during startup silence period avoid penalizing nodes, ignore all notifications if !r.afterSilencePeriod() { lg.Trace().Msg("ignoring invalid control message notification for peer during silence period") - return + return nil } record, err := r.spamScoreCache.Adjust(notification.PeerID, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { @@ -451,6 +464,8 @@ func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification( lg.Debug(). Float64("spam_record_penalty", record.Penalty). Msg("applied misbehaviour penalty and updated application specific penalty") + + return nil } // afterSilencePeriod returns true if registry silence period is over, false otherwise. From a5c7b8dad55fba41b51410925f05c52ff23b3bab Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 10:13:54 -0800 Subject: [PATCH 02/52] wip --- network/p2p/scoring/registry.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index 9b46a2edd9f..4e69890e966 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -140,7 +140,7 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis } lg := config.Logger.With().Str("module", "app_score_registry").Logger() - store := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, + appSpecificScoreHS := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, lg.With().Str("component", "app_specific_score_update").Logger(), metrics.GossipSubAppSpecificScoreUpdateQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) @@ -161,10 +161,10 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis } reg.appScoreUpdateWorkerPool = worker.NewWorkerPoolBuilder[peer.ID](lg.With().Str("component", "app_specific_score_update_worker_pool").Logger(), - store, + appSpecificScoreHS, reg.processAppSpecificScoreUpdateWork).Build() - pool := worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, store, reg.handleMisbehaviourReport).Build() + pool := worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, appSpecificScoreHS, reg.handleMisbehaviourReport).Build() reg.workerPool = pool builder := component.NewComponentManagerBuilder() From 9cad443b71487fd7367e3c1cc7264622e561a2a9 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:19:44 -0800 Subject: [PATCH 03/52] cleans up distributor and inspector suit --- network/p2p/builder/inspector/aggregate.go | 38 ------- network/p2p/builder/inspector/suite.go | 93 ---------------- .../p2p/distributor/gossipsub_inspector.go | 99 ----------------- .../distributor/gossipsub_inspector_test.go | 101 ------------------ .../gossip_sub_inspector_notif_distributor.go | 86 --------------- ..._sub_inspector_notification_distributor.go | 87 --------------- .../p2p/mock/gossip_sub_inspector_suite.go | 99 ----------------- 7 files changed, 603 deletions(-) delete mode 100644 network/p2p/builder/inspector/aggregate.go delete mode 100644 network/p2p/builder/inspector/suite.go delete mode 100644 network/p2p/distributor/gossipsub_inspector.go delete mode 100644 network/p2p/distributor/gossipsub_inspector_test.go delete mode 100644 network/p2p/mock/gossip_sub_inspector_notif_distributor.go delete mode 100644 network/p2p/mock/gossip_sub_inspector_notification_distributor.go delete mode 100644 network/p2p/mock/gossip_sub_inspector_suite.go diff --git a/network/p2p/builder/inspector/aggregate.go b/network/p2p/builder/inspector/aggregate.go deleted file mode 100644 index 604a888fb45..00000000000 --- a/network/p2p/builder/inspector/aggregate.go +++ /dev/null @@ -1,38 +0,0 @@ -package inspector - -import ( - "github.com/hashicorp/go-multierror" - pubsub "github.com/libp2p/go-libp2p-pubsub" - "github.com/libp2p/go-libp2p/core/peer" - - "github.com/onflow/flow-go/network/p2p" -) - -// AggregateRPCInspector gossip sub RPC inspector that combines multiple RPC inspectors into a single inspector. Each -// individual inspector will be invoked synchronously. -type AggregateRPCInspector struct { - inspectors []p2p.GossipSubRPCInspector -} - -// NewAggregateRPCInspector returns new aggregate RPC inspector. -func NewAggregateRPCInspector(inspectors ...p2p.GossipSubRPCInspector) *AggregateRPCInspector { - return &AggregateRPCInspector{ - inspectors: inspectors, - } -} - -// Inspect func with the p2p.GossipSubAppSpecificRpcInspector func signature that will invoke all the configured inspectors. -func (a *AggregateRPCInspector) Inspect(peerID peer.ID, rpc *pubsub.RPC) error { - var errs *multierror.Error - for _, inspector := range a.inspectors { - err := inspector.Inspect(peerID, rpc) - if err != nil { - errs = multierror.Append(errs, err) - } - } - return errs.ErrorOrNil() -} - -func (a *AggregateRPCInspector) Inspectors() []p2p.GossipSubRPCInspector { - return a.inspectors -} diff --git a/network/p2p/builder/inspector/suite.go b/network/p2p/builder/inspector/suite.go deleted file mode 100644 index b1b35d8bc2c..00000000000 --- a/network/p2p/builder/inspector/suite.go +++ /dev/null @@ -1,93 +0,0 @@ -package inspector - -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" - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/inspector/validation" -) - -// GossipSubInspectorSuite encapsulates what is exposed to the libp2p node regarding the gossipsub RPC inspectors as -// well as their notification distributors. -type GossipSubInspectorSuite struct { - component.Component - aggregatedInspector *AggregateRPCInspector - validationInspector *validation.ControlMsgValidationInspector - ctrlMsgInspectDistributor p2p.GossipSubInspectorNotifDistributor -} - -// TODO: this can be simplified as there is no more need for the aggregated inspector. -var _ p2p.GossipSubInspectorSuite = (*GossipSubInspectorSuite)(nil) - -// NewGossipSubInspectorSuite creates a new GossipSubInspectorSuite. -// The suite is composed of the aggregated inspector, which is used to inspect the gossipsub rpc messages, and the -// control message notification distributor, which is used to notify consumers when a misbehaving peer regarding gossipsub -// control messages is detected. -// The suite is also a component, which is used to start and stop the rpc inspectors. -// Args: -// - metricsInspector: the control message metrics inspector. -// - validationInspector: the gossipsub validation control message validation inspector. -// - ctrlMsgInspectDistributor: the notification distributor that is used to notify consumers when a misbehaving peer -// -// regarding gossipsub control messages is detected. -// Returns: -// - the new GossipSubInspectorSuite. -func NewGossipSubInspectorSuite( - validationInspector *validation.ControlMsgValidationInspector, - ctrlMsgInspectDistributor p2p.GossipSubInspectorNotifDistributor) *GossipSubInspectorSuite { - inspectors := []p2p.GossipSubRPCInspector{validationInspector} - s := &GossipSubInspectorSuite{ - ctrlMsgInspectDistributor: ctrlMsgInspectDistributor, - validationInspector: validationInspector, - aggregatedInspector: NewAggregateRPCInspector(inspectors...), - } - - builder := component.NewComponentManagerBuilder() - for _, rpcInspector := range inspectors { - rpcInspector := rpcInspector // capture loop variable - builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - rpcInspector.Start(ctx) - - select { - case <-ctx.Done(): - case <-rpcInspector.Ready(): - ready() - } - - <-rpcInspector.Done() - }) - } - - s.Component = builder.Build() - return s -} - -// InspectFunc returns the inspect function that is used to inspect the gossipsub rpc messages. -// This function follows a dependency injection pattern, where the inspect function is injected into the gossipsu, and -// is called whenever a gossipsub rpc message is received. -func (s *GossipSubInspectorSuite) InspectFunc() func(peer.ID, *pubsub.RPC) error { - return s.aggregatedInspector.Inspect -} - -// AddInvalidControlMessageConsumer adds a consumer to the invalid control message notification distributor. -// This consumer is notified when a misbehaving peer regarding gossipsub control messages is detected. This follows a pub/sub -// pattern where the consumer is notified when a new notification is published. -// A consumer is only notified once for each notification, and only receives notifications that were published after it was added. -func (s *GossipSubInspectorSuite) AddInvalidControlMessageConsumer(c p2p.GossipSubInvCtrlMsgNotifConsumer) { - s.ctrlMsgInspectDistributor.AddConsumer(c) -} - -// ActiveClustersChanged is called when the list of active collection nodes cluster is changed. -// GossipSubInspectorSuite consumes this event and forwards it to all the respective rpc inspectors, that are -// concerned with this cluster-based topics (i.e., channels), so that they can update their internal state. -func (s *GossipSubInspectorSuite) ActiveClustersChanged(list flow.ChainIDList) { - for _, rpcInspector := range s.aggregatedInspector.Inspectors() { - if r, ok := rpcInspector.(p2p.GossipSubMsgValidationRpcInspector); ok { - r.ActiveClustersChanged(list) - } - } -} diff --git a/network/p2p/distributor/gossipsub_inspector.go b/network/p2p/distributor/gossipsub_inspector.go deleted file mode 100644 index f3152ad5d79..00000000000 --- a/network/p2p/distributor/gossipsub_inspector.go +++ /dev/null @@ -1,99 +0,0 @@ -package distributor - -import ( - "sync" - - "github.com/rs/zerolog" - - "github.com/onflow/flow-go/engine" - "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 ( - // DefaultGossipSubInspectorNotificationQueueCacheSize is the default cache size for the gossipsub rpc inspector notification queue. - DefaultGossipSubInspectorNotificationQueueCacheSize = 10_000 - // defaultGossipSubInspectorNotificationQueueWorkerCount is the default number of workers that will process the gossipsub rpc inspector notifications. - defaultGossipSubInspectorNotificationQueueWorkerCount = 1 -) - -var _ p2p.GossipSubInspectorNotifDistributor = (*GossipSubInspectorNotifDistributor)(nil) - -// GossipSubInspectorNotifDistributor is a component that distributes gossipsub rpc inspector notifications to -// registered consumers in a non-blocking manner and asynchronously. It is thread-safe and can be used concurrently from -// multiple goroutines. The distribution is done by a worker pool. The worker pool is configured with a queue that has a -// fixed size. If the queue is full, the notification is discarded. The queue size and the number of workers can be -// configured. -type GossipSubInspectorNotifDistributor struct { - component.Component - cm *component.ComponentManager - logger zerolog.Logger - - consumerLock sync.RWMutex // protects the consumer field from concurrent updates - consumers []p2p.GossipSubInvCtrlMsgNotifConsumer -} - -// DefaultGossipSubInspectorNotificationDistributor returns a new GossipSubInspectorNotifDistributor component with the default configuration. -func DefaultGossipSubInspectorNotificationDistributor(logger zerolog.Logger, opts ...queue.HeroStoreConfigOption) *GossipSubInspectorNotifDistributor { - cfg := &queue.HeroStoreConfig{ - SizeLimit: DefaultGossipSubInspectorNotificationQueueCacheSize, - Collector: metrics.NewNoopCollector(), - } - - for _, opt := range opts { - opt(cfg) - } - - store := queue.NewHeroStore(cfg.SizeLimit, logger, cfg.Collector) - return NewGossipSubInspectorNotificationDistributor(logger, store) -} - -// NewGossipSubInspectorNotificationDistributor returns a new GossipSubInspectorNotifDistributor component. -// It takes a message store to store the notifications in memory and process them asynchronously. -func NewGossipSubInspectorNotificationDistributor(log zerolog.Logger, store engine.MessageStore) *GossipSubInspectorNotifDistributor { - lg := log.With().Str("component", "gossipsub_rpc_inspector_distributor").Logger() - - d := &GossipSubInspectorNotifDistributor{ - logger: lg, - } - - d.cm = cm.Build() - d.Component = d.cm - - return d -} - -// Distribute distributes the gossipsub rpc inspector notification to all registered consumers. -// The distribution is done asynchronously and non-blocking. The notification is added to a queue and processed by a worker pool. -// DistributeEvent in this implementation does not return an error, but it logs a warning if the queue is full. -func (g *GossipSubInspectorNotifDistributor) Distribute(notification *p2p.InvCtrlMsgNotif) error { - -} - -// AddConsumer adds a consumer to the distributor. The consumer will be called when 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 (g *GossipSubInspectorNotifDistributor) AddConsumer(consumer p2p.GossipSubInvCtrlMsgNotifConsumer) { - g.consumerLock.Lock() - defer g.consumerLock.Unlock() - - g.consumers = append(g.consumers, consumer) -} - -// distribute calls the ConsumeEvent method of all registered consumers. It is called by the workers of the worker pool. -// It is concurrency safe and can be called concurrently by multiple workers. However, the consumers may be blocking -// on the ConsumeEvent method. -func (g *GossipSubInspectorNotifDistributor) distribute(notification *p2p.InvCtrlMsgNotif) error { - g.consumerLock.RLock() - defer g.consumerLock.RUnlock() - - g.logger.Trace().Msg("distributing gossipsub rpc inspector notification") - for _, consumer := range g.consumers { - consumer.OnInvalidControlMessageNotification(notification) - } - g.logger.Trace().Msg("gossipsub rpc inspector notification distributed") - - return nil -} diff --git a/network/p2p/distributor/gossipsub_inspector_test.go b/network/p2p/distributor/gossipsub_inspector_test.go deleted file mode 100644 index 43d26d8fc26..00000000000 --- a/network/p2p/distributor/gossipsub_inspector_test.go +++ /dev/null @@ -1,101 +0,0 @@ -package distributor_test - -import ( - "context" - "fmt" - "math/rand" - "sync" - "testing" - "time" - - "github.com/libp2p/go-libp2p/core/peer" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - - "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/network/p2p" - "github.com/onflow/flow-go/network/p2p/distributor" - p2pmsg "github.com/onflow/flow-go/network/p2p/message" - mockp2p "github.com/onflow/flow-go/network/p2p/mock" - "github.com/onflow/flow-go/utils/unittest" -) - -// TestGossipSubInspectorNotification tests the GossipSub inspector notification 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. -func TestGossipSubInspectorNotification(t *testing.T) { - g := distributor.DefaultGossipSubInspectorNotificationDistributor(unittest.Logger()) - - c1 := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) - c2 := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) - - g.AddConsumer(c1) - g.AddConsumer(c2) - - tt := invalidControlMessageNotificationListFixture(t, 100) - - c1Done := sync.WaitGroup{} - c1Done.Add(len(tt)) - c1Seen := unittest.NewProtectedMap[peer.ID, struct{}]() - c1.On("OnInvalidControlMessageNotification", mock.Anything).Run(func(args mock.Arguments) { - notification, ok := args.Get(0).(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - - require.Contains(t, tt, notification) - - // ensure consumer see each peer once - require.False(t, c1Seen.Has(notification.PeerID)) - c1Seen.Add(notification.PeerID, struct{}{}) - - c1Done.Done() - }).Return() - - c2Done := sync.WaitGroup{} - c2Done.Add(len(tt)) - c2Seen := unittest.NewProtectedMap[peer.ID, struct{}]() - c2.On("OnInvalidControlMessageNotification", mock.Anything).Run(func(args mock.Arguments) { - notification, ok := args.Get(0).(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - - require.Contains(t, tt, notification) - // ensure consumer see each peer once - require.False(t, c2Seen.Has(notification.PeerID)) - c2Seen.Add(notification.PeerID, struct{}{}) - - c2Done.Done() - }).Return() - - cancelCtx, cancel := context.WithCancel(context.Background()) - defer cancel() - ctx, _ := irrecoverable.WithSignaler(cancelCtx) - g.Start(ctx) - - unittest.RequireCloseBefore(t, g.Ready(), 100*time.Millisecond, "could not start distributor") - - for i := 0; i < len(tt); i++ { - go func(i int) { - require.NoError(t, g.Distribute(tt[i])) - }(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, g.Done(), 100*time.Millisecond, "could not stop distributor") -} - -func invalidControlMessageNotificationListFixture(t *testing.T, n int) []*p2p.InvCtrlMsgNotif { - list := make([]*p2p.InvCtrlMsgNotif, n) - for i := 0; i < n; i++ { - list[i] = invalidControlMessageNotificationFixture(t) - } - return list -} - -func invalidControlMessageNotificationFixture(t *testing.T) *p2p.InvCtrlMsgNotif { - return &p2p.InvCtrlMsgNotif{ - PeerID: unittest.PeerIdFixture(t), - MsgType: []p2pmsg.ControlMessageType{p2pmsg.CtrlMsgGraft, p2pmsg.CtrlMsgPrune, p2pmsg.CtrlMsgIHave, p2pmsg.CtrlMsgIWant}[rand.Intn(4)], - Error: fmt.Errorf("this is an error"), - } -} diff --git a/network/p2p/mock/gossip_sub_inspector_notif_distributor.go b/network/p2p/mock/gossip_sub_inspector_notif_distributor.go deleted file mode 100644 index b378c9fac2b..00000000000 --- a/network/p2p/mock/gossip_sub_inspector_notif_distributor.go +++ /dev/null @@ -1,86 +0,0 @@ -// Code generated by mockery v2.21.4. DO NOT EDIT. - -package mockp2p - -import ( - irrecoverable "github.com/onflow/flow-go/module/irrecoverable" - mock "github.com/stretchr/testify/mock" - - p2p "github.com/onflow/flow-go/network/p2p" -) - -// GossipSubInspectorNotifDistributor is an autogenerated mock type for the GossipSubInspectorNotifDistributor type -type GossipSubInspectorNotifDistributor struct { - mock.Mock -} - -// AddConsumer provides a mock function with given fields: _a0 -func (_m *GossipSubInspectorNotifDistributor) AddConsumer(_a0 p2p.GossipSubInvCtrlMsgNotifConsumer) { - _m.Called(_a0) -} - -// Distribute provides a mock function with given fields: notification -func (_m *GossipSubInspectorNotifDistributor) Distribute(notification *p2p.InvCtrlMsgNotif) error { - ret := _m.Called(notification) - - var r0 error - if rf, ok := ret.Get(0).(func(*p2p.InvCtrlMsgNotif) error); ok { - r0 = rf(notification) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Done provides a mock function with given fields: -func (_m *GossipSubInspectorNotifDistributor) 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 *GossipSubInspectorNotifDistributor) 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 *GossipSubInspectorNotifDistributor) Start(_a0 irrecoverable.SignalerContext) { - _m.Called(_a0) -} - -type mockConstructorTestingTNewGossipSubInspectorNotifDistributor interface { - mock.TestingT - Cleanup(func()) -} - -// NewGossipSubInspectorNotifDistributor creates a new instance of GossipSubInspectorNotifDistributor. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewGossipSubInspectorNotifDistributor(t mockConstructorTestingTNewGossipSubInspectorNotifDistributor) *GossipSubInspectorNotifDistributor { - mock := &GossipSubInspectorNotifDistributor{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/network/p2p/mock/gossip_sub_inspector_notification_distributor.go b/network/p2p/mock/gossip_sub_inspector_notification_distributor.go deleted file mode 100644 index f44f7a2c480..00000000000 --- a/network/p2p/mock/gossip_sub_inspector_notification_distributor.go +++ /dev/null @@ -1,87 +0,0 @@ -// Code generated by mockery v2.21.4. DO NOT EDIT. - -package mockp2p - -import ( - mock "github.com/stretchr/testify/mock" - - irrecoverable "github.com/onflow/flow-go/module/irrecoverable" - - p2p "github.com/onflow/flow-go/network/p2p" -) - -// GossipSubInspectorNotificationDistributor is an autogenerated mock type for the GossipSubInspectorNotificationDistributor type -type GossipSubInspectorNotificationDistributor struct { - mock.Mock -} - -// AddConsumer provides a mock function with given fields: _a0 -func (_m *GossipSubInspectorNotificationDistributor) AddConsumer(_a0 p2p.GossipSubInvCtrlMsgNotifConsumer) { - _m.Called(_a0) -} - -// DistributeInvalidControlMessageNotification provides a mock function with given fields: notification -func (_m *GossipSubInspectorNotificationDistributor) Distribute(notification *p2p.InvCtrlMsgNotif) error { - ret := _m.Called(notification) - - var r0 error - if rf, ok := ret.Get(0).(func(*p2p.InvCtrlMsgNotif) error); ok { - r0 = rf(notification) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Done provides a mock function with given fields: -func (_m *GossipSubInspectorNotificationDistributor) 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 *GossipSubInspectorNotificationDistributor) 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 *GossipSubInspectorNotificationDistributor) Start(_a0 irrecoverable.SignalerContext) { - _m.Called(_a0) -} - -type mockConstructorTestingTNewGossipSubInspectorNotificationDistributor interface { - mock.TestingT - Cleanup(func()) -} - -// NewGossipSubInspectorNotificationDistributor creates a new instance of GossipSubInspectorNotificationDistributor. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewGossipSubInspectorNotificationDistributor(t mockConstructorTestingTNewGossipSubInspectorNotificationDistributor) *GossipSubInspectorNotificationDistributor { - mock := &GossipSubInspectorNotificationDistributor{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/network/p2p/mock/gossip_sub_inspector_suite.go b/network/p2p/mock/gossip_sub_inspector_suite.go deleted file mode 100644 index 90c7e5b15d7..00000000000 --- a/network/p2p/mock/gossip_sub_inspector_suite.go +++ /dev/null @@ -1,99 +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" - - peer "github.com/libp2p/go-libp2p/core/peer" - - pubsub "github.com/libp2p/go-libp2p-pubsub" -) - -// GossipSubInspectorSuite is an autogenerated mock type for the GossipSubInspectorSuite type -type GossipSubInspectorSuite struct { - mock.Mock -} - -// ActiveClustersChanged provides a mock function with given fields: _a0 -func (_m *GossipSubInspectorSuite) ActiveClustersChanged(_a0 flow.ChainIDList) { - _m.Called(_a0) -} - -// AddInvalidControlMessageConsumer provides a mock function with given fields: _a0 -func (_m *GossipSubInspectorSuite) AddInvalidControlMessageConsumer(_a0 p2p.GossipSubInvCtrlMsgNotifConsumer) { - _m.Called(_a0) -} - -// Done provides a mock function with given fields: -func (_m *GossipSubInspectorSuite) 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 -} - -// InspectFunc provides a mock function with given fields: -func (_m *GossipSubInspectorSuite) InspectFunc() func(peer.ID, *pubsub.RPC) error { - ret := _m.Called() - - var r0 func(peer.ID, *pubsub.RPC) error - if rf, ok := ret.Get(0).(func() func(peer.ID, *pubsub.RPC) error); ok { - r0 = rf() - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(func(peer.ID, *pubsub.RPC) error) - } - } - - return r0 -} - -// Ready provides a mock function with given fields: -func (_m *GossipSubInspectorSuite) 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 *GossipSubInspectorSuite) Start(_a0 irrecoverable.SignalerContext) { - _m.Called(_a0) -} - -type mockConstructorTestingTNewGossipSubInspectorSuite interface { - mock.TestingT - Cleanup(func()) -} - -// NewGossipSubInspectorSuite creates a new instance of GossipSubInspectorSuite. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewGossipSubInspectorSuite(t mockConstructorTestingTNewGossipSubInspectorSuite) *GossipSubInspectorSuite { - mock := &GossipSubInspectorSuite{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} From bb8ac360868b7ef6eaf77c4894b1db83a4e23246 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:21:04 -0800 Subject: [PATCH 04/52] renames a method on builder --- .../corruptlibp2p/pubsub_adapter_config.go | 2 +- network/p2p/mock/pub_sub_adapter_config.go | 2 +- network/p2p/node/gossipSubAdapterConfig.go | 22 +++++++++---------- network/p2p/pubsub.go | 2 +- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/insecure/corruptlibp2p/pubsub_adapter_config.go b/insecure/corruptlibp2p/pubsub_adapter_config.go index 1bae78dd872..d90aa38ce5d 100644 --- a/insecure/corruptlibp2p/pubsub_adapter_config.go +++ b/insecure/corruptlibp2p/pubsub_adapter_config.go @@ -153,7 +153,7 @@ func (c *CorruptPubSubAdapterConfig) ScoreTracer() p2p.PeerScoreTracer { return c.scoreTracer } -func (c *CorruptPubSubAdapterConfig) WithInspectorSuite(_ p2p.GossipSubInspectorSuite) { +func (c *CorruptPubSubAdapterConfig) WithRpcInspector(_ p2p.GossipSubInspectorSuite) { // CorruptPubSub does not support inspector suite. This is a no-op. } diff --git a/network/p2p/mock/pub_sub_adapter_config.go b/network/p2p/mock/pub_sub_adapter_config.go index 113ef45a163..db232f524b0 100644 --- a/network/p2p/mock/pub_sub_adapter_config.go +++ b/network/p2p/mock/pub_sub_adapter_config.go @@ -15,7 +15,7 @@ type PubSubAdapterConfig struct { } // WithInspectorSuite provides a mock function with given fields: _a0 -func (_m *PubSubAdapterConfig) WithInspectorSuite(_a0 p2p.GossipSubInspectorSuite) { +func (_m *PubSubAdapterConfig) WithRpcInspector(_a0 p2p.GossipSubInspectorSuite) { _m.Called(_a0) } diff --git a/network/p2p/node/gossipSubAdapterConfig.go b/network/p2p/node/gossipSubAdapterConfig.go index f4069930612..b73d207297a 100644 --- a/network/p2p/node/gossipSubAdapterConfig.go +++ b/network/p2p/node/gossipSubAdapterConfig.go @@ -14,11 +14,11 @@ import ( // GossipSubAdapterConfig is a wrapper around libp2p pubsub options that // implements the PubSubAdapterConfig interface for the Flow network. type GossipSubAdapterConfig struct { - options []pubsub.Option - scoreTracer p2p.PeerScoreTracer - scoreOption p2p.ScoreOptionBuilder - pubsubTracer p2p.PubSubTracer - inspectorSuite p2p.GossipSubInspectorSuite // currently only used to manage the lifecycle. + options []pubsub.Option + scoreTracer p2p.PeerScoreTracer + scoreOption p2p.ScoreOptionBuilder + pubsubTracer p2p.PubSubTracer + inspector p2p.GossipSubRPCInspector // currently only used to manage the lifecycle. } var _ p2p.PubSubAdapterConfig = (*GossipSubAdapterConfig)(nil) @@ -82,9 +82,9 @@ func (g *GossipSubAdapterConfig) WithMessageIdFunction(f func([]byte) string) { // - suite: the inspector suite to use // Returns: // -None -func (g *GossipSubAdapterConfig) WithInspectorSuite(suite p2p.GossipSubInspectorSuite) { - g.options = append(g.options, pubsub.WithAppSpecificRpcInspector(suite.InspectFunc())) - g.inspectorSuite = suite +func (g *GossipSubAdapterConfig) WithRpcInspector(inspector p2p.GossipSubMsgValidationRpcInspector) { + g.options = append(g.options, pubsub.WithAppSpecificRpcInspector(inspector.Inspect)) + g.inspector = inspector } // WithTracer adds a tracer option to the config. @@ -120,15 +120,15 @@ func (g *GossipSubAdapterConfig) ScoringComponent() component.Component { return g.scoreOption } -// InspectorSuiteComponent returns the component that manages the lifecycle of the inspector suite. +// RpcInspectorComponent returns the component that manages the lifecycle of the inspector suite. // This is used to start and stop the inspector suite by the PubSubAdapter. // Args: // - None // // Returns: // - component.Component: the component that manages the lifecycle of the inspector suite. -func (g *GossipSubAdapterConfig) InspectorSuiteComponent() component.Component { - return g.inspectorSuite +func (g *GossipSubAdapterConfig) RpcInspectorComponent() component.Component { + return g.inspector } // WithScoreTracer sets the tracer for the peer score. diff --git a/network/p2p/pubsub.go b/network/p2p/pubsub.go index 99ba5dfdf38..50c8c3548c1 100644 --- a/network/p2p/pubsub.go +++ b/network/p2p/pubsub.go @@ -83,7 +83,7 @@ type PubSubAdapterConfig interface { // WithScoreTracer sets the tracer for the underlying pubsub score implementation. // This is used to expose the local scoring table of the GossipSub node to its higher level components. WithScoreTracer(tracer PeerScoreTracer) - WithInspectorSuite(GossipSubInspectorSuite) + WithRpcInspector(GossipSubMsgValidationRpcInspector) } // GossipSubRPCInspector app specific RPC inspector used to inspect and validate incoming RPC messages before they are processed by libp2p. From 308b7802ff4b9674617618f5ff55310b7ce8bec3 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:21:22 -0800 Subject: [PATCH 05/52] removes inspector suite factory func --- network/p2p/builder.go | 39 --------------------------------------- 1 file changed, 39 deletions(-) diff --git a/network/p2p/builder.go b/network/p2p/builder.go index cbc71475511..9ee4fd58cba 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -12,13 +12,8 @@ import ( madns "github.com/multiformats/go-multiaddr-dns" "github.com/rs/zerolog" - "github.com/onflow/flow-go/model/flow" - "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/metrics" - flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" - p2pconfig "github.com/onflow/flow-go/network/p2p/config" ) type GossipSubFactoryFunc func(context.Context, zerolog.Logger, host.Host, PubSubAdapterConfig, CollectionClusterChangesConsumer) (PubSubAdapter, error) @@ -65,13 +60,6 @@ type GossipSubBuilder interface { // If the routing system has already been set, a fatal error is logged. SetRoutingSystem(routing.Routing) - // OverrideDefaultRpcInspectorSuiteFactory overrides the default RPC inspector suite factory of the builder. - // A default RPC inspector suite factory is provided by the node. This function overrides the default factory. - // The purpose of override is to allow the node to provide a custom RPC inspector suite factory for sake of testing - // or experimentation. - // It is NOT recommended to override the default RPC inspector suite factory in production unless you know what you are doing. - OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorSuiteFactoryFunc) - // Build creates a new GossipSub pubsub system. // It returns the newly created GossipSub pubsub system and any errors encountered during its creation. // @@ -85,32 +73,6 @@ type GossipSubBuilder interface { Build(irrecoverable.SignalerContext) (PubSubAdapter, error) } -// GossipSubRpcInspectorSuiteFactoryFunc is a function that creates a new RPC inspector suite. It is used to create -// RPC inspectors for the gossipsub protocol. The RPC inspectors are used to inspect and validate -// incoming RPC messages before they are processed by the gossipsub protocol. -// Args: -// - logger: logger to use -// - sporkID: spork ID of the node -// - cfg: configuration for the RPC inspectors -// - metrics: metrics to use for the RPC inspectors -// - heroCacheMetricsFactory: metrics factory for the hero cache -// - networkingType: networking type of the node, i.e., public or private -// - identityProvider: identity provider of the node -// Returns: -// - p2p.GossipSubInspectorSuite: new RPC inspector suite -// - error: error if any, any returned error is irrecoverable. -type GossipSubRpcInspectorSuiteFactoryFunc func( - irrecoverable.SignalerContext, - zerolog.Logger, - flow.Identifier, - *p2pconfig.RpcInspectorParameters, - module.GossipSubMetrics, - metrics.HeroCacheMetricsFactory, - flownet.NetworkingType, - module.IdentityProvider, - func() TopicProvider, -) (GossipSubInspectorSuite, error) - // NodeBuilder is a builder pattern for creating a libp2p Node instance. type NodeBuilder interface { SetBasicResolver(madns.BasicResolver) NodeBuilder @@ -142,7 +104,6 @@ type NodeBuilder interface { // none OverrideNodeConstructor(NodeConstructor) NodeBuilder SetGossipSubFactory(GossipSubFactoryFunc, GossipSubAdapterConfigFunc) NodeBuilder - OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorSuiteFactoryFunc) NodeBuilder Build() (LibP2PNode, error) } From f1571b4051225c0a717b2416b16c69ed7996a354 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:21:39 -0800 Subject: [PATCH 06/52] removes distributor and inspector suite interfaces --- network/p2p/consumers.go | 37 +------------------------------------ 1 file changed, 1 insertion(+), 36 deletions(-) diff --git a/network/p2p/consumers.go b/network/p2p/consumers.go index f079a3864af..25ccf2678f5 100644 --- a/network/p2p/consumers.go +++ b/network/p2p/consumers.go @@ -1,29 +1,11 @@ package p2p import ( - pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/peer" - "github.com/onflow/flow-go/module/component" p2pmsg "github.com/onflow/flow-go/network/p2p/message" ) -// 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. -type GossipSubInspectorNotifDistributor interface { - component.Component - // Distribute 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. - Distribute(notification *InvCtrlMsgNotif) 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(GossipSubInvCtrlMsgNotifConsumer) -} - // CtrlMsgTopicType represents the type of the topic within a control message. type CtrlMsgTopicType uint64 @@ -90,23 +72,6 @@ func NewInvalidControlMessageNotification(peerID peer.ID, ctlMsgType p2pmsg.Cont type GossipSubInvCtrlMsgNotifConsumer interface { // OnInvalidControlMessageNotification is called when a new invalid control message notification is distributed. // Any error on consuming event must handle internally. - // The implementation must be concurrency safe, but can be blocking. + // The implementation must be concurrency safe and non-blocking. OnInvalidControlMessageNotification(*InvCtrlMsgNotif) } - -// GossipSubInspectorSuite is the interface for the GossipSub inspector suite. -// It encapsulates the rpc inspectors and the notification distributors. -type GossipSubInspectorSuite interface { - component.Component - CollectionClusterChangesConsumer - // InspectFunc returns the inspect function that is used to inspect the gossipsub rpc messages. - // This function follows a dependency injection pattern, where the inspect function is injected into the gossipsu, and - // is called whenever a gossipsub rpc message is received. - InspectFunc() func(peer.ID, *pubsub.RPC) error - - // AddInvalidControlMessageConsumer adds a consumer to the invalid control message notification distributor. - // This consumer is notified when a misbehaving peer regarding gossipsub control messages is detected. This follows a pub/sub - // pattern where the consumer is notified when a new notification is published. - // A consumer is only notified once for each notification, and only receives notifications that were published after it was added. - AddInvalidControlMessageConsumer(GossipSubInvCtrlMsgNotifConsumer) -} From 5e344cad9fdcbefa5155d9b07674ce2f1e516811 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:22:02 -0800 Subject: [PATCH 07/52] adds notification consumer to score option --- network/p2p/scoring/score_option.go | 39 +++++++++++++---------------- 1 file changed, 17 insertions(+), 22 deletions(-) diff --git a/network/p2p/scoring/score_option.go b/network/p2p/scoring/score_option.go index c0377974a14..6de94b1d07c 100644 --- a/network/p2p/scoring/score_option.go +++ b/network/p2p/scoring/score_option.go @@ -33,19 +33,19 @@ type ScoreOption struct { defaultTopicScoreParams *pubsub.TopicScoreParams validator p2p.SubscriptionValidator appScoreFunc func(peer.ID) float64 + appScoreRegistry *GossipSubAppSpecificScoreRegistry } type ScoreOptionConfig struct { - logger zerolog.Logger - params p2pconfig.ScoringParameters - provider module.IdentityProvider - heroCacheMetricsFactory metrics.HeroCacheMetricsFactory - appScoreFunc func(peer.ID) float64 - topicParams []func(map[string]*pubsub.TopicScoreParams) - registerNotificationConsumerFunc func(p2p.GossipSubInvCtrlMsgNotifConsumer) - getDuplicateMessageCount func(id peer.ID) float64 - scoringRegistryMetricsCollector module.GossipSubScoringRegistryMetrics - networkingType network.NetworkingType + logger zerolog.Logger + params p2pconfig.ScoringParameters + provider module.IdentityProvider + heroCacheMetricsFactory metrics.HeroCacheMetricsFactory + appScoreFunc func(peer.ID) float64 + topicParams []func(map[string]*pubsub.TopicScoreParams) + getDuplicateMessageCount func(id peer.ID) float64 + scoringRegistryMetricsCollector module.GossipSubScoringRegistryMetrics + networkingType network.NetworkingType } // NewScoreOptionConfig creates a new configuration for the GossipSub peer scoring option. @@ -93,13 +93,6 @@ func (c *ScoreOptionConfig) OverrideTopicScoreParams(topic channels.Topic, topic }) } -// SetRegisterNotificationConsumerFunc sets the function to register the notification consumer for the penalty option. -// ScoreOption uses this function to register the notification consumer for the pubsub system so that it can receive -// notifications of invalid control messages. -func (c *ScoreOptionConfig) SetRegisterNotificationConsumerFunc(f func(p2p.GossipSubInvCtrlMsgNotifConsumer)) { - c.registerNotificationConsumerFunc = f -} - // NewScoreOption creates a new penalty option with the given configuration. func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) (*ScoreOption, error) { throttledSampler := logging.BurstSampler(cfg.params.PeerScoring.Protocol.MaxDebugLogs, time.Second) @@ -210,11 +203,6 @@ func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) ( Msg("decay interval is overridden, should never happen in production") } - // registers the score registry as the consumer of the invalid control message notifications - if cfg.registerNotificationConsumerFunc != nil { - cfg.registerNotificationConsumerFunc(scoreRegistry) - } - s.peerScoreParams.AppSpecificScore = s.appScoreFunc // apply the topic penalty parameters if any. @@ -276,3 +264,10 @@ func (s *ScoreOption) TopicScoreParams(topic *pubsub.Topic) *pubsub.TopicScorePa } return params } + +// OnInvalidControlMessageNotification is called when a new invalid control message notification is distributed. +// Any error on consuming event must handle internally. +// The implementation must be concurrency safe and non-blocking. +func (s *ScoreOption) OnInvalidControlMessageNotification(notif *p2p.InvCtrlMsgNotif) { + s.appScoreRegistry.OnInvalidControlMessageNotification(notif) +} From 4a63d82b387badd418159614c6c982f293f976ae Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:22:31 -0800 Subject: [PATCH 08/52] moves worker pool to score registry --- network/p2p/scoring/registry.go | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index 4e69890e966..d54c98f4ce0 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -65,8 +65,8 @@ type GossipSubAppSpecificScoreRegistry struct { appScoreCache p2p.GossipSubApplicationSpecificScoreCache // appScoreUpdateWorkerPool is the worker pool for handling the application specific score update of peers in a non-blocking way. - appScoreUpdateWorkerPool *worker.Pool[peer.ID] - workerPool *worker.Pool[*p2p.InvCtrlMsgNotif] + appScoreUpdateWorkerPool *worker.Pool[peer.ID] + invCtrlMsgNotifWorkerPool *worker.Pool[*p2p.InvCtrlMsgNotif] appSpecificScoreParams p2pconfig.ApplicationSpecificScoreParameters duplicateMessageThreshold float64 @@ -140,9 +140,6 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis } lg := config.Logger.With().Str("module", "app_score_registry").Logger() - appSpecificScoreHS := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, - lg.With().Str("component", "app_specific_score_update").Logger(), - metrics.GossipSubAppSpecificScoreUpdateQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) reg := &GossipSubAppSpecificScoreRegistry{ logger: config.Logger.With().Str("module", "app_score_registry").Logger(), @@ -160,12 +157,16 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis collector: config.Collector, } + appSpecificScoreHS := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, + lg.With().Str("component", "app_specific_score_update").Logger(), + metrics.GossipSubAppSpecificScoreUpdateQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) reg.appScoreUpdateWorkerPool = worker.NewWorkerPoolBuilder[peer.ID](lg.With().Str("component", "app_specific_score_update_worker_pool").Logger(), appSpecificScoreHS, reg.processAppSpecificScoreUpdateWork).Build() - pool := worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, appSpecificScoreHS, reg.handleMisbehaviourReport).Build() - reg.workerPool = pool + invalidCtrlMsgNotifHS := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, lg.With().Str("component", "invalid_control_message_notification_queue").Logger(), + metrics.RpcInspectorNotificationQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) + reg.invCtrlMsgNotifWorkerPool = worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, invalidCtrlMsgNotifHS, reg.handleMisbehaviourReport).Build() builder := component.NewComponentManagerBuilder() builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { @@ -189,7 +190,7 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis } reg.silencePeriodStartTime = time.Now() ready() - }).AddWorker(pool.WorkerLogic()) // we must NOT have more than one worker for processing notifications; handling notifications are NOT idempotent. + }).AddWorker(reg.invCtrlMsgNotifWorkerPool.WorkerLogic()) // we must NOT have more than one worker for processing notifications; handling notifications are NOT idempotent. for i := 0; i < config.Parameters.ScoreUpdateWorkerNum; i++ { builder.AddWorker(reg.appScoreUpdateWorkerPool.WorkerLogic()) @@ -409,7 +410,7 @@ func (r *GossipSubAppSpecificScoreRegistry) duplicateMessagesPenalty(pid peer.ID // The implementation must be concurrency safe, but can be blocking. func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification(notification *p2p.InvCtrlMsgNotif) { lg := r.logger.With().Str("peer_id", p2plogging.PeerId(notification.PeerID)).Logger() - if ok := r.workerPool.Submit(notification); !ok { + if ok := r.invCtrlMsgNotifWorkerPool.Submit(notification); !ok { // we use a queue with a fixed size, so this can happen when queue is full or when the notification is duplicate. // TODO: we have to add a metric for this case. lg.Warn().Msg("gossipsub rpc inspector notification queue is full or notification is duplicate, discarding notification") From e7acb98ddf791474a348703427bfa3bc99a89ecc Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:23:01 -0800 Subject: [PATCH 09/52] refactors gossipsub builder to use the inspector than suite --- .../p2p/builder/gossipsub/gossipSubBuilder.go | 105 +++++------------- 1 file changed, 27 insertions(+), 78 deletions(-) diff --git a/network/p2p/builder/gossipsub/gossipSubBuilder.go b/network/p2p/builder/gossipsub/gossipSubBuilder.go index fd35c884b43..71558ddbefc 100644 --- a/network/p2p/builder/gossipsub/gossipSubBuilder.go +++ b/network/p2p/builder/gossipsub/gossipSubBuilder.go @@ -12,14 +12,10 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" - "github.com/onflow/flow-go/module/mempool/queue" - "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" p2pbuilderconfig "github.com/onflow/flow-go/network/p2p/builder/config" - inspectorbuilder "github.com/onflow/flow-go/network/p2p/builder/inspector" p2pconfig "github.com/onflow/flow-go/network/p2p/config" - "github.com/onflow/flow-go/network/p2p/distributor" "github.com/onflow/flow-go/network/p2p/inspector/validation" p2pnode "github.com/onflow/flow-go/network/p2p/node" "github.com/onflow/flow-go/network/p2p/scoring" @@ -40,12 +36,11 @@ type Builder struct { gossipSubConfigFunc p2p.GossipSubAdapterConfigFunc // gossipSubTracer is a callback interface that is called by the gossipsub implementation upon // certain events. Currently, we use it to log and observe the local mesh of the node. - gossipSubTracer p2p.PubSubTracer - scoreOptionConfig *scoring.ScoreOptionConfig - idProvider module.IdentityProvider - routingSystem routing.Routing - rpcInspectorSuiteFactory p2p.GossipSubRpcInspectorSuiteFactoryFunc - gossipSubCfg *p2pconfig.GossipSubParameters + gossipSubTracer p2p.PubSubTracer + scoreOptionConfig *scoring.ScoreOptionConfig + idProvider module.IdentityProvider + routingSystem routing.Routing + gossipSubCfg *p2pconfig.GossipSubParameters } var _ p2p.GossipSubBuilder = (*Builder)(nil) @@ -128,13 +123,6 @@ func (g *Builder) SetRoutingSystem(routingSystem routing.Routing) { g.routingSystem = routingSystem } -// OverrideDefaultRpcInspectorSuiteFactory overrides the default rpc inspector suite factory. -// Note: this function should only be used for testing purposes. Never override the default rpc inspector suite factory unless you know what you are doing. -func (g *Builder) OverrideDefaultRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorSuiteFactoryFunc) { - g.logger.Warn().Msg("overriding default rpc inspector suite factory") - g.rpcInspectorSuiteFactory = factory -} - // NewGossipSubBuilder returns a new gossipsub builder. // Args: // - logger: the logger of the node. @@ -191,9 +179,8 @@ func NewGossipSubBuilder(logger zerolog.Logger, meshTracer.DuplicateMessageCount, networkType, ), - rpcInspectorSuiteFactory: defaultInspectorSuite(meshTracer), - gossipSubTracer: meshTracer, - gossipSubCfg: gossipSubCfg, + gossipSubTracer: meshTracer, + gossipSubCfg: gossipSubCfg, } return b @@ -215,45 +202,6 @@ func defaultGossipSubAdapterConfig() p2p.GossipSubAdapterConfigFunc { } } -// defaultInspectorSuite returns the default inspector suite factory function. It is used to create the default inspector suite. -// Inspector suite is utilized to inspect the incoming gossipsub rpc messages from different perspectives. -// Note: always use the default inspector suite factory function to create the inspector suite (unless you know what you are doing). -// todo: this function can be simplified. -func defaultInspectorSuite(rpcTracker p2p.RpcControlTracking) p2p.GossipSubRpcInspectorSuiteFactoryFunc { - return func(ctx irrecoverable.SignalerContext, - logger zerolog.Logger, - sporkId flow.Identifier, - inspectorCfg *p2pconfig.RpcInspectorParameters, - gossipSubMetrics module.GossipSubMetrics, - heroCacheMetricsFactory metrics.HeroCacheMetricsFactory, - networkType network.NetworkingType, - idProvider module.IdentityProvider, - topicProvider func() p2p.TopicProvider) (p2p.GossipSubInspectorSuite, error) { - - notificationDistributor := distributor.DefaultGossipSubInspectorNotificationDistributor(logger, []queue.HeroStoreConfigOption{ - queue.WithHeroStoreSizeLimit(inspectorCfg.NotificationCacheSize), - queue.WithHeroStoreCollector(metrics.RpcInspectorNotificationQueueMetricFactory(heroCacheMetricsFactory, networkType))}...) - - params := &validation.InspectorParams{ - Logger: logger, - SporkID: sporkId, - Config: &inspectorCfg.Validation, - Distributor: notificationDistributor, - HeroCacheMetricsFactory: heroCacheMetricsFactory, - IdProvider: idProvider, - InspectorMetrics: gossipSubMetrics, - RpcTracker: rpcTracker, - NetworkingType: networkType, - TopicOracle: topicProvider, - } - rpcValidationInspector, err := validation.NewControlMsgValidationInspector(params) - if err != nil { - return nil, fmt.Errorf("failed to create new control message valiadation inspector: %w", err) - } - return inspectorbuilder.NewGossipSubInspectorSuite(rpcValidationInspector, notificationDistributor), nil - } -} - // Build creates a new GossipSub pubsub system. // It returns the newly created GossipSub pubsub system and any errors encountered during its creation. // Arguments: @@ -282,22 +230,6 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e gossipSubConfigs.WithSubscriptionFilter(g.subscriptionFilter) } - inspectorSuite, err := g.rpcInspectorSuiteFactory(ctx, - g.logger, - g.sporkId, - &g.gossipSubCfg.RpcInspector, - g.metricsCfg.Metrics, - g.metricsCfg.HeroCacheFactory, - g.networkType, - g.idProvider, - func() p2p.TopicProvider { - return gossipSub - }) - if err != nil { - return nil, fmt.Errorf("could not create gossipsub inspector suite: %w", err) - } - gossipSubConfigs.WithInspectorSuite(inspectorSuite) - var scoreOpt *scoring.ScoreOption var scoreTracer p2p.PeerScoreTracer // currently, peer scoring is not supported for public networks. @@ -319,8 +251,6 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e if err != nil { return nil, fmt.Errorf("could not create subscription provider: %w", err) } - - g.scoreOptionConfig.SetRegisterNotificationConsumerFunc(inspectorSuite.AddInvalidControlMessageConsumer) scoreOpt, err = scoring.NewScoreOption(g.scoreOptionConfig, subscriptionProvider) if err != nil { return nil, fmt.Errorf("could not create gossipsub score option: %w", err) @@ -337,6 +267,25 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e Msg("gossipsub peer scoring is disabled") } + rpcValidationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ + Logger: g.logger, + SporkID: g.sporkId, + Config: &g.gossipSubCfg.RpcInspector.Validation, + HeroCacheMetricsFactory: g.metricsCfg.HeroCacheFactory, + IdProvider: g.idProvider, + InspectorMetrics: g.metricsCfg.Metrics, + RpcTracker: g.gossipSubTracer, + NetworkingType: g.networkType, + InvalidControlMessageNotificationConsumer: scoreOpt, + TopicOracle: func() p2p.TopicProvider { + return gossipSub + }, + }) + if err != nil { + return nil, fmt.Errorf("failed to create new rpc valiadation inspector: %w", err) + } + gossipSubConfigs.WithRpcInspector(rpcValidationInspector) + if g.gossipSubTracer != nil { gossipSubConfigs.WithTracer(g.gossipSubTracer) } @@ -345,7 +294,7 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e return nil, fmt.Errorf("could not create gossipsub: host is nil") } - gossipSub, err = g.gossipSubFactory(ctx, g.logger, g.h, gossipSubConfigs, inspectorSuite) + gossipSub, err = g.gossipSubFactory(ctx, g.logger, g.h, gossipSubConfigs, rpcValidationInspector) if err != nil { return nil, fmt.Errorf("could not create gossipsub: %w", err) } From e5e86d6a8e5062f1bbc04eba74bd29a76d518ed0 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:23:32 -0800 Subject: [PATCH 10/52] adds invalid control message notification consumer as a base dependency --- .../control_message_validation_inspector.go | 59 +++++++------------ 1 file changed, 21 insertions(+), 38 deletions(-) diff --git a/network/p2p/inspector/validation/control_message_validation_inspector.go b/network/p2p/inspector/validation/control_message_validation_inspector.go index 705e709a8cb..20bd05f6830 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector.go @@ -62,8 +62,6 @@ type ControlMsgValidationInspector struct { metrics module.GossipSubRpcValidationInspectorMetrics // config control message validation configurations. config *p2pconfig.RpcValidationInspector - // distributor used to disseminate invalid RPC message notifications. - distributor p2p.GossipSubInspectorNotifDistributor // workerPool queue that stores *InspectRPCRequest that will be processed by component workers. workerPool *worker.Pool[*InspectRPCRequest] // tracker is a map that associates the hash of a peer's ID with the @@ -81,6 +79,10 @@ type ControlMsgValidationInspector struct { networkingType network.NetworkingType // topicOracle callback used to retrieve the current subscribed topics of the libp2p node. topicOracle func() p2p.TopicProvider + // notificationConsumer the consumer that will be notified when a misbehavior is detected upon inspection of an RPC. + // For each RPC, at most one notification is sent to the consumer. + // Each notification acts as a penalty to the peer's score. + notificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer } type InspectorParams struct { @@ -90,8 +92,6 @@ type InspectorParams struct { SporkID flow.Identifier `validate:"required"` // Config inspector configuration. Config *p2pconfig.RpcValidationInspector `validate:"required"` - // Distributor gossipsub inspector notification distributor. - Distributor p2p.GossipSubInspectorNotifDistributor `validate:"required"` // HeroCacheMetricsFactory the metrics factory. HeroCacheMetricsFactory metrics.HeroCacheMetricsFactory `validate:"required"` // IdProvider identity provider is used to get the flow identifier for a peer. @@ -105,6 +105,11 @@ type InspectorParams struct { // TopicOracle callback used to retrieve the current subscribed topics of the libp2p node. // It is set as a callback to avoid circular dependencies between the topic oracle and the inspector. TopicOracle func() p2p.TopicProvider `validate:"required"` + + // InvalidControlMessageNotificationConsumer the consumer that will be notified when a misbehavior is detected upon inspection of an RPC. + // For each RPC, at most one notification is sent to the consumer. + // Each notification acts as a penalty to the peer's score. + InvalidControlMessageNotificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer `validate:"required"` } var _ component.Component = (*ControlMsgValidationInspector)(nil) @@ -143,16 +148,16 @@ func NewControlMsgValidationInspector(params *InspectorParams) (*ControlMsgValid } c := &ControlMsgValidationInspector{ - logger: lg, - sporkID: params.SporkID, - config: params.Config, - distributor: params.Distributor, - tracker: clusterPrefixedTracker, - rpcTracker: params.RpcTracker, - idProvider: params.IdProvider, - metrics: params.InspectorMetrics, - networkingType: params.NetworkingType, - topicOracle: params.TopicOracle, + logger: lg, + sporkID: params.SporkID, + config: params.Config, + tracker: clusterPrefixedTracker, + rpcTracker: params.RpcTracker, + idProvider: params.IdProvider, + metrics: params.InspectorMetrics, + networkingType: params.NetworkingType, + topicOracle: params.TopicOracle, + notificationConsumer: params.InvalidControlMessageNotificationConsumer, } store := queue.NewHeroStore(params.Config.InspectionQueue.Size, params.Logger, inspectMsgQueueCacheCollector) @@ -162,22 +167,6 @@ func NewControlMsgValidationInspector(params *InspectorParams) (*ControlMsgValid c.workerPool = pool builder := component.NewComponentManagerBuilder() - builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - c.logger.Debug().Msg("starting rpc inspector distributor") - c.ctx = ctx - c.distributor.Start(ctx) - select { - case <-ctx.Done(): - c.logger.Debug().Msg("rpc inspector distributor startup aborted; context cancelled") - case <-c.distributor.Ready(): - c.logger.Debug().Msg("rpc inspector distributor started") - ready() - } - <-ctx.Done() - c.logger.Debug().Msg("rpc inspector distributor stopped") - <-c.distributor.Done() - c.logger.Debug().Msg("rpc inspector distributor shutdown complete") - }) for i := 0; i < c.config.InspectionQueue.NumberOfWorkers; i++ { builder.AddWorker(pool.WorkerLogic()) } @@ -1081,14 +1070,8 @@ func (c *ControlMsgValidationInspector) logAndDistributeAsyncInspectErrs(req *In c.metrics.OnUnstakedPeerInspectionFailed() lg.Warn().Msg("control message received from unstaked peer") default: - distErr := c.distributor.Distribute(p2p.NewInvalidControlMessageNotification(req.Peer, ctlMsgType, err, count, topicType)) - if distErr != nil { - lg.Error(). - Err(distErr). - Msg("failed to distribute invalid control message notification") - return - } - lg.Error().Msg("rpc control message async inspection failed") + c.notificationConsumer.OnInvalidControlMessageNotification(p2p.NewInvalidControlMessageNotification(req.Peer, ctlMsgType, err, count, topicType)) + lg.Error().Msg("rpc control message async inspection failed, notification sent") c.metrics.OnInvalidControlMessageNotificationSent() } } From 90c2bcd294e35777bcc87b08e4bcfd34c8bf7d10 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:23:46 -0800 Subject: [PATCH 11/52] removes overriding inspector suite func --- network/p2p/builder/libp2pNodeBuilder.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/network/p2p/builder/libp2pNodeBuilder.go b/network/p2p/builder/libp2pNodeBuilder.go index 7d9709ae457..a47a3eef5ca 100644 --- a/network/p2p/builder/libp2pNodeBuilder.go +++ b/network/p2p/builder/libp2pNodeBuilder.go @@ -180,11 +180,6 @@ func (builder *LibP2PNodeBuilder) OverrideNodeConstructor(f p2p.NodeConstructor) return builder } -func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorSuiteFactoryFunc) p2p.NodeBuilder { - builder.gossipSubBuilder.OverrideDefaultRpcInspectorSuiteFactory(factory) - return builder -} - // Build creates a new libp2p node using the configured options. func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { var opts []libp2p.Option From 7cde571d6373a28dbb61fec37abb49982b3971bc Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 8 Feb 2024 16:23:56 -0800 Subject: [PATCH 12/52] renames a method --- network/p2p/node/gossipSubAdapter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/p2p/node/gossipSubAdapter.go b/network/p2p/node/gossipSubAdapter.go index d1acf5af376..3ada14efd5c 100644 --- a/network/p2p/node/gossipSubAdapter.go +++ b/network/p2p/node/gossipSubAdapter.go @@ -112,7 +112,7 @@ func NewGossipSubAdapter(ctx context.Context, a.localMeshTracer = tracer } - if inspectorSuite := gossipSubConfig.InspectorSuiteComponent(); inspectorSuite != nil { + if inspectorSuite := gossipSubConfig.RpcInspectorComponent(); inspectorSuite != nil { builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { a.logger.Info().Msg("starting inspector suite") inspectorSuite.Start(ctx) From 77589bde1e174ec99e11b699d8ae25da9d536945 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 9 Feb 2024 09:51:34 -0800 Subject: [PATCH 13/52] generates mocks --- network/p2p/mock/gossip_sub_builder.go | 5 ----- network/p2p/mock/node_builder.go | 16 ---------------- network/p2p/mock/pub_sub_adapter_config.go | 10 +++++----- 3 files changed, 5 insertions(+), 26 deletions(-) diff --git a/network/p2p/mock/gossip_sub_builder.go b/network/p2p/mock/gossip_sub_builder.go index 13342243e3b..25e819f6d60 100644 --- a/network/p2p/mock/gossip_sub_builder.go +++ b/network/p2p/mock/gossip_sub_builder.go @@ -51,11 +51,6 @@ func (_m *GossipSubBuilder) EnableGossipSubScoringWithOverride(_a0 *p2p.PeerScor _m.Called(_a0) } -// OverrideDefaultRpcInspectorSuiteFactory provides a mock function with given fields: _a0 -func (_m *GossipSubBuilder) OverrideDefaultRpcInspectorSuiteFactory(_a0 p2p.GossipSubRpcInspectorSuiteFactoryFunc) { - _m.Called(_a0) -} - // SetGossipSubConfigFunc provides a mock function with given fields: _a0 func (_m *GossipSubBuilder) SetGossipSubConfigFunc(_a0 p2p.GossipSubAdapterConfigFunc) { _m.Called(_a0) diff --git a/network/p2p/mock/node_builder.go b/network/p2p/mock/node_builder.go index 3b10dcfb0c8..6fa12bae038 100644 --- a/network/p2p/mock/node_builder.go +++ b/network/p2p/mock/node_builder.go @@ -53,22 +53,6 @@ func (_m *NodeBuilder) Build() (p2p.LibP2PNode, error) { return r0, r1 } -// OverrideDefaultRpcInspectorSuiteFactory provides a mock function with given fields: _a0 -func (_m *NodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(_a0 p2p.GossipSubRpcInspectorSuiteFactoryFunc) p2p.NodeBuilder { - ret := _m.Called(_a0) - - var r0 p2p.NodeBuilder - if rf, ok := ret.Get(0).(func(p2p.GossipSubRpcInspectorSuiteFactoryFunc) p2p.NodeBuilder); ok { - r0 = rf(_a0) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(p2p.NodeBuilder) - } - } - - return r0 -} - // OverrideGossipSubScoringConfig provides a mock function with given fields: _a0 func (_m *NodeBuilder) OverrideGossipSubScoringConfig(_a0 *p2p.PeerScoringConfigOverride) p2p.NodeBuilder { ret := _m.Called(_a0) diff --git a/network/p2p/mock/pub_sub_adapter_config.go b/network/p2p/mock/pub_sub_adapter_config.go index db232f524b0..9db204914a9 100644 --- a/network/p2p/mock/pub_sub_adapter_config.go +++ b/network/p2p/mock/pub_sub_adapter_config.go @@ -14,11 +14,6 @@ type PubSubAdapterConfig struct { mock.Mock } -// WithInspectorSuite provides a mock function with given fields: _a0 -func (_m *PubSubAdapterConfig) WithRpcInspector(_a0 p2p.GossipSubInspectorSuite) { - _m.Called(_a0) -} - // WithMessageIdFunction provides a mock function with given fields: f func (_m *PubSubAdapterConfig) WithMessageIdFunction(f func([]byte) string) { _m.Called(f) @@ -29,6 +24,11 @@ func (_m *PubSubAdapterConfig) WithRoutingDiscovery(_a0 routing.ContentRouting) _m.Called(_a0) } +// WithRpcInspector provides a mock function with given fields: _a0 +func (_m *PubSubAdapterConfig) WithRpcInspector(_a0 p2p.GossipSubMsgValidationRpcInspector) { + _m.Called(_a0) +} + // WithScoreOption provides a mock function with given fields: _a0 func (_m *PubSubAdapterConfig) WithScoreOption(_a0 p2p.ScoreOptionBuilder) { _m.Called(_a0) From 006768c4150a4d28c2b474c5f61d4c5d3ba5bdd2 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 12 Feb 2024 14:42:18 -0800 Subject: [PATCH 14/52] Revert "removes overriding inspector suite func" This reverts commit 90c2bcd294e35777bcc87b08e4bcfd34c8bf7d10. --- network/p2p/builder/libp2pNodeBuilder.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/network/p2p/builder/libp2pNodeBuilder.go b/network/p2p/builder/libp2pNodeBuilder.go index a47a3eef5ca..7d9709ae457 100644 --- a/network/p2p/builder/libp2pNodeBuilder.go +++ b/network/p2p/builder/libp2pNodeBuilder.go @@ -180,6 +180,11 @@ func (builder *LibP2PNodeBuilder) OverrideNodeConstructor(f p2p.NodeConstructor) return builder } +func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorSuiteFactoryFunc) p2p.NodeBuilder { + builder.gossipSubBuilder.OverrideDefaultRpcInspectorSuiteFactory(factory) + return builder +} + // Build creates a new libp2p node using the configured options. func (builder *LibP2PNodeBuilder) Build() (p2p.LibP2PNode, error) { var opts []libp2p.Option From 7fde32dbd4d0a4c9e48f14792bc41aa8b65e8726 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 12 Feb 2024 14:42:48 -0800 Subject: [PATCH 15/52] Revert "removes inspector suite factory func" This reverts commit 308b7802ff4b9674617618f5ff55310b7ce8bec3. --- network/p2p/builder.go | 39 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/network/p2p/builder.go b/network/p2p/builder.go index 9ee4fd58cba..cbc71475511 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -12,8 +12,13 @@ import ( madns "github.com/multiformats/go-multiaddr-dns" "github.com/rs/zerolog" + "github.com/onflow/flow-go/model/flow" + "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/metrics" + flownet "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/channels" + p2pconfig "github.com/onflow/flow-go/network/p2p/config" ) type GossipSubFactoryFunc func(context.Context, zerolog.Logger, host.Host, PubSubAdapterConfig, CollectionClusterChangesConsumer) (PubSubAdapter, error) @@ -60,6 +65,13 @@ type GossipSubBuilder interface { // If the routing system has already been set, a fatal error is logged. SetRoutingSystem(routing.Routing) + // OverrideDefaultRpcInspectorSuiteFactory overrides the default RPC inspector suite factory of the builder. + // A default RPC inspector suite factory is provided by the node. This function overrides the default factory. + // The purpose of override is to allow the node to provide a custom RPC inspector suite factory for sake of testing + // or experimentation. + // It is NOT recommended to override the default RPC inspector suite factory in production unless you know what you are doing. + OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorSuiteFactoryFunc) + // Build creates a new GossipSub pubsub system. // It returns the newly created GossipSub pubsub system and any errors encountered during its creation. // @@ -73,6 +85,32 @@ type GossipSubBuilder interface { Build(irrecoverable.SignalerContext) (PubSubAdapter, error) } +// GossipSubRpcInspectorSuiteFactoryFunc is a function that creates a new RPC inspector suite. It is used to create +// RPC inspectors for the gossipsub protocol. The RPC inspectors are used to inspect and validate +// incoming RPC messages before they are processed by the gossipsub protocol. +// Args: +// - logger: logger to use +// - sporkID: spork ID of the node +// - cfg: configuration for the RPC inspectors +// - metrics: metrics to use for the RPC inspectors +// - heroCacheMetricsFactory: metrics factory for the hero cache +// - networkingType: networking type of the node, i.e., public or private +// - identityProvider: identity provider of the node +// Returns: +// - p2p.GossipSubInspectorSuite: new RPC inspector suite +// - error: error if any, any returned error is irrecoverable. +type GossipSubRpcInspectorSuiteFactoryFunc func( + irrecoverable.SignalerContext, + zerolog.Logger, + flow.Identifier, + *p2pconfig.RpcInspectorParameters, + module.GossipSubMetrics, + metrics.HeroCacheMetricsFactory, + flownet.NetworkingType, + module.IdentityProvider, + func() TopicProvider, +) (GossipSubInspectorSuite, error) + // NodeBuilder is a builder pattern for creating a libp2p Node instance. type NodeBuilder interface { SetBasicResolver(madns.BasicResolver) NodeBuilder @@ -104,6 +142,7 @@ type NodeBuilder interface { // none OverrideNodeConstructor(NodeConstructor) NodeBuilder SetGossipSubFactory(GossipSubFactoryFunc, GossipSubAdapterConfigFunc) NodeBuilder + OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorSuiteFactoryFunc) NodeBuilder Build() (LibP2PNode, error) } From 5ee4b873c21317a157b35cee0a1d53c8a61bc3ac Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Mon, 12 Feb 2024 16:25:10 -0800 Subject: [PATCH 16/52] fixes rpc inspector tests --- .../corruptlibp2p/pubsub_adapter_config.go | 2 +- .../test/gossipsub/rpc_inspector/utils.go | 17 - .../validation_inspector_test.go | 305 ++++++++---------- network/p2p/builder.go | 18 +- .../p2p/builder/gossipsub/gossipSubBuilder.go | 59 +++- network/p2p/builder/libp2pNodeBuilder.go | 4 +- network/p2p/mock/gossip_sub_builder.go | 5 + ... gossip_sub_rpc_inspector_factory_func.go} | 22 +- network/p2p/mock/node_builder.go | 16 + network/p2p/test/fixtures.go | 20 +- 10 files changed, 229 insertions(+), 239 deletions(-) rename network/p2p/mock/{gossip_sub_rpc_inspector_suite_factory_func.go => gossip_sub_rpc_inspector_factory_func.go} (60%) diff --git a/insecure/corruptlibp2p/pubsub_adapter_config.go b/insecure/corruptlibp2p/pubsub_adapter_config.go index d90aa38ce5d..fe90db0a9b2 100644 --- a/insecure/corruptlibp2p/pubsub_adapter_config.go +++ b/insecure/corruptlibp2p/pubsub_adapter_config.go @@ -153,7 +153,7 @@ func (c *CorruptPubSubAdapterConfig) ScoreTracer() p2p.PeerScoreTracer { return c.scoreTracer } -func (c *CorruptPubSubAdapterConfig) WithRpcInspector(_ p2p.GossipSubInspectorSuite) { +func (c *CorruptPubSubAdapterConfig) WithRpcInspector(_ p2p.GossipSubMsgValidationRpcInspector) { // CorruptPubSub does not support inspector suite. This is a no-op. } diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/utils.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/utils.go index 7d773870b8c..3b91ae8884f 100644 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/utils.go +++ b/insecure/integration/functional/test/gossipsub/rpc_inspector/utils.go @@ -7,17 +7,13 @@ import ( "testing" "time" - mockery "github.com/stretchr/testify/mock" - "github.com/onflow/flow-go/config" - "github.com/onflow/flow-go/insecure/corruptlibp2p" "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network/channels" "github.com/onflow/flow-go/network/p2p" - mockp2p "github.com/onflow/flow-go/network/p2p/mock" p2ptest "github.com/onflow/flow-go/network/p2p/test" "github.com/onflow/flow-go/network/p2p/tracer" "github.com/onflow/flow-go/utils/unittest" @@ -45,19 +41,6 @@ func randomClusterPrefixedTopic() channels.Topic { return channels.Topic(channels.SyncCluster(flow.ChainID(fmt.Sprintf("%d", rand.Uint64())))) } -type onNotificationDissemination func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) -type mockDistributorOption func(*mockp2p.GossipSubInspectorNotificationDistributor, *corruptlibp2p.GossipSubRouterSpammer) - -func withExpectedNotificationDissemination(expectedNumOfTotalNotif int, f onNotificationDissemination) mockDistributorOption { - return func(distributor *mockp2p.GossipSubInspectorNotificationDistributor, spammer *corruptlibp2p.GossipSubRouterSpammer) { - distributor. - On("Distribute", mockery.Anything). - Times(expectedNumOfTotalNotif). - Run(f(spammer)). - Return(nil) - } -} - func meshTracerFixture(flowConfig *config.FlowConfig, idProvider module.IdentityProvider) *tracer.GossipSubMeshTracer { meshTracerCfg := &tracer.GossipSubMeshTracerConfig{ Logger: unittest.Logger(), diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go index e8cf9bb1566..7f794527f94 100644 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go +++ b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go @@ -56,30 +56,6 @@ func TestValidationInspector_InvalidTopicId_Detection(t *testing.T) { invIHaveNotifCount := atomic.NewUint64(0) done := make(chan struct{}) expectedNumOfTotalNotif := 9 - // ensure expected notifications are disseminated with expected error - inspectDisseminatedNotifyFunc := func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) { - return func(args mockery.Arguments) { - count.Inc() - notification, ok := args[0].(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") - require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - require.True(t, channels.IsInvalidTopicErr(notification.Error)) - switch notification.MsgType { - case p2pmsg.CtrlMsgGraft: - invGraftNotifCount.Inc() - case p2pmsg.CtrlMsgPrune: - invPruneNotifCount.Inc() - case p2pmsg.CtrlMsgIHave: - invIHaveNotifCount.Inc() - default: - require.Fail(t, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) - } - if count.Load() == uint64(expectedNumOfTotalNotif) { - close(done) - } - } - } idProvider := mock.NewIdentityProvider(t) spammer := corruptlibp2p.NewGossipSubRouterSpammer(t, sporkID, role, idProvider) @@ -87,9 +63,28 @@ func TestValidationInspector_InvalidTopicId_Detection(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - withExpectedNotificationDissemination(expectedNumOfTotalNotif, inspectDisseminatedNotifyFunc)(distributor, spammer) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + count.Inc() + notification, ok := args[0].(*p2p.InvCtrlMsgNotif) + require.True(t, ok) + require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") + require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) + require.True(t, channels.IsInvalidTopicErr(notification.Error)) + switch notification.MsgType { + case p2pmsg.CtrlMsgGraft: + invGraftNotifCount.Inc() + case p2pmsg.CtrlMsgPrune: + invPruneNotifCount.Inc() + case p2pmsg.CtrlMsgIHave: + invIHaveNotifCount.Inc() + default: + require.Fail(t, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) + } + if count.Load() == uint64(expectedNumOfTotalNotif) { + close(done) + } + }).Return().Times(expectedNumOfTotalNotif) meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() @@ -97,12 +92,12 @@ func TestValidationInspector_InvalidTopicId_Detection(t *testing.T) { Logger: unittest.Logger(), SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -193,30 +188,6 @@ func TestValidationInspector_DuplicateTopicId_Detection(t *testing.T) { invGraftNotifCount := atomic.NewUint64(0) invPruneNotifCount := atomic.NewUint64(0) invIHaveNotifCount := atomic.NewUint64(0) - inspectDisseminatedNotifyFunc := func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) { - return func(args mockery.Arguments) { - count.Inc() - notification, ok := args[0].(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") - require.True(t, validation.IsDuplicateTopicErr(notification.Error)) - require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - switch notification.MsgType { - case p2pmsg.CtrlMsgGraft: - invGraftNotifCount.Inc() - case p2pmsg.CtrlMsgPrune: - invPruneNotifCount.Inc() - case p2pmsg.CtrlMsgIHave: - invIHaveNotifCount.Inc() - default: - require.Fail(t, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) - } - - if count.Load() == int64(expectedNumOfTotalNotif) { - close(done) - } - } - } idProvider := mock.NewIdentityProvider(t) spammer := corruptlibp2p.NewGossipSubRouterSpammer(t, sporkID, role, idProvider) @@ -224,21 +195,42 @@ func TestValidationInspector_DuplicateTopicId_Detection(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - withExpectedNotificationDissemination(expectedNumOfTotalNotif, inspectDisseminatedNotifyFunc)(distributor, spammer) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + count.Inc() + notification, ok := args[0].(*p2p.InvCtrlMsgNotif) + require.True(t, ok) + require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") + require.True(t, validation.IsDuplicateTopicErr(notification.Error)) + require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) + switch notification.MsgType { + case p2pmsg.CtrlMsgGraft: + invGraftNotifCount.Inc() + case p2pmsg.CtrlMsgPrune: + invPruneNotifCount.Inc() + case p2pmsg.CtrlMsgIHave: + invIHaveNotifCount.Inc() + default: + require.Fail(t, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) + } + + if count.Load() == int64(expectedNumOfTotalNotif) { + close(done) + } + }).Return().Times(expectedNumOfTotalNotif) + meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() validationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: unittest.Logger(), SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -291,54 +283,45 @@ func TestValidationInspector_IHaveDuplicateMessageId_Detection(t *testing.T) { flowConfig, err := config.DefaultConfig() require.NoError(t, err) inspectorConfig := flowConfig.NetworkConfig.GossipSub.RpcInspector.Validation - inspectorConfig.InspectionQueue.NumberOfWorkers = 1 count := atomic.NewInt64(0) done := make(chan struct{}) expectedNumOfTotalNotif := 1 invIHaveNotifCount := atomic.NewUint64(0) - inspectDisseminatedNotifyFunc := func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) { - return func(args mockery.Arguments) { - count.Inc() - notification, ok := args[0].(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") - require.True(t, validation.IsDuplicateMessageIDErr(notification.Error)) - require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - require.True(t, - notification.MsgType == p2pmsg.CtrlMsgIHave, - fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) - invIHaveNotifCount.Inc() - - if count.Load() == int64(expectedNumOfTotalNotif) { - close(done) - } - } - } - idProvider := mock.NewIdentityProvider(t) spammer := corruptlibp2p.NewGossipSubRouterSpammer(t, sporkID, role, idProvider) ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + count.Inc() + notification, ok := args[0].(*p2p.InvCtrlMsgNotif) + require.True(t, ok) + require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") + require.True(t, validation.IsDuplicateMessageIDErr(notification.Error)) + require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) + require.True(t, notification.MsgType == p2pmsg.CtrlMsgIHave, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) + invIHaveNotifCount.Inc() + + if count.Load() == int64(expectedNumOfTotalNotif) { + close(done) + } + }).Return().Times(expectedNumOfTotalNotif) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - withExpectedNotificationDissemination(expectedNumOfTotalNotif, inspectDisseminatedNotifyFunc)(distributor, spammer) meshTracer := meshTracerFixture(flowConfig, idProvider) - topicProvider := newMockUpdatableTopicProvider() validationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: unittest.Logger(), SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -412,49 +395,46 @@ func TestValidationInspector_UnknownClusterId_Detection(t *testing.T) { expectedNumOfTotalNotif := 2 invGraftNotifCount := atomic.NewUint64(0) invPruneNotifCount := atomic.NewUint64(0) - inspectDisseminatedNotifyFunc := func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) { - return func(args mockery.Arguments) { - count.Inc() - notification, ok := args[0].(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - require.Equal(t, notification.TopicType, p2p.CtrlMsgTopicTypeClusterPrefixed) - require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - require.True(t, channels.IsUnknownClusterIDErr(notification.Error)) - switch notification.MsgType { - case p2pmsg.CtrlMsgGraft: - invGraftNotifCount.Inc() - case p2pmsg.CtrlMsgPrune: - invPruneNotifCount.Inc() - default: - require.Fail(t, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) - } - - if count.Load() == int64(expectedNumOfTotalNotif) { - close(done) - } - } - } idProvider := mock.NewIdentityProvider(t) spammer := corruptlibp2p.NewGossipSubRouterSpammer(t, sporkID, role, idProvider) ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - withExpectedNotificationDissemination(expectedNumOfTotalNotif, inspectDisseminatedNotifyFunc)(distributor, spammer) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + count.Inc() + notification, ok := args[0].(*p2p.InvCtrlMsgNotif) + require.True(t, ok) + require.Equal(t, notification.TopicType, p2p.CtrlMsgTopicTypeClusterPrefixed) + require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) + require.True(t, channels.IsUnknownClusterIDErr(notification.Error)) + switch notification.MsgType { + case p2pmsg.CtrlMsgGraft: + invGraftNotifCount.Inc() + case p2pmsg.CtrlMsgPrune: + invPruneNotifCount.Inc() + default: + require.Fail(t, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) + } + + if count.Load() == int64(expectedNumOfTotalNotif) { + close(done) + } + }).Return().Times(expectedNumOfTotalNotif) + meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() validationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: unittest.Logger(), SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -503,6 +483,7 @@ func TestValidationInspector_UnknownClusterId_Detection(t *testing.T) { // cluster prefix hard threshold when the active cluster IDs not set and an invalid control message notification is disseminated with the expected error. // This test involves Graft control messages. func TestValidationInspector_ActiveClusterIdsNotSet_Graft_Detection(t *testing.T) { + // TODO: this test does not implement the documented behavior, it should be fixed, a notification is expected to be sent, but no mocking is done. role := flow.RoleConsensus sporkID := unittest.IdentifierFixture() flowConfig, err := config.DefaultConfig() @@ -534,8 +515,7 @@ func TestValidationInspector_ActiveClusterIdsNotSet_Graft_Detection(t *testing.T ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() @@ -543,12 +523,12 @@ func TestValidationInspector_ActiveClusterIdsNotSet_Graft_Detection(t *testing.T Logger: logger, SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: inspectorIdProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -591,6 +571,7 @@ func TestValidationInspector_ActiveClusterIdsNotSet_Graft_Detection(t *testing.T // cluster prefix hard threshold when the active cluster IDs not set and an invalid control message notification is disseminated with the expected error. // This test involves Prune control messages. func TestValidationInspector_ActiveClusterIdsNotSet_Prune_Detection(t *testing.T) { + // TODO: this test does not implement the documented behavior, it should be fixed, a notification is expected to be sent, but no mocking is done. role := flow.RoleConsensus sporkID := unittest.IdentifierFixture() flowConfig, err := config.DefaultConfig() @@ -620,8 +601,7 @@ func TestValidationInspector_ActiveClusterIdsNotSet_Prune_Detection(t *testing.T ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() inspectorIdProvider := mock.NewIdentityProvider(t) @@ -629,12 +609,12 @@ func TestValidationInspector_ActiveClusterIdsNotSet_Prune_Detection(t *testing.T Logger: logger, SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: inspectorIdProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -675,6 +655,8 @@ func TestValidationInspector_ActiveClusterIdsNotSet_Prune_Detection(t *testing.T // TestValidationInspector_Unstaked_Node_Detection ensures that RPC control message inspector disseminates an invalid control message notification when an unstaked peer // sends a control message for a cluster prefixed topic. func TestValidationInspector_UnstakedNode_Detection(t *testing.T) { + // TODO: we must implement checking rpcs for unstaked peers right at the beginning of the inspection process. + unittest.SkipUnless(t, unittest.TEST_TODO, "the feature this test is testing is not yet implemented") role := flow.RoleConsensus sporkID := unittest.IdentifierFixture() flowConfig, err := config.DefaultConfig() @@ -710,22 +692,24 @@ func TestValidationInspector_UnstakedNode_Detection(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - meshTracer := meshTracerFixture(flowConfig, idProvider) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + // TODO: here we should assert the notification received + }).Return().Once() + meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() inspectorIdProvider := mock.NewIdentityProvider(t) validationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: logger, SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: inspectorIdProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -784,24 +768,6 @@ func TestValidationInspector_InspectIWants_CacheMissThreshold(t *testing.T) { controlMessageCount := int64(1) cacheMissThresholdNotifCount := atomic.NewUint64(0) done := make(chan struct{}) - // ensure expected notifications are disseminated with expected error - inspectDisseminatedNotifyFunc := func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) { - return func(args mockery.Arguments) { - notification, ok := args[0].(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") - require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - require.True(t, - notification.MsgType == p2pmsg.CtrlMsgIWant, - fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) - require.True(t, validation.IsIWantCacheMissThresholdErr(notification.Error)) - - cacheMissThresholdNotifCount.Inc() - if cacheMissThresholdNotifCount.Load() == 1 { - close(done) - } - } - } idProvider := mock.NewIdentityProvider(t) spammer := corruptlibp2p.NewGossipSubRouterSpammer(t, sporkID, role, idProvider) @@ -809,22 +775,33 @@ func TestValidationInspector_InspectIWants_CacheMissThreshold(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - withExpectedNotificationDissemination(1, inspectDisseminatedNotifyFunc)(distributor, spammer) - meshTracer := meshTracerFixture(flowConfig, idProvider) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + notification, ok := args[0].(*p2p.InvCtrlMsgNotif) + require.True(t, ok) + require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") + require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) + require.True(t, notification.MsgType == p2pmsg.CtrlMsgIWant, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) + require.True(t, validation.IsIWantCacheMissThresholdErr(notification.Error)) + + cacheMissThresholdNotifCount.Inc() + if cacheMissThresholdNotifCount.Load() == 1 { + close(done) + } + }).Return().Once() + meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() validationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: unittest.Logger(), SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -918,48 +895,42 @@ func TestValidationInspector_InspectRpcPublishMessages(t *testing.T) { // first create 4 valid messages publishMsgs := unittest.GossipSubMessageFixtures(4, topic.String(), unittest.WithFrom(spammer.SpammerNode.ID())) publishMsgs = append(publishMsgs, invalidPublishMsgs...) - // ensure expected notifications are disseminated with expected error - inspectDisseminatedNotifyFunc := func(spammer *corruptlibp2p.GossipSubRouterSpammer) func(args mockery.Arguments) { - return func(args mockery.Arguments) { - notification, ok := args[0].(*p2p.InvCtrlMsgNotif) - require.True(t, ok) - require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") - require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - require.True(t, - notification.MsgType == p2pmsg.RpcPublishMessage, - fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) - require.True(t, validation.IsInvalidRpcPublishMessagesErr(notification.Error)) - require.Contains(t, - notification.Error.Error(), - fmt.Sprintf("%d error(s) encountered", len(invalidPublishMsgs)), - fmt.Sprintf("expected %d errors, an error for each invalid pubsub message", len(invalidPublishMsgs))) - require.Contains(t, notification.Error.Error(), fmt.Sprintf("received rpc publish message from unstaked peer: %s", unknownPeerID)) - require.Contains(t, notification.Error.Error(), fmt.Sprintf("received rpc publish message from ejected peer: %s", ejectedIdentityPeerID)) - notificationCount.Inc() - if notificationCount.Load() == 1 { - close(done) - } - } - } ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) - withExpectedNotificationDissemination(1, inspectDisseminatedNotifyFunc)(distributor, spammer) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) + consumer.On("OnInvalidControlMessageNotification", mockery.Anything).Run(func(args mockery.Arguments) { + notification, ok := args[0].(*p2p.InvCtrlMsgNotif) + require.True(t, ok) + require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") + require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) + require.True(t, notification.MsgType == p2pmsg.RpcPublishMessage, fmt.Sprintf("unexpected control message type %s error: %s", notification.MsgType, notification.Error)) + require.True(t, validation.IsInvalidRpcPublishMessagesErr(notification.Error)) + require.Contains(t, + notification.Error.Error(), + fmt.Sprintf("%d error(s) encountered", len(invalidPublishMsgs)), + fmt.Sprintf("expected %d errors, an error for each invalid pubsub message", len(invalidPublishMsgs))) + require.Contains(t, notification.Error.Error(), fmt.Sprintf("received rpc publish message from unstaked peer: %s", unknownPeerID)) + require.Contains(t, notification.Error.Error(), fmt.Sprintf("received rpc publish message from ejected peer: %s", ejectedIdentityPeerID)) + notificationCount.Inc() + if notificationCount.Load() == 1 { + close(done) + } + }).Return().Once() + meshTracer := meshTracerFixture(flowConfig, idProvider) topicProvider := newMockUpdatableTopicProvider() validationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: unittest.Logger(), SporkID: sporkID, Config: &inspectorConfig, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: meshTracer, NetworkingType: network.PrivateNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, diff --git a/network/p2p/builder.go b/network/p2p/builder.go index cbc71475511..af615ea3a41 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -65,12 +65,12 @@ type GossipSubBuilder interface { // If the routing system has already been set, a fatal error is logged. SetRoutingSystem(routing.Routing) - // OverrideDefaultRpcInspectorSuiteFactory overrides the default RPC inspector suite factory of the builder. + // OverrideDefaultRpcInspectorFactory overrides the default RPC inspector suite factory of the builder. // A default RPC inspector suite factory is provided by the node. This function overrides the default factory. // The purpose of override is to allow the node to provide a custom RPC inspector suite factory for sake of testing // or experimentation. // It is NOT recommended to override the default RPC inspector suite factory in production unless you know what you are doing. - OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorSuiteFactoryFunc) + OverrideDefaultRpcInspectorFactory(GossipSubRpcInspectorFactoryFunc) // Build creates a new GossipSub pubsub system. // It returns the newly created GossipSub pubsub system and any errors encountered during its creation. @@ -85,8 +85,8 @@ type GossipSubBuilder interface { Build(irrecoverable.SignalerContext) (PubSubAdapter, error) } -// GossipSubRpcInspectorSuiteFactoryFunc is a function that creates a new RPC inspector suite. It is used to create -// RPC inspectors for the gossipsub protocol. The RPC inspectors are used to inspect and validate +// GossipSubRpcInspectorFactoryFunc is a function that creates a new RPC inspector. It is used to create +// an RPC inspector for the gossipsub protocol. The RPC inspectors are used to inspect and validate // incoming RPC messages before they are processed by the gossipsub protocol. // Args: // - logger: logger to use @@ -97,10 +97,9 @@ type GossipSubBuilder interface { // - networkingType: networking type of the node, i.e., public or private // - identityProvider: identity provider of the node // Returns: -// - p2p.GossipSubInspectorSuite: new RPC inspector suite +// - GossipSubRPCInspector: new RPC inspector suite // - error: error if any, any returned error is irrecoverable. -type GossipSubRpcInspectorSuiteFactoryFunc func( - irrecoverable.SignalerContext, +type GossipSubRpcInspectorFactoryFunc func( zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, @@ -109,7 +108,8 @@ type GossipSubRpcInspectorSuiteFactoryFunc func( flownet.NetworkingType, module.IdentityProvider, func() TopicProvider, -) (GossipSubInspectorSuite, error) + GossipSubInvCtrlMsgNotifConsumer, +) (GossipSubMsgValidationRpcInspector, error) // NodeBuilder is a builder pattern for creating a libp2p Node instance. type NodeBuilder interface { @@ -142,7 +142,7 @@ type NodeBuilder interface { // none OverrideNodeConstructor(NodeConstructor) NodeBuilder SetGossipSubFactory(GossipSubFactoryFunc, GossipSubAdapterConfigFunc) NodeBuilder - OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorSuiteFactoryFunc) NodeBuilder + OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorFactoryFunc) NodeBuilder Build() (LibP2PNode, error) } diff --git a/network/p2p/builder/gossipsub/gossipSubBuilder.go b/network/p2p/builder/gossipsub/gossipSubBuilder.go index 71558ddbefc..66fff418dde 100644 --- a/network/p2p/builder/gossipsub/gossipSubBuilder.go +++ b/network/p2p/builder/gossipsub/gossipSubBuilder.go @@ -12,6 +12,7 @@ import ( "github.com/onflow/flow-go/model/flow" "github.com/onflow/flow-go/module" "github.com/onflow/flow-go/module/irrecoverable" + "github.com/onflow/flow-go/module/metrics" "github.com/onflow/flow-go/network" "github.com/onflow/flow-go/network/p2p" p2pbuilderconfig "github.com/onflow/flow-go/network/p2p/builder/config" @@ -34,6 +35,7 @@ type Builder struct { subscriptionFilter pubsub.SubscriptionFilter gossipSubFactory p2p.GossipSubFactoryFunc gossipSubConfigFunc p2p.GossipSubAdapterConfigFunc + rpcInspectorFactory p2p.GossipSubRpcInspectorFactoryFunc // gossipSubTracer is a callback interface that is called by the gossipsub implementation upon // certain events. Currently, we use it to log and observe the local mesh of the node. gossipSubTracer p2p.PubSubTracer @@ -55,6 +57,11 @@ func (g *Builder) SetHost(h host.Host) { g.h = h } +func (g *Builder) OverrideDefaultRpcInspectorFactory(factoryFunc p2p.GossipSubRpcInspectorFactoryFunc) { + g.logger.Warn().Msg("overriding default rpc inspector factory, not recommended for production") + g.rpcInspectorFactory = factoryFunc +} + // SetSubscriptionFilter sets the subscription filter of the builder. // If the subscription filter has already been set, a fatal error is logged. func (g *Builder) SetSubscriptionFilter(subscriptionFilter pubsub.SubscriptionFilter) { @@ -179,13 +186,39 @@ func NewGossipSubBuilder(logger zerolog.Logger, meshTracer.DuplicateMessageCount, networkType, ), - gossipSubTracer: meshTracer, - gossipSubCfg: gossipSubCfg, + gossipSubTracer: meshTracer, + gossipSubCfg: gossipSubCfg, + rpcInspectorFactory: defaultRpcInspectorFactory(meshTracer), } return b } +func defaultRpcInspectorFactory(tracer p2p.PubSubTracer) p2p.GossipSubRpcInspectorFactoryFunc { + return func(logger zerolog.Logger, + sporkId flow.Identifier, + rpcInspectorConfig *p2pconfig.RpcInspectorParameters, + inspectorMetrics module.GossipSubMetrics, + heroCacheMetrics metrics.HeroCacheMetricsFactory, + networkingType network.NetworkingType, + idProvider module.IdentityProvider, + topicProvider func() p2p.TopicProvider, + notificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error) { + return validation.NewControlMsgValidationInspector(&validation.InspectorParams{ + Logger: logger.With().Str("component", "rpc-inspector").Logger(), + SporkID: sporkId, + Config: &rpcInspectorConfig.Validation, + HeroCacheMetricsFactory: heroCacheMetrics, + IdProvider: idProvider, + InspectorMetrics: inspectorMetrics, + RpcTracker: tracer, + NetworkingType: networkingType, + InvalidControlMessageNotificationConsumer: notificationConsumer, + TopicOracle: topicProvider, + }) + } +} + // defaultGossipSubFactory returns the default gossipsub factory function. It is used to create the default gossipsub factory. // Note: always use the default gossipsub factory function to create the gossipsub factory (unless you know what you are doing). func defaultGossipSubFactory() p2p.GossipSubFactoryFunc { @@ -267,20 +300,18 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e Msg("gossipsub peer scoring is disabled") } - rpcValidationInspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ - Logger: g.logger, - SporkID: g.sporkId, - Config: &g.gossipSubCfg.RpcInspector.Validation, - HeroCacheMetricsFactory: g.metricsCfg.HeroCacheFactory, - IdProvider: g.idProvider, - InspectorMetrics: g.metricsCfg.Metrics, - RpcTracker: g.gossipSubTracer, - NetworkingType: g.networkType, - InvalidControlMessageNotificationConsumer: scoreOpt, - TopicOracle: func() p2p.TopicProvider { + rpcValidationInspector, err := g.rpcInspectorFactory( + g.logger, + g.sporkId, + &g.gossipSubCfg.RpcInspector, + g.metricsCfg.Metrics, + g.metricsCfg.HeroCacheFactory, + g.networkType, + g.idProvider, + func() p2p.TopicProvider { return gossipSub }, - }) + scoreOpt) if err != nil { return nil, fmt.Errorf("failed to create new rpc valiadation inspector: %w", err) } diff --git a/network/p2p/builder/libp2pNodeBuilder.go b/network/p2p/builder/libp2pNodeBuilder.go index 7d9709ae457..b02f43c8adc 100644 --- a/network/p2p/builder/libp2pNodeBuilder.go +++ b/network/p2p/builder/libp2pNodeBuilder.go @@ -180,8 +180,8 @@ func (builder *LibP2PNodeBuilder) OverrideNodeConstructor(f p2p.NodeConstructor) return builder } -func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorSuiteFactoryFunc) p2p.NodeBuilder { - builder.gossipSubBuilder.OverrideDefaultRpcInspectorSuiteFactory(factory) +func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { + builder.gossipSubBuilder.OverrideDefaultRpcInspectorFactory(factory) return builder } diff --git a/network/p2p/mock/gossip_sub_builder.go b/network/p2p/mock/gossip_sub_builder.go index 25e819f6d60..a5d4a846c56 100644 --- a/network/p2p/mock/gossip_sub_builder.go +++ b/network/p2p/mock/gossip_sub_builder.go @@ -51,6 +51,11 @@ func (_m *GossipSubBuilder) EnableGossipSubScoringWithOverride(_a0 *p2p.PeerScor _m.Called(_a0) } +// OverrideDefaultRpcInspectorFactory provides a mock function with given fields: _a0 +func (_m *GossipSubBuilder) OverrideDefaultRpcInspectorFactory(_a0 p2p.GossipSubRpcInspectorFactoryFunc) { + _m.Called(_a0) +} + // SetGossipSubConfigFunc provides a mock function with given fields: _a0 func (_m *GossipSubBuilder) SetGossipSubConfigFunc(_a0 p2p.GossipSubAdapterConfigFunc) { _m.Called(_a0) diff --git a/network/p2p/mock/gossip_sub_rpc_inspector_suite_factory_func.go b/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go similarity index 60% rename from network/p2p/mock/gossip_sub_rpc_inspector_suite_factory_func.go rename to network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go index 7b419f29c48..c4191cd7f5c 100644 --- a/network/p2p/mock/gossip_sub_rpc_inspector_suite_factory_func.go +++ b/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go @@ -21,25 +21,25 @@ import ( zerolog "github.com/rs/zerolog" ) -// GossipSubRpcInspectorSuiteFactoryFunc is an autogenerated mock type for the GossipSubRpcInspectorSuiteFactoryFunc type -type GossipSubRpcInspectorSuiteFactoryFunc struct { +// GossipSubRpcInspectorFactoryFunc is an autogenerated mock type for the GossipSubRpcInspectorFactoryFunc type +type GossipSubRpcInspectorFactoryFunc struct { mock.Mock } // Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8 -func (_m *GossipSubRpcInspectorSuiteFactoryFunc) Execute(_a0 irrecoverable.SignalerContext, _a1 zerolog.Logger, _a2 flow.Identifier, _a3 *p2pconfig.RpcInspectorParameters, _a4 module.GossipSubMetrics, _a5 metrics.HeroCacheMetricsFactory, _a6 network.NetworkingType, _a7 module.IdentityProvider, _a8 func() p2p.TopicProvider) (p2p.GossipSubInspectorSuite, error) { +func (_m *GossipSubRpcInspectorFactoryFunc) Execute(_a0 irrecoverable.SignalerContext, _a1 zerolog.Logger, _a2 flow.Identifier, _a3 *p2pconfig.RpcInspectorParameters, _a4 module.GossipSubMetrics, _a5 metrics.HeroCacheMetricsFactory, _a6 network.NetworkingType, _a7 module.IdentityProvider, _a8 func() p2p.TopicProvider) (p2p.GossipSubRPCInspector, error) { ret := _m.Called(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) - var r0 p2p.GossipSubInspectorSuite + var r0 p2p.GossipSubRPCInspector var r1 error - if rf, ok := ret.Get(0).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) (p2p.GossipSubInspectorSuite, error)); ok { + if rf, ok := ret.Get(0).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) (p2p.GossipSubRPCInspector, error)); ok { return rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } - if rf, ok := ret.Get(0).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) p2p.GossipSubInspectorSuite); ok { + if rf, ok := ret.Get(0).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) p2p.GossipSubRPCInspector); ok { r0 = rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(p2p.GossipSubInspectorSuite) + r0 = ret.Get(0).(p2p.GossipSubRPCInspector) } } @@ -52,14 +52,14 @@ func (_m *GossipSubRpcInspectorSuiteFactoryFunc) Execute(_a0 irrecoverable.Signa return r0, r1 } -type mockConstructorTestingTNewGossipSubRpcInspectorSuiteFactoryFunc interface { +type mockConstructorTestingTNewGossipSubRpcInspectorFactoryFunc interface { mock.TestingT Cleanup(func()) } -// NewGossipSubRpcInspectorSuiteFactoryFunc creates a new instance of GossipSubRpcInspectorSuiteFactoryFunc. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewGossipSubRpcInspectorSuiteFactoryFunc(t mockConstructorTestingTNewGossipSubRpcInspectorSuiteFactoryFunc) *GossipSubRpcInspectorSuiteFactoryFunc { - mock := &GossipSubRpcInspectorSuiteFactoryFunc{} +// NewGossipSubRpcInspectorFactoryFunc creates a new instance of GossipSubRpcInspectorFactoryFunc. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +func NewGossipSubRpcInspectorFactoryFunc(t mockConstructorTestingTNewGossipSubRpcInspectorFactoryFunc) *GossipSubRpcInspectorFactoryFunc { + mock := &GossipSubRpcInspectorFactoryFunc{} mock.Mock.Test(t) t.Cleanup(func() { mock.AssertExpectations(t) }) diff --git a/network/p2p/mock/node_builder.go b/network/p2p/mock/node_builder.go index 6fa12bae038..523cae41c2a 100644 --- a/network/p2p/mock/node_builder.go +++ b/network/p2p/mock/node_builder.go @@ -53,6 +53,22 @@ func (_m *NodeBuilder) Build() (p2p.LibP2PNode, error) { return r0, r1 } +// OverrideDefaultRpcInspectorSuiteFactory provides a mock function with given fields: _a0 +func (_m *NodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(_a0 p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { + ret := _m.Called(_a0) + + var r0 p2p.NodeBuilder + if rf, ok := ret.Get(0).(func(p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder); ok { + r0 = rf(_a0) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(p2p.NodeBuilder) + } + } + + return r0 +} + // OverrideGossipSubScoringConfig provides a mock function with given fields: _a0 func (_m *NodeBuilder) OverrideGossipSubScoringConfig(_a0 *p2p.PeerScoringConfigOverride) p2p.NodeBuilder { ret := _m.Called(_a0) diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index a11e8acda84..abeea7cc6cc 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -247,7 +247,7 @@ type NodeFixtureParameters struct { GossipSubConfig p2p.GossipSubAdapterConfigFunc MetricsCfg *p2pbuilderconfig.MetricsConfig ResourceManager network.ResourceManager - GossipSubRpcInspectorSuiteFactory p2p.GossipSubRpcInspectorSuiteFactoryFunc + GossipSubRpcInspectorSuiteFactory p2p.GossipSubRpcInspectorFactoryFunc FlowConfig *config.FlowConfig UnicastRateLimiterDistributor p2p.UnicastRateLimiterDistributor } @@ -258,7 +258,7 @@ func WithUnicastRateLimitDistributor(distributor p2p.UnicastRateLimiterDistribut } } -func OverrideGossipSubRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorSuiteFactoryFunc) NodeFixtureParameterOption { +func OverrideGossipSubRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) NodeFixtureParameterOption { return func(p *NodeFixtureParameters) { p.GossipSubRpcInspectorSuiteFactory = factory } @@ -809,22 +809,6 @@ func NewConnectionGater(idProvider module.IdentityProvider, allowListFilter p2p. return connection.NewConnGater(unittest.Logger(), idProvider, connection.WithOnInterceptPeerDialFilters(filters), connection.WithOnInterceptSecuredFilters(filters)) } -// MockInspectorNotificationDistributorReadyDoneAware mocks the Ready and Done methods of the distributor to return a channel that is already closed, -// so that the distributor is considered ready and done when the test needs. -func MockInspectorNotificationDistributorReadyDoneAware(d *mockp2p.GossipSubInspectorNotificationDistributor) { - d.On("Start", mockery.Anything).Return().Maybe() - d.On("Ready").Return(func() <-chan struct{} { - ch := make(chan struct{}) - close(ch) - return ch - }()).Maybe() - d.On("Done").Return(func() <-chan struct{} { - ch := make(chan struct{}) - close(ch) - return ch - }()).Maybe() -} - // MockScoringRegistrySubscriptionValidatorReadyDoneAware mocks the Ready and Done methods of the subscription validator to return a channel that is already closed, // so that the distributor is considered ready and done when the test needs. func MockScoringRegistrySubscriptionValidatorReadyDoneAware(s *mockp2p.SubscriptionValidator) { From aea636a3097d595e5c7b2eea9fdddf0f4f5c8f99 Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Mon, 12 Feb 2024 23:40:06 -0500 Subject: [PATCH 17/52] add invalid topic ID threshold configuration and flags --- config/default-config.yml | 6 ++++++ network/netconf/flags.go | 8 ++++++++ network/p2p/config/gossipsub_rpc_inspectors.go | 9 +++++++++ 3 files changed, 23 insertions(+) diff --git a/config/default-config.yml b/config/default-config.yml index c4d162331e9..54e96c35d52 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -160,6 +160,9 @@ network-config: # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. # A topic id is considered duplicate if it appears more than once in a single GRAFT or PRUNE message. duplicate-topic-id-threshold: 50 + # Maximum number of total invalid topic ids in a single GRAFT or PRUNE message, ideally this should be 0 but we allow for some tolerance + # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. + invalid-topic-id-threshold: 50 ihave: # The maximum allowed number of iHave messages in a single RPC message. # Each iHave message represents the list of message ids. When the total number of iHave messages @@ -181,6 +184,9 @@ network-config: # Ideally, an iHave message should not have any duplicate message IDs, hence a message id is considered duplicate when it is repeated more than once # within the same iHave message. When the total number of duplicate message ids in a single iHave message exceeds this threshold, the inspection of message will fail. duplicate-message-id-threshold: 100 + # Maximum number of total invalid topic ids in a single IHAVE message, ideally this should be 0 but we allow for some tolerance + # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. + invalid-topic-id-threshold: 50 iwant: # The maximum allowed number of iWant messages in a single RPC message. # Each iWant message represents the list of message ids. When the total number of iWant messages diff --git a/network/netconf/flags.go b/network/netconf/flags.go index d6e58a4b340..82c333be9dd 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -116,7 +116,9 @@ func AllFlagNames() []string { BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.MessageIdCountThreshold), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.DuplicateTopicIdThresholdKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.DuplicateMessageIdThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.InvalidTopicIdThresholdKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.DuplicateTopicIdThresholdKey), + BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.InvalidTopicIdThresholdKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.MessageCountThreshold), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IWantConfigKey, p2pconfig.MessageCountThreshold), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IWantConfigKey, p2pconfig.MessageIdCountThreshold), @@ -353,6 +355,9 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { "the max allowed duplicate topic IDs across all ihave control messages in a single RPC message, if exceeded a misbehavior report will be created") flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.DuplicateMessageIdThresholdKey), config.GossipSub.RpcInspector.Validation.IHave.DuplicateMessageIdThreshold, + "the max allowed invalid topics in a single ihave control message, if exceeded a misbehavior report will be created") + flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.InvalidTopicIdThresholdKey), + config.GossipSub.RpcInspector.Validation.IHave.InvalidTopicThreshold, "the max allowed duplicate message IDs in a single ihave control message, if exceeded a misbehavior report will be created") flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.MessageCountThreshold), config.GossipSub.RpcInspector.Validation.GraftPrune.MessageCountThreshold, @@ -378,6 +383,9 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.DuplicateTopicIdThresholdKey), config.GossipSub.RpcInspector.Validation.GraftPrune.DuplicateTopicIdThreshold, "the max allowed duplicate topic IDs across all graft or prune control messages in a single RPC message, if exceeded a misbehavior report will be created") + flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.InvalidTopicIdThresholdKey), + config.GossipSub.RpcInspector.Validation.GraftPrune.InvalidTopicThreshold, + "the max allowed invalid topic across all graft or prune control messages in a single RPC message, if exceeded a misbehavior report will be created") flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.SubscriptionProviderKey, p2pconfig.UpdateIntervalKey), config.GossipSub.SubscriptionProvider.UpdateInterval, diff --git a/network/p2p/config/gossipsub_rpc_inspectors.go b/network/p2p/config/gossipsub_rpc_inspectors.go index a2e4b9f180e..490d304534d 100644 --- a/network/p2p/config/gossipsub_rpc_inspectors.go +++ b/network/p2p/config/gossipsub_rpc_inspectors.go @@ -138,6 +138,10 @@ type GraftPruneRpcInspectionParameters struct { // Ideally, a GRAFT or PRUNE message should not have any duplicate topics, hence a topic ID is counted as a duplicate only if it is repeated more than once. // When the total number of duplicate topic ids in a single GRAFT or PRUNE message exceeds this threshold, the inspection of message will fail. DuplicateTopicIdThreshold int `validate:"gte=0" mapstructure:"duplicate-topic-id-threshold"` + + // InvalidTopicIdThreshold Maximum number of total invalid topic ids in a single GRAFT or PRUNE message, ideally this should be 0 but we allow for some tolerance + // to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. + InvalidTopicIdThreshold int `validate:"gte=0" mapstructure:"invalid-topic-id-threshold"` } const ( @@ -145,6 +149,7 @@ const ( MessageIdCountThreshold = "message-id-count-threshold" CacheMissThresholdKey = "cache-miss-threshold" DuplicateMsgIDThresholdKey = "duplicate-message-id-threshold" + InvalidTopicIdThresholdKey = "invalid-topic-id-threshold" ) // IWantRpcInspectionParameters contains the "numerical values" for iwant rpc control inspection. @@ -207,6 +212,10 @@ type IHaveRpcInspectionParameters struct { // Ideally, an iHave message should not have any duplicate message IDs, hence a message id is considered duplicate when it is repeated more than once // within the same iHave message. When the total number of duplicate message ids in a single iHave message exceeds this threshold, the inspection of message will fail. DuplicateMessageIdThreshold int `validate:"gte=0" mapstructure:"duplicate-message-id-threshold"` + + // InvalidTopicIdThreshold Maximum number of total invalid topic ids in a single IHAVE message, ideally this should be 0 but we allow for some tolerance + // to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. + InvalidTopicIdThreshold int `validate:"gte=0" mapstructure:"invalid-topic-id-threshold"` } const ( From 1d572c56f92b4f3d08d32ee8e4482328d45d042e Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Tue, 13 Feb 2024 02:53:33 -0500 Subject: [PATCH 18/52] add invalid topic unit tests above/below threshold for each control message type --- .../gossipsub_rpc_validation_inspector.go | 2 +- network/netconf/flags.go | 10 +- .../control_message_validation_inspector.go | 18 +- ...ntrol_message_validation_inspector_test.go | 271 +++++++++++++++++- 4 files changed, 280 insertions(+), 21 deletions(-) diff --git a/module/metrics/gossipsub_rpc_validation_inspector.go b/module/metrics/gossipsub_rpc_validation_inspector.go index 6b79e8c477d..7460486b426 100644 --- a/module/metrics/gossipsub_rpc_validation_inspector.go +++ b/module/metrics/gossipsub_rpc_validation_inspector.go @@ -323,7 +323,7 @@ func (c *GossipSubRpcValidationInspectorMetrics) AsyncProcessingFinished(duratio c.rpcCtrlMsgAsyncProcessingTimeHistogram.Observe(duration.Seconds()) } -// OnControlMessageIDsTruncated tracks the number of times a control message was truncated. +// OnControlMessagesTruncated tracks the number of times a control message was truncated. // Args: // // messageType: the type of the control message that was truncated diff --git a/network/netconf/flags.go b/network/netconf/flags.go index 82c333be9dd..44767ca4969 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -355,10 +355,12 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { "the max allowed duplicate topic IDs across all ihave control messages in a single RPC message, if exceeded a misbehavior report will be created") flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.DuplicateMessageIdThresholdKey), config.GossipSub.RpcInspector.Validation.IHave.DuplicateMessageIdThreshold, - "the max allowed invalid topics in a single ihave control message, if exceeded a misbehavior report will be created") - flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.InvalidTopicIdThresholdKey), - config.GossipSub.RpcInspector.Validation.IHave.InvalidTopicThreshold, "the max allowed duplicate message IDs in a single ihave control message, if exceeded a misbehavior report will be created") + flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.InvalidTopicIdThresholdKey), + config.GossipSub.RpcInspector.Validation.IHave.InvalidTopicIdThreshold, + "the max allowed invalid topics in a single ihave control message, if exceeded a misbehavior report will be created", + ) + flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.MessageCountThreshold), config.GossipSub.RpcInspector.Validation.GraftPrune.MessageCountThreshold, "threshold for the number of graft or prune control messages to accept on a single RPC message, if exceeded the RPC message will be sampled and truncated") @@ -384,7 +386,7 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { config.GossipSub.RpcInspector.Validation.GraftPrune.DuplicateTopicIdThreshold, "the max allowed duplicate topic IDs across all graft or prune control messages in a single RPC message, if exceeded a misbehavior report will be created") flags.Int(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.GraftPruneKey, p2pconfig.InvalidTopicIdThresholdKey), - config.GossipSub.RpcInspector.Validation.GraftPrune.InvalidTopicThreshold, + config.GossipSub.RpcInspector.Validation.GraftPrune.InvalidTopicIdThreshold, "the max allowed invalid topic across all graft or prune control messages in a single RPC message, if exceeded a misbehavior report will be created") flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.SubscriptionProviderKey, p2pconfig.UpdateIntervalKey), diff --git a/network/p2p/inspector/validation/control_message_validation_inspector.go b/network/p2p/inspector/validation/control_message_validation_inspector.go index 705e709a8cb..6547075c119 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector.go @@ -366,6 +366,7 @@ func (c *ControlMsgValidationInspector) inspectGraftMessages(from peer.ID, graft duplicateTopicTracker := make(duplicateStrTracker) totalDuplicateTopicIds := 0 + totalInvalidTopicIdErrs := 0 defer func() { // regardless of inspection result, update metrics c.metrics.OnGraftMessageInspected(totalDuplicateTopicIds) @@ -384,9 +385,12 @@ func (c *ControlMsgValidationInspector) inspectGraftMessages(from peer.ID, graft } err, ctrlMsgType := c.validateTopic(from, topic, activeClusterIDS) if err != nil { + totalInvalidTopicIdErrs++ // TODO: consider adding a threshold for this error similar to the duplicate topic id threshold. c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgGraft) - return err, ctrlMsgType + if totalInvalidTopicIdErrs > c.config.GraftPrune.InvalidTopicIdThreshold { + return err, ctrlMsgType + } } } return nil, p2p.CtrlMsgNonClusterTopicType @@ -413,6 +417,7 @@ func (c *ControlMsgValidationInspector) inspectPruneMessages(from peer.ID, prune } tracker := make(duplicateStrTracker) totalDuplicateTopicIds := 0 + totalInvalidTopicIdErrs := 0 defer func() { // regardless of inspection result, update metrics c.metrics.OnPruneMessageInspected(totalDuplicateTopicIds) @@ -430,9 +435,12 @@ func (c *ControlMsgValidationInspector) inspectPruneMessages(from peer.ID, prune } err, ctrlMsgType := c.validateTopic(from, topic, activeClusterIDS) if err != nil { + totalInvalidTopicIdErrs++ // TODO: consider adding a threshold for this error similar to the duplicate topic id threshold. c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgPrune) - return err, ctrlMsgType + if totalInvalidTopicIdErrs > c.config.GraftPrune.InvalidTopicIdThreshold { + return err, ctrlMsgType + } } } return nil, p2p.CtrlMsgNonClusterTopicType @@ -471,6 +479,7 @@ func (c *ControlMsgValidationInspector) inspectIHaveMessages(from peer.ID, ihave totalMessageIds := 0 totalDuplicateTopicIds := 0 totalDuplicateMessageIds := 0 + totalInvalidTopicIdErrs := 0 defer func() { // regardless of inspection result, update metrics c.metrics.OnIHaveMessagesInspected(totalDuplicateTopicIds, totalDuplicateMessageIds) @@ -483,9 +492,12 @@ func (c *ControlMsgValidationInspector) inspectIHaveMessages(from peer.ID, ihave // first check if the topic is valid, fail fast if it is not err, ctrlMsgType := c.validateTopic(from, channels.Topic(topic), activeClusterIDS) if err != nil { + totalInvalidTopicIdErrs++ // TODO: consider adding a threshold for this error similar to the duplicate topic id threshold. c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgIHave) - return err, ctrlMsgType + if totalInvalidTopicIdErrs > c.config.IHave.InvalidTopicIdThreshold { + return err, ctrlMsgType + } } // then track the topic ensuring it is not beyond a duplicate threshold. diff --git a/network/p2p/inspector/validation/control_message_validation_inspector_test.go b/network/p2p/inspector/validation/control_message_validation_inspector_test.go index 02fc72f3a44..f5f8d1b8722 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector_test.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector_test.go @@ -343,21 +343,79 @@ func TestControlMessageInspection_ValidRpc(t *testing.T) { unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") } -// TestGraftInspection_InvalidTopic ensures inspector disseminates an invalid control message notification for -// graft messages when the topic is invalid. -func TestGraftInspection_InvalidTopic(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) - // create unknown topic - unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) +// TestGraftInspection_InvalidTopic_BelowThreshold ensures inspector does not disseminate an invalid control message notification for +// graft messages when the invalid topic id count does not exceed the configured threshold. +func TestGraftInspection_InvalidTopic_BelowThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicGrafts []*pubsub_pb.ControlGraft + var malformedTopicGrafts []*pubsub_pb.ControlGraft + var invalidSporkIDTopicGrafts []*pubsub_pb.ControlGraft + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicGrafts = append(unknownTopicGrafts, unittest.P2PRPCGraftFixture(&unknownTopic)) + malformedTopicGrafts = append(malformedTopicGrafts, unittest.P2PRPCGraftFixture(&malformedTopic)) + invalidSporkIDTopicGrafts = append(invalidSporkIDTopicGrafts, unittest.P2PRPCGraftFixture(&invalidSporkIDTopic)) + } // avoid unknown topics errors - topicProviderOracle.UpdateTopics([]string{unknownTopic, malformedTopic, invalidSporkIDTopic}) - unknownTopicGraft := unittest.P2PRPCGraftFixture(&unknownTopic) - malformedTopicGraft := unittest.P2PRPCGraftFixture(&malformedTopic) - invalidSporkIDTopicGraft := unittest.P2PRPCGraftFixture(&invalidSporkIDTopic) + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(unknownTopicGrafts...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(malformedTopicGrafts...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(invalidSporkIDTopicGrafts...)) - unknownTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(unknownTopicGraft)) - malformedTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(malformedTopicGraft)) - invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(invalidSporkIDTopicGraft)) + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + // no notification should be disseminated for valid messages as long as the number of invalid topic ids is below the threshold + distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(3 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + +// TestGraftInspection_InvalidTopic_AboveThreshold ensures inspector disseminates an invalid control message notification for +// graft messages when the invalid topic id count exceeds the configured threshold. +func TestGraftInspection_InvalidTopic_AboveThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicGrafts []*pubsub_pb.ControlGraft + var malformedTopicGrafts []*pubsub_pb.ControlGraft + var invalidSporkIDTopicGrafts []*pubsub_pb.ControlGraft + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold+1; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicGrafts = append(unknownTopicGrafts, unittest.P2PRPCGraftFixture(&unknownTopic)) + malformedTopicGrafts = append(malformedTopicGrafts, unittest.P2PRPCGraftFixture(&malformedTopic)) + invalidSporkIDTopicGrafts = append(invalidSporkIDTopicGrafts, unittest.P2PRPCGraftFixture(&invalidSporkIDTopic)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(unknownTopicGrafts...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(malformedTopicGrafts...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(invalidSporkIDTopicGrafts...)) from := unittest.PeerIdFixture(t) checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgGraft, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) @@ -370,6 +428,7 @@ func TestGraftInspection_InvalidTopic(t *testing.T) { require.NoError(t, inspector.Inspect(from, unknownTopicReq)) require.NoError(t, inspector.Inspect(from, malformedTopicReq)) require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + // sleep for 1 second to ensure rpc's is processed time.Sleep(time.Second) cancel() @@ -439,6 +498,99 @@ func TestGraftInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") } +// TestPruneInspection_InvalidTopic_BelowThreshold ensures inspector does not disseminate an invalid control message notification for +// prune messages when the invalid topic id count does not exceed the configured threshold. +func TestPruneInspection_InvalidTopic_BelowThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicPrunes []*pubsub_pb.ControlPrune + var malformedTopicPrunes []*pubsub_pb.ControlPrune + var invalidSporkIDTopicPrunes []*pubsub_pb.ControlPrune + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicPrunes = append(unknownTopicPrunes, unittest.P2PRPCPruneFixture(&unknownTopic)) + malformedTopicPrunes = append(malformedTopicPrunes, unittest.P2PRPCPruneFixture(&malformedTopic)) + invalidSporkIDTopicPrunes = append(invalidSporkIDTopicPrunes, unittest.P2PRPCPruneFixture(&invalidSporkIDTopic)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(unknownTopicPrunes...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(malformedTopicPrunes...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(invalidSporkIDTopicPrunes...)) + + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + // no notification should be disseminated for valid messages as long as the number of invalid topic ids is below the threshold + distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(2 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + +// TestPruneInspection_InvalidTopic_AboveThreshold ensures inspector disseminates an invalid control message notification for +// prune messages when the invalid topic id count exceeds the configured threshold. +func TestPruneInspection_InvalidTopic_AboveThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicPrunes []*pubsub_pb.ControlPrune + var malformedTopicPrunes []*pubsub_pb.ControlPrune + var invalidSporkIDTopicPrunes []*pubsub_pb.ControlPrune + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold+1; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicPrunes = append(unknownTopicPrunes, unittest.P2PRPCPruneFixture(&unknownTopic)) + malformedTopicPrunes = append(malformedTopicPrunes, unittest.P2PRPCPruneFixture(&malformedTopic)) + invalidSporkIDTopicPrunes = append(invalidSporkIDTopicPrunes, unittest.P2PRPCPruneFixture(&invalidSporkIDTopic)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(unknownTopicPrunes...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(malformedTopicPrunes...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(invalidSporkIDTopicPrunes...)) + + from := unittest.PeerIdFixture(t) + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgPrune, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + // TestPruneInspection_DuplicateTopicIds_AboveThreshold ensures inspector disseminates an invalid control message notification for // prune messages when the number of duplicate topic ids is above the threshold. func TestPruneInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { @@ -572,6 +724,99 @@ func TestIHaveInspection_InvalidTopic(t *testing.T) { unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") } +// TestIHaveInspection_InvalidTopic_BelowThreshold ensures inspector does not disseminate an invalid control message notification for +// ihave messages when the invalid topic id count does not exceed the configured threshold. +func TestIHaveInspection_InvalidTopic_BelowThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicIHaves []*pubsub_pb.ControlIHave + var malformedTopicIHaves []*pubsub_pb.ControlIHave + var invalidSporkIDTopicIHaves []*pubsub_pb.ControlIHave + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicIHaves = append(unknownTopicIHaves, unittest.P2PRPCIHaveFixture(&unknownTopic, unittest.IdentifierListFixture(5).Strings()...)) + malformedTopicIHaves = append(malformedTopicIHaves, unittest.P2PRPCIHaveFixture(&malformedTopic, unittest.IdentifierListFixture(5).Strings()...)) + invalidSporkIDTopicIHaves = append(invalidSporkIDTopicIHaves, unittest.P2PRPCIHaveFixture(&invalidSporkIDTopic, unittest.IdentifierListFixture(5).Strings()...)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(unknownTopicIHaves...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(malformedTopicIHaves...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(invalidSporkIDTopicIHaves...)) + + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + // no notification should be disseminated for valid messages as long as the number of invalid topic ids is below the threshold + distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(2 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + +// TestIHaveInspection_InvalidTopic_AboveThreshold ensures inspector disseminates an invalid control message notification for +// ihave messages when the invalid topic id count exceeds the configured threshold. +func TestIHaveInspection_InvalidTopic_AboveThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicIHaves []*pubsub_pb.ControlIHave + var malformedTopicIHaves []*pubsub_pb.ControlIHave + var invalidSporkIDTopicIHaves []*pubsub_pb.ControlIHave + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold+1; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicIHaves = append(unknownTopicIHaves, unittest.P2PRPCIHaveFixture(&unknownTopic, unittest.IdentifierListFixture(5).Strings()...)) + malformedTopicIHaves = append(malformedTopicIHaves, unittest.P2PRPCIHaveFixture(&malformedTopic, unittest.IdentifierListFixture(5).Strings()...)) + invalidSporkIDTopicIHaves = append(invalidSporkIDTopicIHaves, unittest.P2PRPCIHaveFixture(&invalidSporkIDTopic, unittest.IdentifierListFixture(5).Strings()...)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(unknownTopicIHaves...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(malformedTopicIHaves...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(invalidSporkIDTopicIHaves...)) + + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) + distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(2 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + // TestIHaveInspection_DuplicateTopicIds_BelowThreshold ensures inspector does not disseminate an invalid control message notification for // iHave messages when duplicate topic ids are below allowed threshold. func TestIHaveInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { From 85527b81bb9c190bf51b1e4b4f35f9e07b39320f Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 10:01:50 -0600 Subject: [PATCH 19/52] Add ability to read or extract payloads from state Added two flags to execution state extraction program: --extract-payloads-by-address produces a file of payloads for specified accounts or for all accounts instead of checkpoint files --use-payload-as-input uses payload file as input instead of checkpoint files The two new flags don't affect migration and other existing functionaly of state extraction program. These two options only affect input and output of state extraction program. In other words, this can be used to extract migrated payloads or extract as-is payloads for specified accounts. --- cmd/util/cmd/execution-state-extract/cmd.go | 116 ++++++- .../execution_state_extract.go | 199 +++++++++-- .../execution_state_extract_test.go | 327 +++++++++++++++++- .../export_payloads.go | 205 +++++++++++ 4 files changed, 797 insertions(+), 50 deletions(-) create mode 100644 cmd/util/cmd/execution-state-extract/export_payloads.go diff --git a/cmd/util/cmd/execution-state-extract/cmd.go b/cmd/util/cmd/execution-state-extract/cmd.go index 55728b428a8..55e3432ba9c 100644 --- a/cmd/util/cmd/execution-state-extract/cmd.go +++ b/cmd/util/cmd/execution-state-extract/cmd.go @@ -3,10 +3,13 @@ package extract import ( "encoding/hex" "path" + "strings" "github.com/rs/zerolog/log" "github.com/spf13/cobra" + runtimeCommon "github.com/onflow/cadence/runtime/common" + "github.com/onflow/flow-go/cmd/util/cmd/common" "github.com/onflow/flow-go/model/bootstrap" "github.com/onflow/flow-go/model/flow" @@ -26,6 +29,8 @@ var ( flagNoReport bool flagValidateMigration bool flagLogVerboseValidationError bool + flagInputPayload bool + flagOutputPayloadByAddresses string ) var Cmd = &cobra.Command{ @@ -68,6 +73,19 @@ func init() { Cmd.Flags().BoolVar(&flagLogVerboseValidationError, "log-verbose-validation-error", false, "log entire Cadence values on validation error (atree migration)") + Cmd.Flags().StringVar( + &flagOutputPayloadByAddresses, + "extract-payloads-by-address", + "", + "extract payloads of specified addresses (comma separated list of hex-encoded addresses or \"all\"", // empty string ignores this flag + ) + + Cmd.Flags().BoolVar( + &flagInputPayload, + "use-payload-as-input", + false, + "use payload file instead of checkpoint file as input", + ) } func run(*cobra.Command, []string) { @@ -112,20 +130,65 @@ func run(*cobra.Command, []string) { log.Info().Msgf("extracting state by state commitment: %x", stateCommitment) } - if len(flagBlockHash) == 0 && len(flagStateCommitment) == 0 { - log.Fatal().Msg("no --block-hash or --state-commitment was specified") + if len(flagBlockHash) == 0 && len(flagStateCommitment) == 0 && !flagInputPayload { + log.Fatal().Msg("no --block-hash or --state-commitment or --use-payload-as-input was specified") } - log.Info().Msgf("Extracting state from %s, exporting root checkpoint to %s, version: %v", - flagExecutionStateDir, - path.Join(flagOutputDir, bootstrap.FilenameWALRootCheckpoint), - 6, - ) + exportPayloads := len(flagOutputPayloadByAddresses) > 0 + + var exportedAddresses []runtimeCommon.Address + + if exportPayloads { + + addresses := strings.Split(flagOutputPayloadByAddresses, ",") + + if len(addresses) == 1 && strings.TrimSpace(addresses[0]) == "all" { + // Extract payloads of the entire state. + log.Info().Msgf("Extracting state from %s, exporting all payloads to %s", + flagExecutionStateDir, + path.Join(flagOutputDir, FilenamePayloads), + ) + } else { + // Extract payloads of specified accounts + for _, hexAddr := range addresses { + b, err := hex.DecodeString(strings.TrimSpace(hexAddr)) + if err != nil { + log.Fatal().Err(err).Msgf("cannot hex decode address %s for payload export", strings.TrimSpace(hexAddr)) + } + + addr, err := runtimeCommon.BytesToAddress(b) + if err != nil { + log.Fatal().Err(err).Msgf("cannot decode address %x for payload export", b) + } + + exportedAddresses = append(exportedAddresses, addr) + } + + log.Info().Msgf("Extracting state from %s, exporting payloads by addresses %v to %s", + flagExecutionStateDir, + flagOutputPayloadByAddresses, + path.Join(flagOutputDir, FilenamePayloads), + ) + } - log.Info().Msgf("Block state commitment: %s from %v, output dir: %s", - hex.EncodeToString(stateCommitment[:]), - flagExecutionStateDir, - flagOutputDir) + } else { + log.Info().Msgf("Extracting state from %s, exporting root checkpoint to %s, version: %v", + flagExecutionStateDir, + path.Join(flagOutputDir, bootstrap.FilenameWALRootCheckpoint), + 6, + ) + } + + if flagInputPayload { + log.Info().Msgf("Payload input from %v, output dir: %s", + flagExecutionStateDir, + flagOutputDir) + } else { + log.Info().Msgf("Block state commitment: %s from %v, output dir: %s", + hex.EncodeToString(stateCommitment[:]), + flagExecutionStateDir, + flagOutputDir) + } // err := ensureCheckpointFileExist(flagExecutionStateDir) // if err != nil { @@ -148,14 +211,29 @@ func run(*cobra.Command, []string) { log.Warn().Msgf("atree migration has verbose validation error logging enabled which may increase size of log") } - err := extractExecutionState( - log.Logger, - flagExecutionStateDir, - stateCommitment, - flagOutputDir, - flagNWorker, - !flagNoMigration, - ) + var err error + if flagInputPayload { + err = extractExecutionStateFromPayloads( + log.Logger, + flagExecutionStateDir, + flagOutputDir, + flagNWorker, + !flagNoMigration, + exportPayloads, + exportedAddresses, + ) + } else { + err = extractExecutionState( + log.Logger, + flagExecutionStateDir, + stateCommitment, + flagOutputDir, + flagNWorker, + !flagNoMigration, + exportPayloads, + exportedAddresses, + ) + } if err != nil { log.Fatal().Err(err).Msgf("error extracting the execution state: %s", err.Error()) diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract.go b/cmd/util/cmd/execution-state-extract/execution_state_extract.go index 90bcd70533d..dc552682886 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract.go @@ -5,7 +5,9 @@ import ( "fmt" "math" "os" + "time" + "github.com/onflow/cadence/runtime/common" "github.com/rs/zerolog" "go.uber.org/atomic" @@ -34,6 +36,8 @@ func extractExecutionState( outputDir string, nWorker int, // number of concurrent worker to migation payloads runMigrations bool, + exportPayloads bool, + exportPayloadsByAddresses []common.Address, ) error { log.Info().Msg("init WAL") @@ -84,30 +88,7 @@ func extractExecutionState( <-compactor.Done() }() - var migrations []ledger.Migration - - if runMigrations { - rwf := reporters.NewReportFileWriterFactory(dir, log) - - migrations = []ledger.Migration{ - migrators.CreateAccountBasedMigration( - log, - nWorker, - []migrators.AccountBasedMigration{ - migrators.NewAtreeRegisterMigrator( - rwf, - flagValidateMigration, - flagLogVerboseValidationError, - ), - - &migrators.DeduplicateContractNamesMigration{}, - - // This will fix storage used discrepancies caused by the - // DeduplicateContractNamesMigration. - &migrators.AccountUsageMigrator{}, - }), - } - } + migrations := newMigrations(log, dir, nWorker, runMigrations) newState := ledger.State(targetHash) @@ -134,6 +115,19 @@ func extractExecutionState( log.Error().Err(err).Msgf("can not generate report for migrated state: %v", newMigratedState) } + if exportPayloads { + payloads := newTrie.AllPayloads() + + exportedPayloadCount, err := createPayloadFile(log, outputDir, payloads, exportPayloadsByAddresses) + if err != nil { + return fmt.Errorf("cannot generate payloads file: %w", err) + } + + log.Info().Msgf("Exported %d payloads out of %d payloads", exportedPayloadCount, len(payloads)) + + return nil + } + migratedState, err := createCheckpoint( newTrie, log, @@ -191,3 +185,160 @@ func writeStatusFile(fileName string, e error) error { err := os.WriteFile(fileName, checkpointStatusJson, 0644) return err } + +func extractExecutionStateFromPayloads( + log zerolog.Logger, + dir string, + outputDir string, + nWorker int, // number of concurrent worker to migation payloads + runMigrations bool, + exportPayloads bool, + exportPayloadsByAddresses []common.Address, +) error { + + payloads, err := readPayloadFile(log, dir) + if err != nil { + return err + } + + log.Info().Msgf("read %d payloads\n", len(payloads)) + + migrations := newMigrations(log, dir, nWorker, runMigrations) + + payloads, err = migratePayloads(log, payloads, migrations) + if err != nil { + return err + } + + if exportPayloads { + exportedPayloadCount, err := createPayloadFile(log, outputDir, payloads, exportPayloadsByAddresses) + if err != nil { + return fmt.Errorf("cannot generate payloads file: %w", err) + } + + log.Info().Msgf("Exported %d payloads out of %d payloads", exportedPayloadCount, len(payloads)) + + return nil + } + + newTrie, err := createTrieFromPayloads(log, payloads) + if err != nil { + return err + } + + migratedState, err := createCheckpoint( + newTrie, + log, + outputDir, + bootstrap.FilenameWALRootCheckpoint, + ) + if err != nil { + return fmt.Errorf("cannot generate the output checkpoint: %w", err) + } + + log.Info().Msgf( + "New state commitment for the exported state is: %s (base64: %s)", + migratedState.String(), + migratedState.Base64(), + ) + + return nil +} + +func migratePayloads(logger zerolog.Logger, payloads []*ledger.Payload, migrations []ledger.Migration) ([]*ledger.Payload, error) { + + if len(migrations) == 0 { + return payloads, nil + } + + var err error + payloadCount := len(payloads) + + // migrate payloads + for i, migrate := range migrations { + logger.Info().Msgf("migration %d/%d is underway", i, len(migrations)) + + start := time.Now() + payloads, err = migrate(payloads) + elapsed := time.Since(start) + + if err != nil { + return nil, fmt.Errorf("error applying migration (%d): %w", i, err) + } + + newPayloadCount := len(payloads) + + if payloadCount != newPayloadCount { + logger.Warn(). + Int("migration_step", i). + Int("expected_size", payloadCount). + Int("outcome_size", newPayloadCount). + Msg("payload counts has changed during migration, make sure this is expected.") + } + logger.Info().Str("timeTaken", elapsed.String()).Msgf("migration %d is done", i) + + payloadCount = newPayloadCount + } + + return payloads, nil +} + +func createTrieFromPayloads(logger zerolog.Logger, payloads []*ledger.Payload) (*trie.MTrie, error) { + // get paths + paths, err := pathfinder.PathsFromPayloads(payloads, complete.DefaultPathFinderVersion) + if err != nil { + return nil, fmt.Errorf("cannot export checkpoint, can't construct paths: %w", err) + } + + logger.Info().Msgf("constructing a new trie with migrated payloads (count: %d)...", len(payloads)) + + emptyTrie := trie.NewEmptyMTrie() + + derefPayloads := make([]ledger.Payload, len(payloads)) + for i, p := range payloads { + derefPayloads[i] = *p + } + + // no need to prune the data since it has already been prunned through migrations + applyPruning := false + newTrie, _, err := trie.NewTrieWithUpdatedRegisters(emptyTrie, paths, derefPayloads, applyPruning) + if err != nil { + return nil, fmt.Errorf("constructing updated trie failed: %w", err) + } + + return newTrie, nil +} + +func newMigrations( + log zerolog.Logger, + dir string, + nWorker int, // number of concurrent worker to migation payloads + runMigrations bool, +) []ledger.Migration { + if runMigrations { + rwf := reporters.NewReportFileWriterFactory(dir, log) + + migrations := []ledger.Migration{ + migrators.CreateAccountBasedMigration( + log, + nWorker, + []migrators.AccountBasedMigration{ + migrators.NewAtreeRegisterMigrator( + rwf, + flagValidateMigration, + flagLogVerboseValidationError, + ), + + &migrators.DeduplicateContractNamesMigration{}, + + // This will fix storage used discrepancies caused by the + // DeduplicateContractNamesMigration. + &migrators.AccountUsageMigrator{}, + }), + } + + return migrations + } + + return nil +} diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go index 2f91ea7d603..39b05ad557f 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go @@ -2,13 +2,17 @@ package extract import ( "crypto/rand" + "encoding/hex" "math" + "strings" "testing" "github.com/rs/zerolog" "github.com/stretchr/testify/require" "go.uber.org/atomic" + runtimeCommon "github.com/onflow/cadence/runtime/common" + "github.com/onflow/flow-go/cmd/util/cmd/common" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/pathfinder" @@ -66,6 +70,8 @@ func TestExtractExecutionState(t *testing.T) { outdir, 10, false, + false, + nil, ) require.Error(t, err) }) @@ -96,7 +102,7 @@ func TestExtractExecutionState(t *testing.T) { var stateCommitment = f.InitialState() - //saved data after updates + // saved data after updates keysValuesByCommit := make(map[string]map[string]keyPair) commitsByBlocks := make(map[flow.Identifier]ledger.State) blocksInOrder := make([]flow.Identifier, size) @@ -108,7 +114,7 @@ func TestExtractExecutionState(t *testing.T) { require.NoError(t, err) stateCommitment, _, err = f.Set(update) - //stateCommitment, err = f.UpdateRegisters(keys, values, stateCommitment) + // stateCommitment, err = f.UpdateRegisters(keys, values, stateCommitment) require.NoError(t, err) // generate random block and map it to state commitment @@ -135,13 +141,13 @@ func TestExtractExecutionState(t *testing.T) { err = db.Close() require.NoError(t, err) - //for blockID, stateCommitment := range commitsByBlocks { + // for blockID, stateCommitment := range commitsByBlocks { for i, blockID := range blocksInOrder { stateCommitment := commitsByBlocks[blockID] - //we need fresh output dir to prevent contamination + // we need fresh output dir to prevent contamination unittest.RunWithTempDir(t, func(outdir string) { Cmd.SetArgs([]string{ @@ -182,7 +188,7 @@ func TestExtractExecutionState(t *testing.T) { require.NoError(t, err) registerValues, err := storage.Get(query) - //registerValues, err := mForest.Read([]byte(stateCommitment), keys) + // registerValues, err := mForest.Read([]byte(stateCommitment), keys) require.NoError(t, err) for i, key := range keys { @@ -190,7 +196,7 @@ func TestExtractExecutionState(t *testing.T) { require.Equal(t, data[key.String()].value, registerValue) } - //make sure blocks after this one are not in checkpoint + // make sure blocks after this one are not in checkpoint // ie - extraction stops after hitting right hash for j := i + 1; j < len(blocksInOrder); j++ { @@ -207,6 +213,312 @@ func TestExtractExecutionState(t *testing.T) { }) } +// TestExtractPayloadsFromExecutionState tests state extraction with checkpoint as input and payload as output. +func TestExtractPayloadsFromExecutionState(t *testing.T) { + + metr := &metrics.NoopCollector{} + + t.Run("all payloads", func(t *testing.T) { + withDirs(t, func(_, execdir, outdir string) { + + const ( + checkpointDistance = math.MaxInt // A large number to prevent checkpoint creation. + checkpointsToKeep = 1 + ) + + size := 10 + + diskWal, err := wal.NewDiskWAL(zerolog.Nop(), nil, metrics.NewNoopCollector(), execdir, size, pathfinder.PathByteSize, wal.SegmentSize) + require.NoError(t, err) + f, err := complete.NewLedger(diskWal, size*10, metr, zerolog.Nop(), complete.DefaultPathFinderVersion) + require.NoError(t, err) + compactor, err := complete.NewCompactor(f, diskWal, zerolog.Nop(), uint(size), checkpointDistance, checkpointsToKeep, atomic.NewBool(false)) + require.NoError(t, err) + <-compactor.Ready() + + var stateCommitment = f.InitialState() + + // Save generated data after updates + keysValues := make(map[string]keyPair) + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + update, err := ledger.NewUpdate(stateCommitment, keys, values) + require.NoError(t, err) + + stateCommitment, _, err = f.Set(update) + require.NoError(t, err) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + } + } + + <-f.Done() + <-compactor.Done() + + tries, err := f.Tries() + require.NoError(t, err) + + err = wal.StoreCheckpointV6SingleThread(tries, execdir, "checkpoint.00000001", zerolog.Nop()) + require.NoError(t, err) + + // Export all payloads + Cmd.SetArgs([]string{ + "--execution-state-dir", execdir, + "--output-dir", outdir, + "--state-commitment", hex.EncodeToString(stateCommitment[:]), + "--no-migration", + "--no-report", + "--extract-payloads-by-address", "all", + "--chain", flow.Emulator.Chain().String()}) + + err = Cmd.Execute() + require.NoError(t, err) + + // Verify exported payloads. + payloadsFromFile, err := readPayloadFile(zerolog.Nop(), outdir) + require.NoError(t, err) + require.Equal(t, len(keysValues), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := keysValues[k.String()] + require.True(t, exist) + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + }) + + t.Run("some payloads", func(t *testing.T) { + withDirs(t, func(_, execdir, outdir string) { + const ( + checkpointDistance = math.MaxInt // A large number to prevent checkpoint creation. + checkpointsToKeep = 1 + ) + + size := 10 + + diskWal, err := wal.NewDiskWAL(zerolog.Nop(), nil, metrics.NewNoopCollector(), execdir, size, pathfinder.PathByteSize, wal.SegmentSize) + require.NoError(t, err) + f, err := complete.NewLedger(diskWal, size*10, metr, zerolog.Nop(), complete.DefaultPathFinderVersion) + require.NoError(t, err) + compactor, err := complete.NewCompactor(f, diskWal, zerolog.Nop(), uint(size), checkpointDistance, checkpointsToKeep, atomic.NewBool(false)) + require.NoError(t, err) + <-compactor.Ready() + + var stateCommitment = f.InitialState() + + // Save generated data after updates + keysValues := make(map[string]keyPair) + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + update, err := ledger.NewUpdate(stateCommitment, keys, values) + require.NoError(t, err) + + stateCommitment, _, err = f.Set(update) + require.NoError(t, err) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + } + } + + <-f.Done() + <-compactor.Done() + + tries, err := f.Tries() + require.NoError(t, err) + + err = wal.StoreCheckpointV6SingleThread(tries, execdir, "checkpoint.00000001", zerolog.Nop()) + require.NoError(t, err) + + const selectedAddressCount = 10 + selectedAddresses := make(map[string]struct{}) + selectedKeysValues := make(map[string]keyPair) + for k, kv := range keysValues { + owner := kv.key.KeyParts[0].Value + if len(owner) != runtimeCommon.AddressLength { + continue + } + + address, err := runtimeCommon.BytesToAddress(owner) + require.NoError(t, err) + + if len(selectedAddresses) < selectedAddressCount { + selectedAddresses[address.Hex()] = struct{}{} + } + + if _, exist := selectedAddresses[address.Hex()]; exist { + selectedKeysValues[k] = kv + } + } + + addresses := make([]string, 0, len(selectedAddresses)) + for address := range selectedAddresses { + addresses = append(addresses, address) + } + + // Export selected payloads + Cmd.SetArgs([]string{ + "--execution-state-dir", execdir, + "--output-dir", outdir, + "--state-commitment", hex.EncodeToString(stateCommitment[:]), + "--no-migration", + "--no-report", + "--extract-payloads-by-address", strings.Join(addresses, ","), + "--chain", flow.Emulator.Chain().String()}) + + err = Cmd.Execute() + require.NoError(t, err) + + // Verify exported payloads. + payloadsFromFile, err := readPayloadFile(zerolog.Nop(), outdir) + require.NoError(t, err) + require.Equal(t, len(selectedKeysValues), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := selectedKeysValues[k.String()] + require.True(t, exist) + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + }) +} + +// TestExtractStateFromPayloads tests state extraction with payload as input. +func TestExtractStateFromPayloads(t *testing.T) { + + t.Run("create checkpoint", func(t *testing.T) { + withDirs(t, func(_, execdir, outdir string) { + size := 10 + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + numOfPayloadWritten, err := createPayloadFile(zerolog.Nop(), execdir, payloads, nil) + require.NoError(t, err) + require.Equal(t, len(payloads), numOfPayloadWritten) + + // Export checkpoint file + Cmd.SetArgs([]string{ + "--execution-state-dir", execdir, + "--output-dir", outdir, + "--no-migration", + "--no-report", + "--use-payload-as-input", + "--extract-payloads-by-address", "", + "--chain", flow.Emulator.Chain().String()}) + + err = Cmd.Execute() + require.NoError(t, err) + + tries, err := wal.OpenAndReadCheckpointV6(outdir, "root.checkpoint", zerolog.Nop()) + require.NoError(t, err) + require.Equal(t, 1, len(tries)) + + // Verify exported checkpoint + payloadsFromFile := tries[0].AllPayloads() + require.NoError(t, err) + require.Equal(t, len(keysValues), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := keysValues[k.String()] + require.True(t, exist) + + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + + }) + + t.Run("create payloads", func(t *testing.T) { + withDirs(t, func(_, execdir, outdir string) { + size := 10 + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + numOfPayloadWritten, err := createPayloadFile(zerolog.Nop(), execdir, payloads, nil) + require.NoError(t, err) + require.Equal(t, len(payloads), numOfPayloadWritten) + + // Export all payloads + Cmd.SetArgs([]string{ + "--execution-state-dir", execdir, + "--output-dir", outdir, + "--no-migration", + "--no-report", + "--use-payload-as-input", + "--extract-payloads-by-address", "all", + "--chain", flow.Emulator.Chain().String()}) + + err = Cmd.Execute() + require.NoError(t, err) + + // Verify exported payloads. + payloadsFromFile, err := readPayloadFile(zerolog.Nop(), outdir) + require.NoError(t, err) + require.Equal(t, len(keysValues), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := keysValues[k.String()] + require.True(t, exist) + + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + }) +} + func getSampleKeyValues(i int) ([]ledger.Key, []ledger.Value) { switch i { case 0: @@ -226,7 +538,8 @@ func getSampleKeyValues(i int) ([]ledger.Key, []ledger.Value) { keys := make([]ledger.Key, 0) values := make([]ledger.Value, 0) for j := 0; j < 10; j++ { - address := make([]byte, 32) + // address := make([]byte, 32) + address := make([]byte, 8) _, err := rand.Read(address) if err != nil { panic(err) diff --git a/cmd/util/cmd/execution-state-extract/export_payloads.go b/cmd/util/cmd/execution-state-extract/export_payloads.go new file mode 100644 index 00000000000..68325dac3de --- /dev/null +++ b/cmd/util/cmd/execution-state-extract/export_payloads.go @@ -0,0 +1,205 @@ +package extract + +import ( + "bufio" + "bytes" + "fmt" + "io" + "os" + "path/filepath" + + "github.com/fxamacker/cbor/v2" + "github.com/rs/zerolog" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/ledger" +) + +const ( + FilenamePayloads = "root.payloads" + + defaultBufioWriteSize = 1024 * 32 + defaultBufioReadSize = 1024 * 32 + + payloadEncodingVersion = 1 +) + +func createPayloadFile( + logger zerolog.Logger, + outputDir string, + payloads []*ledger.Payload, + addresses []common.Address, +) (int, error) { + payloadFile := filepath.Join(outputDir, FilenamePayloads) + + f, err := os.Create(payloadFile) + if err != nil { + return 0, fmt.Errorf("can't create %s: %w", payloadFile, err) + } + defer f.Close() + + writer := bufio.NewWriterSize(f, defaultBufioWriteSize) + if err != nil { + return 0, fmt.Errorf("can't create bufio writer for %s: %w", payloadFile, err) + } + defer writer.Flush() + + includeAllPayloads := len(addresses) == 0 + + if includeAllPayloads { + return writeAllPayloads(logger, writer, payloads) + } + + return writeSelectedPayloads(logger, writer, payloads, addresses) +} + +func writeAllPayloads(logger zerolog.Logger, w io.Writer, payloads []*ledger.Payload) (int, error) { + logger.Info().Msgf("writing %d payloads to file", len(payloads)) + + enc := cbor.NewEncoder(w) + + // Encode number of payloads + err := enc.Encode(len(payloads)) + if err != nil { + return 0, fmt.Errorf("failed to encode number of payloads %d in CBOR: %w", len(payloads), err) + } + + var payloadScratchBuffer [1024 * 2]byte + for _, p := range payloads { + + buf := ledger.EncodeAndAppendPayloadWithoutPrefix(payloadScratchBuffer[:0], p, payloadEncodingVersion) + + // Encode payload + err = enc.Encode(buf) + if err != nil { + return 0, err + } + } + + return len(payloads), nil +} + +func writeSelectedPayloads(logger zerolog.Logger, w io.Writer, payloads []*ledger.Payload, addresses []common.Address) (int, error) { + var includedPayloadCount int + + includedFlags := make([]bool, len(payloads)) + for i, p := range payloads { + include, err := includePayloadByAddresses(p, addresses) + if err != nil { + return 0, err + } + + includedFlags[i] = include + + if include { + includedPayloadCount++ + } + } + + logger.Info().Msgf("writing %d payloads to file", includedPayloadCount) + + enc := cbor.NewEncoder(w) + + // Encode number of payloads + err := enc.Encode(includedPayloadCount) + if err != nil { + return 0, fmt.Errorf("failed to encode number of payloads %d in CBOR: %w", includedPayloadCount, err) + } + + var payloadScratchBuffer [1024 * 2]byte + for i, included := range includedFlags { + if !included { + continue + } + + p := payloads[i] + + buf := ledger.EncodeAndAppendPayloadWithoutPrefix(payloadScratchBuffer[:0], p, payloadEncodingVersion) + + // Encode payload + err = enc.Encode(buf) + if err != nil { + return 0, err + } + } + + return includedPayloadCount, nil +} + +func includePayloadByAddresses(payload *ledger.Payload, addresses []common.Address) (bool, error) { + if len(addresses) == 0 { + // Include all payloads + return true, nil + } + + for _, address := range addresses { + k, err := payload.Key() + if err != nil { + return false, fmt.Errorf("failed to get key from payload: %w", err) + } + + owner := k.KeyParts[0].Value + if bytes.Equal(owner, address[:]) { + return true, nil + } + } + + return false, nil +} + +func readPayloadFile(logger zerolog.Logger, inputDir string) ([]*ledger.Payload, error) { + payloadFile := filepath.Join(inputDir, FilenamePayloads) + + if _, err := os.Stat(payloadFile); os.IsNotExist(err) { + return nil, fmt.Errorf("%s doesn't exist", payloadFile) + } + + f, err := os.Open(payloadFile) + if err != nil { + return nil, fmt.Errorf("failed to open %s: %w", payloadFile, err) + } + defer f.Close() + + r := bufio.NewReaderSize(f, defaultBufioReadSize) + if err != nil { + return nil, fmt.Errorf("failed to create bufio reader for %s: %w", payloadFile, err) + } + + dec := cbor.NewDecoder(r) + + // Decode number of payloads + var payloadCount int + err = dec.Decode(&payloadCount) + if err != nil { + return nil, fmt.Errorf("failed to decode number of payload in CBOR: %w", err) + } + + logger.Info().Msgf("reading %d payloads from file", payloadCount) + + payloads := make([]*ledger.Payload, 0, payloadCount) + + for { + var rawPayload []byte + err := dec.Decode(&rawPayload) + if err == io.EOF { + break + } + if err != nil { + return nil, fmt.Errorf("failed to decode payload in CBOR: %w", err) + } + + payload, err := ledger.DecodePayloadWithoutPrefix(rawPayload, false, payloadEncodingVersion) + if err != nil { + return nil, fmt.Errorf("failed to decode payload 0x%x: %w", rawPayload, err) + } + + payloads = append(payloads, payload) + } + + if payloadCount != len(payloads) { + return nil, fmt.Errorf("failed to decode %s: expect %d payloads, got %d payloads", payloadFile, payloadCount, len(payloads)) + } + + return payloads, nil +} From 9b1fc05aec9b651d99603de1c75409d390a10917 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 11:45:44 -0600 Subject: [PATCH 20/52] Add t.Parallel() to test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Bastian Müller --- .../cmd/execution-state-extract/execution_state_extract_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go index 39b05ad557f..2bdd4e6b083 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go @@ -215,7 +215,7 @@ func TestExtractExecutionState(t *testing.T) { // TestExtractPayloadsFromExecutionState tests state extraction with checkpoint as input and payload as output. func TestExtractPayloadsFromExecutionState(t *testing.T) { - + t.Parallel() metr := &metrics.NoopCollector{} t.Run("all payloads", func(t *testing.T) { From 5d5eb09c0f7289647effe987893826bf80138edd Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 12:04:38 -0600 Subject: [PATCH 21/52] Remove t.Parallel() to clean output dir correctly --- .../cmd/execution-state-extract/execution_state_extract_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go index 2bdd4e6b083..3e2441d853a 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go @@ -215,7 +215,6 @@ func TestExtractExecutionState(t *testing.T) { // TestExtractPayloadsFromExecutionState tests state extraction with checkpoint as input and payload as output. func TestExtractPayloadsFromExecutionState(t *testing.T) { - t.Parallel() metr := &metrics.NoopCollector{} t.Run("all payloads", func(t *testing.T) { From 392b8423e506d2d773fea80a99068ecc8d44c660 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 16:53:10 -0600 Subject: [PATCH 22/52] Refactor & add flags for payloads in state extraction Refactored payload file related functionality to be more reusable. Added flags: --input-payload-filename --output-payload-filename --- cmd/util/cmd/execution-state-extract/cmd.go | 152 +++++++++++------- .../execution_state_extract.go | 26 ++- .../execution_state_extract_test.go | 50 ++++-- .../util/payload_file.go} | 13 +- 4 files changed, 157 insertions(+), 84 deletions(-) rename cmd/util/{cmd/execution-state-extract/export_payloads.go => ledger/util/payload_file.go} (93%) diff --git a/cmd/util/cmd/execution-state-extract/cmd.go b/cmd/util/cmd/execution-state-extract/cmd.go index 55e3432ba9c..760c7ff0975 100644 --- a/cmd/util/cmd/execution-state-extract/cmd.go +++ b/cmd/util/cmd/execution-state-extract/cmd.go @@ -2,6 +2,8 @@ package extract import ( "encoding/hex" + "fmt" + "os" "path" "strings" @@ -29,7 +31,8 @@ var ( flagNoReport bool flagValidateMigration bool flagLogVerboseValidationError bool - flagInputPayload bool + flagInputPayloadFileName string + flagOutputPayloadFileName string flagOutputPayloadByAddresses string ) @@ -74,17 +77,27 @@ func init() { "log entire Cadence values on validation error (atree migration)") Cmd.Flags().StringVar( - &flagOutputPayloadByAddresses, - "extract-payloads-by-address", + &flagInputPayloadFileName, + "input-payload-filename", "", - "extract payloads of specified addresses (comma separated list of hex-encoded addresses or \"all\"", // empty string ignores this flag + "input payload file", ) - Cmd.Flags().BoolVar( - &flagInputPayload, - "use-payload-as-input", - false, - "use payload file instead of checkpoint file as input", + Cmd.Flags().StringVar( + &flagOutputPayloadFileName, + "output-payload-filename", + "", + "output payload file", + ) + + Cmd.Flags().StringVar( + // Extract payloads of specified addresses (comma separated list of hex-encoded addresses) + // to file specified by --output-payload-filename. + // If no address is specified (empty string) then this flag is ignored. + &flagOutputPayloadByAddresses, + "extract-payloads-by-address", + "", + "extract payloads of addresses (comma separated hex-encoded addresses) to file specified by output-payload-filename", ) } @@ -96,6 +109,18 @@ func run(*cobra.Command, []string) { return } + if len(flagBlockHash) == 0 && len(flagStateCommitment) == 0 && len(flagInputPayloadFileName) == 0 { + log.Fatal().Msg("--block-hash or --state-commitment or --input-payload-filename must be specified") + } + + if len(flagInputPayloadFileName) > 0 && (len(flagBlockHash) > 0 || len(flagStateCommitment) > 0) { + log.Fatal().Msg("--input-payload-filename cannot be used with --block-hash or --state-commitment") + } + + if len(flagOutputPayloadFileName) == 0 && len(flagOutputPayloadByAddresses) > 0 { + log.Fatal().Msg("--extract-payloads-by-address requires --output-payload-filename to be specified") + } + if len(flagBlockHash) > 0 { blockID, err := flow.HexStringToIdentifier(flagBlockHash) if err != nil { @@ -130,64 +155,37 @@ func run(*cobra.Command, []string) { log.Info().Msgf("extracting state by state commitment: %x", stateCommitment) } - if len(flagBlockHash) == 0 && len(flagStateCommitment) == 0 && !flagInputPayload { - log.Fatal().Msg("no --block-hash or --state-commitment or --use-payload-as-input was specified") + if len(flagInputPayloadFileName) > 0 { + if _, err := os.Stat(flagInputPayloadFileName); os.IsNotExist(err) { + log.Fatal().Msgf("payload input file %s doesn't exist", flagInputPayloadFileName) + } } - exportPayloads := len(flagOutputPayloadByAddresses) > 0 + if len(flagOutputPayloadFileName) > 0 { + if _, err := os.Stat(flagOutputPayloadFileName); os.IsExist(err) { + log.Fatal().Msgf("payload output file %s exists", flagOutputPayloadFileName) + } + } var exportedAddresses []runtimeCommon.Address - if exportPayloads { + if len(flagOutputPayloadByAddresses) > 0 { addresses := strings.Split(flagOutputPayloadByAddresses, ",") - if len(addresses) == 1 && strings.TrimSpace(addresses[0]) == "all" { - // Extract payloads of the entire state. - log.Info().Msgf("Extracting state from %s, exporting all payloads to %s", - flagExecutionStateDir, - path.Join(flagOutputDir, FilenamePayloads), - ) - } else { - // Extract payloads of specified accounts - for _, hexAddr := range addresses { - b, err := hex.DecodeString(strings.TrimSpace(hexAddr)) - if err != nil { - log.Fatal().Err(err).Msgf("cannot hex decode address %s for payload export", strings.TrimSpace(hexAddr)) - } - - addr, err := runtimeCommon.BytesToAddress(b) - if err != nil { - log.Fatal().Err(err).Msgf("cannot decode address %x for payload export", b) - } - - exportedAddresses = append(exportedAddresses, addr) + for _, hexAddr := range addresses { + b, err := hex.DecodeString(strings.TrimSpace(hexAddr)) + if err != nil { + log.Fatal().Err(err).Msgf("cannot hex decode address %s for payload export", strings.TrimSpace(hexAddr)) } - log.Info().Msgf("Extracting state from %s, exporting payloads by addresses %v to %s", - flagExecutionStateDir, - flagOutputPayloadByAddresses, - path.Join(flagOutputDir, FilenamePayloads), - ) - } - - } else { - log.Info().Msgf("Extracting state from %s, exporting root checkpoint to %s, version: %v", - flagExecutionStateDir, - path.Join(flagOutputDir, bootstrap.FilenameWALRootCheckpoint), - 6, - ) - } + addr, err := runtimeCommon.BytesToAddress(b) + if err != nil { + log.Fatal().Err(err).Msgf("cannot decode address %x for payload export", b) + } - if flagInputPayload { - log.Info().Msgf("Payload input from %v, output dir: %s", - flagExecutionStateDir, - flagOutputDir) - } else { - log.Info().Msgf("Block state commitment: %s from %v, output dir: %s", - hex.EncodeToString(stateCommitment[:]), - flagExecutionStateDir, - flagOutputDir) + exportedAddresses = append(exportedAddresses, addr) + } } // err := ensureCheckpointFileExist(flagExecutionStateDir) @@ -211,15 +209,51 @@ func run(*cobra.Command, []string) { log.Warn().Msgf("atree migration has verbose validation error logging enabled which may increase size of log") } + var inputMsg string + if len(flagInputPayloadFileName) > 0 { + // Input is payloads + inputMsg = fmt.Sprintf("reading payloads from %s", flagInputPayloadFileName) + } else { + // Input is execution state + inputMsg = fmt.Sprintf("reading block state commitment %s from %s", + hex.EncodeToString(stateCommitment[:]), + flagExecutionStateDir, + ) + } + + var outputMsg string + if len(flagOutputPayloadFileName) > 0 { + // Output is payload file + if len(exportedAddresses) == 0 { + outputMsg = fmt.Sprintf("exporting all payloads to %s", flagOutputPayloadFileName) + } else { + outputMsg = fmt.Sprintf( + "exporting payloads by addresses %v to %s", + flagOutputPayloadByAddresses, + flagOutputPayloadFileName, + ) + } + } else { + // Output is checkpoint files + outputMsg = fmt.Sprintf( + "exporting root checkpoint to %s, version: %d", + path.Join(flagOutputDir, bootstrap.FilenameWALRootCheckpoint), + 6, + ) + } + + log.Info().Msgf("%s, %s", inputMsg, outputMsg) + var err error - if flagInputPayload { + if len(flagInputPayloadFileName) > 0 { err = extractExecutionStateFromPayloads( log.Logger, flagExecutionStateDir, flagOutputDir, flagNWorker, !flagNoMigration, - exportPayloads, + flagInputPayloadFileName, + flagOutputPayloadFileName, exportedAddresses, ) } else { @@ -230,7 +264,7 @@ func run(*cobra.Command, []string) { flagOutputDir, flagNWorker, !flagNoMigration, - exportPayloads, + flagOutputPayloadFileName, exportedAddresses, ) } diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract.go b/cmd/util/cmd/execution-state-extract/execution_state_extract.go index dc552682886..4fa7697f44a 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract.go @@ -13,6 +13,7 @@ import ( migrators "github.com/onflow/flow-go/cmd/util/ledger/migrations" "github.com/onflow/flow-go/cmd/util/ledger/reporters" + "github.com/onflow/flow-go/cmd/util/ledger/util" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/hash" "github.com/onflow/flow-go/ledger/common/pathfinder" @@ -36,7 +37,7 @@ func extractExecutionState( outputDir string, nWorker int, // number of concurrent worker to migation payloads runMigrations bool, - exportPayloads bool, + outputPayloadFile string, exportPayloadsByAddresses []common.Address, ) error { @@ -115,10 +116,16 @@ func extractExecutionState( log.Error().Err(err).Msgf("can not generate report for migrated state: %v", newMigratedState) } + exportPayloads := len(outputPayloadFile) > 0 if exportPayloads { payloads := newTrie.AllPayloads() - exportedPayloadCount, err := createPayloadFile(log, outputDir, payloads, exportPayloadsByAddresses) + exportedPayloadCount, err := util.CreatePayloadFile( + log, + outputPayloadFile, + payloads, + exportPayloadsByAddresses, + ) if err != nil { return fmt.Errorf("cannot generate payloads file: %w", err) } @@ -192,16 +199,17 @@ func extractExecutionStateFromPayloads( outputDir string, nWorker int, // number of concurrent worker to migation payloads runMigrations bool, - exportPayloads bool, + inputPayloadFile string, + outputPayloadFile string, exportPayloadsByAddresses []common.Address, ) error { - payloads, err := readPayloadFile(log, dir) + payloads, err := util.ReadPayloadFile(log, inputPayloadFile) if err != nil { return err } - log.Info().Msgf("read %d payloads\n", len(payloads)) + log.Info().Msgf("read %d payloads", len(payloads)) migrations := newMigrations(log, dir, nWorker, runMigrations) @@ -210,8 +218,14 @@ func extractExecutionStateFromPayloads( return err } + exportPayloads := len(outputPayloadFile) > 0 if exportPayloads { - exportedPayloadCount, err := createPayloadFile(log, outputDir, payloads, exportPayloadsByAddresses) + exportedPayloadCount, err := util.CreatePayloadFile( + log, + outputPayloadFile, + payloads, + exportPayloadsByAddresses, + ) if err != nil { return fmt.Errorf("cannot generate payloads file: %w", err) } diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go index 3e2441d853a..d193a0169ef 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go @@ -4,6 +4,7 @@ import ( "crypto/rand" "encoding/hex" "math" + "path/filepath" "strings" "testing" @@ -14,6 +15,7 @@ import ( runtimeCommon "github.com/onflow/cadence/runtime/common" "github.com/onflow/flow-go/cmd/util/cmd/common" + "github.com/onflow/flow-go/cmd/util/ledger/util" "github.com/onflow/flow-go/ledger" "github.com/onflow/flow-go/ledger/common/pathfinder" "github.com/onflow/flow-go/ledger/complete" @@ -70,7 +72,7 @@ func TestExtractExecutionState(t *testing.T) { outdir, 10, false, - false, + "", nil, ) require.Error(t, err) @@ -217,6 +219,8 @@ func TestExtractExecutionState(t *testing.T) { func TestExtractPayloadsFromExecutionState(t *testing.T) { metr := &metrics.NoopCollector{} + const payloadFileName = "root.payload" + t.Run("all payloads", func(t *testing.T) { withDirs(t, func(_, execdir, outdir string) { @@ -225,6 +229,8 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { checkpointsToKeep = 1 ) + outputPayloadFileName := filepath.Join(outdir, payloadFileName) + size := 10 diskWal, err := wal.NewDiskWAL(zerolog.Nop(), nil, metrics.NewNoopCollector(), execdir, size, pathfinder.PathByteSize, wal.SegmentSize) @@ -273,14 +279,14 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { "--state-commitment", hex.EncodeToString(stateCommitment[:]), "--no-migration", "--no-report", - "--extract-payloads-by-address", "all", + "--output-payload-filename", outputPayloadFileName, "--chain", flow.Emulator.Chain().String()}) err = Cmd.Execute() require.NoError(t, err) // Verify exported payloads. - payloadsFromFile, err := readPayloadFile(zerolog.Nop(), outdir) + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), outputPayloadFileName) require.NoError(t, err) require.Equal(t, len(keysValues), len(payloadsFromFile)) @@ -302,6 +308,8 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { checkpointsToKeep = 1 ) + outputPayloadFileName := filepath.Join(outdir, payloadFileName) + size := 10 diskWal, err := wal.NewDiskWAL(zerolog.Nop(), nil, metrics.NewNoopCollector(), execdir, size, pathfinder.PathByteSize, wal.SegmentSize) @@ -376,6 +384,7 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { "--state-commitment", hex.EncodeToString(stateCommitment[:]), "--no-migration", "--no-report", + "--output-payload-filename", outputPayloadFileName, "--extract-payloads-by-address", strings.Join(addresses, ","), "--chain", flow.Emulator.Chain().String()}) @@ -383,7 +392,7 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { require.NoError(t, err) // Verify exported payloads. - payloadsFromFile, err := readPayloadFile(zerolog.Nop(), outdir) + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), outputPayloadFileName) require.NoError(t, err) require.Equal(t, len(selectedKeysValues), len(payloadsFromFile)) @@ -402,10 +411,14 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { // TestExtractStateFromPayloads tests state extraction with payload as input. func TestExtractStateFromPayloads(t *testing.T) { + const payloadFileName = "root.payload" + t.Run("create checkpoint", func(t *testing.T) { withDirs(t, func(_, execdir, outdir string) { size := 10 + inputPayloadFileName := filepath.Join(execdir, payloadFileName) + // Generate some data keysValues := make(map[string]keyPair) var payloads []*ledger.Payload @@ -423,7 +436,12 @@ func TestExtractStateFromPayloads(t *testing.T) { } } - numOfPayloadWritten, err := createPayloadFile(zerolog.Nop(), execdir, payloads, nil) + numOfPayloadWritten, err := util.CreatePayloadFile( + zerolog.Nop(), + inputPayloadFileName, + payloads, + nil, + ) require.NoError(t, err) require.Equal(t, len(payloads), numOfPayloadWritten) @@ -433,7 +451,9 @@ func TestExtractStateFromPayloads(t *testing.T) { "--output-dir", outdir, "--no-migration", "--no-report", - "--use-payload-as-input", + "--state-commitment", "", + "--input-payload-filename", inputPayloadFileName, + "--output-payload-filename", "", "--extract-payloads-by-address", "", "--chain", flow.Emulator.Chain().String()}) @@ -464,6 +484,9 @@ func TestExtractStateFromPayloads(t *testing.T) { t.Run("create payloads", func(t *testing.T) { withDirs(t, func(_, execdir, outdir string) { + inputPayloadFileName := filepath.Join(execdir, payloadFileName) + outputPayloadFileName := filepath.Join(outdir, "selected.payload") + size := 10 // Generate some data @@ -483,7 +506,12 @@ func TestExtractStateFromPayloads(t *testing.T) { } } - numOfPayloadWritten, err := createPayloadFile(zerolog.Nop(), execdir, payloads, nil) + numOfPayloadWritten, err := util.CreatePayloadFile( + zerolog.Nop(), + inputPayloadFileName, + payloads, + nil, + ) require.NoError(t, err) require.Equal(t, len(payloads), numOfPayloadWritten) @@ -493,15 +521,17 @@ func TestExtractStateFromPayloads(t *testing.T) { "--output-dir", outdir, "--no-migration", "--no-report", - "--use-payload-as-input", - "--extract-payloads-by-address", "all", + "--state-commitment", "", + "--input-payload-filename", inputPayloadFileName, + "--output-payload-filename", outputPayloadFileName, + "--extract-payloads-by-address", "", "--chain", flow.Emulator.Chain().String()}) err = Cmd.Execute() require.NoError(t, err) // Verify exported payloads. - payloadsFromFile, err := readPayloadFile(zerolog.Nop(), outdir) + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), outputPayloadFileName) require.NoError(t, err) require.Equal(t, len(keysValues), len(payloadsFromFile)) diff --git a/cmd/util/cmd/execution-state-extract/export_payloads.go b/cmd/util/ledger/util/payload_file.go similarity index 93% rename from cmd/util/cmd/execution-state-extract/export_payloads.go rename to cmd/util/ledger/util/payload_file.go index 68325dac3de..ef226d89f86 100644 --- a/cmd/util/cmd/execution-state-extract/export_payloads.go +++ b/cmd/util/ledger/util/payload_file.go @@ -1,4 +1,4 @@ -package extract +package util import ( "bufio" @@ -6,7 +6,6 @@ import ( "fmt" "io" "os" - "path/filepath" "github.com/fxamacker/cbor/v2" "github.com/rs/zerolog" @@ -17,21 +16,18 @@ import ( ) const ( - FilenamePayloads = "root.payloads" - defaultBufioWriteSize = 1024 * 32 defaultBufioReadSize = 1024 * 32 payloadEncodingVersion = 1 ) -func createPayloadFile( +func CreatePayloadFile( logger zerolog.Logger, - outputDir string, + payloadFile string, payloads []*ledger.Payload, addresses []common.Address, ) (int, error) { - payloadFile := filepath.Join(outputDir, FilenamePayloads) f, err := os.Create(payloadFile) if err != nil { @@ -148,8 +144,7 @@ func includePayloadByAddresses(payload *ledger.Payload, addresses []common.Addre return false, nil } -func readPayloadFile(logger zerolog.Logger, inputDir string) ([]*ledger.Payload, error) { - payloadFile := filepath.Join(inputDir, FilenamePayloads) +func ReadPayloadFile(logger zerolog.Logger, payloadFile string) ([]*ledger.Payload, error) { if _, err := os.Stat(payloadFile); os.IsNotExist(err) { return nil, fmt.Errorf("%s doesn't exist", payloadFile) From 74776662f6cc3413c7dbb284847d7de32f7efb26 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 17:04:07 -0600 Subject: [PATCH 23/52] Optimize payload filtering in state extraction --- cmd/util/ledger/util/payload_file.go | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/cmd/util/ledger/util/payload_file.go b/cmd/util/ledger/util/payload_file.go index ef226d89f86..6524cce8261 100644 --- a/cmd/util/ledger/util/payload_file.go +++ b/cmd/util/ledger/util/payload_file.go @@ -129,13 +129,14 @@ func includePayloadByAddresses(payload *ledger.Payload, addresses []common.Addre return true, nil } - for _, address := range addresses { - k, err := payload.Key() - if err != nil { - return false, fmt.Errorf("failed to get key from payload: %w", err) - } + k, err := payload.Key() + if err != nil { + return false, fmt.Errorf("failed to get key from payload: %w", err) + } + + owner := k.KeyParts[0].Value - owner := k.KeyParts[0].Value + for _, address := range addresses { if bytes.Equal(owner, address[:]) { return true, nil } From 67ec2e2d24900af8fa74d02255f6086081970e85 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 17:29:01 -0600 Subject: [PATCH 24/52] Add tests for payload file related functionality --- cmd/util/ledger/util/payload_file_test.go | 272 ++++++++++++++++++++++ 1 file changed, 272 insertions(+) create mode 100644 cmd/util/ledger/util/payload_file_test.go diff --git a/cmd/util/ledger/util/payload_file_test.go b/cmd/util/ledger/util/payload_file_test.go new file mode 100644 index 00000000000..d37da30444f --- /dev/null +++ b/cmd/util/ledger/util/payload_file_test.go @@ -0,0 +1,272 @@ +package util_test + +import ( + "bytes" + "crypto/rand" + "path/filepath" + "testing" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/require" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/cmd/util/ledger/util" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/utils/unittest" +) + +type keyPair struct { + key ledger.Key + value ledger.Value +} + +func TestPayloadFile(t *testing.T) { + + const fileName = "root.payload" + + t.Run("without filter", func(t *testing.T) { + unittest.RunWithTempDir(t, func(datadir string) { + size := 10 + + payloadFileName := filepath.Join(datadir, fileName) + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + numOfPayloadWritten, err := util.CreatePayloadFile( + zerolog.Nop(), + payloadFileName, + payloads, + nil, + ) + require.NoError(t, err) + require.Equal(t, len(payloads), numOfPayloadWritten) + + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), payloadFileName) + require.NoError(t, err) + require.Equal(t, len(payloads), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := keysValues[k.String()] + require.True(t, exist) + + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + }) + + t.Run("with filter", func(t *testing.T) { + unittest.RunWithTempDir(t, func(datadir string) { + size := 10 + + payloadFileName := filepath.Join(datadir, fileName) + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + const selectedAddressCount = 10 + selectedAddresses := make(map[common.Address]struct{}) + selectedKeysValues := make(map[string]keyPair) + for k, kv := range keysValues { + owner := kv.key.KeyParts[0].Value + if len(owner) != common.AddressLength { + continue + } + + address, err := common.BytesToAddress(owner) + require.NoError(t, err) + + if len(selectedAddresses) < selectedAddressCount { + selectedAddresses[address] = struct{}{} + } + + if _, exist := selectedAddresses[address]; exist { + selectedKeysValues[k] = kv + } + } + + addresses := make([]common.Address, 0, len(selectedAddresses)) + for address := range selectedAddresses { + addresses = append(addresses, address) + } + + numOfPayloadWritten, err := util.CreatePayloadFile( + zerolog.Nop(), + payloadFileName, + payloads, + addresses, + ) + require.NoError(t, err) + require.Equal(t, len(selectedKeysValues), numOfPayloadWritten) + + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), payloadFileName) + require.NoError(t, err) + require.Equal(t, len(selectedKeysValues), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := selectedKeysValues[k.String()] + require.True(t, exist) + + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + }) + + t.Run("no payloads found with filter", func(t *testing.T) { + emptyAddress := common.Address{} + + unittest.RunWithTempDir(t, func(datadir string) { + size := 10 + + payloadFileName := filepath.Join(datadir, fileName) + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + if bytes.Equal(key.KeyParts[0].Value, emptyAddress[:]) { + continue + } + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + numOfPayloadWritten, err := util.CreatePayloadFile( + zerolog.Nop(), + payloadFileName, + payloads, + []common.Address{emptyAddress}, + ) + require.NoError(t, err) + require.Equal(t, 0, numOfPayloadWritten) + + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), payloadFileName) + require.NoError(t, err) + require.Equal(t, 0, len(payloadsFromFile)) + }) + }) +} + +func getSampleKeyValues(i int) ([]ledger.Key, []ledger.Value) { + switch i { + case 0: + return []ledger.Key{getKey("", "uuid"), getKey("", "account_address_state")}, + []ledger.Value{[]byte{'1'}, []byte{'A'}} + case 1: + return []ledger.Key{getKey("ADDRESS", "public_key_count"), + getKey("ADDRESS", "public_key_0"), + getKey("ADDRESS", "exists"), + getKey("ADDRESS", "storage_used")}, + []ledger.Value{[]byte{1}, []byte("PUBLICKEYXYZ"), []byte{1}, []byte{100}} + case 2: + // TODO change the contract_names to CBOR encoding + return []ledger.Key{getKey("ADDRESS", "contract_names"), getKey("ADDRESS", "code.mycontract")}, + []ledger.Value{[]byte("mycontract"), []byte("CONTRACT Content")} + default: + keys := make([]ledger.Key, 0) + values := make([]ledger.Value, 0) + for j := 0; j < 10; j++ { + // address := make([]byte, 32) + address := make([]byte, 8) + _, err := rand.Read(address) + if err != nil { + panic(err) + } + keys = append(keys, getKey(string(address), "test")) + values = append(values, getRandomCadenceValue()) + } + return keys, values + } +} + +func getKey(owner, key string) ledger.Key { + return ledger.Key{KeyParts: []ledger.KeyPart{ + {Type: uint16(0), Value: []byte(owner)}, + {Type: uint16(2), Value: []byte(key)}, + }, + } +} + +func getRandomCadenceValue() ledger.Value { + + randomPart := make([]byte, 10) + _, err := rand.Read(randomPart) + if err != nil { + panic(err) + } + valueBytes := []byte{ + // magic prefix + 0x0, 0xca, 0xde, 0x0, 0x4, + // tag + 0xd8, 132, + // array, 5 items follow + 0x85, + + // tag + 0xd8, 193, + // UTF-8 string, length 4 + 0x64, + // t, e, s, t + 0x74, 0x65, 0x73, 0x74, + + // nil + 0xf6, + + // positive integer 1 + 0x1, + + // array, 0 items follow + 0x80, + + // UTF-8 string, length 10 + 0x6a, + 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, + } + + valueBytes = append(valueBytes, randomPart...) + return ledger.Value(valueBytes) +} From 48edc602cff221067aad53e0bb7e1f0622c2f9a2 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 17:55:52 -0600 Subject: [PATCH 25/52] Add utility to extract payloads by addresses This utility can be used to create a subset of execution state which can save time during development, testing, and support/troubleshooting. --- .../cmd/extract-payloads-by-address/cmd.go | 262 ++++++++++++++++++ .../extract_payloads_test.go | 241 ++++++++++++++++ 2 files changed, 503 insertions(+) create mode 100644 cmd/util/cmd/extract-payloads-by-address/cmd.go create mode 100644 cmd/util/cmd/extract-payloads-by-address/extract_payloads_test.go diff --git a/cmd/util/cmd/extract-payloads-by-address/cmd.go b/cmd/util/cmd/extract-payloads-by-address/cmd.go new file mode 100644 index 00000000000..acf54c07b49 --- /dev/null +++ b/cmd/util/cmd/extract-payloads-by-address/cmd.go @@ -0,0 +1,262 @@ +package extractpayloads + +import ( + "bufio" + "bytes" + "encoding/binary" + "encoding/hex" + "fmt" + "io" + "os" + "strings" + + "github.com/fxamacker/cbor/v2" + "github.com/rs/zerolog" + "github.com/rs/zerolog/log" + "github.com/spf13/cobra" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/ledger" +) + +const ( + defaultBufioWriteSize = 1024 * 32 + defaultBufioReadSize = 1024 * 32 + + payloadEncodingVersion = 1 +) + +var ( + flagInputPayloadFileName string + flagOutputPayloadFileName string + flagAddresses string +) + +var Cmd = &cobra.Command{ + Use: "extract-payload-by-address", + Short: "Read payload file and generate payload file containing payloads with specified addresses", + Run: run, +} + +func init() { + Cmd.Flags().StringVar( + &flagInputPayloadFileName, + "input-filename", + "", + "Input payload file name") + _ = Cmd.MarkFlagRequired("input-filename") + + Cmd.Flags().StringVar( + &flagOutputPayloadFileName, + "output-filename", + "", + "Output payload file name") + _ = Cmd.MarkFlagRequired("output-filename") + + Cmd.Flags().StringVar( + &flagAddresses, + "addresses", + "", + "extract payloads of addresses (comma separated hex-encoded addresses) to file specified by output-payload-filename", + ) + _ = Cmd.MarkFlagRequired("addresses") +} + +func run(*cobra.Command, []string) { + + if _, err := os.Stat(flagInputPayloadFileName); os.IsNotExist(err) { + log.Fatal().Msgf("Input file %s doesn't exist", flagInputPayloadFileName) + } + + if _, err := os.Stat(flagOutputPayloadFileName); os.IsExist(err) { + log.Fatal().Msgf("Output file %s exists", flagOutputPayloadFileName) + } + + addresses, err := parseAddresses(strings.Split(flagAddresses, ",")) + if err != nil { + log.Fatal().Err(err) + } + + log.Info().Msgf( + "extracting payloads with address %v from %s to %s", + addresses, + flagInputPayloadFileName, + flagOutputPayloadFileName, + ) + + numOfPayloadWritten, err := extractPayloads(log.Logger, flagInputPayloadFileName, flagOutputPayloadFileName, addresses) + if err != nil { + log.Fatal().Err(err) + } + + err = overwritePayloadCountInFile(flagOutputPayloadFileName, numOfPayloadWritten) + if err != nil { + log.Fatal().Err(err) + } +} + +func overwritePayloadCountInFile(output string, numOfPayloadWritten int) error { + in, err := os.OpenFile(output, os.O_RDWR, 0644) + if err != nil { + return fmt.Errorf("failed to open %s to write payload count: %w", output, err) + } + defer in.Close() + + var data [9]byte + data[0] = 0x1b + binary.BigEndian.PutUint64(data[1:], uint64(numOfPayloadWritten)) + + n, err := in.WriteAt(data[:], 0) + if err != nil { + return fmt.Errorf("failed to overwrite number of payloads in %s: %w", output, err) + } + if n != len(data) { + return fmt.Errorf("failed to overwrite number of payloads in %s: wrote %d bytes, expect %d bytes", output, n, len(data)) + } + + return nil +} + +func extractPayloads(log zerolog.Logger, input, output string, addresses []common.Address) (int, error) { + in, err := os.Open(input) + if err != nil { + return 0, fmt.Errorf("failed to open %s: %w", input, err) + } + defer in.Close() + + reader := bufio.NewReaderSize(in, defaultBufioReadSize) + if err != nil { + return 0, fmt.Errorf("failed to create bufio reader for %s: %w", input, err) + } + + out, err := os.Create(output) + if err != nil { + return 0, fmt.Errorf("failed to open %s: %w", output, err) + } + defer out.Close() + + writer := bufio.NewWriterSize(out, defaultBufioWriteSize) + if err != nil { + return 0, fmt.Errorf("failed to create bufio writer for %s: %w", output, err) + } + defer writer.Flush() + + // Preserve 9-bytes header for number of payloads. + var head [9]byte + _, err = writer.Write(head[:]) + if err != nil { + return 0, fmt.Errorf("failed to write header for %s: %w", output, err) + } + + // Need to flush buffer before encoding payloads. + writer.Flush() + + enc := cbor.NewEncoder(writer) + + const logIntervalForPayloads = 1_000_000 + count := 0 + err = readPayloadFile(log, reader, func(rawPayload []byte) error { + + payload, err := ledger.DecodePayloadWithoutPrefix(rawPayload, false, payloadEncodingVersion) + if err != nil { + return fmt.Errorf("failed to decode payload 0x%x: %w", rawPayload, err) + } + + k, err := payload.Key() + if err != nil { + return err + } + + owner := k.KeyParts[0].Value + + include := false + for _, address := range addresses { + if bytes.Equal(owner, address[:]) { + include = true + break + } + } + + if include { + err = enc.Encode(rawPayload) + if err != nil { + return fmt.Errorf("failed to encode payload: %w", err) + } + + count++ + if count%logIntervalForPayloads == 0 { + log.Info().Msgf("wrote %d payloads", count) + } + } + + return nil + }) + if err != nil { + return 0, err + } + + log.Info().Msgf("wrote %d payloads", count) + return count, nil +} + +func parseAddresses(hexAddresses []string) ([]common.Address, error) { + if len(hexAddresses) == 0 { + return nil, fmt.Errorf("at least one address must be provided") + } + + addresses := make([]common.Address, len(hexAddresses)) + for i, hexAddr := range hexAddresses { + b, err := hex.DecodeString(strings.TrimSpace(hexAddr)) + if err != nil { + return nil, fmt.Errorf("address is not hex encoded %s: %w", strings.TrimSpace(hexAddr), err) + } + + addr, err := common.BytesToAddress(b) + if err != nil { + return nil, fmt.Errorf("cannot decode address %x", b) + } + + addresses[i] = addr + } + + return addresses, nil +} + +func readPayloadFile(log zerolog.Logger, r io.Reader, processPayload func([]byte) error) error { + dec := cbor.NewDecoder(r) + + var payloadCount int + err := dec.Decode(&payloadCount) + if err != nil { + return err + } + + log.Info().Msgf("Processing input file with %d payloads", payloadCount) + + const logIntervalForPayloads = 1_000_000 + count := 0 + for { + var rawPayload []byte + err = dec.Decode(&rawPayload) + if err == io.EOF { + break + } + if err != nil { + return err + } + + err = processPayload(rawPayload) + if err != nil { + return err + } + + count++ + if count%logIntervalForPayloads == 0 { + log.Info().Msgf("processed %d payloads", count) + } + } + + log.Info().Msgf("processed %d payloads", count) + return nil +} diff --git a/cmd/util/cmd/extract-payloads-by-address/extract_payloads_test.go b/cmd/util/cmd/extract-payloads-by-address/extract_payloads_test.go new file mode 100644 index 00000000000..443fed54518 --- /dev/null +++ b/cmd/util/cmd/extract-payloads-by-address/extract_payloads_test.go @@ -0,0 +1,241 @@ +package extractpayloads + +import ( + "bytes" + "crypto/rand" + "encoding/hex" + "path/filepath" + "strings" + "testing" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/require" + + "github.com/onflow/cadence/runtime/common" + + "github.com/onflow/flow-go/cmd/util/ledger/util" + "github.com/onflow/flow-go/ledger" + "github.com/onflow/flow-go/utils/unittest" +) + +type keyPair struct { + key ledger.Key + value ledger.Value +} + +func TestExtractPayloads(t *testing.T) { + + t.Run("some payloads", func(t *testing.T) { + + unittest.RunWithTempDir(t, func(datadir string) { + + inputFile := filepath.Join(datadir, "input.payload") + outputFile := filepath.Join(datadir, "output.payload") + + size := 10 + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + numOfPayloadWritten, err := util.CreatePayloadFile(zerolog.Nop(), inputFile, payloads, nil) + require.NoError(t, err) + require.Equal(t, len(payloads), numOfPayloadWritten) + + const selectedAddressCount = 10 + selectedAddresses := make(map[string]struct{}) + selectedKeysValues := make(map[string]keyPair) + for k, kv := range keysValues { + owner := kv.key.KeyParts[0].Value + if len(owner) != common.AddressLength { + continue + } + + address, err := common.BytesToAddress(owner) + require.NoError(t, err) + + if len(selectedAddresses) < selectedAddressCount { + selectedAddresses[address.Hex()] = struct{}{} + } + + if _, exist := selectedAddresses[address.Hex()]; exist { + selectedKeysValues[k] = kv + } + } + + addresses := make([]string, 0, len(selectedAddresses)) + for address := range selectedAddresses { + addresses = append(addresses, address) + } + + // Export selected payloads + Cmd.SetArgs([]string{ + "--input-filename", inputFile, + "--output-filename", outputFile, + "--addresses", strings.Join(addresses, ","), + }) + + err = Cmd.Execute() + require.NoError(t, err) + + // Verify exported payloads. + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), outputFile) + require.NoError(t, err) + require.Equal(t, len(selectedKeysValues), len(payloadsFromFile)) + + for _, payloadFromFile := range payloadsFromFile { + k, err := payloadFromFile.Key() + require.NoError(t, err) + + kv, exist := selectedKeysValues[k.String()] + require.True(t, exist) + require.Equal(t, kv.value, payloadFromFile.Value()) + } + }) + }) + + t.Run("no payloads", func(t *testing.T) { + + emptyAddress := common.Address{} + + unittest.RunWithTempDir(t, func(datadir string) { + + inputFile := filepath.Join(datadir, "input.payload") + outputFile := filepath.Join(datadir, "output.payload") + + size := 10 + + // Generate some data + keysValues := make(map[string]keyPair) + var payloads []*ledger.Payload + + for i := 0; i < size; i++ { + keys, values := getSampleKeyValues(i) + + for j, key := range keys { + if bytes.Equal(key.KeyParts[0].Value, emptyAddress[:]) { + continue + } + keysValues[key.String()] = keyPair{ + key: key, + value: values[j], + } + + payloads = append(payloads, ledger.NewPayload(key, values[j])) + } + } + + numOfPayloadWritten, err := util.CreatePayloadFile(zerolog.Nop(), inputFile, payloads, nil) + require.NoError(t, err) + require.Equal(t, len(payloads), numOfPayloadWritten) + + // Export selected payloads + Cmd.SetArgs([]string{ + "--input-filename", inputFile, + "--output-filename", outputFile, + "--addresses", hex.EncodeToString(emptyAddress[:]), + }) + + err = Cmd.Execute() + require.NoError(t, err) + + // Verify exported payloads. + payloadsFromFile, err := util.ReadPayloadFile(zerolog.Nop(), outputFile) + require.NoError(t, err) + require.Equal(t, 0, len(payloadsFromFile)) + }) + }) +} + +func getSampleKeyValues(i int) ([]ledger.Key, []ledger.Value) { + switch i { + case 0: + return []ledger.Key{getKey("", "uuid"), getKey("", "account_address_state")}, + []ledger.Value{[]byte{'1'}, []byte{'A'}} + case 1: + return []ledger.Key{getKey("ADDRESS", "public_key_count"), + getKey("ADDRESS", "public_key_0"), + getKey("ADDRESS", "exists"), + getKey("ADDRESS", "storage_used")}, + []ledger.Value{[]byte{1}, []byte("PUBLICKEYXYZ"), []byte{1}, []byte{100}} + case 2: + // TODO change the contract_names to CBOR encoding + return []ledger.Key{getKey("ADDRESS", "contract_names"), getKey("ADDRESS", "code.mycontract")}, + []ledger.Value{[]byte("mycontract"), []byte("CONTRACT Content")} + default: + keys := make([]ledger.Key, 0) + values := make([]ledger.Value, 0) + for j := 0; j < 10; j++ { + // address := make([]byte, 32) + address := make([]byte, 8) + _, err := rand.Read(address) + if err != nil { + panic(err) + } + keys = append(keys, getKey(string(address), "test")) + values = append(values, getRandomCadenceValue()) + } + return keys, values + } +} + +func getKey(owner, key string) ledger.Key { + return ledger.Key{KeyParts: []ledger.KeyPart{ + {Type: uint16(0), Value: []byte(owner)}, + {Type: uint16(2), Value: []byte(key)}, + }, + } +} + +func getRandomCadenceValue() ledger.Value { + + randomPart := make([]byte, 10) + _, err := rand.Read(randomPart) + if err != nil { + panic(err) + } + valueBytes := []byte{ + // magic prefix + 0x0, 0xca, 0xde, 0x0, 0x4, + // tag + 0xd8, 132, + // array, 5 items follow + 0x85, + + // tag + 0xd8, 193, + // UTF-8 string, length 4 + 0x64, + // t, e, s, t + 0x74, 0x65, 0x73, 0x74, + + // nil + 0xf6, + + // positive integer 1 + 0x1, + + // array, 0 items follow + 0x80, + + // UTF-8 string, length 10 + 0x6a, + 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, + } + + valueBytes = append(valueBytes, randomPart...) + return ledger.Value(valueBytes) +} From 75e10997e1b8b74d4b6a648412bbc59c6d8a1479 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Tue, 13 Feb 2024 18:28:43 -0600 Subject: [PATCH 26/52] Refactor to use named magic number for CBOR data --- cmd/util/cmd/extract-payloads-by-address/cmd.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/util/cmd/extract-payloads-by-address/cmd.go b/cmd/util/cmd/extract-payloads-by-address/cmd.go index acf54c07b49..3e384bf5d05 100644 --- a/cmd/util/cmd/extract-payloads-by-address/cmd.go +++ b/cmd/util/cmd/extract-payloads-by-address/cmd.go @@ -103,8 +103,10 @@ func overwritePayloadCountInFile(output string, numOfPayloadWritten int) error { } defer in.Close() + const cbor8BytesPositiveIntegerIndicator = 0x1b + var data [9]byte - data[0] = 0x1b + data[0] = cbor8BytesPositiveIntegerIndicator binary.BigEndian.PutUint64(data[1:], uint64(numOfPayloadWritten)) n, err := in.WriteAt(data[:], 0) From 46c1c7a56a3bb8691a6d5ec1c037f10a0349c95f Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Tue, 13 Feb 2024 23:43:18 -0500 Subject: [PATCH 27/52] update tests and add metrics --- module/metrics.go | 21 ++++- .../gossipsub_rpc_validation_inspector.go | 81 ++++++++++++++++++- module/metrics/noop.go | 36 +++++---- .../control_message_validation_inspector.go | 9 +-- ...ntrol_message_validation_inspector_test.go | 75 ++--------------- 5 files changed, 124 insertions(+), 98 deletions(-) diff --git a/module/metrics.go b/module/metrics.go index 834a7ec04ef..2d23549f2f5 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -310,12 +310,17 @@ type GossipSubRpcValidationInspectorMetrics interface { // // duplicateTopicIds: the total number of duplicate topic ids received by the node on the iHave messages at the end of the async inspection of the RPC. // duplicateMessageIds: the number of duplicate message ids received by the node on the iHave messages at the end of the async inspection of the RPC. - OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) + // invalidTopicIds: the number of invalid message ids received by the node on the iHave messages at the end of the async inspection of the RPC. + OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds, invalidTopicIds int) // OnIHaveDuplicateTopicIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of duplicate topic ids // received by the node on the iHave messages of that RPC exceeding the threshold, which results in a misbehaviour report. OnIHaveDuplicateTopicIdsExceedThreshold() + // OnIHaveInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of invalid topic ids + // received by the node on the iHave messages of that RPC exceeding the threshold, which results in a misbehaviour report. + OnIHaveInvalidTopicIdsExceedThreshold() + // OnIHaveDuplicateMessageIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of duplicate message ids // received by the node on an iHave message exceeding the threshold, which results in a misbehaviour report. OnIHaveDuplicateMessageIdsExceedThreshold() @@ -343,19 +348,29 @@ type GossipSubRpcValidationInspectorMetrics interface { // received by the node on prune messages of the same RPC excesses threshold, which results in a misbehaviour report. OnPruneDuplicateTopicIdsExceedThreshold() + // OnPruneInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of prune messages for an RPC failed due to the number of invalid topic ids + // received by the node on prune messages of the same RPC excesses threshold, which results in a misbehaviour report. + OnPruneInvalidTopicIdsExceedThreshold() + // OnPruneMessageInspected is called at the end of the async inspection of prune messages of the RPC, regardless of the result of the inspection. // Args: // duplicateTopicIds: the number of duplicate topic ids received by the node on the prune messages of the RPC at the end of the async inspection prunes. - OnPruneMessageInspected(duplicateTopicIds int) + // invalidTopicIds: the number of invalid topic ids received by the node on the prune messages at the end of the async inspection of a single RPC. + OnPruneMessageInspected(duplicateTopicIds, invalidTopicIds int) // OnGraftDuplicateTopicIdsExceedThreshold tracks the number of times that the async inspection of the graft messages of a single RPC failed due to the number of duplicate topic ids // received by the node on graft messages of the same RPC excesses threshold, which results in a misbehaviour report. OnGraftDuplicateTopicIdsExceedThreshold() + // OnGraftInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of the graft messages of a single RPC failed due to the number of invalid topic ids + // received by the node on graft messages of the same RPC excesses threshold, which results in a misbehaviour report. + OnGraftInvalidTopicIdsExceedThreshold() + // OnGraftMessageInspected is called at the end of the async inspection of graft messages of a single RPC, regardless of the result of the inspection. // Args: // duplicateTopicIds: the number of duplicate topic ids received by the node on the graft messages at the end of the async inspection of a single RPC. - OnGraftMessageInspected(duplicateTopicIds int) + // invalidTopicIds: the number of invalid topic ids received by the node on the graft messages at the end of the async inspection of a single RPC. + OnGraftMessageInspected(duplicateTopicIds, invalidTopicIds int) // OnPublishMessageInspected is called at the end of the async inspection of publish messages of a single RPC, regardless of the result of the inspection. // It tracks the total number of errors detected during the async inspection of the rpc together with their individual breakdown. diff --git a/module/metrics/gossipsub_rpc_validation_inspector.go b/module/metrics/gossipsub_rpc_validation_inspector.go index 7460486b426..3dac6cbadc9 100644 --- a/module/metrics/gossipsub_rpc_validation_inspector.go +++ b/module/metrics/gossipsub_rpc_validation_inspector.go @@ -33,17 +33,23 @@ type GossipSubRpcValidationInspectorMetrics struct { // graft inspection graftDuplicateTopicIdsHistogram prometheus.Histogram + graftInvalidTopicIdsHistogram prometheus.Histogram graftDuplicateTopicIdsExceedThresholdCount prometheus.Counter + graftInvalidTopicIdsExceedThresholdCount prometheus.Counter // prune inspection pruneDuplicateTopicIdsHistogram prometheus.Histogram + pruneInvalidTopicIdsHistogram prometheus.Histogram pruneDuplicateTopicIdsExceedThresholdCount prometheus.Counter + pruneInvalidTopicIdsExceedThresholdCount prometheus.Counter // iHave inspection iHaveDuplicateMessageIdHistogram prometheus.Histogram iHaveDuplicateTopicIdHistogram prometheus.Histogram + iHaveInvalidTopicIdHistogram prometheus.Histogram iHaveDuplicateMessageIdExceedThresholdCount prometheus.Counter iHaveDuplicateTopicIdExceedThresholdCount prometheus.Counter + iHaveInvalidTopicIdExceedThresholdCount prometheus.Counter // iWant inspection iWantDuplicateMessageIdHistogram prometheus.Histogram @@ -167,6 +173,14 @@ func NewGossipSubRPCValidationInspectorMetrics(prefix string) *GossipSubRpcValid Help: "number of duplicate topic ids received from gossipsub protocol during the async inspection of a single RPC", }) + gc.iHaveInvalidTopicIdHistogram = promauto.NewHistogram(prometheus.HistogramOpts{ + Namespace: namespaceNetwork, + Subsystem: subsystemGossip, + Buckets: []float64{1, 100, 1000}, + Name: gc.prefix + "rpc_inspection_ihave_invalid_topic_ids_count", + Help: "number of invalid topic ids received from gossipsub protocol during the async inspection of a single RPC", + }) + gc.iHaveDuplicateMessageIdExceedThresholdCount = promauto.NewCounter(prometheus.CounterOpts{ Namespace: namespaceNetwork, Subsystem: subsystemGossip, @@ -181,6 +195,13 @@ func NewGossipSubRPCValidationInspectorMetrics(prefix string) *GossipSubRpcValid Help: "total number of times that the async inspection of iHave messages failed due to the number of duplicate topic ids exceeding the threshold", }) + gc.iHaveInvalidTopicIdExceedThresholdCount = promauto.NewCounter(prometheus.CounterOpts{ + Namespace: namespaceNetwork, + Subsystem: subsystemGossip, + Name: gc.prefix + "rpc_inspection_ihave_invalid_topic_ids_exceed_threshold_total", + Help: "total number of times that the async inspection of iHave messages failed due to the number of invalid topic ids exceeding the threshold", + }) + gc.iWantDuplicateMessageIdHistogram = promauto.NewHistogram(prometheus.HistogramOpts{ Namespace: namespaceNetwork, Subsystem: subsystemGossip, @@ -247,6 +268,14 @@ func NewGossipSubRPCValidationInspectorMetrics(prefix string) *GossipSubRpcValid Help: "number of duplicate topic ids on graft messages of a single RPC during the async inspection, regardless of the result of the inspection", }) + gc.graftInvalidTopicIdsHistogram = promauto.NewHistogram(prometheus.HistogramOpts{ + Namespace: namespaceNetwork, + Subsystem: subsystemGossip, + Name: gc.prefix + "rpc_inspection_graft_invalid_topic_ids_count", + Buckets: []float64{1, 100, 1000}, + Help: "number of invalid topic ids on graft messages of a single RPC during the async inspection, regardless of the result of the inspection", + }) + gc.graftDuplicateTopicIdsExceedThresholdCount = promauto.NewCounter(prometheus.CounterOpts{ Namespace: namespaceNetwork, Subsystem: subsystemGossip, @@ -254,6 +283,13 @@ func NewGossipSubRPCValidationInspectorMetrics(prefix string) *GossipSubRpcValid Help: "number of times that the async inspection of graft messages of an rpc failed due to the number of duplicate topic ids exceeding the threshold", }) + gc.graftInvalidTopicIdsExceedThresholdCount = promauto.NewCounter(prometheus.CounterOpts{ + Namespace: namespaceNetwork, + Subsystem: subsystemGossip, + Name: gc.prefix + "rpc_inspection_graft_invalid_topic_ids_exceed_threshold_total", + Help: "number of times that the async inspection of graft messages of an rpc failed due to the number of invalid topic ids exceeding the threshold", + }) + gc.pruneDuplicateTopicIdsHistogram = promauto.NewHistogram(prometheus.HistogramOpts{ Namespace: namespaceNetwork, Subsystem: subsystemGossip, @@ -262,6 +298,14 @@ func NewGossipSubRPCValidationInspectorMetrics(prefix string) *GossipSubRpcValid Help: "number of duplicate topic ids on prune messages of a single RPC during the async inspection, regardless of the result of the inspection", }) + gc.pruneInvalidTopicIdsHistogram = promauto.NewHistogram(prometheus.HistogramOpts{ + Namespace: namespaceNetwork, + Subsystem: subsystemGossip, + Buckets: []float64{1, 100, 1000}, + Name: gc.prefix + "rpc_inspection_prune_invalid_topic_ids_count", + Help: "number of invalid topic ids on prune messages of a single RPC during the async inspection, regardless of the result of the inspection", + }) + gc.pruneDuplicateTopicIdsExceedThresholdCount = promauto.NewCounter(prometheus.CounterOpts{ Namespace: namespaceNetwork, Subsystem: subsystemGossip, @@ -269,6 +313,13 @@ func NewGossipSubRPCValidationInspectorMetrics(prefix string) *GossipSubRpcValid Help: "number of times that the async inspection of prune messages failed due to the number of duplicate topic ids exceeding the threshold", }) + gc.pruneInvalidTopicIdsExceedThresholdCount = promauto.NewCounter(prometheus.CounterOpts{ + Namespace: namespaceNetwork, + Subsystem: subsystemGossip, + Name: gc.prefix + "rpc_inspection_prune_invalid_topic_ids_exceed_threshold_total", + Help: "number of times that the async inspection of prune messages failed due to the number of invalid topic ids exceeding the threshold", + }) + gc.publishMessageInspectedErrHistogram = promauto.NewHistogram(prometheus.HistogramOpts{ Namespace: namespaceNetwork, Subsystem: subsystemGossip, @@ -407,9 +458,11 @@ func (c *GossipSubRpcValidationInspectorMetrics) OnIWantCacheMissMessageIdsExcee // // duplicateTopicIds: the total number of duplicate topic ids received by the node on the iHave messages at the end of the async inspection of the RPC. // duplicateMessageIds: the number of duplicate message ids received by the node on the iHave messages at the end of the async inspection of the RPC. -func (c *GossipSubRpcValidationInspectorMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) { +// invalidTopicIds: the number of invalid message ids received by the node on the iHave messages at the end of the async inspection of the RPC. +func (c *GossipSubRpcValidationInspectorMetrics) OnIHaveMessagesInspected(duplicateTopicIds, duplicateMessageIds, invalidTopicIds int) { c.iHaveDuplicateTopicIdHistogram.Observe(float64(duplicateTopicIds)) c.iHaveDuplicateMessageIdHistogram.Observe(float64(duplicateMessageIds)) + c.iHaveInvalidTopicIdHistogram.Observe(float64(invalidTopicIds)) } // OnIHaveDuplicateTopicIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of duplicate topic ids @@ -424,6 +477,12 @@ func (c *GossipSubRpcValidationInspectorMetrics) OnIHaveDuplicateMessageIdsExcee c.iHaveDuplicateMessageIdExceedThresholdCount.Inc() } +// OnIHaveInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of invalid topic ids +// received by the node on the iHave messages of that RPC exceeding the threshold, which results in a misbehaviour report. +func (c *GossipSubRpcValidationInspectorMetrics) OnIHaveInvalidTopicIdsExceedThreshold() { + c.iHaveInvalidTopicIdExceedThresholdCount.Inc() +} + // OnInvalidTopicIdDetectedForControlMessage tracks the number of times that the async inspection of a control message type on a single RPC failed due to an invalid topic id. // Args: // - messageType: the type of the control message that was truncated. @@ -455,12 +514,20 @@ func (c *GossipSubRpcValidationInspectorMetrics) OnPruneDuplicateTopicIdsExceedT c.pruneDuplicateTopicIdsExceedThresholdCount.Inc() } +// OnPruneInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of prune messages for an RPC failed due to the number of invalid topic ids +// received by the node on prune messages of the same RPC excesses threshold, which results in a misbehaviour report. +func (c *GossipSubRpcValidationInspectorMetrics) OnPruneInvalidTopicIdsExceedThreshold() { + c.pruneInvalidTopicIdsExceedThresholdCount.Inc() +} + // OnPruneMessageInspected is called at the end of the async inspection of prune messages of the RPC, regardless of the result of the inspection. // Args: // // duplicateTopicIds: the number of duplicate topic ids received by the node on the prune messages of the RPC at the end of the async inspection prunes. -func (c *GossipSubRpcValidationInspectorMetrics) OnPruneMessageInspected(duplicateTopicIds int) { +// invalidTopicIds: the number of invalid message ids received by the node on the prune messages at the end of the async inspection of the RPC. +func (c *GossipSubRpcValidationInspectorMetrics) OnPruneMessageInspected(duplicateTopicIds, invalidTopicIds int) { c.pruneDuplicateTopicIdsHistogram.Observe(float64(duplicateTopicIds)) + c.pruneInvalidTopicIdsHistogram.Observe(float64(invalidTopicIds)) } // OnGraftDuplicateTopicIdsExceedThreshold tracks the number of times that the async inspection of a graft message failed due to the number of duplicate topic ids. @@ -469,12 +536,20 @@ func (c *GossipSubRpcValidationInspectorMetrics) OnGraftDuplicateTopicIdsExceedT c.graftDuplicateTopicIdsExceedThresholdCount.Inc() } +// OnGraftInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of the graft messages of a single RPC failed due to the number of invalid topic ids +// received by the node on graft messages of the same RPC excesses threshold, which results in a misbehaviour report. +func (c *GossipSubRpcValidationInspectorMetrics) OnGraftInvalidTopicIdsExceedThreshold() { + c.graftInvalidTopicIdsExceedThresholdCount.Inc() +} + // OnGraftMessageInspected is called at the end of the async inspection of graft messages of a single RPC, regardless of the result of the inspection. // Args: // // duplicateTopicIds: the number of duplicate topic ids received by the node on the graft messages at the end of the async inspection of a single RPC. -func (c *GossipSubRpcValidationInspectorMetrics) OnGraftMessageInspected(duplicateTopicIds int) { +// invalidTopicIds: the number of invalid message ids received by the node on the graft messages at the end of the async inspection of the RPC. +func (c *GossipSubRpcValidationInspectorMetrics) OnGraftMessageInspected(duplicateTopicIds, invalidTopicIds int) { c.graftDuplicateTopicIdsHistogram.Observe(float64(duplicateTopicIds)) + c.graftInvalidTopicIdsHistogram.Observe(float64(invalidTopicIds)) } // OnPublishMessageInspected is called at the end of the async inspection of publish messages of a single RPC, regardless of the result of the inspection. diff --git a/module/metrics/noop.go b/module/metrics/noop.go index 04a6d80b70e..6610c3620a7 100644 --- a/module/metrics/noop.go +++ b/module/metrics/noop.go @@ -322,24 +322,28 @@ func (nc *NoopCollector) OnControlMessagesTruncated(messageType p2pmsg.ControlMe } func (nc *NoopCollector) OnIncomingRpcReceived(iHaveCount, iWantCount, graftCount, pruneCount, msgCount int) { } -func (nc *NoopCollector) AsyncProcessingStarted() {} -func (nc *NoopCollector) AsyncProcessingFinished(time.Duration) {} -func (nc *NoopCollector) OnIWantMessagesInspected(duplicateCount int, cacheMissCount int) {} -func (nc *NoopCollector) OnIWantDuplicateMessageIdsExceedThreshold() {} -func (nc *NoopCollector) OnIWantCacheMissMessageIdsExceedThreshold() {} -func (nc *NoopCollector) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) {} -func (nc *NoopCollector) OnIHaveDuplicateTopicIdsExceedThreshold() {} -func (nc *NoopCollector) OnIHaveDuplicateMessageIdsExceedThreshold() {} +func (nc *NoopCollector) AsyncProcessingStarted() {} +func (nc *NoopCollector) AsyncProcessingFinished(time.Duration) {} +func (nc *NoopCollector) OnIWantMessagesInspected(duplicateCount int, cacheMissCount int) {} +func (nc *NoopCollector) OnIWantDuplicateMessageIdsExceedThreshold() {} +func (nc *NoopCollector) OnIWantCacheMissMessageIdsExceedThreshold() {} +func (nc *NoopCollector) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds, invalidTopicIds int) { +} +func (nc *NoopCollector) OnIHaveDuplicateTopicIdsExceedThreshold() {} +func (nc *NoopCollector) OnIHaveInvalidTopicIdsExceedThreshold() {} +func (nc *NoopCollector) OnIHaveDuplicateMessageIdsExceedThreshold() {} func (nc *NoopCollector) OnInvalidTopicIdDetectedForControlMessage(messageType p2pmsg.ControlMessageType) { } -func (nc *NoopCollector) OnActiveClusterIDsNotSetErr() {} -func (nc *NoopCollector) OnUnstakedPeerInspectionFailed() {} -func (nc *NoopCollector) OnInvalidControlMessageNotificationSent() {} -func (nc *NoopCollector) OnPublishMessagesInspectionErrorExceedsThreshold() {} -func (nc *NoopCollector) OnPruneDuplicateTopicIdsExceedThreshold() {} -func (nc *NoopCollector) OnPruneMessageInspected(duplicateTopicIds int) {} -func (nc *NoopCollector) OnGraftDuplicateTopicIdsExceedThreshold() {} -func (nc *NoopCollector) OnGraftMessageInspected(duplicateTopicIds int) {} +func (nc *NoopCollector) OnActiveClusterIDsNotSetErr() {} +func (nc *NoopCollector) OnUnstakedPeerInspectionFailed() {} +func (nc *NoopCollector) OnInvalidControlMessageNotificationSent() {} +func (nc *NoopCollector) OnPublishMessagesInspectionErrorExceedsThreshold() {} +func (nc *NoopCollector) OnPruneDuplicateTopicIdsExceedThreshold() {} +func (nc *NoopCollector) OnPruneInvalidTopicIdsExceedThreshold() {} +func (nc *NoopCollector) OnPruneMessageInspected(duplicateTopicIds, invalidTopicIds int) {} +func (nc *NoopCollector) OnGraftDuplicateTopicIdsExceedThreshold() {} +func (nc *NoopCollector) OnGraftInvalidTopicIdsExceedThreshold() {} +func (nc *NoopCollector) OnGraftMessageInspected(duplicateTopicIds, invalidTopicIds int) {} func (nc *NoopCollector) OnPublishMessageInspected(totalErrCount int, invalidTopicIdsCount int, invalidSubscriptionsCount int, invalidSendersCount int) { } diff --git a/network/p2p/inspector/validation/control_message_validation_inspector.go b/network/p2p/inspector/validation/control_message_validation_inspector.go index 6547075c119..308869731af 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector.go @@ -369,7 +369,7 @@ func (c *ControlMsgValidationInspector) inspectGraftMessages(from peer.ID, graft totalInvalidTopicIdErrs := 0 defer func() { // regardless of inspection result, update metrics - c.metrics.OnGraftMessageInspected(totalDuplicateTopicIds) + c.metrics.OnGraftMessageInspected(totalDuplicateTopicIds, totalInvalidTopicIdErrs) }() for _, graft := range grafts { @@ -386,7 +386,6 @@ func (c *ControlMsgValidationInspector) inspectGraftMessages(from peer.ID, graft err, ctrlMsgType := c.validateTopic(from, topic, activeClusterIDS) if err != nil { totalInvalidTopicIdErrs++ - // TODO: consider adding a threshold for this error similar to the duplicate topic id threshold. c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgGraft) if totalInvalidTopicIdErrs > c.config.GraftPrune.InvalidTopicIdThreshold { return err, ctrlMsgType @@ -420,7 +419,7 @@ func (c *ControlMsgValidationInspector) inspectPruneMessages(from peer.ID, prune totalInvalidTopicIdErrs := 0 defer func() { // regardless of inspection result, update metrics - c.metrics.OnPruneMessageInspected(totalDuplicateTopicIds) + c.metrics.OnPruneMessageInspected(totalDuplicateTopicIds, totalInvalidTopicIdErrs) }() for _, prune := range prunes { topic := channels.Topic(prune.GetTopicID()) @@ -436,7 +435,6 @@ func (c *ControlMsgValidationInspector) inspectPruneMessages(from peer.ID, prune err, ctrlMsgType := c.validateTopic(from, topic, activeClusterIDS) if err != nil { totalInvalidTopicIdErrs++ - // TODO: consider adding a threshold for this error similar to the duplicate topic id threshold. c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgPrune) if totalInvalidTopicIdErrs > c.config.GraftPrune.InvalidTopicIdThreshold { return err, ctrlMsgType @@ -482,7 +480,7 @@ func (c *ControlMsgValidationInspector) inspectIHaveMessages(from peer.ID, ihave totalInvalidTopicIdErrs := 0 defer func() { // regardless of inspection result, update metrics - c.metrics.OnIHaveMessagesInspected(totalDuplicateTopicIds, totalDuplicateMessageIds) + c.metrics.OnIHaveMessagesInspected(totalDuplicateTopicIds, totalDuplicateMessageIds, totalInvalidTopicIdErrs) }() for _, ihave := range ihaves { messageIds := ihave.GetMessageIDs() @@ -493,7 +491,6 @@ func (c *ControlMsgValidationInspector) inspectIHaveMessages(from peer.ID, ihave err, ctrlMsgType := c.validateTopic(from, channels.Topic(topic), activeClusterIDS) if err != nil { totalInvalidTopicIdErrs++ - // TODO: consider adding a threshold for this error similar to the duplicate topic id threshold. c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgIHave) if totalInvalidTopicIdErrs > c.config.IHave.InvalidTopicIdThreshold { return err, ctrlMsgType diff --git a/network/p2p/inspector/validation/control_message_validation_inspector_test.go b/network/p2p/inspector/validation/control_message_validation_inspector_test.go index f5f8d1b8722..6ebea061322 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector_test.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector_test.go @@ -129,12 +129,11 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() + distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) - // unittest.RequireCloseBefore(t, inspector.Ready(), 100*time.Millisecond, "inspector did not start") // topic validation not performed, so we can use random strings prunesGreaterThanMaxSampleSize := unittest.P2PRPCFixture(unittest.WithPrunes(unittest.P2PRPCPruneFixtures(unittest.IdentifierListFixture(2000).Strings()...)...)) require.Greater(t, len(prunesGreaterThanMaxSampleSize.GetControl().GetPrune()), graftPruneMessageMaxSampleSize) @@ -162,7 +161,7 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() + distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -194,7 +193,7 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() + distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -658,72 +657,6 @@ func TestPrueInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") } -// TestPruneInspection_InvalidTopic ensures inspector disseminates an invalid control message notification for -// prune messages when the topic is invalid. -func TestPruneInspection_InvalidTopic(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) - // create unknown topic - unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) - unknownTopicPrune := unittest.P2PRPCPruneFixture(&unknownTopic) - malformedTopicPrune := unittest.P2PRPCPruneFixture(&malformedTopic) - invalidSporkIDTopicPrune := unittest.P2PRPCPruneFixture(&invalidSporkIDTopic) - // avoid unknown topics errors - topicProviderOracle.UpdateTopics([]string{unknownTopic, malformedTopic, invalidSporkIDTopic}) - unknownTopicRpc := unittest.P2PRPCFixture(unittest.WithPrunes(unknownTopicPrune)) - malformedTopicRpc := unittest.P2PRPCFixture(unittest.WithPrunes(malformedTopicPrune)) - invalidSporkIDTopicRpc := unittest.P2PRPCFixture(unittest.WithPrunes(invalidSporkIDTopicPrune)) - - from := unittest.PeerIdFixture(t) - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgPrune, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) - rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) - - inspector.Start(signalerCtx) - unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) - - require.NoError(t, inspector.Inspect(from, unknownTopicRpc)) - require.NoError(t, inspector.Inspect(from, malformedTopicRpc)) - require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicRpc)) - // sleep for 1 second to ensure rpc's is processed - time.Sleep(time.Second) - cancel() - unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") -} - -// TestIHaveInspection_InvalidTopic ensures inspector disseminates an invalid control message notification for -// iHave messages when the topic is invalid. -func TestIHaveInspection_InvalidTopic(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) - // create unknown topic - unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) - // avoid unknown topics errors - topicProviderOracle.UpdateTopics([]string{unknownTopic, malformedTopic, invalidSporkIDTopic}) - unknownTopicIhave := unittest.P2PRPCIHaveFixture(&unknownTopic, unittest.IdentifierListFixture(5).Strings()...) - malformedTopicIhave := unittest.P2PRPCIHaveFixture(&malformedTopic, unittest.IdentifierListFixture(5).Strings()...) - invalidSporkIDTopicIhave := unittest.P2PRPCIHaveFixture(&invalidSporkIDTopic, unittest.IdentifierListFixture(5).Strings()...) - - unknownTopicRpc := unittest.P2PRPCFixture(unittest.WithIHaves(unknownTopicIhave)) - malformedTopicRpc := unittest.P2PRPCFixture(unittest.WithIHaves(malformedTopicIhave)) - invalidSporkIDTopicRpc := unittest.P2PRPCFixture(unittest.WithIHaves(invalidSporkIDTopicIhave)) - - from := unittest.PeerIdFixture(t) - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) - rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) - inspector.Start(signalerCtx) - unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) - - require.NoError(t, inspector.Inspect(from, unknownTopicRpc)) - require.NoError(t, inspector.Inspect(from, malformedTopicRpc)) - require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicRpc)) - // sleep for 1 second to ensure rpc's is processed - time.Sleep(time.Second) - cancel() - unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") -} - // TestIHaveInspection_InvalidTopic_BelowThreshold ensures inspector does not disseminate an invalid control message notification for // ihave messages when the invalid topic id count does not exceed the configured threshold. func TestIHaveInspection_InvalidTopic_BelowThreshold(t *testing.T) { @@ -1342,6 +1275,8 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { // the 11th unknown cluster ID error should cause an error params.Config.ClusterPrefixedMessage.HardThreshold = 10 + // disable invalid topic threshold return an error always + params.Config.GraftPrune.InvalidTopicIdThreshold = 0 }) clusterID := flow.ChainID(unittest.IdentifierFixture().String()) clusterPrefixedTopic := channels.Topic(fmt.Sprintf("%s/%s", channels.SyncCluster(clusterID), sporkID)).String() From 0e9356d8198c16cbf0843c9455216aaf5f541207 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 14 Feb 2024 16:50:40 -0800 Subject: [PATCH 28/52] fixes inspector validation test --- ...ntrol_message_validation_inspector_test.go | 176 +++++++++--------- 1 file changed, 88 insertions(+), 88 deletions(-) diff --git a/network/p2p/inspector/validation/control_message_validation_inspector_test.go b/network/p2p/inspector/validation/control_message_validation_inspector_test.go index 02fc72f3a44..d8c59c4ddcb 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector_test.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector_test.go @@ -29,7 +29,6 @@ import ( "github.com/onflow/flow-go/network/p2p/inspector/validation" p2pmsg "github.com/onflow/flow-go/network/p2p/message" mockp2p "github.com/onflow/flow-go/network/p2p/mock" - p2ptest "github.com/onflow/flow-go/network/p2p/test" "github.com/onflow/flow-go/utils/unittest" ) @@ -39,19 +38,19 @@ func TestNewControlMsgValidationInspector(t *testing.T) { sporkID := unittest.IdentifierFixture() flowConfig, err := config.DefaultConfig() require.NoError(t, err, "failed to get default flow config") - distributor := mockp2p.NewGossipSubInspectorNotifDistributor(t) + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) idProvider := mockmodule.NewIdentityProvider(t) topicProvider := internal.NewMockUpdatableTopicProvider() inspector, err := validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: unittest.Logger(), SporkID: sporkID, Config: &flowConfig.NetworkConfig.GossipSub.RpcInspector.Validation, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: mockp2p.NewRpcControlTracking(t), NetworkingType: network.PublicNetwork, + InvalidControlMessageNotificationConsumer: consumer, TopicOracle: func() p2p.TopicProvider { return topicProvider }, @@ -64,18 +63,18 @@ func TestNewControlMsgValidationInspector(t *testing.T) { Logger: unittest.Logger(), SporkID: unittest.IdentifierFixture(), Config: nil, - Distributor: nil, IdProvider: nil, HeroCacheMetricsFactory: nil, InspectorMetrics: nil, RpcTracker: nil, TopicOracle: nil, + InvalidControlMessageNotificationConsumer: nil, }) require.Nil(t, inspector) require.Error(t, err) s := err.Error() require.Contains(t, s, "validation for 'Config' failed on the 'required'") - require.Contains(t, s, "validation for 'Distributor' failed on the 'required'") + require.Contains(t, s, "validation for 'InvalidControlMessageNotificationConsumer' failed on the 'required'") require.Contains(t, s, "validation for 'IdProvider' failed on the 'required'") require.Contains(t, s, "validation for 'HeroCacheMetricsFactory' failed on the 'required'") require.Contains(t, s, "validation for 'InspectorMetrics' failed on the 'required'") @@ -91,11 +90,11 @@ func TestNewControlMsgValidationInspector(t *testing.T) { func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { t.Run("graft truncation", func(t *testing.T) { graftPruneMessageMaxSampleSize := 1000 - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.GraftPrune.MessageCountThreshold = graftPruneMessageMaxSampleSize }) // topic validation is ignored set any topic oracle - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() inspector.Start(signalerCtx) @@ -123,13 +122,13 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { t.Run("prune truncation", func(t *testing.T) { graftPruneMessageMaxSampleSize := 1000 - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.GraftPrune.MessageCountThreshold = graftPruneMessageMaxSampleSize }) // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -156,13 +155,13 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { t.Run("ihave message id truncation", func(t *testing.T) { maxSampleSize := 1000 - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.IHave.MessageCountThreshold = maxSampleSize }) // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -188,13 +187,13 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { t.Run("ihave message ids truncation", func(t *testing.T) { maxMessageIDSampleSize := 1000 - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.IHave.MessageIdCountThreshold = maxMessageIDSampleSize }) // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Twice() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -226,13 +225,13 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { t.Run("iwant message truncation", func(t *testing.T) { maxSampleSize := uint(100) - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.IWant.MessageCountThreshold = maxSampleSize }) // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -257,13 +256,13 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { t.Run("iwant message id truncation", func(t *testing.T) { maxMessageIDSampleSize := 1000 - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.IWant.MessageIdCountThreshold = maxMessageIDSampleSize }) // topic validation is ignored set any topic oracle rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -295,8 +294,8 @@ func TestControlMessageValidationInspector_truncateRPC(t *testing.T) { // TestControlMessageInspection_ValidRpc ensures inspector does not disseminate invalid control message notifications for a valid RPC. func TestControlMessageInspection_ValidRpc(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) - defer distributor.AssertNotCalled(t, "Distribute") + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") topics := []string{ fmt.Sprintf("%s/%s", channels.TestNetworkChannel, sporkID), @@ -346,7 +345,7 @@ func TestControlMessageInspection_ValidRpc(t *testing.T) { // TestGraftInspection_InvalidTopic ensures inspector disseminates an invalid control message notification for // graft messages when the topic is invalid. func TestGraftInspection_InvalidTopic(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) // create unknown topic unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) // avoid unknown topics errors @@ -362,7 +361,7 @@ func TestGraftInspection_InvalidTopic(t *testing.T) { from := unittest.PeerIdFixture(t) checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgGraft, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -379,7 +378,7 @@ func TestGraftInspection_InvalidTopic(t *testing.T) { // TestGraftInspection_DuplicateTopicIds_BelowThreshold ensures inspector does not disseminate invalid control message notifications // for a valid RPC with duplicate graft topic ids below the threshold. func TestGraftInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) duplicateTopic := fmt.Sprintf("%s/%s", channels.TestNetworkChannel, sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{duplicateTopic}) @@ -393,7 +392,7 @@ func TestGraftInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { rpc := unittest.P2PRPCFixture(unittest.WithGrafts(grafts...)) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() // no notification should be disseminated for valid messages as long as the number of duplicates is below the threshold - distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 100*time.Millisecond, inspector) @@ -406,7 +405,7 @@ func TestGraftInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { } func TestGraftInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) duplicateTopic := fmt.Sprintf("%s/%s", channels.TestNetworkChannel, sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{duplicateTopic}) @@ -420,7 +419,7 @@ func TestGraftInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { rpc := unittest.P2PRPCFixture(unittest.WithGrafts(grafts...)) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(func(args mock.Arguments) { + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(func(args mock.Arguments) { notification, ok := args[0].(*p2p.InvCtrlMsgNotif) require.True(t, ok) require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "expected p2p.CtrlMsgNonClusterTopicType notification type, no RPC with cluster prefixed topic sent in this test") @@ -442,7 +441,7 @@ func TestGraftInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { // TestPruneInspection_DuplicateTopicIds_AboveThreshold ensures inspector disseminates an invalid control message notification for // prune messages when the number of duplicate topic ids is above the threshold. func TestPruneInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) duplicateTopic := fmt.Sprintf("%s/%s", channels.TestNetworkChannel, sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{duplicateTopic}) @@ -457,7 +456,7 @@ func TestPruneInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { rpc := unittest.P2PRPCFixture(unittest.WithPrunes(prunes...)) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(func(args mock.Arguments) { + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(func(args mock.Arguments) { notification, ok := args[0].(*p2p.InvCtrlMsgNotif) require.True(t, ok) require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "expected p2p.CtrlMsgNonClusterTopicType notification type, no RPC with cluster prefixed topic sent in this test") @@ -478,8 +477,8 @@ func TestPruneInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { // TestPruneInspection_DuplicateTopicIds_BelowThreshold ensures inspector does not disseminate invalid control message notifications // for a valid RPC with duplicate prune topic ids below the threshold. -func TestPrueInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) +func TestPruneInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) duplicateTopic := fmt.Sprintf("%s/%s", channels.TestNetworkChannel, sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{duplicateTopic}) @@ -494,7 +493,7 @@ func TestPrueInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() // no notification should be disseminated for valid messages as long as the number of duplicates is below the threshold - distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 100*time.Millisecond, inspector) @@ -509,7 +508,7 @@ func TestPrueInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { // TestPruneInspection_InvalidTopic ensures inspector disseminates an invalid control message notification for // prune messages when the topic is invalid. func TestPruneInspection_InvalidTopic(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) // create unknown topic unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) unknownTopicPrune := unittest.P2PRPCPruneFixture(&unknownTopic) @@ -525,7 +524,7 @@ func TestPruneInspection_InvalidTopic(t *testing.T) { checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgPrune, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -542,7 +541,7 @@ func TestPruneInspection_InvalidTopic(t *testing.T) { // TestIHaveInspection_InvalidTopic ensures inspector disseminates an invalid control message notification for // iHave messages when the topic is invalid. func TestIHaveInspection_InvalidTopic(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) // create unknown topic unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) // avoid unknown topics errors @@ -559,7 +558,8 @@ func TestIHaveInspection_InvalidTopic(t *testing.T) { checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -575,7 +575,7 @@ func TestIHaveInspection_InvalidTopic(t *testing.T) { // TestIHaveInspection_DuplicateTopicIds_BelowThreshold ensures inspector does not disseminate an invalid control message notification for // iHave messages when duplicate topic ids are below allowed threshold. func TestIHaveInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) validTopic := fmt.Sprintf("%s/%s", channels.PushBlocks.String(), sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{validTopic}) @@ -595,7 +595,7 @@ func TestIHaveInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() // no notification should be disseminated for valid messages as long as the number of duplicates is below the threshold - distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -609,7 +609,7 @@ func TestIHaveInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { // TestIHaveInspection_DuplicateTopicIds_AboveThreshold ensures inspector disseminate an invalid control message notification for // iHave messages when duplicate topic ids are above allowed threshold. func TestIHaveInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) validTopic := fmt.Sprintf("%s/%s", channels.PushBlocks.String(), sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{validTopic}) @@ -629,7 +629,7 @@ func TestIHaveInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { // one notification should be disseminated for invalid messages when the number of duplicates exceeds the threshold checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, validation.IsDuplicateTopicErr, p2p.CtrlMsgNonClusterTopicType) - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -643,7 +643,7 @@ func TestIHaveInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { // TestIHaveInspection_DuplicateMessageIds_BelowThreshold ensures inspector does not disseminate an invalid control message notification for // iHave messages when duplicate message ids are below allowed threshold. func TestIHaveInspection_DuplicateMessageIds_BelowThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) validTopic := fmt.Sprintf("%s/%s", channels.PushBlocks.String(), sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{validTopic}) @@ -662,7 +662,7 @@ func TestIHaveInspection_DuplicateMessageIds_BelowThreshold(t *testing.T) { rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() // no notification should be disseminated for valid messages as long as the number of duplicates is below the threshold - distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -676,7 +676,7 @@ func TestIHaveInspection_DuplicateMessageIds_BelowThreshold(t *testing.T) { // TestIHaveInspection_DuplicateMessageIds_AboveThreshold ensures inspector disseminates an invalid control message notification for // iHave messages when duplicate message ids are above allowed threshold. func TestIHaveInspection_DuplicateMessageIds_AboveThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t) validTopic := fmt.Sprintf("%s/%s", channels.PushBlocks.String(), sporkID) // avoid unknown topics errors topicProviderOracle.UpdateTopics([]string{validTopic}) @@ -696,7 +696,7 @@ func TestIHaveInspection_DuplicateMessageIds_AboveThreshold(t *testing.T) { // one notification should be disseminated for invalid messages when the number of duplicates exceeds the threshold checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, validation.IsDuplicateMessageIDErr, p2p.CtrlMsgNonClusterTopicType) - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -710,7 +710,7 @@ func TestIHaveInspection_DuplicateMessageIds_AboveThreshold(t *testing.T) { // TestIWantInspection_DuplicateMessageIds_BelowThreshold ensures inspector does not disseminate an invalid control message notification for // iWant messages when duplicate message ids are below allowed threshold. func TestIWantInspection_DuplicateMessageIds_BelowThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t) // oracle must be set even though iWant messages do not have topic IDs duplicateMsgID := unittest.IdentifierFixture() duplicates := flow.IdentifierList{} @@ -726,7 +726,8 @@ func TestIWantInspection_DuplicateMessageIds_BelowThreshold(t *testing.T) { duplicateMsgIDRpc := unittest.P2PRPCFixture(unittest.WithIWants(duplicateMsgIDIWant)) from := unittest.PeerIdFixture(t) - distributor.AssertNotCalled(t, "Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + // no notification should be disseminated for valid messages as long as the number of duplicates is below the threshold + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Run(func(args mock.Arguments) { id, ok := args[0].(string) @@ -746,7 +747,7 @@ func TestIWantInspection_DuplicateMessageIds_BelowThreshold(t *testing.T) { // TestIWantInspection_DuplicateMessageIds_AboveThreshold ensures inspector disseminates invalid control message notifications for iWant messages when duplicate message ids exceeds allowed threshold. func TestIWantInspection_DuplicateMessageIds_AboveThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t) + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t) // oracle must be set even though iWant messages do not have topic IDs duplicateMsgID := unittest.IdentifierFixture() duplicates := flow.IdentifierList{} @@ -763,7 +764,7 @@ func TestIWantInspection_DuplicateMessageIds_AboveThreshold(t *testing.T) { from := unittest.PeerIdFixture(t) checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIWant, validation.IsIWantDuplicateMsgIDThresholdErr, p2p.CtrlMsgNonClusterTopicType) - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Run(func(args mock.Arguments) { id, ok := args[0].(string) @@ -783,7 +784,7 @@ func TestIWantInspection_DuplicateMessageIds_AboveThreshold(t *testing.T) { // TestIWantInspection_CacheMiss_AboveThreshold ensures inspector disseminates invalid control message notifications for iWant messages when cache misses exceeds allowed threshold. func TestIWantInspection_CacheMiss_AboveThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { // set high cache miss threshold to ensure we only disseminate notification when it is exceeded params.Config.IWant.CacheMissThreshold = 900 }) @@ -792,7 +793,7 @@ func TestIWantInspection_CacheMiss_AboveThreshold(t *testing.T) { from := unittest.PeerIdFixture(t) checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIWant, validation.IsIWantCacheMissThresholdErr, p2p.CtrlMsgNonClusterTopicType) - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() // return false each time to eventually force a notification to be disseminated when the cache miss count finally exceeds the 90% threshold allIwantsChecked := sync.WaitGroup{} @@ -826,12 +827,12 @@ func TestIWantInspection_CacheMiss_AboveThreshold(t *testing.T) { } func TestIWantInspection_CacheMiss_BelowThreshold(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { // set high cache miss threshold to ensure that we do not disseminate notification in this test params.Config.IWant.CacheMissThreshold = 99 }) // oracle must be set even though iWant messages do not have topic IDs - defer distributor.AssertNotCalled(t, "Distribute") + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") msgIds := unittest.IdentifierListFixture(98).Strings() // one less than cache miss threshold inspectMsgRpc := unittest.P2PRPCFixture(unittest.WithIWants(unittest.P2PRPCIWantFixture(msgIds...))) @@ -863,7 +864,7 @@ func TestIWantInspection_CacheMiss_BelowThreshold(t *testing.T) { // TestControlMessageInspection_ExceedingErrThreshold ensures inspector disseminates invalid control message notifications for RPCs that exceed the configured error threshold. func TestPublishMessageInspection_ExceedingErrThreshold(t *testing.T) { errThreshold := 500 - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.PublishMessages.ErrorThreshold = errThreshold }) // create unknown topic @@ -894,7 +895,7 @@ func TestPublishMessageInspection_ExceedingErrThreshold(t *testing.T) { rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() checkNotification := checkNotificationFunc(t, from, p2pmsg.RpcPublishMessage, validation.IsInvalidRpcPublishMessagesErr, p2p.CtrlMsgNonClusterTopicType) - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -909,7 +910,7 @@ func TestPublishMessageInspection_ExceedingErrThreshold(t *testing.T) { // TestControlMessageInspection_MissingSubscription ensures inspector disseminates invalid control message notifications for RPCs that the peer is not subscribed to. func TestPublishMessageInspection_MissingSubscription(t *testing.T) { errThreshold := 500 - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.PublishMessages.ErrorThreshold = errThreshold }) pubsubMsgs := unittest.GossipSubMessageFixtures(errThreshold+1, fmt.Sprintf("%s/%s", channels.TestNetworkChannel, sporkID)) @@ -917,8 +918,7 @@ func TestPublishMessageInspection_MissingSubscription(t *testing.T) { rpc := unittest.P2PRPCFixture(unittest.WithPubsubMessages(pubsubMsgs...)) checkNotification := checkNotificationFunc(t, from, p2pmsg.RpcPublishMessage, validation.IsInvalidRpcPublishMessagesErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -932,7 +932,7 @@ func TestPublishMessageInspection_MissingSubscription(t *testing.T) { // TestPublishMessageInspection_MissingTopic ensures inspector disseminates invalid control message notifications for published messages with missing topics. func TestPublishMessageInspection_MissingTopic(t *testing.T) { errThreshold := 500 - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { // 5 invalid pubsub messages will force notification dissemination params.Config.PublishMessages.ErrorThreshold = errThreshold }) @@ -944,7 +944,7 @@ func TestPublishMessageInspection_MissingTopic(t *testing.T) { from := unittest.PeerIdFixture(t) checkNotification := checkNotificationFunc(t, from, p2pmsg.RpcPublishMessage, validation.IsInvalidRpcPublishMessagesErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -976,7 +976,7 @@ func TestRpcInspectionDeactivatedOnPublicNetwork(t *testing.T) { // TestControlMessageInspection_Unstaked_From ensures inspector disseminates invalid control message notifications for published messages from unstaked peers. func TestPublishMessageInspection_Unstaked_From(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { // override the inspector and params, run the inspector in private mode params.NetworkingType = network.PrivateNetwork }) @@ -989,7 +989,7 @@ func TestPublishMessageInspection_Unstaked_From(t *testing.T) { rpc := unittest.P2PRPCFixture(unittest.WithPubsubMessages(pubsubMsgs...)) checkNotification := checkNotificationFunc(t, from, p2pmsg.RpcPublishMessage, validation.IsInvalidRpcPublishMessagesErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -1002,7 +1002,7 @@ func TestPublishMessageInspection_Unstaked_From(t *testing.T) { // TestControlMessageInspection_Ejected_From ensures inspector disseminates invalid control message notifications for published messages from ejected peers. func TestPublishMessageInspection_Ejected_From(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { // override the inspector and params, run the inspector in private mode params.NetworkingType = network.PrivateNetwork }) @@ -1016,7 +1016,7 @@ func TestPublishMessageInspection_Ejected_From(t *testing.T) { rpc := unittest.P2PRPCFixture(unittest.WithPubsubMessages(pubsubMsgs...)) checkNotification := checkNotificationFunc(t, from, p2pmsg.RpcPublishMessage, validation.IsInvalidRpcPublishMessagesErr, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -1030,8 +1030,8 @@ func TestPublishMessageInspection_Ejected_From(t *testing.T) { // TestNewControlMsgValidationInspector_validateClusterPrefixedTopic ensures cluster prefixed topics are validated as expected. func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testing.T) { t.Run("validateClusterPrefixedTopic should not return an error for valid cluster prefixed topics", func(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t) - defer distributor.AssertNotCalled(t, "Distribute") + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t) + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") clusterID := flow.ChainID(unittest.IdentifierFixture().String()) clusterPrefixedTopic := channels.Topic(fmt.Sprintf("%s/%s", channels.SyncCluster(clusterID), sporkID)).String() topicProviderOracle.UpdateTopics([]string{clusterPrefixedTopic}) @@ -1051,11 +1051,11 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin }) t.Run("validateClusterPrefixedTopic should not return error if cluster prefixed hard threshold not exceeded for unknown cluster ids", func(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, _ := inspectorFixture(t, func(params *validation.InspectorParams) { // set hard threshold to small number , ensure that a single unknown cluster prefix id does not cause a notification to be disseminated params.Config.ClusterPrefixedMessage.HardThreshold = 2 }) - defer distributor.AssertNotCalled(t, "Distribute") + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") clusterID := flow.ChainID(unittest.IdentifierFixture().String()) clusterPrefixedTopic := channels.Topic(fmt.Sprintf("%s/%s", channels.SyncCluster(clusterID), sporkID)).String() from := unittest.PeerIdFixture(t) @@ -1074,8 +1074,8 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin }) t.Run("validateClusterPrefixedTopic should return an error when sender is unstaked", func(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t) - defer distributor.AssertNotCalled(t, "Distribute") + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t) + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") clusterID := flow.ChainID(unittest.IdentifierFixture().String()) clusterPrefixedTopic := channels.Topic(fmt.Sprintf("%s/%s", channels.SyncCluster(clusterID), sporkID)).String() topicProviderOracle.UpdateTopics([]string{clusterPrefixedTopic}) @@ -1094,7 +1094,7 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin }) t.Run("validateClusterPrefixedTopic should return error if cluster prefixed hard threshold exceeded for unknown cluster ids", func(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { // the 11th unknown cluster ID error should cause an error params.Config.ClusterPrefixedMessage.HardThreshold = 10 }) @@ -1108,7 +1108,7 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin inspectMsgRpc := unittest.P2PRPCFixture(unittest.WithGrafts(unittest.P2PRPCGraftFixture(&clusterPrefixedTopic))) inspector.ActiveClustersChanged(flow.ChainIDList{flow.ChainID(unittest.IdentifierFixture().String())}) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -1124,8 +1124,8 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin // TestControlMessageValidationInspector_ActiveClustersChanged validates the expected update of the active cluster IDs list. func TestControlMessageValidationInspector_ActiveClustersChanged(t *testing.T) { - inspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, _ := inspectorFixture(t) - defer distributor.AssertNotCalled(t, "Distribute") + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, _ := inspectorFixture(t) + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") identity := unittest.IdentityFixture() idProvider.On("ByPeerID", mock.AnythingOfType("peer.ID")).Return(identity, true).Times(5) activeClusterIds := make(flow.ChainIDList, 0) @@ -1158,7 +1158,7 @@ func TestControlMessageValidationInspector_TruncationConfigToggle(t *testing.T) expectedLogStrs := map[string]struct{}{validation.RPCTruncationDisabledWarning: {}} logCounter := atomic.NewInt64(0) logger := hookedLogger(logCounter, zerolog.TraceLevel, expectedLogStrs) - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.GraftPrune.MessageCountThreshold = numOfMsgs params.Logger = logger // disable truncation for all control message types @@ -1166,7 +1166,7 @@ func TestControlMessageValidationInspector_TruncationConfigToggle(t *testing.T) }) // topic validation is ignored set any topic oracle - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() inspector.Start(signalerCtx) @@ -1211,7 +1211,7 @@ func TestControlMessageValidationInspector_TruncationConfigToggle(t *testing.T) } logCounter := atomic.NewInt64(0) logger := hookedLogger(logCounter, zerolog.TraceLevel, expectedLogStrs) - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.GraftPrune.MessageCountThreshold = numOfMsgs params.Logger = logger // disable truncation for all control message types individually @@ -1224,7 +1224,7 @@ func TestControlMessageValidationInspector_TruncationConfigToggle(t *testing.T) }) // topic validation is ignored set any topic oracle - distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Maybe() rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() inspector.Start(signalerCtx) @@ -1267,14 +1267,14 @@ func TestControlMessageValidationInspector_InspectionConfigToggle(t *testing.T) expectedLogStrs := map[string]struct{}{validation.RPCInspectionDisabledWarning: {}} logCounter := atomic.NewInt64(0) logger := hookedLogger(logCounter, zerolog.TraceLevel, expectedLogStrs) - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Logger = logger // disable inspector for all control message types params.Config.InspectionProcess.Inspect.Disabled = true }) - // distribute should never be called when inspection is disabled - defer distributor.AssertNotCalled(t, "Distribute") + // notification consumer should never be called when inspection is disabled + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() inspector.Start(signalerCtx) @@ -1309,7 +1309,7 @@ func TestControlMessageValidationInspector_InspectionConfigToggle(t *testing.T) } logCounter := atomic.NewInt64(0) logger := hookedLogger(logCounter, zerolog.TraceLevel, expectedLogStrs) - inspector, signalerCtx, cancel, distributor, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { + inspector, signalerCtx, cancel, consumer, rpcTracker, _, _, _ := inspectorFixture(t, func(params *validation.InspectorParams) { params.Config.GraftPrune.MessageCountThreshold = numOfMsgs params.Logger = logger // disable inspection for all control message types individually @@ -1320,8 +1320,8 @@ func TestControlMessageValidationInspector_InspectionConfigToggle(t *testing.T) params.Config.InspectionProcess.Inspect.EnablePublish = false }) - // distribute should never be called when inspection is disabled - defer distributor.AssertNotCalled(t, "Distribute") + // notification consumer should never be called when inspection is disabled + defer consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification") rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() rpcTracker.On("WasIHaveRPCSent", mock.AnythingOfType("string")).Return(true).Maybe() inspector.Start(signalerCtx) @@ -1379,17 +1379,17 @@ func checkNotificationFunc(t *testing.T, func inspectorFixture(t *testing.T, opts ...func(params *validation.InspectorParams)) (*validation.ControlMsgValidationInspector, *irrecoverable.MockSignalerContext, - context.CancelFunc, - *mockp2p.GossipSubInspectorNotificationDistributor, + context.CancelFunc, *mockp2p.GossipSubInvalidControlMessageNotificationConsumer, *mockp2p.RpcControlTracking, flow.Identifier, *mockmodule.IdentityProvider, *internal.MockUpdatableTopicProvider) { + sporkID := unittest.IdentifierFixture() flowConfig, err := config.DefaultConfig() require.NoError(t, err) - distributor := mockp2p.NewGossipSubInspectorNotificationDistributor(t) - p2ptest.MockInspectorNotificationDistributorReadyDoneAware(distributor) + + consumer := mockp2p.NewGossipSubInvalidControlMessageNotificationConsumer(t) idProvider := mockmodule.NewIdentityProvider(t) rpcTracker := mockp2p.NewRpcControlTracking(t) topicProviderOracle := internal.NewMockUpdatableTopicProvider() @@ -1397,12 +1397,12 @@ func inspectorFixture(t *testing.T, opts ...func(params *validation.InspectorPar Logger: unittest.Logger(), SporkID: sporkID, Config: &flowConfig.NetworkConfig.GossipSub.RpcInspector.Validation, - Distributor: distributor, IdProvider: idProvider, HeroCacheMetricsFactory: metrics.NewNoopHeroCacheMetricsFactory(), InspectorMetrics: metrics.NewNoopCollector(), RpcTracker: rpcTracker, - NetworkingType: network.PublicNetwork, + InvalidControlMessageNotificationConsumer: consumer, + NetworkingType: network.PublicNetwork, TopicOracle: func() p2p.TopicProvider { return topicProviderOracle }, @@ -1414,7 +1414,7 @@ func inspectorFixture(t *testing.T, opts ...func(params *validation.InspectorPar require.NoError(t, err, "failed to create control message validation inspector fixture") ctx, cancel := context.WithCancel(context.Background()) signalerCtx := irrecoverable.NewMockSignalerContext(t, ctx) - return validationInspector, signalerCtx, cancel, distributor, rpcTracker, sporkID, idProvider, topicProviderOracle + return validationInspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, idProvider, topicProviderOracle } // utility function to track the number of logs expected logs for the expected log level. From 6eaa46dedf6609da8414145e5a991efadc0a1921 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 14 Feb 2024 17:31:12 -0800 Subject: [PATCH 29/52] fixes tests --- network/p2p/builder.go | 2 +- network/p2p/builder/libp2pNodeBuilder.go | 2 +- network/p2p/mock/node_builder.go | 2 +- network/p2p/scoring/registry.go | 1 + network/p2p/scoring/score_option.go | 3 +- network/p2p/scoring/scoring_test.go | 95 ++++++++---------------- network/p2p/test/fixtures.go | 52 ++++++------- 7 files changed, 63 insertions(+), 94 deletions(-) diff --git a/network/p2p/builder.go b/network/p2p/builder.go index af615ea3a41..5c928c7512c 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -142,7 +142,7 @@ type NodeBuilder interface { // none OverrideNodeConstructor(NodeConstructor) NodeBuilder SetGossipSubFactory(GossipSubFactoryFunc, GossipSubAdapterConfigFunc) NodeBuilder - OverrideDefaultRpcInspectorSuiteFactory(GossipSubRpcInspectorFactoryFunc) NodeBuilder + OverrideDefaultRpcInspectorFactory(GossipSubRpcInspectorFactoryFunc) NodeBuilder Build() (LibP2PNode, error) } diff --git a/network/p2p/builder/libp2pNodeBuilder.go b/network/p2p/builder/libp2pNodeBuilder.go index b02f43c8adc..a725cf962a3 100644 --- a/network/p2p/builder/libp2pNodeBuilder.go +++ b/network/p2p/builder/libp2pNodeBuilder.go @@ -180,7 +180,7 @@ func (builder *LibP2PNodeBuilder) OverrideNodeConstructor(f p2p.NodeConstructor) return builder } -func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { +func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { builder.gossipSubBuilder.OverrideDefaultRpcInspectorFactory(factory) return builder } diff --git a/network/p2p/mock/node_builder.go b/network/p2p/mock/node_builder.go index 523cae41c2a..39e01bb0fa8 100644 --- a/network/p2p/mock/node_builder.go +++ b/network/p2p/mock/node_builder.go @@ -54,7 +54,7 @@ func (_m *NodeBuilder) Build() (p2p.LibP2PNode, error) { } // OverrideDefaultRpcInspectorSuiteFactory provides a mock function with given fields: _a0 -func (_m *NodeBuilder) OverrideDefaultRpcInspectorSuiteFactory(_a0 p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { +func (_m *NodeBuilder) OverrideDefaultRpcInspectorFactory(_a0 p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { ret := _m.Called(_a0) var r0 p2p.NodeBuilder diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index d54c98f4ce0..6c022fa4494 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -413,6 +413,7 @@ func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification( if ok := r.invCtrlMsgNotifWorkerPool.Submit(notification); !ok { // we use a queue with a fixed size, so this can happen when queue is full or when the notification is duplicate. // TODO: we have to add a metric for this case. + // TODO: we should not have deduplication for this case, as we need to penalize the peer for each misbehaviour, we need to add a nonce to the notification. lg.Warn().Msg("gossipsub rpc inspector notification queue is full or notification is duplicate, discarding notification") } lg.Trace().Msg("gossipsub rpc inspector notification submitted to the queue") diff --git a/network/p2p/scoring/score_option.go b/network/p2p/scoring/score_option.go index 6de94b1d07c..a6095951651 100644 --- a/network/p2p/scoring/score_option.go +++ b/network/p2p/scoring/score_option.go @@ -180,7 +180,8 @@ func NewScoreOption(cfg *ScoreOptionConfig, provider p2p.SubscriptionProvider) ( MeshMessageDeliveriesWindow: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveriesWindow, MeshMessageDeliveriesActivation: cfg.params.PeerScoring.Internal.TopicParameters.MeshMessageDeliveryActivation, }, - appScoreFunc: scoreRegistry.AppSpecificScoreFunc(), + appScoreFunc: scoreRegistry.AppSpecificScoreFunc(), + appScoreRegistry: scoreRegistry, } // set the app specific penalty function for the penalty option diff --git a/network/p2p/scoring/scoring_test.go b/network/p2p/scoring/scoring_test.go index cee819d3c85..2dbfff81ff3 100644 --- a/network/p2p/scoring/scoring_test.go +++ b/network/p2p/scoring/scoring_test.go @@ -7,7 +7,6 @@ import ( "testing" "time" - pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/peer" "github.com/rs/zerolog" mocktestify "github.com/stretchr/testify/mock" @@ -16,7 +15,6 @@ import ( "github.com/onflow/flow-go/config" "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/id" "github.com/onflow/flow-go/module/irrecoverable" "github.com/onflow/flow-go/module/metrics" @@ -26,55 +24,11 @@ import ( "github.com/onflow/flow-go/network/p2p" p2pconfig "github.com/onflow/flow-go/network/p2p/config" p2pmsg "github.com/onflow/flow-go/network/p2p/message" + mockp2p "github.com/onflow/flow-go/network/p2p/mock" p2ptest "github.com/onflow/flow-go/network/p2p/test" "github.com/onflow/flow-go/utils/unittest" ) -// mockInspectorSuite is a mock implementation of the GossipSubInspectorSuite interface. -// It is used to test the impact of invalid control messages on the scoring and connectivity of nodes in a network. -type mockInspectorSuite struct { - component.Component - t *testing.T - consumer p2p.GossipSubInvCtrlMsgNotifConsumer -} - -// ensures that mockInspectorSuite implements the GossipSubInspectorSuite interface. -var _ p2p.GossipSubInspectorSuite = (*mockInspectorSuite)(nil) - -func (m *mockInspectorSuite) AddInvalidControlMessageConsumer(consumer p2p.GossipSubInvCtrlMsgNotifConsumer) { - require.Nil(m.t, m.consumer) - m.consumer = consumer -} -func (m *mockInspectorSuite) ActiveClustersChanged(_ flow.ChainIDList) { - // no-op -} - -// newMockInspectorSuite creates a new mockInspectorSuite. -// Args: -// - t: the test object used for assertions. -// Returns: -// - a new mockInspectorSuite. -func newMockInspectorSuite(t *testing.T) *mockInspectorSuite { - i := &mockInspectorSuite{ - t: t, - } - - builder := component.NewComponentManagerBuilder() - builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { - ready() - <-ctx.Done() - }) - - i.Component = builder.Build() - return i -} - -// InspectFunc returns a function that is called when a node receives a control message. -// In this mock implementation, the function does nothing. -func (m *mockInspectorSuite) InspectFunc() func(peer.ID, *pubsub.RPC) error { - return nil -} - // TestInvalidCtrlMsgScoringIntegration tests the impact of invalid control messages on the scoring and connectivity of nodes in a network. // It creates a network of 2 nodes, and sends a set of control messages with invalid topic IDs to one of the nodes. // It then checks that the node receiving the invalid control messages decreases its score for the peer spamming the invalid messages, and @@ -86,26 +40,25 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { sporkId := unittest.IdentifierFixture() idProvider := mock.NewIdentityProvider(t) - inspectorSuite1 := newMockInspectorSuite(t) - factory := func( - irrecoverable.SignalerContext, - zerolog.Logger, - flow.Identifier, - *p2pconfig.RpcInspectorParameters, - module.GossipSubMetrics, - metrics.HeroCacheMetricsFactory, - flownet.NetworkingType, - module.IdentityProvider, - func() p2p.TopicProvider) (p2p.GossipSubInspectorSuite, error) { - // override the gossipsub rpc inspector suite factory to return the mock inspector suite - return inspectorSuite1, nil - } - cfg, err := config.DefaultConfig() require.NoError(t, err) cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond // speed up the test + var notificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer + inspector := mockp2p.NewGossipSubMsgValidationRpcInspector(t) + inspector.On("Inspect", mocktestify.Anything, mocktestify.Anything).Return(nil) // no-op for the inspector + inspector.On("ActiveClustersChanged", mocktestify.Anything).Return().Maybe() // no-op for the inspector + inspector.On("Start", mocktestify.Anything).Return(nil) // no-op for the inspector + + // mocking the Ready and Done channels to be closed + done := make(chan struct{}) + close(done) + f := func() <-chan struct{} { + return done + } + inspector.On("Ready").Return(f()) // no-op for the inspector + inspector.On("Done").Return(f()) // no-op for the inspector node1, id1 := p2ptest.NodeFixture( t, sporkId, @@ -113,7 +66,19 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { idProvider, p2ptest.WithRole(flow.RoleConsensus), p2ptest.OverrideFlowConfig(cfg), - p2ptest.OverrideGossipSubRpcInspectorSuiteFactory(factory)) + p2ptest.OverrideGossipSubRpcInspectorFactory(func(logger zerolog.Logger, + _ flow.Identifier, + _ *p2pconfig.RpcInspectorParameters, + _ module.GossipSubMetrics, + _ metrics.HeroCacheMetricsFactory, + _ flownet.NetworkingType, + _ module.IdentityProvider, + _ func() p2p.TopicProvider, + consumer p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error) { + // short-wire the consumer + notificationConsumer = consumer + return inspector, nil + })) node2, id2 := p2ptest.NodeFixture( t, @@ -125,6 +90,8 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { ids := flow.IdentityList{&id1, &id2} nodes := []p2p.LibP2PNode{node1, node2} + // suppressing "peers provider not set error" + p2ptest.RegisterPeerProviders(t, nodes) provider := id.NewFixedIdentityProvider(ids) idProvider.On("ByPeerID", mocktestify.Anything).Return( @@ -148,7 +115,7 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { // simulates node2 spamming node1 with invalid gossipsub control messages until node2 gets dissallow listed. // since the decay will start lower than .99 and will only be incremented by default .01, we need to spam a lot of messages so that the node gets disallow listed for i := 0; i < 750; i++ { - inspectorSuite1.consumer.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ + notificationConsumer.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ PeerID: node2.ID(), MsgType: p2pmsg.ControlMessageTypes()[rand.Intn(len(p2pmsg.ControlMessageTypes()))], Error: fmt.Errorf("invalid control message"), diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index abeea7cc6cc..05b8e1bc952 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -161,8 +161,8 @@ func NodeFixture(t *testing.T, }) } - if parameters.GossipSubRpcInspectorSuiteFactory != nil { - builder.OverrideDefaultRpcInspectorSuiteFactory(parameters.GossipSubRpcInspectorSuiteFactory) + if parameters.GossipSubRpcInspectorFactory != nil { + builder.OverrideDefaultRpcInspectorFactory(parameters.GossipSubRpcInspectorFactory) } if parameters.ResourceManager != nil { @@ -228,28 +228,28 @@ func RegisterPeerProviders(_ *testing.T, nodes []p2p.LibP2PNode) { type NodeFixtureParameterOption func(*NodeFixtureParameters) type NodeFixtureParameters struct { - HandlerFunc network.StreamHandler - NetworkingType flownet.NetworkingType - Unicasts []protocols.ProtocolName - Key crypto.PrivateKey - Address string - DhtOptions []dht.Option - Role flow.Role - Logger zerolog.Logger - PeerScoringEnabled bool - IdProvider module.IdentityProvider - PeerScoringConfigOverride *p2p.PeerScoringConfigOverride - PeerManagerConfig *p2pbuilderconfig.PeerManagerConfig - PeerProvider p2p.PeersProvider // peer manager parameter - ConnGater p2p.ConnectionGater - ConnManager connmgr.ConnManager - GossipSubFactory p2p.GossipSubFactoryFunc - GossipSubConfig p2p.GossipSubAdapterConfigFunc - MetricsCfg *p2pbuilderconfig.MetricsConfig - ResourceManager network.ResourceManager - GossipSubRpcInspectorSuiteFactory p2p.GossipSubRpcInspectorFactoryFunc - FlowConfig *config.FlowConfig - UnicastRateLimiterDistributor p2p.UnicastRateLimiterDistributor + HandlerFunc network.StreamHandler + NetworkingType flownet.NetworkingType + Unicasts []protocols.ProtocolName + Key crypto.PrivateKey + Address string + DhtOptions []dht.Option + Role flow.Role + Logger zerolog.Logger + PeerScoringEnabled bool + IdProvider module.IdentityProvider + PeerScoringConfigOverride *p2p.PeerScoringConfigOverride + PeerManagerConfig *p2pbuilderconfig.PeerManagerConfig + PeerProvider p2p.PeersProvider // peer manager parameter + ConnGater p2p.ConnectionGater + ConnManager connmgr.ConnManager + GossipSubFactory p2p.GossipSubFactoryFunc + GossipSubConfig p2p.GossipSubAdapterConfigFunc + MetricsCfg *p2pbuilderconfig.MetricsConfig + ResourceManager network.ResourceManager + GossipSubRpcInspectorFactory p2p.GossipSubRpcInspectorFactoryFunc + FlowConfig *config.FlowConfig + UnicastRateLimiterDistributor p2p.UnicastRateLimiterDistributor } func WithUnicastRateLimitDistributor(distributor p2p.UnicastRateLimiterDistributor) NodeFixtureParameterOption { @@ -258,9 +258,9 @@ func WithUnicastRateLimitDistributor(distributor p2p.UnicastRateLimiterDistribut } } -func OverrideGossipSubRpcInspectorSuiteFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) NodeFixtureParameterOption { +func OverrideGossipSubRpcInspectorFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) NodeFixtureParameterOption { return func(p *NodeFixtureParameters) { - p.GossipSubRpcInspectorSuiteFactory = factory + p.GossipSubRpcInspectorFactory = factory } } From 86a9ba466193222e79f697e9c8c0d6142bb80d95 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 14 Feb 2024 17:34:22 -0800 Subject: [PATCH 30/52] re-generates mocks --- .../mock/gossip_sub_rpc_inspector_factory_func.go | 14 ++++++-------- network/p2p/mock/node_builder.go | 2 +- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go b/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go index c4191cd7f5c..deeb2c309f0 100644 --- a/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go +++ b/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go @@ -4,8 +4,6 @@ package mockp2p import ( flow "github.com/onflow/flow-go/model/flow" - irrecoverable "github.com/onflow/flow-go/module/irrecoverable" - metrics "github.com/onflow/flow-go/module/metrics" mock "github.com/stretchr/testify/mock" @@ -27,23 +25,23 @@ type GossipSubRpcInspectorFactoryFunc struct { } // Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8 -func (_m *GossipSubRpcInspectorFactoryFunc) Execute(_a0 irrecoverable.SignalerContext, _a1 zerolog.Logger, _a2 flow.Identifier, _a3 *p2pconfig.RpcInspectorParameters, _a4 module.GossipSubMetrics, _a5 metrics.HeroCacheMetricsFactory, _a6 network.NetworkingType, _a7 module.IdentityProvider, _a8 func() p2p.TopicProvider) (p2p.GossipSubRPCInspector, error) { +func (_m *GossipSubRpcInspectorFactoryFunc) Execute(_a0 zerolog.Logger, _a1 flow.Identifier, _a2 *p2pconfig.RpcInspectorParameters, _a3 module.GossipSubMetrics, _a4 metrics.HeroCacheMetricsFactory, _a5 network.NetworkingType, _a6 module.IdentityProvider, _a7 func() p2p.TopicProvider, _a8 p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error) { ret := _m.Called(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) - var r0 p2p.GossipSubRPCInspector + var r0 p2p.GossipSubMsgValidationRpcInspector var r1 error - if rf, ok := ret.Get(0).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) (p2p.GossipSubRPCInspector, error)); ok { + if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error)); ok { return rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } - if rf, ok := ret.Get(0).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) p2p.GossipSubRPCInspector); ok { + if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) p2p.GossipSubMsgValidationRpcInspector); ok { r0 = rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(p2p.GossipSubRPCInspector) + r0 = ret.Get(0).(p2p.GossipSubMsgValidationRpcInspector) } } - if rf, ok := ret.Get(1).(func(irrecoverable.SignalerContext, zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider) error); ok { + if rf, ok := ret.Get(1).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) error); ok { r1 = rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } else { r1 = ret.Error(1) diff --git a/network/p2p/mock/node_builder.go b/network/p2p/mock/node_builder.go index 39e01bb0fa8..acadd5e34a2 100644 --- a/network/p2p/mock/node_builder.go +++ b/network/p2p/mock/node_builder.go @@ -53,7 +53,7 @@ func (_m *NodeBuilder) Build() (p2p.LibP2PNode, error) { return r0, r1 } -// OverrideDefaultRpcInspectorSuiteFactory provides a mock function with given fields: _a0 +// OverrideDefaultRpcInspectorFactory provides a mock function with given fields: _a0 func (_m *NodeBuilder) OverrideDefaultRpcInspectorFactory(_a0 p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { ret := _m.Called(_a0) From 2a21a44d04a94afaf6879e48912b96f2f2c520de Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 15 Feb 2024 18:27:53 -0800 Subject: [PATCH 31/52] fixing tests --- .../corruptlibp2p/pubsub_adapter_config.go | 2 +- network/p2p/builder.go | 2 +- .../p2p/builder/gossipsub/gossipSubBuilder.go | 7 +- .../control_message_validation_inspector.go | 2 +- ...gossip_sub_msg_validation_rpc_inspector.go | 104 ------------------ network/p2p/mock/gossip_sub_rpc_inspector.go | 7 ++ .../gossip_sub_rpc_inspector_factory_func.go | 10 +- network/p2p/mock/pub_sub_adapter_config.go | 2 +- network/p2p/node/gossipSubAdapterConfig.go | 2 +- network/p2p/pubsub.go | 20 +--- network/p2p/scoring/noopConsumer.go | 19 ++++ network/p2p/scoring/registry.go | 6 + network/p2p/scoring/scoring_test.go | 4 +- 13 files changed, 55 insertions(+), 132 deletions(-) delete mode 100644 network/p2p/mock/gossip_sub_msg_validation_rpc_inspector.go create mode 100644 network/p2p/scoring/noopConsumer.go diff --git a/insecure/corruptlibp2p/pubsub_adapter_config.go b/insecure/corruptlibp2p/pubsub_adapter_config.go index fe90db0a9b2..adc3337d629 100644 --- a/insecure/corruptlibp2p/pubsub_adapter_config.go +++ b/insecure/corruptlibp2p/pubsub_adapter_config.go @@ -153,7 +153,7 @@ func (c *CorruptPubSubAdapterConfig) ScoreTracer() p2p.PeerScoreTracer { return c.scoreTracer } -func (c *CorruptPubSubAdapterConfig) WithRpcInspector(_ p2p.GossipSubMsgValidationRpcInspector) { +func (c *CorruptPubSubAdapterConfig) WithRpcInspector(_ p2p.GossipSubRPCInspector) { // CorruptPubSub does not support inspector suite. This is a no-op. } diff --git a/network/p2p/builder.go b/network/p2p/builder.go index 5c928c7512c..9140f83ce59 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -109,7 +109,7 @@ type GossipSubRpcInspectorFactoryFunc func( module.IdentityProvider, func() TopicProvider, GossipSubInvCtrlMsgNotifConsumer, -) (GossipSubMsgValidationRpcInspector, error) +) (GossipSubRPCInspector, error) // NodeBuilder is a builder pattern for creating a libp2p Node instance. type NodeBuilder interface { diff --git a/network/p2p/builder/gossipsub/gossipSubBuilder.go b/network/p2p/builder/gossipsub/gossipSubBuilder.go index 66fff418dde..faed8090b43 100644 --- a/network/p2p/builder/gossipsub/gossipSubBuilder.go +++ b/network/p2p/builder/gossipsub/gossipSubBuilder.go @@ -203,7 +203,7 @@ func defaultRpcInspectorFactory(tracer p2p.PubSubTracer) p2p.GossipSubRpcInspect networkingType network.NetworkingType, idProvider module.IdentityProvider, topicProvider func() p2p.TopicProvider, - notificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error) { + notificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubRPCInspector, error) { return validation.NewControlMsgValidationInspector(&validation.InspectorParams{ Logger: logger.With().Str("component", "rpc-inspector").Logger(), SporkID: sporkId, @@ -265,6 +265,7 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e var scoreOpt *scoring.ScoreOption var scoreTracer p2p.PeerScoreTracer + var consumer p2p.GossipSubInvCtrlMsgNotifConsumer // currently, peer scoring is not supported for public networks. if g.gossipSubCfg.PeerScoringEnabled && g.networkType != network.PublicNetwork { // wires the gossipsub score option to the subscription provider. @@ -289,6 +290,7 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e return nil, fmt.Errorf("could not create gossipsub score option: %w", err) } gossipSubConfigs.WithScoreOption(scoreOpt) + consumer = scoreOpt // the score option is the consumer of the invalid control message notifications. if g.gossipSubCfg.RpcTracer.ScoreTracerInterval > 0 { scoreTracer = tracer.NewGossipSubScoreTracer(g.logger, g.idProvider, g.metricsCfg.Metrics, g.gossipSubCfg.RpcTracer.ScoreTracerInterval) @@ -298,6 +300,7 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e g.logger.Warn(). Str(logging.KeyNetworkingSecurity, "true"). Msg("gossipsub peer scoring is disabled") + consumer = scoring.NewNoopInvCtrlMsgNotifConsumer() // no-op consumer as peer scoring is disabled. } rpcValidationInspector, err := g.rpcInspectorFactory( @@ -311,7 +314,7 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e func() p2p.TopicProvider { return gossipSub }, - scoreOpt) + consumer) if err != nil { return nil, fmt.Errorf("failed to create new rpc valiadation inspector: %w", err) } diff --git a/network/p2p/inspector/validation/control_message_validation_inspector.go b/network/p2p/inspector/validation/control_message_validation_inspector.go index 20bd05f6830..06dca48f2c7 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector.go @@ -113,7 +113,7 @@ type InspectorParams struct { } var _ component.Component = (*ControlMsgValidationInspector)(nil) -var _ p2p.GossipSubMsgValidationRpcInspector = (*ControlMsgValidationInspector)(nil) +var _ p2p.GossipSubRPCInspector = (*ControlMsgValidationInspector)(nil) var _ protocol.Consumer = (*ControlMsgValidationInspector)(nil) // NewControlMsgValidationInspector returns new ControlMsgValidationInspector diff --git a/network/p2p/mock/gossip_sub_msg_validation_rpc_inspector.go b/network/p2p/mock/gossip_sub_msg_validation_rpc_inspector.go deleted file mode 100644 index 41d3a409533..00000000000 --- a/network/p2p/mock/gossip_sub_msg_validation_rpc_inspector.go +++ /dev/null @@ -1,104 +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" - - peer "github.com/libp2p/go-libp2p/core/peer" - - pubsub "github.com/libp2p/go-libp2p-pubsub" -) - -// GossipSubMsgValidationRpcInspector is an autogenerated mock type for the GossipSubMsgValidationRpcInspector type -type GossipSubMsgValidationRpcInspector struct { - mock.Mock -} - -// ActiveClustersChanged provides a mock function with given fields: _a0 -func (_m *GossipSubMsgValidationRpcInspector) ActiveClustersChanged(_a0 flow.ChainIDList) { - _m.Called(_a0) -} - -// Done provides a mock function with given fields: -func (_m *GossipSubMsgValidationRpcInspector) 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 -} - -// Inspect provides a mock function with given fields: _a0, _a1 -func (_m *GossipSubMsgValidationRpcInspector) Inspect(_a0 peer.ID, _a1 *pubsub.RPC) error { - ret := _m.Called(_a0, _a1) - - var r0 error - if rf, ok := ret.Get(0).(func(peer.ID, *pubsub.RPC) error); ok { - r0 = rf(_a0, _a1) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Name provides a mock function with given fields: -func (_m *GossipSubMsgValidationRpcInspector) Name() string { - ret := _m.Called() - - var r0 string - if rf, ok := ret.Get(0).(func() string); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(string) - } - - return r0 -} - -// Ready provides a mock function with given fields: -func (_m *GossipSubMsgValidationRpcInspector) 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 *GossipSubMsgValidationRpcInspector) Start(_a0 irrecoverable.SignalerContext) { - _m.Called(_a0) -} - -type mockConstructorTestingTNewGossipSubMsgValidationRpcInspector interface { - mock.TestingT - Cleanup(func()) -} - -// NewGossipSubMsgValidationRpcInspector creates a new instance of GossipSubMsgValidationRpcInspector. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -func NewGossipSubMsgValidationRpcInspector(t mockConstructorTestingTNewGossipSubMsgValidationRpcInspector) *GossipSubMsgValidationRpcInspector { - mock := &GossipSubMsgValidationRpcInspector{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/network/p2p/mock/gossip_sub_rpc_inspector.go b/network/p2p/mock/gossip_sub_rpc_inspector.go index fa7453b5bc2..24123537b23 100644 --- a/network/p2p/mock/gossip_sub_rpc_inspector.go +++ b/network/p2p/mock/gossip_sub_rpc_inspector.go @@ -3,7 +3,9 @@ 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" peer "github.com/libp2p/go-libp2p/core/peer" @@ -16,6 +18,11 @@ type GossipSubRPCInspector struct { mock.Mock } +// ActiveClustersChanged provides a mock function with given fields: _a0 +func (_m *GossipSubRPCInspector) ActiveClustersChanged(_a0 flow.ChainIDList) { + _m.Called(_a0) +} + // Done provides a mock function with given fields: func (_m *GossipSubRPCInspector) Done() <-chan struct{} { ret := _m.Called() diff --git a/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go b/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go index deeb2c309f0..141da62f9e5 100644 --- a/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go +++ b/network/p2p/mock/gossip_sub_rpc_inspector_factory_func.go @@ -25,19 +25,19 @@ type GossipSubRpcInspectorFactoryFunc struct { } // Execute provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8 -func (_m *GossipSubRpcInspectorFactoryFunc) Execute(_a0 zerolog.Logger, _a1 flow.Identifier, _a2 *p2pconfig.RpcInspectorParameters, _a3 module.GossipSubMetrics, _a4 metrics.HeroCacheMetricsFactory, _a5 network.NetworkingType, _a6 module.IdentityProvider, _a7 func() p2p.TopicProvider, _a8 p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error) { +func (_m *GossipSubRpcInspectorFactoryFunc) Execute(_a0 zerolog.Logger, _a1 flow.Identifier, _a2 *p2pconfig.RpcInspectorParameters, _a3 module.GossipSubMetrics, _a4 metrics.HeroCacheMetricsFactory, _a5 network.NetworkingType, _a6 module.IdentityProvider, _a7 func() p2p.TopicProvider, _a8 p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubRPCInspector, error) { ret := _m.Called(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) - var r0 p2p.GossipSubMsgValidationRpcInspector + var r0 p2p.GossipSubRPCInspector var r1 error - if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error)); ok { + if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubRPCInspector, error)); ok { return rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } - if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) p2p.GossipSubMsgValidationRpcInspector); ok { + if rf, ok := ret.Get(0).(func(zerolog.Logger, flow.Identifier, *p2pconfig.RpcInspectorParameters, module.GossipSubMetrics, metrics.HeroCacheMetricsFactory, network.NetworkingType, module.IdentityProvider, func() p2p.TopicProvider, p2p.GossipSubInvCtrlMsgNotifConsumer) p2p.GossipSubRPCInspector); ok { r0 = rf(_a0, _a1, _a2, _a3, _a4, _a5, _a6, _a7, _a8) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(p2p.GossipSubMsgValidationRpcInspector) + r0 = ret.Get(0).(p2p.GossipSubRPCInspector) } } diff --git a/network/p2p/mock/pub_sub_adapter_config.go b/network/p2p/mock/pub_sub_adapter_config.go index 9db204914a9..980d6cb71f3 100644 --- a/network/p2p/mock/pub_sub_adapter_config.go +++ b/network/p2p/mock/pub_sub_adapter_config.go @@ -25,7 +25,7 @@ func (_m *PubSubAdapterConfig) WithRoutingDiscovery(_a0 routing.ContentRouting) } // WithRpcInspector provides a mock function with given fields: _a0 -func (_m *PubSubAdapterConfig) WithRpcInspector(_a0 p2p.GossipSubMsgValidationRpcInspector) { +func (_m *PubSubAdapterConfig) WithRpcInspector(_a0 p2p.GossipSubRPCInspector) { _m.Called(_a0) } diff --git a/network/p2p/node/gossipSubAdapterConfig.go b/network/p2p/node/gossipSubAdapterConfig.go index b73d207297a..e9b102a6e81 100644 --- a/network/p2p/node/gossipSubAdapterConfig.go +++ b/network/p2p/node/gossipSubAdapterConfig.go @@ -82,7 +82,7 @@ func (g *GossipSubAdapterConfig) WithMessageIdFunction(f func([]byte) string) { // - suite: the inspector suite to use // Returns: // -None -func (g *GossipSubAdapterConfig) WithRpcInspector(inspector p2p.GossipSubMsgValidationRpcInspector) { +func (g *GossipSubAdapterConfig) WithRpcInspector(inspector p2p.GossipSubRPCInspector) { g.options = append(g.options, pubsub.WithAppSpecificRpcInspector(inspector.Inspect)) g.inspector = inspector } diff --git a/network/p2p/pubsub.go b/network/p2p/pubsub.go index 50c8c3548c1..97741d0820e 100644 --- a/network/p2p/pubsub.go +++ b/network/p2p/pubsub.go @@ -83,14 +83,18 @@ type PubSubAdapterConfig interface { // WithScoreTracer sets the tracer for the underlying pubsub score implementation. // This is used to expose the local scoring table of the GossipSub node to its higher level components. WithScoreTracer(tracer PeerScoreTracer) - WithRpcInspector(GossipSubMsgValidationRpcInspector) + WithRpcInspector(GossipSubRPCInspector) } -// GossipSubRPCInspector app specific RPC inspector used to inspect and validate incoming RPC messages before they are processed by libp2p. +// GossipSubRPCInspector abstracts the general behavior of an app specific RPC inspector specifically +// used to inspect and validate incoming. It is used to implement custom message validation logic. It is injected into +// the GossipSubRouter and run on every incoming RPC message before the message is processed by libp2p. If the message +// is invalid the RPC message will be dropped. // Implementations must: // - be concurrency safe // - be non-blocking type GossipSubRPCInspector interface { + collection.ClusterEvents component.Component // Name returns the name of the rpc inspector. @@ -102,18 +106,6 @@ type GossipSubRPCInspector interface { Inspect(peer.ID, *pubsub.RPC) error } -// GossipSubMsgValidationRpcInspector abstracts the general behavior of an app specific RPC inspector specifically -// used to inspect and validate incoming. It is used to implement custom message validation logic. It is injected into -// the GossipSubRouter and run on every incoming RPC message before the message is processed by libp2p. If the message -// is invalid the RPC message will be dropped. -// Implementations must: -// - be concurrency safe -// - be non-blocking -type GossipSubMsgValidationRpcInspector interface { - collection.ClusterEvents - GossipSubRPCInspector -} - // Topic is the abstraction of the underlying pubsub topic that is used by the Flow network. type Topic interface { // String returns the topic name as a string. diff --git a/network/p2p/scoring/noopConsumer.go b/network/p2p/scoring/noopConsumer.go new file mode 100644 index 00000000000..b3eaa95ee8e --- /dev/null +++ b/network/p2p/scoring/noopConsumer.go @@ -0,0 +1,19 @@ +package scoring + +import "github.com/onflow/flow-go/network/p2p" + +// NoopInvCtrlMsgNotifConsumer is a no-op implementation of the p2p.GossipSubInvCtrlMsgNotifConsumer interface. +// It is used to consume invalid control message notifications from the GossipSub pubsub system and take no action. +// It is mainly used for cases when the peer scoring system is disabled. +type NoopInvCtrlMsgNotifConsumer struct { +} + +func NewNoopInvCtrlMsgNotifConsumer() *NoopInvCtrlMsgNotifConsumer { + return &NoopInvCtrlMsgNotifConsumer{} +} + +var _ p2p.GossipSubInvCtrlMsgNotifConsumer = (*NoopInvCtrlMsgNotifConsumer)(nil) + +func (n NoopInvCtrlMsgNotifConsumer) OnInvalidControlMessageNotification(_ *p2p.InvCtrlMsgNotif) { + // no-op +} diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index 6c022fa4494..989aed92425 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -419,6 +419,12 @@ func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification( lg.Trace().Msg("gossipsub rpc inspector notification submitted to the queue") } +// handleMisbehaviourReport is the worker function that is called by the worker pool to handle the misbehaviour report of a peer. +// The function is called in a non-blocking way, and the worker pool is used to limit the number of concurrent executions of the function. +// Args: +// - notification: the notification of the misbehaviour report of a peer. +// Returns: +// - error: an error if the update failed; any returned error is an irrecoverable error and indicates a bug or misconfiguration. func (r *GossipSubAppSpecificScoreRegistry) handleMisbehaviourReport(notification *p2p.InvCtrlMsgNotif) error { // we use mutex to ensure the method is concurrency safe. lg := r.logger.With(). diff --git a/network/p2p/scoring/scoring_test.go b/network/p2p/scoring/scoring_test.go index 2dbfff81ff3..f448cd271bd 100644 --- a/network/p2p/scoring/scoring_test.go +++ b/network/p2p/scoring/scoring_test.go @@ -46,7 +46,7 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { cfg.NetworkConfig.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL = 10 * time.Millisecond // speed up the test var notificationConsumer p2p.GossipSubInvCtrlMsgNotifConsumer - inspector := mockp2p.NewGossipSubMsgValidationRpcInspector(t) + inspector := mockp2p.NewGossipSubRPCInspector(t) inspector.On("Inspect", mocktestify.Anything, mocktestify.Anything).Return(nil) // no-op for the inspector inspector.On("ActiveClustersChanged", mocktestify.Anything).Return().Maybe() // no-op for the inspector inspector.On("Start", mocktestify.Anything).Return(nil) // no-op for the inspector @@ -74,7 +74,7 @@ func TestInvalidCtrlMsgScoringIntegration(t *testing.T) { _ flownet.NetworkingType, _ module.IdentityProvider, _ func() p2p.TopicProvider, - consumer p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubMsgValidationRpcInspector, error) { + consumer p2p.GossipSubInvCtrlMsgNotifConsumer) (p2p.GossipSubRPCInspector, error) { // short-wire the consumer notificationConsumer = consumer return inspector, nil From 3275b22aec70c9b6758afbc89fadb7c64155aa01 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 15 Feb 2024 18:29:09 -0800 Subject: [PATCH 32/52] fixes TestScoreRegistry_SpamRecordWithoutDuplicateMessagesPenalty --- network/p2p/scoring/registry_test.go | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/network/p2p/scoring/registry_test.go b/network/p2p/scoring/registry_test.go index db74553ac2e..3d85d4032ea 100644 --- a/network/p2p/scoring/registry_test.go +++ b/network/p2p/scoring/registry_test.go @@ -600,12 +600,20 @@ func testScoreRegistrySpamRecordWithoutDuplicateMessagesPenalty(t *testing.T, me MsgType: messageType, }) - // the penalty should now be updated in the spamRecords - record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. - assert.True(t, ok) - assert.NoError(t, err) - unittest.RequireNumericallyClose(t, expectedPenalty, record.Penalty, 0.01) - assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + require.Eventually(t, func() bool { + // the notification is processed asynchronously, and the penalty should eventually be updated in the spamRecords + record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. + if !ok { + return false + } + require.NoError(t, err) + if !unittest.AreNumericallyClose(expectedPenalty, record.Penalty, 10e-2) { + return false + } + require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + + return true + }, 5*time.Second, 10*time.Millisecond) queryTime := time.Now() // eventually, the app specific score should be updated in the cache. From d72ccac99c2cec5465297b121cb26fde408a2407 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 15 Feb 2024 18:43:45 -0800 Subject: [PATCH 33/52] fixes testScoreRegistryPeerWithSpamRecord --- network/p2p/scoring/registry_test.go | 30 +++++++++++++++------------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/network/p2p/scoring/registry_test.go b/network/p2p/scoring/registry_test.go index 3d85d4032ea..fed4e9a66d3 100644 --- a/network/p2p/scoring/registry_test.go +++ b/network/p2p/scoring/registry_test.go @@ -169,29 +169,31 @@ func testScoreRegistryPeerWithSpamRecord(t *testing.T, messageType p2pmsg.Contro MsgType: messageType, }) - // the penalty should now be updated in the spamRecords - record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. - assert.True(t, ok) - assert.NoError(t, err) - assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) // penalty should be updated to -10. - assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. - queryTime := time.Now() - // eventually, the app specific score should be updated in the cache. require.Eventually(t, func() bool { - // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. - score := reg.AppSpecificScoreFunc()(peerID) + // the notification is processed asynchronously, and the penalty should eventually be updated in the spamRecords + record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. + if !ok { + return false + } + require.NoError(t, err) + if !unittest.AreNumericallyClose(expectedPenalty, record.Penalty, 10e-2) { + return false + } + require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + + // eventually, the app specific score should be updated in the cache. // this peer has a spam record, with no subscription penalty. Hence, the app specific score should only be the spam penalty, // and the peer should be deprived of the default reward for its valid staked role. - // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 0.1% error. - return unittest.AreNumericallyClose(expectedPenalty, score, 10e-4) - }, 5*time.Second, 100*time.Millisecond) + // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 1% error. + return unittest.AreNumericallyClose(expectedPenalty, reg.AppSpecificScoreFunc()(peerID), 10e-2) + }, 5*time.Second, 10*time.Millisecond) // the app specific score should now be updated in the cache. score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) require.True(t, updated.After(queryTime)) - require.True(t, unittest.AreNumericallyClose(expectedPenalty, score, 10e-4)) + require.True(t, unittest.AreNumericallyClose(expectedPenalty, score, 10e-2)) // stop the registry. cancel() From 832d28552610b138334994d360a88bee4cea1aee Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 15 Feb 2024 18:54:54 -0800 Subject: [PATCH 34/52] fixes TestScoreRegistry_SpamRecordWithUnknownIdentity --- network/p2p/scoring/registry_test.go | 40 +++++++++++++++------------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/network/p2p/scoring/registry_test.go b/network/p2p/scoring/registry_test.go index fed4e9a66d3..2d0958dc67e 100644 --- a/network/p2p/scoring/registry_test.go +++ b/network/p2p/scoring/registry_test.go @@ -185,15 +185,15 @@ func testScoreRegistryPeerWithSpamRecord(t *testing.T, messageType p2pmsg.Contro // eventually, the app specific score should be updated in the cache. // this peer has a spam record, with no subscription penalty. Hence, the app specific score should only be the spam penalty, // and the peer should be deprived of the default reward for its valid staked role. - // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 1% error. - return unittest.AreNumericallyClose(expectedPenalty, reg.AppSpecificScoreFunc()(peerID), 10e-2) + // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 5% error. + return unittest.AreNumericallyClose(expectedPenalty, reg.AppSpecificScoreFunc()(peerID), 0.05) }, 5*time.Second, 10*time.Millisecond) // the app specific score should now be updated in the cache. score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) require.True(t, updated.After(queryTime)) - require.True(t, unittest.AreNumericallyClose(expectedPenalty, score, 10e-2)) + require.True(t, unittest.AreNumericallyClose(expectedPenalty, score, 0.1)) // account for maximum 10% error due to decays and asynchrony. // stop the registry. cancel() @@ -275,31 +275,33 @@ func testScoreRegistrySpamRecordWithUnknownIdentity(t *testing.T, messageType p2 MsgType: messageType, }) - // the penalty should now be updated. - record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. - require.True(t, ok) - require.NoError(t, err) - require.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) // penalty should be updated to -10, we account for decay. - require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. - queryTime := time.Now() - // eventually, the app specific score should be updated in the cache. require.Eventually(t, func() bool { - // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. - score := reg.AppSpecificScoreFunc()(peerID) + // the notification is processed asynchronously, and the penalty should eventually be updated in the spamRecords + record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. + if !ok { + return false + } + require.NoError(t, err) + if !unittest.AreNumericallyClose(expectedPenalty, record.Penalty, 10e-2) { + return false + } + require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + + // eventually, the app specific score should be updated in the cache. // the peer has spam record as well as an unknown identity. Hence, the app specific score should be the spam penalty // and the staking penalty. - // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 0.1% error. - return unittest.AreNumericallyClose(expectedPenalty+scoreOptParameters.UnknownIdentityPenalty, score, 0.01) - }, 5*time.Second, 10*time.Millisecond) + // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 5% error. + return unittest.AreNumericallyClose(expectedPenalty+scoreOptParameters.UnknownIdentityPenalty, reg.AppSpecificScoreFunc()(peerID), 0.05) + }, 5*time.Second, 100*time.Millisecond) // the app specific score should now be updated in the cache. score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) + fmt.Println("updated", updated, "queryTime", queryTime) require.True(t, updated.After(queryTime)) - - unittest.RequireNumericallyClose(t, expectedPenalty+scoreOptParameters.UnknownIdentityPenalty, score, 0.01) - assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + fmt.Println("score", score, "expected", expectedPenalty+scoreOptParameters.UnknownIdentityPenalty) + unittest.RequireNumericallyClose(t, expectedPenalty+scoreOptParameters.UnknownIdentityPenalty, score, 0.1) // account for maximum 10% error due to decays and asynchrony. // stop the registry. cancel() From 29302d6c95db45cec8a385dab9c719c1fa916963 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 15 Feb 2024 18:57:50 -0800 Subject: [PATCH 35/52] fixes TestScoreRegistry_SpamRecordWithSubscriptionPenalty --- network/p2p/scoring/registry_test.go | 30 +++++++++++++++------------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/network/p2p/scoring/registry_test.go b/network/p2p/scoring/registry_test.go index 2d0958dc67e..d5a2922bb24 100644 --- a/network/p2p/scoring/registry_test.go +++ b/network/p2p/scoring/registry_test.go @@ -384,29 +384,31 @@ func testScoreRegistrySpamRecordWithSubscriptionPenalty(t *testing.T, messageTyp MsgType: messageType, }) - // the penalty should now be updated. - record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. - assert.True(t, ok) - assert.NoError(t, err) - assert.Less(t, math.Abs(expectedPenalty-record.Penalty), 10e-3) - assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. - queryTime := time.Now() - // eventually, the app specific score should be updated in the cache. require.Eventually(t, func() bool { - // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. - score := reg.AppSpecificScoreFunc()(peerID) + // the notification is processed asynchronously, and the penalty should eventually be updated in the spamRecords + record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. + if !ok { + return false + } + require.NoError(t, err) + if !unittest.AreNumericallyClose(expectedPenalty, record.Penalty, 10e-2) { + return false + } + require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + + // eventually, the app specific score should be updated in the cache. // the peer has spam record as well as an unknown identity. Hence, the app specific score should be the spam penalty // and the staking penalty. - // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 0.1% error. - return unittest.AreNumericallyClose(expectedPenalty+scoreOptParameters.InvalidSubscriptionPenalty, score, 0.01) - }, 5*time.Second, 10*time.Millisecond) + // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 5% error. + return unittest.AreNumericallyClose(expectedPenalty+scoreOptParameters.InvalidSubscriptionPenalty, reg.AppSpecificScoreFunc()(peerID), 0.05) + }, 5*time.Second, 100*time.Millisecond) // the app specific score should now be updated in the cache. score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) require.True(t, updated.After(queryTime)) - unittest.RequireNumericallyClose(t, expectedPenalty+scoreOptParameters.InvalidSubscriptionPenalty, score, 0.01) + unittest.RequireNumericallyClose(t, expectedPenalty+scoreOptParameters.InvalidSubscriptionPenalty, score, 0.1) // account for maximum 10% error due to decays and asynchrony. // stop the registry. cancel() From 27bbd0f009f47411f0662fcf34c246055dcb8ca5 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Thu, 15 Feb 2024 19:35:56 -0800 Subject: [PATCH 36/52] fixes all scoring tests --- network/p2p/scoring/registry_test.go | 74 +++++++++++++++++++--------- 1 file changed, 51 insertions(+), 23 deletions(-) diff --git a/network/p2p/scoring/registry_test.go b/network/p2p/scoring/registry_test.go index d5a2922bb24..68d8ca0b129 100644 --- a/network/p2p/scoring/registry_test.go +++ b/network/p2p/scoring/registry_test.go @@ -497,28 +497,29 @@ func testScoreRegistrySpamRecordWithDuplicateMessagesPenalty(t *testing.T, messa MsgType: messageType, }) - // the penalty should now be updated in the spamRecords - record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. - assert.True(t, ok) - assert.NoError(t, err) - unittest.RequireNumericallyClose(t, expectedPenalty, record.Penalty, 0.01) - assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. - queryTime := time.Now() - // expected penalty should include the expected duplicate messages penalty - expectedPenalty = expectedPenalty + expectedDuplicateMessagesPenalty - // eventually, the app specific score should be updated in the cache. require.Eventually(t, func() bool { - // calling the app specific score function when there is no app specific score in the cache should eventually update the cache. - score := reg.AppSpecificScoreFunc()(peerID) - return unittest.AreNumericallyClose(expectedPenalty, score, 10e-4) + // the notification is processed asynchronously, and the penalty should eventually be updated in the spamRecords + record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. + if !ok { + return false + } + require.NoError(t, err) + if !unittest.AreNumericallyClose(expectedPenalty, record.Penalty, 10e-2) { + return false + } + require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + + // eventually, the app specific score should be updated in the cache. + // As the app specific score in the cache and spam penalty in the spamRecords are updated at different times, we account for 5% error. + return unittest.AreNumericallyClose(expectedPenalty+expectedDuplicateMessagesPenalty, reg.AppSpecificScoreFunc()(peerID), 0.05) }, 5*time.Second, 100*time.Millisecond) // the app specific score should now be updated in the cache. score, updated, exists = appScoreCache.Get(peerID) // get the score from the cache. require.True(t, exists) require.True(t, updated.After(queryTime)) - unittest.RequireNumericallyClose(t, expectedPenalty, score, 10e-3) + unittest.RequireNumericallyClose(t, expectedPenalty+expectedDuplicateMessagesPenalty, score, 0.1) // account for maximum 10% error due to decays and asynchrony. // stop the registry. cancel() @@ -978,15 +979,22 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { // for a spam record should be reduced to the MinimumSpamPenaltyDecayFactor prevDecay := scoringRegistryParameters.SpamRecordCache.Decay.MaximumSpamPenaltyDecayFactor tolerance := 0.1 + require.Eventually(t, func() bool { reg.OnInvalidControlMessageNotification(&p2p.InvCtrlMsgNotif{ PeerID: peer1, MsgType: p2pmsg.CtrlMsgPrune, }) + + // the spam penalty should eventually updated in the spamRecords record, err, ok := spamRecords.Get(peer1) require.NoError(t, err) - require.True(t, ok) - assert.Less(t, math.Abs(prevDecay-record.Decay), tolerance) + if !ok { + return false + } + if math.Abs(prevDecay-record.Decay) > tolerance { + return false + } prevDecay = record.Decay return record.Decay == scoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor }, 5*time.Second, 500*time.Millisecond) @@ -996,6 +1004,14 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { PeerID: peer2, MsgType: p2pmsg.CtrlMsgPrune, }) + + // eventually the spam record should appear in the cache + require.Eventually(t, func() bool { + _, err, ok := spamRecords.Get(peer2) + require.NoError(t, err) + return ok + }, 5*time.Second, 10*time.Millisecond) + // reduce penalty and increase Decay to scoring.MinimumSpamPenaltyDecayFactor record, err := spamRecords.Adjust(peer2, func(record p2p.GossipSubSpamRecord) p2p.GossipSubSpamRecord { record.Penalty = -.1 @@ -1024,9 +1040,12 @@ func TestScoreRegistry_TestSpamRecordDecayAdjustment(t *testing.T) { PeerID: peer2, MsgType: p2pmsg.CtrlMsgPrune, }) + // the spam penalty should eventually updated in the spamRecords record, err, ok := spamRecords.Get(peer1) require.NoError(t, err) - require.True(t, ok) + if !ok { + return false + } return record.Decay == scoringRegistryParameters.SpamRecordCache.Decay.MinimumSpamPenaltyDecayFactor }, 5*time.Second, 500*time.Millisecond) @@ -1105,12 +1124,20 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { // expected penalty should be penaltyValueFixtures().GraftMisbehaviour * (1 + clusterReductionFactor) expectedPenalty := penaltyValueFixture(ctlMsgType) * (1 + penaltyValueFixtures().ClusterPrefixedReductionFactor) - // the penalty should now be updated in the spamRecords - record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. - assert.True(t, ok) - assert.NoError(t, err) - unittest.RequireNumericallyClose(t, expectedPenalty, record.Penalty, 0.02) - assert.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) + require.Eventually(t, func() bool { + // the notification is processed asynchronously, and the penalty should eventually be updated in the spamRecords + record, err, ok := spamRecords.Get(peerID) // get the record from the spamRecords. + if !ok { + return false + } + require.NoError(t, err) + if !unittest.AreNumericallyClose(expectedPenalty, record.Penalty, 10e-2) { + return false + } + require.Equal(t, scoring.InitAppScoreRecordStateFunc(maximumSpamPenaltyDecayFactor)().Decay, record.Decay) // decay should be initialized to the initial state. + return true + }, 5*time.Second, 100*time.Millisecond) + // this peer has a spam record, with no subscription penalty. Hence, the app specific score should only be the spam penalty, // and the peer should be deprived of the default reward for its valid staked role. score := reg.AppSpecificScoreFunc()(peerID) @@ -1130,6 +1157,7 @@ func TestPeerSpamPenaltyClusterPrefixed(t *testing.T) { // TestScoringRegistrySilencePeriod ensures that the scoring registry does not penalize nodes during the silence period, and // starts to penalize nodes only after the silence period is over. func TestScoringRegistrySilencePeriod(t *testing.T) { + unittest.SkipUnless(t, unittest.TEST_TODO, "requires notification be unique (e.g., nonce or timestamp) to avoid duplicate notifications being ignored.") peerID := unittest.PeerIdFixture(t) silenceDuration := 5 * time.Second silencedNotificationLogs := atomic.NewInt32(0) From ee2ca4c433719f66ac28abba05ac7facc0ff12ba Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Fri, 16 Feb 2024 08:54:28 -0600 Subject: [PATCH 37/52] Add suggested comments from feedback Co-authored-by: Leo Zhang --- cmd/util/cmd/execution-state-extract/cmd.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/cmd/util/cmd/execution-state-extract/cmd.go b/cmd/util/cmd/execution-state-extract/cmd.go index 760c7ff0975..5cfeea0312f 100644 --- a/cmd/util/cmd/execution-state-extract/cmd.go +++ b/cmd/util/cmd/execution-state-extract/cmd.go @@ -76,6 +76,12 @@ func init() { Cmd.Flags().BoolVar(&flagLogVerboseValidationError, "log-verbose-validation-error", false, "log entire Cadence values on validation error (atree migration)") + // If specified, the state will consist of payloads from the given input payload file. + // If not specified, then the state will be extracted from the latest checkpoint file. + // This flag can be used to reduce total duration of migrations when state extraction involves + // multiple migrations because it helps avoid repeatedly reading from checkpoint file to rebuild trie. + // The input payload file must be created by state extraction running with either + // flagOutputPayloadFileName or flagOutputPayloadByAddresses. Cmd.Flags().StringVar( &flagInputPayloadFileName, "input-payload-filename", @@ -117,6 +123,7 @@ func run(*cobra.Command, []string) { log.Fatal().Msg("--input-payload-filename cannot be used with --block-hash or --state-commitment") } + // When flagOutputPayloadByAddresses is specified, flagOutputPayloadFileName is required. if len(flagOutputPayloadFileName) == 0 && len(flagOutputPayloadByAddresses) > 0 { log.Fatal().Msg("--extract-payloads-by-address requires --output-payload-filename to be specified") } @@ -242,7 +249,7 @@ func run(*cobra.Command, []string) { ) } - log.Info().Msgf("%s, %s", inputMsg, outputMsg) + log.Info().Msgf("state extraction plan: %s, %s", inputMsg, outputMsg) var err error if len(flagInputPayloadFileName) > 0 { From 182bcf6f9a46619a9a590296863556645be18314 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 16 Feb 2024 10:01:17 -0800 Subject: [PATCH 38/52] adds documentations to the gossipsub builder --- .../p2p/builder/gossipsub/gossipSubBuilder.go | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/network/p2p/builder/gossipsub/gossipSubBuilder.go b/network/p2p/builder/gossipsub/gossipSubBuilder.go index faed8090b43..da3b74943b5 100644 --- a/network/p2p/builder/gossipsub/gossipSubBuilder.go +++ b/network/p2p/builder/gossipsub/gossipSubBuilder.go @@ -57,8 +57,16 @@ func (g *Builder) SetHost(h host.Host) { g.h = h } +// OverrideDefaultRpcInspectorFactory overrides the default rpc inspector factory of the builder. +// If the rpc inspector factory has already been set, a warning is logged. +// Note: it is not recommended to override the default rpc inspector factory in production unless you know what you are doing. +// The purpose of this function is to allow for testing and development. +// Args: +// - factoryFunc: the factory function to override the default rpc inspector factory. +// Returns: +// none func (g *Builder) OverrideDefaultRpcInspectorFactory(factoryFunc p2p.GossipSubRpcInspectorFactoryFunc) { - g.logger.Warn().Msg("overriding default rpc inspector factory, not recommended for production") + g.logger.Warn().Bool(logging.KeySuspicious, true).Msg("overriding default rpc inspector factory, not recommended for production") g.rpcInspectorFactory = factoryFunc } @@ -194,6 +202,12 @@ func NewGossipSubBuilder(logger zerolog.Logger, return b } +// defaultRpcInspectorFactory returns the default rpc inspector factory function. It is used to create the default rpc inspector factory. +// Note: always use the default rpc inspector factory function to create the rpc inspector factory (unless you know what you are doing). +// Args: +// - tracer: the tracer of the node. +// Returns: +// - a new rpc inspector factory function. func defaultRpcInspectorFactory(tracer p2p.PubSubTracer) p2p.GossipSubRpcInspectorFactoryFunc { return func(logger zerolog.Logger, sporkId flow.Identifier, @@ -263,8 +277,16 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e gossipSubConfigs.WithSubscriptionFilter(g.subscriptionFilter) } + // scoreOpt is the score option for the GossipSub pubsub system. It is a self-contained component that is used carry over the + // peer scoring parameters (including the entire app-specific score function) and inject it into the GossipSub pubsub system at creation time. var scoreOpt *scoring.ScoreOption + // scoreTracer is the peer score tracer for the GossipSub pubsub system. It is used to trace the peer scores. + // It is only created if peer scoring is enabled. Otherwise, it is nil. var scoreTracer p2p.PeerScoreTracer + // consumer is the consumer of the invalid control message notifications; i.e., the component that should be nlotified when + // an RPC validation fails. This component is responsible for taking action on the notification. Currently, the score option + // is the consumer of the invalid control message notifications. + // When the peer scoring is disabled, the consumer is a no-op consumer. var consumer p2p.GossipSubInvCtrlMsgNotifConsumer // currently, peer scoring is not supported for public networks. if g.gossipSubCfg.PeerScoringEnabled && g.networkType != network.PublicNetwork { @@ -299,7 +321,7 @@ func (g *Builder) Build(ctx irrecoverable.SignalerContext) (p2p.PubSubAdapter, e } else { g.logger.Warn(). Str(logging.KeyNetworkingSecurity, "true"). - Msg("gossipsub peer scoring is disabled") + Msg("gossipsub peer scoring is disabled, no-op consumer will be used for invalid control message notifications.") consumer = scoring.NewNoopInvCtrlMsgNotifConsumer() // no-op consumer as peer scoring is disabled. } From 52678304bf165c0ec39593c727fc76679a7ae5e3 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 16 Feb 2024 10:06:05 -0800 Subject: [PATCH 39/52] adds documentation and renames a method --- insecure/corruptlibp2p/libp2p_node_factory.go | 2 +- network/p2p/builder.go | 25 ++++++++++++++- network/p2p/builder/libp2pNodeBuilder.go | 16 +++++++++- network/p2p/mock/node_builder.go | 32 +++++++++---------- network/p2p/test/fixtures.go | 2 +- 5 files changed, 57 insertions(+), 20 deletions(-) diff --git a/insecure/corruptlibp2p/libp2p_node_factory.go b/insecure/corruptlibp2p/libp2p_node_factory.go index 5bc70a50f0c..79002c1c6ed 100644 --- a/insecure/corruptlibp2p/libp2p_node_factory.go +++ b/insecure/corruptlibp2p/libp2p_node_factory.go @@ -164,5 +164,5 @@ func CorruptGossipSubConfigFactoryWithInspector(inspector func(peer.ID, *corrupt func overrideWithCorruptGossipSub(builder p2p.NodeBuilder, opts ...CorruptPubSubAdapterConfigOption) { factory := CorruptGossipSubFactory() - builder.SetGossipSubFactory(factory, CorruptGossipSubConfigFactory(opts...)) + builder.OverrideGossipSubFactory(factory, CorruptGossipSubConfigFactory(opts...)) } diff --git a/network/p2p/builder.go b/network/p2p/builder.go index 9140f83ce59..207235f7347 100644 --- a/network/p2p/builder.go +++ b/network/p2p/builder.go @@ -141,8 +141,31 @@ type NodeBuilder interface { // Returns: // none OverrideNodeConstructor(NodeConstructor) NodeBuilder - SetGossipSubFactory(GossipSubFactoryFunc, GossipSubAdapterConfigFunc) NodeBuilder + + // OverrideGossipSubFactory overrides the default gossipsub factory for the GossipSub protocol. + // The purpose of override is to allow the node to provide a custom gossipsub factory for sake of testing or experimentation. + // Note: it is not recommended to override the default gossipsub factory in production unless you know what you are doing. + // Args: + // - factory: custom gossipsub factory + // Returns: + // - NodeBuilder: the node builder + OverrideGossipSubFactory(GossipSubFactoryFunc, GossipSubAdapterConfigFunc) NodeBuilder + + // OverrideDefaultRpcInspectorFactory overrides the default rpc inspector factory for the GossipSub protocol. + // The purpose of override is to allow the node to provide a custom rpc inspector factory for sake of testing or experimentation. + // Note: it is not recommended to override the default rpc inspector factory in production unless you know what you are doing. + // Args: + // - factory: custom rpc inspector factory + // Returns: + // - NodeBuilder: the node builder OverrideDefaultRpcInspectorFactory(GossipSubRpcInspectorFactoryFunc) NodeBuilder + + // Build creates a new libp2p node. It returns the newly created libp2p node and any errors encountered during its creation. + // Args: + // none + // Returns: + // - LibP2PNode: a new libp2p node + // - error: if an error occurs during the creation of the libp2p node, it is returned. Otherwise, nil is returned. Any error returned is unexpected and should be handled as irrecoverable. Build() (LibP2PNode, error) } diff --git a/network/p2p/builder/libp2pNodeBuilder.go b/network/p2p/builder/libp2pNodeBuilder.go index a725cf962a3..96254160e2c 100644 --- a/network/p2p/builder/libp2pNodeBuilder.go +++ b/network/p2p/builder/libp2pNodeBuilder.go @@ -146,7 +146,14 @@ func (builder *LibP2PNodeBuilder) SetRoutingSystem(f func(context.Context, host. return builder } -func (builder *LibP2PNodeBuilder) SetGossipSubFactory(gf p2p.GossipSubFactoryFunc, cf p2p.GossipSubAdapterConfigFunc) p2p.NodeBuilder { +// OverrideGossipSubFactory overrides the default gossipsub factory for the GossipSub protocol. +// The purpose of override is to allow the node to provide a custom gossipsub factory for sake of testing or experimentation. +// Note: it is not recommended to override the default gossipsub factory in production unless you know what you are doing. +// Args: +// - factory: custom gossipsub factory +// Returns: +// - NodeBuilder: the node builder +func (builder *LibP2PNodeBuilder) OverrideGossipSubFactory(gf p2p.GossipSubFactoryFunc, cf p2p.GossipSubAdapterConfigFunc) p2p.NodeBuilder { builder.gossipSubBuilder.SetGossipSubFactory(gf) builder.gossipSubBuilder.SetGossipSubConfigFunc(cf) return builder @@ -180,6 +187,13 @@ func (builder *LibP2PNodeBuilder) OverrideNodeConstructor(f p2p.NodeConstructor) return builder } +// OverrideDefaultRpcInspectorFactory overrides the default rpc inspector factory for the GossipSub protocol. +// The purpose of override is to allow the node to provide a custom rpc inspector factory for sake of testing or experimentation. +// Note: it is not recommended to override the default rpc inspector factory in production unless you know what you are doing. +// Args: +// - factory: custom rpc inspector factory +// Returns: +// - NodeBuilder: the node builder func (builder *LibP2PNodeBuilder) OverrideDefaultRpcInspectorFactory(factory p2p.GossipSubRpcInspectorFactoryFunc) p2p.NodeBuilder { builder.gossipSubBuilder.OverrideDefaultRpcInspectorFactory(factory) return builder diff --git a/network/p2p/mock/node_builder.go b/network/p2p/mock/node_builder.go index acadd5e34a2..5229be19fbc 100644 --- a/network/p2p/mock/node_builder.go +++ b/network/p2p/mock/node_builder.go @@ -69,6 +69,22 @@ func (_m *NodeBuilder) OverrideDefaultRpcInspectorFactory(_a0 p2p.GossipSubRpcIn return r0 } +// OverrideGossipSubFactory provides a mock function with given fields: _a0, _a1 +func (_m *NodeBuilder) OverrideGossipSubFactory(_a0 p2p.GossipSubFactoryFunc, _a1 p2p.GossipSubAdapterConfigFunc) p2p.NodeBuilder { + ret := _m.Called(_a0, _a1) + + var r0 p2p.NodeBuilder + if rf, ok := ret.Get(0).(func(p2p.GossipSubFactoryFunc, p2p.GossipSubAdapterConfigFunc) p2p.NodeBuilder); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(p2p.NodeBuilder) + } + } + + return r0 +} + // OverrideGossipSubScoringConfig provides a mock function with given fields: _a0 func (_m *NodeBuilder) OverrideGossipSubScoringConfig(_a0 *p2p.PeerScoringConfigOverride) p2p.NodeBuilder { ret := _m.Called(_a0) @@ -149,22 +165,6 @@ func (_m *NodeBuilder) SetConnectionManager(_a0 connmgr.ConnManager) p2p.NodeBui return r0 } -// SetGossipSubFactory provides a mock function with given fields: _a0, _a1 -func (_m *NodeBuilder) SetGossipSubFactory(_a0 p2p.GossipSubFactoryFunc, _a1 p2p.GossipSubAdapterConfigFunc) p2p.NodeBuilder { - ret := _m.Called(_a0, _a1) - - var r0 p2p.NodeBuilder - if rf, ok := ret.Get(0).(func(p2p.GossipSubFactoryFunc, p2p.GossipSubAdapterConfigFunc) p2p.NodeBuilder); ok { - r0 = rf(_a0, _a1) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(p2p.NodeBuilder) - } - } - - return r0 -} - // SetResourceManager provides a mock function with given fields: _a0 func (_m *NodeBuilder) SetResourceManager(_a0 network.ResourceManager) p2p.NodeBuilder { ret := _m.Called(_a0) diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 05b8e1bc952..67c90ff0ad4 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -178,7 +178,7 @@ func NodeFixture(t *testing.T, } if parameters.GossipSubFactory != nil && parameters.GossipSubConfig != nil { - builder.SetGossipSubFactory(parameters.GossipSubFactory, parameters.GossipSubConfig) + builder.OverrideGossipSubFactory(parameters.GossipSubFactory, parameters.GossipSubConfig) } if parameters.ConnManager != nil { From 03fb25599879b71826e0990e293ca2242ef91717 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 16 Feb 2024 10:09:13 -0800 Subject: [PATCH 40/52] adds a godoc --- network/p2p/consumers.go | 1 + network/p2p/scoring/registry.go | 1 + network/p2p/scoring/score_option.go | 1 + 3 files changed, 3 insertions(+) diff --git a/network/p2p/consumers.go b/network/p2p/consumers.go index 25ccf2678f5..21e4bd4dfe3 100644 --- a/network/p2p/consumers.go +++ b/network/p2p/consumers.go @@ -73,5 +73,6 @@ type GossipSubInvCtrlMsgNotifConsumer interface { // OnInvalidControlMessageNotification is called when a new invalid control message notification is distributed. // Any error on consuming event must handle internally. // The implementation must be concurrency safe and non-blocking. + // Note: there is no real-time guarantee on processing the notification. OnInvalidControlMessageNotification(*InvCtrlMsgNotif) } diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index 989aed92425..cdd5f897c02 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -408,6 +408,7 @@ func (r *GossipSubAppSpecificScoreRegistry) duplicateMessagesPenalty(pid peer.ID // OnInvalidControlMessageNotification is called when a new invalid control message notification is distributed. // Any error on consuming event must handle internally. // The implementation must be concurrency safe, but can be blocking. +// Note: there is no real-time guarantee on processing the notification. func (r *GossipSubAppSpecificScoreRegistry) OnInvalidControlMessageNotification(notification *p2p.InvCtrlMsgNotif) { lg := r.logger.With().Str("peer_id", p2plogging.PeerId(notification.PeerID)).Logger() if ok := r.invCtrlMsgNotifWorkerPool.Submit(notification); !ok { diff --git a/network/p2p/scoring/score_option.go b/network/p2p/scoring/score_option.go index a6095951651..3136478176b 100644 --- a/network/p2p/scoring/score_option.go +++ b/network/p2p/scoring/score_option.go @@ -269,6 +269,7 @@ func (s *ScoreOption) TopicScoreParams(topic *pubsub.Topic) *pubsub.TopicScorePa // OnInvalidControlMessageNotification is called when a new invalid control message notification is distributed. // Any error on consuming event must handle internally. // The implementation must be concurrency safe and non-blocking. +// Note: there is no real-time guarantee on processing the notification. func (s *ScoreOption) OnInvalidControlMessageNotification(notif *p2p.InvCtrlMsgNotif) { s.appScoreRegistry.OnInvalidControlMessageNotification(notif) } From f7d5549aae15f242331d67ebfba3dbf2525c3f01 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 16 Feb 2024 10:14:02 -0800 Subject: [PATCH 41/52] removes an unused function --- network/p2p/test/fixtures.go | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/network/p2p/test/fixtures.go b/network/p2p/test/fixtures.go index 67c90ff0ad4..550906e2724 100644 --- a/network/p2p/test/fixtures.go +++ b/network/p2p/test/fixtures.go @@ -19,7 +19,6 @@ import ( discoveryBackoff "github.com/libp2p/go-libp2p/p2p/discovery/backoff" "github.com/onflow/crypto" "github.com/rs/zerolog" - mockery "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" @@ -37,7 +36,6 @@ import ( p2pbuilderconfig "github.com/onflow/flow-go/network/p2p/builder/config" "github.com/onflow/flow-go/network/p2p/connection" p2pdht "github.com/onflow/flow-go/network/p2p/dht" - mockp2p "github.com/onflow/flow-go/network/p2p/mock" "github.com/onflow/flow-go/network/p2p/unicast/protocols" "github.com/onflow/flow-go/network/p2p/utils" validator "github.com/onflow/flow-go/network/validator/pubsub" @@ -809,22 +807,6 @@ func NewConnectionGater(idProvider module.IdentityProvider, allowListFilter p2p. return connection.NewConnGater(unittest.Logger(), idProvider, connection.WithOnInterceptPeerDialFilters(filters), connection.WithOnInterceptSecuredFilters(filters)) } -// MockScoringRegistrySubscriptionValidatorReadyDoneAware mocks the Ready and Done methods of the subscription validator to return a channel that is already closed, -// so that the distributor is considered ready and done when the test needs. -func MockScoringRegistrySubscriptionValidatorReadyDoneAware(s *mockp2p.SubscriptionValidator) { - s.On("Start", mockery.Anything).Return().Maybe() - s.On("Ready").Return(func() <-chan struct{} { - ch := make(chan struct{}) - close(ch) - return ch - }()).Maybe() - s.On("Done").Return(func() <-chan struct{} { - ch := make(chan struct{}) - close(ch) - return ch - }()).Maybe() -} - // GossipSubRpcFixtures returns a slice of random message IDs for testing. // Args: // - t: *testing.T instance From 149f309017b0fcbfaf8669b62728d6d38eae8634 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Fri, 16 Feb 2024 10:26:45 -0800 Subject: [PATCH 42/52] adds config for size of hero store for notification queue --- config/default-config.yml | 4 ++- network/netconf/flags.go | 48 +++++++++++++++++++++++++--- network/p2p/config/score_registry.go | 10 ++++-- network/p2p/scoring/registry.go | 10 +++--- 4 files changed, 58 insertions(+), 14 deletions(-) diff --git a/config/default-config.yml b/config/default-config.yml index c4d162331e9..268eaebd2d6 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -538,12 +538,14 @@ network-config: # number of workers that asynchronously update the app specific score requests when they are expired. score-update-worker-num: 5 # size of the queue used by the worker pool for the app specific score update requests. The queue is used to buffer the app specific score update requests - # before they are processed by the worker pool. The queue size must be larger than total number of peers in the network. + # before they are processed by the worker pool. The queue size must be larger than 10x total number of peers in the network. # The queue is deduplicated based on the peer ids ensuring that there is only one app specific score update request per peer in the queue. score-update-request-queue-size: 10_000 # score ttl is the time to live for the app specific score. Once the score is expired; a new request will be sent to the app specific score provider to update the score. # until the score is updated, the previous score will be used. score-ttl: 1m + # size of the queue used by the score registry to buffer the invalid control message notifications before they are processed by the worker pool. The queue size must be larger than 10x total number of peers in the network. + invalid-control-message-notification-queue-size: 10_000 spam-record-cache: # size of cache used to track spam records at gossipsub. Each peer id is mapped to a spam record that keeps track of the spam score for that peer. # cache should be big enough to keep track of the entire network's size. Otherwise, the local node's view of the network will be incomplete due to cache eviction. diff --git a/network/netconf/flags.go b/network/netconf/flags.go index d6e58a4b340..05412d01813 100644 --- a/network/netconf/flags.go +++ b/network/netconf/flags.go @@ -108,9 +108,23 @@ func AllFlagNames() []string { BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.GraftKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.PruneKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IHaveKey), - BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IHaveKey, p2pconfig.MessageIDKey), + BuildFlagName(gossipsubKey, + p2pconfig.RpcInspectorKey, + p2pconfig.ValidationConfigKey, + p2pconfig.ProcessKey, + p2pconfig.TruncationKey, + p2pconfig.EnableKey, + p2pconfig.IHaveKey, + p2pconfig.MessageIDKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IWantConfigKey), - BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IWantKey, p2pconfig.MessageIDKey), + BuildFlagName(gossipsubKey, + p2pconfig.RpcInspectorKey, + p2pconfig.ValidationConfigKey, + p2pconfig.ProcessKey, + p2pconfig.TruncationKey, + p2pconfig.EnableKey, + p2pconfig.IWantKey, + p2pconfig.MessageIDKey), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.MessageCountThreshold), BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.IHaveConfigKey, p2pconfig.MessageIdCountThreshold), @@ -166,6 +180,7 @@ func AllFlagNames() []string { BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.StartupSilenceDurationKey), BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateWorkerNumKey), BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateRequestQueueSizeKey), + BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.InvalidControlMessageNotificationQueueSizeKey), BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreTTLKey), BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.SpamRecordCacheKey, p2pconfig.CacheSizeKey), @@ -332,13 +347,27 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Bool(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IHaveKey), config.GossipSub.RpcInspector.Validation.InspectionProcess.Truncate.EnableIHave, "disable ihave control message truncation") - flags.Bool(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IHaveKey, p2pconfig.MessageIDKey), + flags.Bool(BuildFlagName(gossipsubKey, + p2pconfig.RpcInspectorKey, + p2pconfig.ValidationConfigKey, + p2pconfig.ProcessKey, + p2pconfig.TruncationKey, + p2pconfig.EnableKey, + p2pconfig.IHaveKey, + p2pconfig.MessageIDKey), config.GossipSub.RpcInspector.Validation.InspectionProcess.Truncate.EnableIHaveMessageIds, "disable ihave message id truncation") flags.Bool(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IWantKey), config.GossipSub.RpcInspector.Validation.InspectionProcess.Truncate.EnableIWant, "disable iwant control message truncation") - flags.Bool(BuildFlagName(gossipsubKey, p2pconfig.RpcInspectorKey, p2pconfig.ValidationConfigKey, p2pconfig.ProcessKey, p2pconfig.TruncationKey, p2pconfig.EnableKey, p2pconfig.IWantKey, p2pconfig.MessageIDKey), + flags.Bool(BuildFlagName(gossipsubKey, + p2pconfig.RpcInspectorKey, + p2pconfig.ValidationConfigKey, + p2pconfig.ProcessKey, + p2pconfig.TruncationKey, + p2pconfig.EnableKey, + p2pconfig.IWantKey, + p2pconfig.MessageIDKey), config.GossipSub.RpcInspector.Validation.InspectionProcess.Truncate.EnableIWantMessageIds, "disable iwant message id truncation") @@ -486,7 +515,13 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.StakedIdentityKey, p2pconfig.RewardKey), config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.StakedIdentityReward, "the reward for staking peers") - flags.Float64(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.PeerScoringKey, p2pconfig.ProtocolKey, p2pconfig.AppSpecificKey, p2pconfig.DuplicateMessageKey, p2pconfig.ThresholdKey), + flags.Float64(BuildFlagName(gossipsubKey, + p2pconfig.ScoreParamsKey, + p2pconfig.PeerScoringKey, + p2pconfig.ProtocolKey, + p2pconfig.AppSpecificKey, + p2pconfig.DuplicateMessageKey, + p2pconfig.ThresholdKey), config.GossipSub.ScoringParameters.PeerScoring.Protocol.AppSpecificScore.DuplicateMessageThreshold, "the peer's duplicate message count threshold above which the peer will be penalized") @@ -499,6 +534,9 @@ func InitializeNetworkFlags(flags *pflag.FlagSet, config *Config) { flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreUpdateRequestQueueSizeKey), config.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreUpdateRequestQueueSize, "size of the app specific score update worker pool queue") + flags.Uint32(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.InvalidControlMessageNotificationQueueSizeKey), + config.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.InvalidControlMessageNotificationQueueSize, + "size of the queue for invalid control message notifications processing") flags.Duration(BuildFlagName(gossipsubKey, p2pconfig.ScoreParamsKey, p2pconfig.ScoringRegistryKey, p2pconfig.AppSpecificScoreRegistryKey, p2pconfig.ScoreTTLKey), config.GossipSub.ScoringParameters.ScoringRegistryParameters.AppSpecificScore.ScoreTTL, "time to live for app specific scores; when expired a new request will be sent to the score update worker pool; till then the expired score will be used") diff --git a/network/p2p/config/score_registry.go b/network/p2p/config/score_registry.go index aa4f8596d24..ef35dc8bf77 100644 --- a/network/p2p/config/score_registry.go +++ b/network/p2p/config/score_registry.go @@ -25,9 +25,10 @@ type ScoringRegistryParameters struct { } const ( - ScoreUpdateWorkerNumKey = "score-update-worker-num" - ScoreUpdateRequestQueueSizeKey = "score-update-request-queue-size" - ScoreTTLKey = "score-ttl" + ScoreUpdateWorkerNumKey = "score-update-worker-num" + ScoreUpdateRequestQueueSizeKey = "score-update-request-queue-size" + ScoreTTLKey = "score-ttl" + InvalidControlMessageNotificationQueueSizeKey = "invalid-control-message-notification-queue-size" ) // AppSpecificScoreParameters is the parameters for the GossipSubAppSpecificScoreRegistry. @@ -39,6 +40,9 @@ type AppSpecificScoreParameters struct { // ScoreUpdateRequestQueueSize is the size of the worker pool for handling the application specific score update of peers in a non-blocking way. ScoreUpdateRequestQueueSize uint32 `validate:"gt=0" mapstructure:"score-update-request-queue-size"` + // InvalidControlMessageNotificationQueueSize is the size of the queue for handling invalid control message notifications in a non-blocking way. + InvalidControlMessageNotificationQueueSize uint32 `validate:"gt=0" mapstructure:"invalid-control-message-notification-queue-size"` + // ScoreTTL is the time to live of the application specific score of a peer; the registry keeps a cached copy of the // application specific score of a peer for this duration. When the duration expires, the application specific score // of the peer is updated asynchronously. As long as the update is in progress, the cached copy of the application diff --git a/network/p2p/scoring/registry.go b/network/p2p/scoring/registry.go index cdd5f897c02..73cea927c69 100644 --- a/network/p2p/scoring/registry.go +++ b/network/p2p/scoring/registry.go @@ -157,16 +157,16 @@ func NewGossipSubAppSpecificScoreRegistry(config *GossipSubAppSpecificScoreRegis collector: config.Collector, } - appSpecificScoreHS := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, + appSpecificScore := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, lg.With().Str("component", "app_specific_score_update").Logger(), metrics.GossipSubAppSpecificScoreUpdateQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) - reg.appScoreUpdateWorkerPool = worker.NewWorkerPoolBuilder[peer.ID](lg.With().Str("component", "app_specific_score_update_worker_pool").Logger(), - appSpecificScoreHS, + reg.appScoreUpdateWorkerPool = worker.NewWorkerPoolBuilder[peer.ID](lg.With().Str("component", "app_specific_score_update_worker_pool").Logger(), appSpecificScore, reg.processAppSpecificScoreUpdateWork).Build() - invalidCtrlMsgNotifHS := queue.NewHeroStore(config.Parameters.ScoreUpdateRequestQueueSize, lg.With().Str("component", "invalid_control_message_notification_queue").Logger(), + invalidCtrlMsgNotificationStore := queue.NewHeroStore(config.Parameters.InvalidControlMessageNotificationQueueSize, + lg.With().Str("component", "invalid_control_message_notification_queue").Logger(), metrics.RpcInspectorNotificationQueueMetricFactory(config.HeroCacheMetricsFactory, config.NetworkingType)) - reg.invCtrlMsgNotifWorkerPool = worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, invalidCtrlMsgNotifHS, reg.handleMisbehaviourReport).Build() + reg.invCtrlMsgNotifWorkerPool = worker.NewWorkerPoolBuilder[*p2p.InvCtrlMsgNotif](lg, invalidCtrlMsgNotificationStore, reg.handleMisbehaviourReport).Build() builder := component.NewComponentManagerBuilder() builder.AddWorker(func(ctx irrecoverable.SignalerContext, ready component.ReadyFunc) { From c03eb77fa6e30a1da4d579705fceae46b0213e41 Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Wed, 21 Feb 2024 09:36:53 -0500 Subject: [PATCH 43/52] Update config/default-config.yml Co-authored-by: Yahya Hassanzadeh, Ph.D. --- config/default-config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/default-config.yml b/config/default-config.yml index 54e96c35d52..ad9388d1292 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -160,7 +160,7 @@ network-config: # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. # A topic id is considered duplicate if it appears more than once in a single GRAFT or PRUNE message. duplicate-topic-id-threshold: 50 - # Maximum number of total invalid topic ids in a single GRAFT or PRUNE message, ideally this should be 0 but we allow for some tolerance + # Maximum number of total invalid topic ids in GRAFTs/PRUNEs of a single RPC, ideally this should be 0 but we allow for some tolerance # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. invalid-topic-id-threshold: 50 ihave: From cc41ebc13428d734b6a0c6ab8d07c3657d5fbbad Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Wed, 21 Feb 2024 09:36:59 -0500 Subject: [PATCH 44/52] Update config/default-config.yml Co-authored-by: Yahya Hassanzadeh, Ph.D. --- config/default-config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/default-config.yml b/config/default-config.yml index ad9388d1292..ff549f4ff72 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -161,7 +161,7 @@ network-config: # A topic id is considered duplicate if it appears more than once in a single GRAFT or PRUNE message. duplicate-topic-id-threshold: 50 # Maximum number of total invalid topic ids in GRAFTs/PRUNEs of a single RPC, ideally this should be 0 but we allow for some tolerance - # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. + # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. Exceeding this threshold causes RPC inspection failure with an invalid control message notification (penalty). invalid-topic-id-threshold: 50 ihave: # The maximum allowed number of iHave messages in a single RPC message. From d1564abfebb439e228eaf7f266e9bad7123874bc Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Wed, 21 Feb 2024 09:37:05 -0500 Subject: [PATCH 45/52] Update config/default-config.yml Co-authored-by: Yahya Hassanzadeh, Ph.D. --- config/default-config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/default-config.yml b/config/default-config.yml index ff549f4ff72..acb009836d1 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -184,7 +184,7 @@ network-config: # Ideally, an iHave message should not have any duplicate message IDs, hence a message id is considered duplicate when it is repeated more than once # within the same iHave message. When the total number of duplicate message ids in a single iHave message exceeds this threshold, the inspection of message will fail. duplicate-message-id-threshold: 100 - # Maximum number of total invalid topic ids in a single IHAVE message, ideally this should be 0 but we allow for some tolerance + # Maximum number of total invalid topic ids in an IHAVE message on a single RPC, ideally this should be 0 but we allow for some tolerance # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. invalid-topic-id-threshold: 50 iwant: From 0ada021f023eef58cf2159dbd5c1e5e1c986d4d7 Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Wed, 21 Feb 2024 09:37:11 -0500 Subject: [PATCH 46/52] Update config/default-config.yml Co-authored-by: Yahya Hassanzadeh, Ph.D. --- config/default-config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/default-config.yml b/config/default-config.yml index acb009836d1..e7108afb20d 100644 --- a/config/default-config.yml +++ b/config/default-config.yml @@ -185,7 +185,7 @@ network-config: # within the same iHave message. When the total number of duplicate message ids in a single iHave message exceeds this threshold, the inspection of message will fail. duplicate-message-id-threshold: 100 # Maximum number of total invalid topic ids in an IHAVE message on a single RPC, ideally this should be 0 but we allow for some tolerance - # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. + # to avoid penalizing peers that are not malicious but are misbehaving due to bugs or other issues. Exceeding this threshold causes RPC inspection failure with an invalid control message notification (penalty). invalid-topic-id-threshold: 50 iwant: # The maximum allowed number of iWant messages in a single RPC message. From a6cd7ba676f405adf8e9b0354b31aee06e960391 Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Wed, 21 Feb 2024 10:38:26 -0500 Subject: [PATCH 47/52] add *ThresholdExceeded errors for invalid topic ID and duplicate topic ID --- .../validation_inspector_test.go | 2 +- .../control_message_validation_inspector.go | 12 +++--- ...ntrol_message_validation_inspector_test.go | 14 +++--- network/p2p/inspector/validation/errors.go | 43 +++++++++++++++++++ .../p2p/inspector/validation/errors_test.go | 24 +++++++++++ 5 files changed, 81 insertions(+), 14 deletions(-) diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go index e8cf9bb1566..e4f4b7e61d6 100644 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go +++ b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go @@ -199,7 +199,7 @@ func TestValidationInspector_DuplicateTopicId_Detection(t *testing.T) { notification, ok := args[0].(*p2p.InvCtrlMsgNotif) require.True(t, ok) require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") - require.True(t, validation.IsDuplicateTopicErr(notification.Error)) + require.True(t, validation.IsDuplicateTopicIDThresholdExceeded(notification.Error)) require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) switch notification.MsgType { case p2pmsg.CtrlMsgGraft: diff --git a/network/p2p/inspector/validation/control_message_validation_inspector.go b/network/p2p/inspector/validation/control_message_validation_inspector.go index 308869731af..09e94143004 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector.go @@ -380,7 +380,7 @@ func (c *ControlMsgValidationInspector) inspectGraftMessages(from peer.ID, graft // check if the total number of duplicates exceeds the configured threshold. if totalDuplicateTopicIds > c.config.GraftPrune.DuplicateTopicIdThreshold { c.metrics.OnGraftDuplicateTopicIdsExceedThreshold() - return NewDuplicateTopicErr(topic.String(), totalDuplicateTopicIds, p2pmsg.CtrlMsgGraft), p2p.CtrlMsgNonClusterTopicType + return NewDuplicateTopicIDThresholdExceeded(totalDuplicateTopicIds, len(grafts), c.config.GraftPrune.DuplicateTopicIdThreshold), p2p.CtrlMsgNonClusterTopicType } } err, ctrlMsgType := c.validateTopic(from, topic, activeClusterIDS) @@ -388,7 +388,7 @@ func (c *ControlMsgValidationInspector) inspectGraftMessages(from peer.ID, graft totalInvalidTopicIdErrs++ c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgGraft) if totalInvalidTopicIdErrs > c.config.GraftPrune.InvalidTopicIdThreshold { - return err, ctrlMsgType + return NewInvalidTopicIDThresholdExceeded(totalInvalidTopicIdErrs, c.config.GraftPrune.InvalidTopicIdThreshold), ctrlMsgType } } } @@ -429,7 +429,7 @@ func (c *ControlMsgValidationInspector) inspectPruneMessages(from peer.ID, prune // check if the total number of duplicates exceeds the configured threshold. if totalDuplicateTopicIds > c.config.GraftPrune.DuplicateTopicIdThreshold { c.metrics.OnPruneDuplicateTopicIdsExceedThreshold() - return NewDuplicateTopicErr(topic.String(), totalDuplicateTopicIds, p2pmsg.CtrlMsgPrune), p2p.CtrlMsgNonClusterTopicType + return NewDuplicateTopicIDThresholdExceeded(totalDuplicateTopicIds, len(prunes), c.config.GraftPrune.DuplicateTopicIdThreshold), p2p.CtrlMsgNonClusterTopicType } } err, ctrlMsgType := c.validateTopic(from, topic, activeClusterIDS) @@ -437,7 +437,7 @@ func (c *ControlMsgValidationInspector) inspectPruneMessages(from peer.ID, prune totalInvalidTopicIdErrs++ c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgPrune) if totalInvalidTopicIdErrs > c.config.GraftPrune.InvalidTopicIdThreshold { - return err, ctrlMsgType + return NewInvalidTopicIDThresholdExceeded(totalInvalidTopicIdErrs, c.config.GraftPrune.InvalidTopicIdThreshold), ctrlMsgType } } } @@ -493,7 +493,7 @@ func (c *ControlMsgValidationInspector) inspectIHaveMessages(from peer.ID, ihave totalInvalidTopicIdErrs++ c.metrics.OnInvalidTopicIdDetectedForControlMessage(p2pmsg.CtrlMsgIHave) if totalInvalidTopicIdErrs > c.config.IHave.InvalidTopicIdThreshold { - return err, ctrlMsgType + return NewInvalidTopicIDThresholdExceeded(totalInvalidTopicIdErrs, c.config.IHave.InvalidTopicIdThreshold), ctrlMsgType } } @@ -503,7 +503,7 @@ func (c *ControlMsgValidationInspector) inspectIHaveMessages(from peer.ID, ihave // the topic is duplicated, check if the total number of duplicates exceeds the configured threshold if totalDuplicateTopicIds > c.config.IHave.DuplicateTopicIdThreshold { c.metrics.OnIHaveDuplicateTopicIdsExceedThreshold() - return NewDuplicateTopicErr(topic, totalDuplicateTopicIds, p2pmsg.CtrlMsgIHave), p2p.CtrlMsgNonClusterTopicType + return NewDuplicateTopicIDThresholdExceeded(totalDuplicateTopicIds, len(ihaves), c.config.IHave.DuplicateTopicIdThreshold), p2p.CtrlMsgNonClusterTopicType } } diff --git a/network/p2p/inspector/validation/control_message_validation_inspector_test.go b/network/p2p/inspector/validation/control_message_validation_inspector_test.go index 6ebea061322..63e29be0fe7 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector_test.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector_test.go @@ -417,7 +417,7 @@ func TestGraftInspection_InvalidTopic_AboveThreshold(t *testing.T) { invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(invalidSporkIDTopicGrafts...)) from := unittest.PeerIdFixture(t) - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgGraft, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgGraft, validation.IsInvalidTopicIDThresholdExceeded, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) @@ -484,7 +484,7 @@ func TestGraftInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "expected p2p.CtrlMsgNonClusterTopicType notification type, no RPC with cluster prefixed topic sent in this test") require.Equal(t, from, notification.PeerID) require.Equal(t, p2pmsg.CtrlMsgGraft, notification.MsgType) - require.True(t, validation.IsDuplicateTopicErr(notification.Error)) + require.True(t, validation.IsDuplicateTopicIDThresholdExceeded(notification.Error)) }) inspector.Start(signalerCtx) @@ -573,7 +573,7 @@ func TestPruneInspection_InvalidTopic_AboveThreshold(t *testing.T) { invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithPrunes(invalidSporkIDTopicPrunes...)) from := unittest.PeerIdFixture(t) - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgPrune, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgPrune, validation.IsInvalidTopicIDThresholdExceeded, p2p.CtrlMsgNonClusterTopicType) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) @@ -614,7 +614,7 @@ func TestPruneInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "expected p2p.CtrlMsgNonClusterTopicType notification type, no RPC with cluster prefixed topic sent in this test") require.Equal(t, from, notification.PeerID) require.Equal(t, p2pmsg.CtrlMsgPrune, notification.MsgType) - require.True(t, validation.IsDuplicateTopicErr(notification.Error)) + require.True(t, validation.IsDuplicateTopicIDThresholdExceeded(notification.Error)) }) inspector.Start(signalerCtx) @@ -734,7 +734,7 @@ func TestIHaveInspection_InvalidTopic_AboveThreshold(t *testing.T) { from := unittest.PeerIdFixture(t) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, channels.IsInvalidTopicErr, p2p.CtrlMsgNonClusterTopicType) + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, validation.IsInvalidTopicIDThresholdExceeded, p2p.CtrlMsgNonClusterTopicType) distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) inspector.Start(signalerCtx) @@ -806,7 +806,7 @@ func TestIHaveInspection_DuplicateTopicIds_AboveThreshold(t *testing.T) { rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() // one notification should be disseminated for invalid messages when the number of duplicates exceeds the threshold - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, validation.IsDuplicateTopicErr, p2p.CtrlMsgNonClusterTopicType) + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, validation.IsDuplicateTopicIDThresholdExceeded, p2p.CtrlMsgNonClusterTopicType) distributor.On("Distribute", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Once().Run(checkNotification) inspector.Start(signalerCtx) unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) @@ -1284,7 +1284,7 @@ func TestNewControlMsgValidationInspector_validateClusterPrefixedTopic(t *testin from := unittest.PeerIdFixture(t) identity := unittest.IdentityFixture() idProvider.On("ByPeerID", from).Return(identity, true).Times(11) - checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgGraft, channels.IsUnknownClusterIDErr, p2p.CtrlMsgTopicTypeClusterPrefixed) + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgGraft, validation.IsInvalidTopicIDThresholdExceeded, p2p.CtrlMsgTopicTypeClusterPrefixed) inspectMsgRpc := unittest.P2PRPCFixture(unittest.WithGrafts(unittest.P2PRPCGraftFixture(&clusterPrefixedTopic))) inspector.ActiveClustersChanged(flow.ChainIDList{flow.ChainID(unittest.IdentifierFixture().String())}) rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() diff --git a/network/p2p/inspector/validation/errors.go b/network/p2p/inspector/validation/errors.go index bb73f9cba9b..be58d768d59 100644 --- a/network/p2p/inspector/validation/errors.go +++ b/network/p2p/inspector/validation/errors.go @@ -173,3 +173,46 @@ func IsInvalidRpcPublishMessagesErr(err error) bool { var e InvalidRpcPublishMessagesErr return errors.As(err, &e) } + +// DuplicateTopicIDThresholdExceeded indicates that the number of duplicate topic IDs exceeds the allowed threshold. +type DuplicateTopicIDThresholdExceeded struct { + duplicates int + sampleSize int + threshold int +} + +func (e DuplicateTopicIDThresholdExceeded) Error() string { + return fmt.Sprintf("%d/%d duplicate topic IDs exceed the allowed threshold: %d", e.duplicates, e.sampleSize, e.threshold) +} + +// NewDuplicateTopicIDThresholdExceeded returns a new DuplicateTopicIDThresholdExceeded error. +func NewDuplicateTopicIDThresholdExceeded(duplicates int, sampleSize int, threshold int) DuplicateTopicIDThresholdExceeded { + return DuplicateTopicIDThresholdExceeded{duplicates, sampleSize, threshold} +} + +// IsDuplicateTopicIDThresholdExceeded returns true if an error is DuplicateTopicIDThresholdExceeded +func IsDuplicateTopicIDThresholdExceeded(err error) bool { + var e DuplicateTopicIDThresholdExceeded + return errors.As(err, &e) +} + +// InvalidTopicIDThresholdExceeded indicates that the number of invalid topic IDs exceeds the allowed threshold. +type InvalidTopicIDThresholdExceeded struct { + invalidCount int + threshold int +} + +func (e InvalidTopicIDThresholdExceeded) Error() string { + return fmt.Sprintf("%d invalid topic IDs exceed the allowed threshold: %d", e.invalidCount, e.threshold) +} + +// NewInvalidTopicIDThresholdExceeded returns a new InvalidTopicIDThresholdExceeded error. +func NewInvalidTopicIDThresholdExceeded(invalidCount, threshold int) InvalidTopicIDThresholdExceeded { + return InvalidTopicIDThresholdExceeded{invalidCount, threshold} +} + +// IsInvalidTopicIDThresholdExceeded returns true if an error is InvalidTopicIDThresholdExceeded. +func IsInvalidTopicIDThresholdExceeded(err error) bool { + var e InvalidTopicIDThresholdExceeded + return errors.As(err, &e) +} diff --git a/network/p2p/inspector/validation/errors_test.go b/network/p2p/inspector/validation/errors_test.go index 29072fbd5f7..cc56ca52fde 100644 --- a/network/p2p/inspector/validation/errors_test.go +++ b/network/p2p/inspector/validation/errors_test.go @@ -105,3 +105,27 @@ func TestInvalidRpcPublishMessagesErrRoundTrip(t *testing.T) { dummyErr := fmt.Errorf("dummy error") assert.False(t, IsInvalidRpcPublishMessagesErr(dummyErr), "IsInvalidRpcPublishMessagesErr should return false for non-InvalidRpcPublishMessagesErr error") } + +// TestErrDuplicateTopicIDThresholdExceededRoundTrip ensures correct error formatting for DuplicateTopicIDThresholdExceeded. +func TestDuplicateTopicIDThresholdExceededRoundTrip(t *testing.T) { + expectedErrorMsg := "3/5 duplicate topic IDs exceed the allowed threshold: 2" + err := NewDuplicateTopicIDThresholdExceeded(3, 5, 2) + assert.Equal(t, expectedErrorMsg, err.Error(), "the error message should be correctly formatted") + // tests the IsDuplicateTopicIDThresholdExceeded function. + assert.True(t, IsDuplicateTopicIDThresholdExceeded(err), "IsDuplicateTopicIDThresholdExceeded should return true for DuplicateTopicIDThresholdExceeded error") + // test IsDuplicateTopicIDThresholdExceeded with a different error type. + dummyErr := fmt.Errorf("dummy error") + assert.False(t, IsDuplicateTopicIDThresholdExceeded(dummyErr), "IsDuplicateTopicIDThresholdExceeded should return false for non-DuplicateTopicIDThresholdExceeded error") +} + +// TestErrInvalidTopicIDThresholdExceededRoundTrip ensures correct error formatting for InvalidTopicIDThresholdExceeded. +func TestInvalidTopicIDThresholdExceededRoundTrip(t *testing.T) { + expectedErrorMsg := "8 invalid topic IDs exceed the allowed threshold: 5" + err := NewInvalidTopicIDThresholdExceeded(8, 5) + assert.Equal(t, expectedErrorMsg, err.Error(), "the error message should be correctly formatted") + // tests the IsInvalidTopicIDThresholdExceeded function. + assert.True(t, IsInvalidTopicIDThresholdExceeded(err), "IsInvalidTopicIDThresholdExceeded should return true for InvalidTopicIDThresholdExceeded error") + // test IsInvalidTopicIDThresholdExceeded with a different error type. + dummyErr := fmt.Errorf("dummy error") + assert.False(t, IsInvalidTopicIDThresholdExceeded(dummyErr), "IsInvalidTopicIDThresholdExceeded should return false for non-InvalidTopicIDThresholdExceeded error") +} From fd268a3d0c6e11e44e48be9d3058202da8539d43 Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Wed, 21 Feb 2024 10:56:17 -0500 Subject: [PATCH 48/52] update mocks --- module/mock/gossip_sub_metrics.go | 33 ++++++++++++++----- ...ip_sub_rpc_validation_inspector_metrics.go | 33 ++++++++++++++----- module/mock/lib_p2_p_metrics.go | 33 ++++++++++++++----- module/mock/network_metrics.go | 33 ++++++++++++++----- 4 files changed, 96 insertions(+), 36 deletions(-) diff --git a/module/mock/gossip_sub_metrics.go b/module/mock/gossip_sub_metrics.go index f7e057ea5ba..f2650ceefc3 100644 --- a/module/mock/gossip_sub_metrics.go +++ b/module/mock/gossip_sub_metrics.go @@ -56,9 +56,14 @@ func (_m *GossipSubMetrics) OnGraftDuplicateTopicIdsExceedThreshold() { _m.Called() } -// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *GossipSubMetrics) OnGraftMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnGraftInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *GossipSubMetrics) OnGraftInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *GossipSubMetrics) OnGraftMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnIHaveControlMessageIdsTruncated provides a mock function with given fields: diff @@ -76,14 +81,19 @@ func (_m *GossipSubMetrics) OnIHaveDuplicateTopicIdsExceedThreshold() { _m.Called() } +// OnIHaveInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *GossipSubMetrics) OnIHaveInvalidTopicIdsExceedThreshold() { + _m.Called() +} + // OnIHaveMessageIDsReceived provides a mock function with given fields: channel, msgIdCount func (_m *GossipSubMetrics) OnIHaveMessageIDsReceived(channel string, msgIdCount int) { _m.Called(channel, msgIdCount) } -// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds -func (_m *GossipSubMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) { - _m.Called(duplicateTopicIds, duplicateMessageIds) +// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds, invalidTopicIds +func (_m *GossipSubMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, duplicateMessageIds, invalidTopicIds) } // OnIPColocationFactorUpdated provides a mock function with given fields: _a0 @@ -216,9 +226,14 @@ func (_m *GossipSubMetrics) OnPruneDuplicateTopicIdsExceedThreshold() { _m.Called() } -// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *GossipSubMetrics) OnPruneMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnPruneInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *GossipSubMetrics) OnPruneInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *GossipSubMetrics) OnPruneMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnPublishMessageInspected provides a mock function with given fields: totalErrCount, invalidTopicIdsCount, invalidSubscriptionsCount, invalidSendersCount diff --git a/module/mock/gossip_sub_rpc_validation_inspector_metrics.go b/module/mock/gossip_sub_rpc_validation_inspector_metrics.go index 84eef02f7ea..b9624e8a32a 100644 --- a/module/mock/gossip_sub_rpc_validation_inspector_metrics.go +++ b/module/mock/gossip_sub_rpc_validation_inspector_metrics.go @@ -40,9 +40,14 @@ func (_m *GossipSubRpcValidationInspectorMetrics) OnGraftDuplicateTopicIdsExceed _m.Called() } -// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *GossipSubRpcValidationInspectorMetrics) OnGraftMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnGraftInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *GossipSubRpcValidationInspectorMetrics) OnGraftInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *GossipSubRpcValidationInspectorMetrics) OnGraftMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnIHaveControlMessageIdsTruncated provides a mock function with given fields: diff @@ -60,14 +65,19 @@ func (_m *GossipSubRpcValidationInspectorMetrics) OnIHaveDuplicateTopicIdsExceed _m.Called() } +// OnIHaveInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *GossipSubRpcValidationInspectorMetrics) OnIHaveInvalidTopicIdsExceedThreshold() { + _m.Called() +} + // OnIHaveMessageIDsReceived provides a mock function with given fields: channel, msgIdCount func (_m *GossipSubRpcValidationInspectorMetrics) OnIHaveMessageIDsReceived(channel string, msgIdCount int) { _m.Called(channel, msgIdCount) } -// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds -func (_m *GossipSubRpcValidationInspectorMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) { - _m.Called(duplicateTopicIds, duplicateMessageIds) +// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds, invalidTopicIds +func (_m *GossipSubRpcValidationInspectorMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, duplicateMessageIds, invalidTopicIds) } // OnIWantCacheMissMessageIdsExceedThreshold provides a mock function with given fields: @@ -115,9 +125,14 @@ func (_m *GossipSubRpcValidationInspectorMetrics) OnPruneDuplicateTopicIdsExceed _m.Called() } -// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *GossipSubRpcValidationInspectorMetrics) OnPruneMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnPruneInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *GossipSubRpcValidationInspectorMetrics) OnPruneInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *GossipSubRpcValidationInspectorMetrics) OnPruneMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnPublishMessageInspected provides a mock function with given fields: totalErrCount, invalidTopicIdsCount, invalidSubscriptionsCount, invalidSendersCount diff --git a/module/mock/lib_p2_p_metrics.go b/module/mock/lib_p2_p_metrics.go index 298a16fd936..3f5f2dc6e2f 100644 --- a/module/mock/lib_p2_p_metrics.go +++ b/module/mock/lib_p2_p_metrics.go @@ -187,9 +187,14 @@ func (_m *LibP2PMetrics) OnGraftDuplicateTopicIdsExceedThreshold() { _m.Called() } -// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *LibP2PMetrics) OnGraftMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnGraftInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *LibP2PMetrics) OnGraftInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *LibP2PMetrics) OnGraftMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnIHaveControlMessageIdsTruncated provides a mock function with given fields: diff @@ -207,14 +212,19 @@ func (_m *LibP2PMetrics) OnIHaveDuplicateTopicIdsExceedThreshold() { _m.Called() } +// OnIHaveInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *LibP2PMetrics) OnIHaveInvalidTopicIdsExceedThreshold() { + _m.Called() +} + // OnIHaveMessageIDsReceived provides a mock function with given fields: channel, msgIdCount func (_m *LibP2PMetrics) OnIHaveMessageIDsReceived(channel string, msgIdCount int) { _m.Called(channel, msgIdCount) } -// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds -func (_m *LibP2PMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) { - _m.Called(duplicateTopicIds, duplicateMessageIds) +// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds, invalidTopicIds +func (_m *LibP2PMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, duplicateMessageIds, invalidTopicIds) } // OnIPColocationFactorUpdated provides a mock function with given fields: _a0 @@ -357,9 +367,14 @@ func (_m *LibP2PMetrics) OnPruneDuplicateTopicIdsExceedThreshold() { _m.Called() } -// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *LibP2PMetrics) OnPruneMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnPruneInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *LibP2PMetrics) OnPruneInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *LibP2PMetrics) OnPruneMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnPublishMessageInspected provides a mock function with given fields: totalErrCount, invalidTopicIdsCount, invalidSubscriptionsCount, invalidSendersCount diff --git a/module/mock/network_metrics.go b/module/mock/network_metrics.go index ad77ea288f0..325a028e088 100644 --- a/module/mock/network_metrics.go +++ b/module/mock/network_metrics.go @@ -217,9 +217,14 @@ func (_m *NetworkMetrics) OnGraftDuplicateTopicIdsExceedThreshold() { _m.Called() } -// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *NetworkMetrics) OnGraftMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnGraftInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *NetworkMetrics) OnGraftInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnGraftMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *NetworkMetrics) OnGraftMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnIHaveControlMessageIdsTruncated provides a mock function with given fields: diff @@ -237,14 +242,19 @@ func (_m *NetworkMetrics) OnIHaveDuplicateTopicIdsExceedThreshold() { _m.Called() } +// OnIHaveInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *NetworkMetrics) OnIHaveInvalidTopicIdsExceedThreshold() { + _m.Called() +} + // OnIHaveMessageIDsReceived provides a mock function with given fields: channel, msgIdCount func (_m *NetworkMetrics) OnIHaveMessageIDsReceived(channel string, msgIdCount int) { _m.Called(channel, msgIdCount) } -// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds -func (_m *NetworkMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int) { - _m.Called(duplicateTopicIds, duplicateMessageIds) +// OnIHaveMessagesInspected provides a mock function with given fields: duplicateTopicIds, duplicateMessageIds, invalidTopicIds +func (_m *NetworkMetrics) OnIHaveMessagesInspected(duplicateTopicIds int, duplicateMessageIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, duplicateMessageIds, invalidTopicIds) } // OnIPColocationFactorUpdated provides a mock function with given fields: _a0 @@ -392,9 +402,14 @@ func (_m *NetworkMetrics) OnPruneDuplicateTopicIdsExceedThreshold() { _m.Called() } -// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds -func (_m *NetworkMetrics) OnPruneMessageInspected(duplicateTopicIds int) { - _m.Called(duplicateTopicIds) +// OnPruneInvalidTopicIdsExceedThreshold provides a mock function with given fields: +func (_m *NetworkMetrics) OnPruneInvalidTopicIdsExceedThreshold() { + _m.Called() +} + +// OnPruneMessageInspected provides a mock function with given fields: duplicateTopicIds, invalidTopicIds +func (_m *NetworkMetrics) OnPruneMessageInspected(duplicateTopicIds int, invalidTopicIds int) { + _m.Called(duplicateTopicIds, invalidTopicIds) } // OnPublishMessageInspected provides a mock function with given fields: totalErrCount, invalidTopicIdsCount, invalidSubscriptionsCount, invalidSendersCount From 3c2d3134b70411a50b9feb705d105015e2e4d300 Mon Sep 17 00:00:00 2001 From: Faye Amacker <33205765+fxamacker@users.noreply.github.com> Date: Wed, 21 Feb 2024 17:45:44 -0600 Subject: [PATCH 49/52] Use latest NewCompactor API --- .../execution-state-extract/execution_state_extract_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go index bc52133f1b9..882c88df898 100644 --- a/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go +++ b/cmd/util/cmd/execution-state-extract/execution_state_extract_test.go @@ -237,7 +237,7 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { require.NoError(t, err) f, err := complete.NewLedger(diskWal, size*10, metr, zerolog.Nop(), complete.DefaultPathFinderVersion) require.NoError(t, err) - compactor, err := complete.NewCompactor(f, diskWal, zerolog.Nop(), uint(size), checkpointDistance, checkpointsToKeep, atomic.NewBool(false)) + compactor, err := complete.NewCompactor(f, diskWal, zerolog.Nop(), uint(size), checkpointDistance, checkpointsToKeep, atomic.NewBool(false), &metrics.NoopCollector{}) require.NoError(t, err) <-compactor.Ready() @@ -316,7 +316,7 @@ func TestExtractPayloadsFromExecutionState(t *testing.T) { require.NoError(t, err) f, err := complete.NewLedger(diskWal, size*10, metr, zerolog.Nop(), complete.DefaultPathFinderVersion) require.NoError(t, err) - compactor, err := complete.NewCompactor(f, diskWal, zerolog.Nop(), uint(size), checkpointDistance, checkpointsToKeep, atomic.NewBool(false)) + compactor, err := complete.NewCompactor(f, diskWal, zerolog.Nop(), uint(size), checkpointDistance, checkpointsToKeep, atomic.NewBool(false), &metrics.NoopCollector{}) require.NoError(t, err) <-compactor.Ready() From 30d0231c05a3167125c335540de05ceedac95556 Mon Sep 17 00:00:00 2001 From: Yahya Hassanzadeh Date: Wed, 21 Feb 2024 16:48:07 -0800 Subject: [PATCH 50/52] comments out flakey test --- .../test/gossipsub/scoring/scoring_test.go | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go index 8cdf52c5e1a..d2caeae9072 100644 --- a/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go +++ b/insecure/integration/functional/test/gossipsub/scoring/scoring_test.go @@ -30,25 +30,26 @@ func TestGossipSubInvalidMessageDelivery_Integration(t *testing.T) { tt := []struct { name string spamMsgFactory func(spammerId peer.ID, victimId peer.ID, topic channels.Topic) *pubsub_pb.Message - }{ - { - name: "unknown peer, invalid signature", - spamMsgFactory: func(spammerId peer.ID, _ peer.ID, topic channels.Topic) *pubsub_pb.Message { - return p2ptest.PubsubMessageFixture(t, p2ptest.WithTopic(topic.String())) - }, - }, + }{ // TODO: unittest.SkipUnless(t, unittest.TEST_FLAKY, "https://github.com/dapperlabs/flow-go/issues/6949") + // { + // + // name: "unknown peer, invalid signature", + // spamMsgFactory: func(spammerId peer.ID, _ peer.ID, topic channels.Topic) *pubsub_pb.Message { + // return p2ptest.PubsubMessageFixture(t, p2ptest.WithTopic(topic.String())) + // }, + // }, { name: "unknown peer, missing signature", spamMsgFactory: func(spammerId peer.ID, _ peer.ID, topic channels.Topic) *pubsub_pb.Message { return p2ptest.PubsubMessageFixture(t, p2ptest.WithTopic(topic.String()), p2ptest.WithoutSignature()) }, }, - //{ + // { // name: "known peer, invalid signature", // spamMsgFactory: func(spammerId peer.ID, _ peer.ID, topic channels.Topic) *pubsub_pb.Message { // return p2ptest.PubsubMessageFixture(t, p2ptest.WithFrom(spammerId), p2ptest.WithTopic(topic.String())) // }, - //}, + // }, { name: "known peer, missing signature", spamMsgFactory: func(spammerId peer.ID, _ peer.ID, topic channels.Topic) *pubsub_pb.Message { From 3e393bb26badee5f510fbc317431fea7041b6bb0 Mon Sep 17 00:00:00 2001 From: Khalil Claybon Date: Thu, 22 Feb 2024 20:16:20 -0500 Subject: [PATCH 51/52] merge master --- .../validation_inspector_test.go | 2 +- module/metrics.go | 2 +- ...ntrol_message_validation_inspector_test.go | 136 ++++++++++++++++++ 3 files changed, 138 insertions(+), 2 deletions(-) diff --git a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go index 088ab934022..22847df96ca 100644 --- a/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go +++ b/insecure/integration/functional/test/gossipsub/rpc_inspector/validation_inspector_test.go @@ -71,7 +71,7 @@ func TestValidationInspector_InvalidTopicId_Detection(t *testing.T) { require.True(t, ok) require.Equal(t, notification.TopicType, p2p.CtrlMsgNonClusterTopicType, "IsClusterPrefixed is expected to be false, no RPC with cluster prefixed topic sent in this test") require.Equal(t, spammer.SpammerNode.ID(), notification.PeerID) - require.True(t, channels.IsInvalidTopicErr(notification.Error)) + require.True(t, validation.IsInvalidTopicIDThresholdExceeded(notification.Error)) switch notification.MsgType { case p2pmsg.CtrlMsgGraft: invGraftNotifCount.Inc() diff --git a/module/metrics.go b/module/metrics.go index 44b0acda605..39342be16b1 100644 --- a/module/metrics.go +++ b/module/metrics.go @@ -320,7 +320,7 @@ type GossipSubRpcValidationInspectorMetrics interface { // OnIHaveInvalidTopicIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of invalid topic ids // received by the node on the iHave messages of that RPC exceeding the threshold, which results in a misbehaviour report. OnIHaveInvalidTopicIdsExceedThreshold() - + // OnIHaveDuplicateMessageIdsExceedThreshold tracks the number of times that the async inspection of iHave messages of a single RPC failed due to the total number of duplicate message ids // received by the node on an iHave message exceeding the threshold, which results in a misbehaviour report. OnIHaveDuplicateMessageIdsExceedThreshold() diff --git a/network/p2p/inspector/validation/control_message_validation_inspector_test.go b/network/p2p/inspector/validation/control_message_validation_inspector_test.go index 93289a3836d..0cd0bfb02dc 100644 --- a/network/p2p/inspector/validation/control_message_validation_inspector_test.go +++ b/network/p2p/inspector/validation/control_message_validation_inspector_test.go @@ -342,6 +342,50 @@ func TestControlMessageInspection_ValidRpc(t *testing.T) { unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") } +// TestGraftInspection_InvalidTopic_BelowThreshold ensures inspector does not disseminate an invalid control message notification for +// graft messages when the invalid topic id count does not exceed the configured threshold. +func TestGraftInspection_InvalidTopic_BelowThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicGrafts []*pubsub_pb.ControlGraft + var malformedTopicGrafts []*pubsub_pb.ControlGraft + var invalidSporkIDTopicGrafts []*pubsub_pb.ControlGraft + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicGrafts = append(unknownTopicGrafts, unittest.P2PRPCGraftFixture(&unknownTopic)) + malformedTopicGrafts = append(malformedTopicGrafts, unittest.P2PRPCGraftFixture(&malformedTopic)) + invalidSporkIDTopicGrafts = append(invalidSporkIDTopicGrafts, unittest.P2PRPCGraftFixture(&invalidSporkIDTopic)) + } + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(unknownTopicGrafts...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(malformedTopicGrafts...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithGrafts(invalidSporkIDTopicGrafts...)) + + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(3 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + // TestGraftInspection_InvalidTopic_AboveThreshold ensures inspector disseminates an invalid control message notification for // graft messages when the invalid topic id count exceeds the configured threshold. func TestGraftInspection_InvalidTopic_AboveThreshold(t *testing.T) { @@ -612,6 +656,98 @@ func TestPruneInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") } +// TestIHaveInspection_InvalidTopic_AboveThreshold ensures inspector disseminates an invalid control message notification for +// ihave messages when the invalid topic id count exceeds the configured threshold. +func TestIHaveInspection_InvalidTopic_AboveThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicIHaves []*pubsub_pb.ControlIHave + var malformedTopicIHaves []*pubsub_pb.ControlIHave + var invalidSporkIDTopicIHaves []*pubsub_pb.ControlIHave + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold+1; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicIHaves = append(unknownTopicIHaves, unittest.P2PRPCIHaveFixture(&unknownTopic, unittest.IdentifierListFixture(5).Strings()...)) + malformedTopicIHaves = append(malformedTopicIHaves, unittest.P2PRPCIHaveFixture(&malformedTopic, unittest.IdentifierListFixture(5).Strings()...)) + invalidSporkIDTopicIHaves = append(invalidSporkIDTopicIHaves, unittest.P2PRPCIHaveFixture(&invalidSporkIDTopic, unittest.IdentifierListFixture(5).Strings()...)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(unknownTopicIHaves...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(malformedTopicIHaves...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(invalidSporkIDTopicIHaves...)) + + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + checkNotification := checkNotificationFunc(t, from, p2pmsg.CtrlMsgIHave, validation.IsInvalidTopicIDThresholdExceeded, p2p.CtrlMsgNonClusterTopicType) + consumer.On("OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")).Return(nil).Times(3).Run(checkNotification) + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(2 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + +// TestIHaveInspection_InvalidTopic_BelowThreshold ensures inspector does not disseminate an invalid control message notification for +// ihave messages when the invalid topic id count does not exceed the configured threshold. +func TestIHaveInspection_InvalidTopic_BelowThreshold(t *testing.T) { + c, err := config.DefaultConfig() + require.NoError(t, err) + cfg := &c.NetworkConfig.GossipSub.RpcInspector.Validation + inspector, signalerCtx, cancel, consumer, rpcTracker, sporkID, _, topicProviderOracle := inspectorFixture(t, func(params *validation.InspectorParams) { + params.Config = cfg + }) + + var unknownTopicIHaves []*pubsub_pb.ControlIHave + var malformedTopicIHaves []*pubsub_pb.ControlIHave + var invalidSporkIDTopicIHaves []*pubsub_pb.ControlIHave + var allTopics []string + for i := 0; i < cfg.GraftPrune.InvalidTopicIdThreshold; i++ { + // create unknown topic + unknownTopic, malformedTopic, invalidSporkIDTopic := invalidTopics(t, sporkID) + allTopics = append(allTopics, unknownTopic, malformedTopic, invalidSporkIDTopic) + unknownTopicIHaves = append(unknownTopicIHaves, unittest.P2PRPCIHaveFixture(&unknownTopic, unittest.IdentifierListFixture(5).Strings()...)) + malformedTopicIHaves = append(malformedTopicIHaves, unittest.P2PRPCIHaveFixture(&malformedTopic, unittest.IdentifierListFixture(5).Strings()...)) + invalidSporkIDTopicIHaves = append(invalidSporkIDTopicIHaves, unittest.P2PRPCIHaveFixture(&invalidSporkIDTopic, unittest.IdentifierListFixture(5).Strings()...)) + } + + // avoid unknown topics errors + topicProviderOracle.UpdateTopics(allTopics) + unknownTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(unknownTopicIHaves...)) + malformedTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(malformedTopicIHaves...)) + invalidSporkIDTopicReq := unittest.P2PRPCFixture(unittest.WithIHaves(invalidSporkIDTopicIHaves...)) + + from := unittest.PeerIdFixture(t) + rpcTracker.On("LastHighestIHaveRPCSize").Return(int64(100)).Maybe() + // no notification should be disseminated for valid messages as long as the number of invalid topic ids is below the threshold + consumer.AssertNotCalled(t, "OnInvalidControlMessageNotification", mock.AnythingOfType("*p2p.InvCtrlMsgNotif")) + inspector.Start(signalerCtx) + unittest.RequireComponentsReadyBefore(t, 1*time.Second, inspector) + + require.NoError(t, inspector.Inspect(from, unknownTopicReq)) + require.NoError(t, inspector.Inspect(from, malformedTopicReq)) + require.NoError(t, inspector.Inspect(from, invalidSporkIDTopicReq)) + + // sleep for 1 second to ensure rpc's is processed + time.Sleep(2 * time.Second) + cancel() + unittest.RequireCloseBefore(t, inspector.Done(), 5*time.Second, "inspector did not stop") +} + // TestIHaveInspection_DuplicateTopicIds_BelowThreshold ensures inspector does not disseminate an invalid control message notification for // iHave messages when duplicate topic ids are below allowed threshold. func TestIHaveInspection_DuplicateTopicIds_BelowThreshold(t *testing.T) { From f4dd026579ce34ea2f9ef1a5b5e3c166608cb331 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bastian=20M=C3=BCller?= Date: Fri, 23 Feb 2024 10:10:49 -0800 Subject: [PATCH 52/52] Fix return type --- integration/benchmark/load/evm_load.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration/benchmark/load/evm_load.go b/integration/benchmark/load/evm_load.go index 3c9dff39f19..344e0d6eedb 100644 --- a/integration/benchmark/load/evm_load.go +++ b/integration/benchmark/load/evm_load.go @@ -132,7 +132,7 @@ func (l *EVMTransferLoad) Setup(log zerolog.Logger, lc LoadContext) error { data: [UInt8], gasLimit: UInt64, value: EVM.Balance - ): [UInt8] { + ): EVM.Result { return self.acc.call( to: to, data: data,